HDDS-692. Use the ProgressBar class in the RandomKeyGenerator freon test. Contributed by Zsolt Horvath.

This commit is contained in:
Márton Elek 2018-11-20 12:56:03 +01:00
parent c946f1b121
commit 4ecdcc9620
3 changed files with 120 additions and 291 deletions

View File

@ -14,177 +14,116 @@
* License for the specific language governing permissions and limitations under * License for the specific language governing permissions and limitations under
* the License. * the License.
*/ */
package org.apache.hadoop.ozone.freon; package org.apache.hadoop.ozone.freon;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.PrintStream; import java.io.PrintStream;
import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicLong;
import java.util.function.Function;
import java.util.function.Supplier; import java.util.function.Supplier;
/** /**
* Run an arbitrary code and print progress on the provided stream. The * Creates and runs a ProgressBar in new Thread which gets printed on
* progressbar stops when: - the provided currentvalue is less the the maxvalue * the provided PrintStream.
* - exception thrown
*/ */
public class ProgressBar { public class ProgressBar {
private static final Logger LOG = LoggerFactory.getLogger(ProgressBar.class);
private static final long REFRESH_INTERVAL = 1000L; private static final long REFRESH_INTERVAL = 1000L;
private PrintStream stream; private final long maxValue;
private AtomicLong currentValue; private final Supplier<Long> currentValue;
private long maxValue; private final Thread progressBar;
private Thread progressBar;
private volatile boolean exception = false; private volatile boolean running;
private long startTime;
private volatile long startTime;
/** /**
* @param stream Used to display the progress * Creates a new ProgressBar instance which prints the progress on the given
* PrintStream when started.
*
* @param stream to display the progress
* @param maxValue Maximum value of the progress * @param maxValue Maximum value of the progress
* @param currentValue Supplier that provides the current value
*/ */
ProgressBar(PrintStream stream, long maxValue) { public ProgressBar(final PrintStream stream, final Long maxValue,
this.stream = stream; final Supplier<Long> currentValue) {
this.maxValue = maxValue; this.maxValue = maxValue;
this.currentValue = new AtomicLong(0); this.currentValue = currentValue;
this.progressBar = new Thread(new ProgressBarThread()); this.progressBar = new Thread(getProgressBar(stream));
this.running = false;
} }
/** /**
* Start a task with a progessbar without any in/out parameters Runnable used * Starts the ProgressBar in a new Thread.
* just a task wrapper. * This is a non blocking call.
*
* @param task Runnable
*/ */
public void start(Runnable task) { public synchronized void start() {
if (!running) {
running = true;
startTime = System.nanoTime(); startTime = System.nanoTime();
try {
progressBar.start(); progressBar.start();
task.run(); }
}
} catch (Exception e) {
exception = true;
} finally {
/**
* Graceful shutdown, waits for the progress bar to complete.
* This is a blocking call.
*/
public synchronized void shutdown() {
if (running) {
try { try {
progressBar.join(); progressBar.join();
running = false;
} catch (InterruptedException e) { } catch (InterruptedException e) {
e.printStackTrace(); LOG.warn("Got interrupted while waiting for the progress bar to " +
"complete.");
} }
} }
} }
/** /**
* Start a task with only out parameters. * Terminates the progress bar. This doesn't wait for the progress bar
* * to complete.
* @param task Supplier that represents the task
* @param <T> Generic return type
* @return Whatever the supllier produces
*/ */
public <T> T start(Supplier<T> task) { public synchronized void terminate() {
if (running) {
startTime = System.nanoTime();
T result = null;
try {
progressBar.start();
result = task.get();
} catch (Exception e) {
exception = true;
} finally {
try { try {
running = false;
progressBar.join(); progressBar.join();
} catch (InterruptedException e) { } catch (InterruptedException e) {
e.printStackTrace(); LOG.warn("Got interrupted while waiting for the progress bar to " +
"complete.");
} }
return result;
} }
} }
/** private Runnable getProgressBar(final PrintStream stream) {
* Start a task with in/out parameters. return () -> {
*
* @param input Input of the function
* @param task A Function that does the task
* @param <T> type of the input
* @param <R> return type
* @return Whatever the Function returns
*/
public <T, R> R start(T input, Function<T, R> task) {
startTime = System.nanoTime();
R result = null;
try {
progressBar.start();
result = task.apply(input);
} catch (Exception e) {
exception = true;
throw e;
} finally {
try {
progressBar.join();
} catch (InterruptedException e) {
e.printStackTrace();
}
return result;
}
}
/**
* Increment the progress with one step.
*/
public void incrementProgress() {
currentValue.incrementAndGet();
}
private class ProgressBarThread implements Runnable {
@Override
public void run() {
try {
stream.println(); stream.println();
long value; while (running && currentValue.get() < maxValue) {
print(stream, currentValue.get());
while ((value = currentValue.get()) < maxValue) { try {
print(value);
if (exception) {
break;
}
Thread.sleep(REFRESH_INTERVAL); Thread.sleep(REFRESH_INTERVAL);
}
if (exception) {
stream.println();
stream.println("Incomplete termination, " + "check log for " +
"exception.");
} else {
print(maxValue);
}
stream.println();
} catch (InterruptedException e) { } catch (InterruptedException e) {
stream.println(e); LOG.warn("ProgressBar was interrupted.");
} }
} }
print(stream, maxValue);
stream.println();
running = false;
};
}
/** /**
* Given current value prints the progress bar. * Given current value prints the progress bar.
* *
* @param value current progress position * @param value current progress position
*/ */
private void print(long value) { private void print(final PrintStream stream, final long value) {
stream.print('\r'); stream.print('\r');
double percent = 100.0 * value / maxValue; double percent = 100.0 * value / maxValue;
StringBuilder sb = new StringBuilder(); StringBuilder sb = new StringBuilder();
@ -206,5 +145,3 @@ private void print(long value) {
stream.print(sb.toString()); stream.print(sb.toString());
} }
} }
}

View File

@ -185,6 +185,7 @@ enum FreonOps {
private ArrayList<Histogram> histograms = new ArrayList<>(); private ArrayList<Histogram> histograms = new ArrayList<>();
private OzoneConfiguration ozoneConfiguration; private OzoneConfiguration ozoneConfiguration;
private ProgressBar progressbar;
RandomKeyGenerator() { RandomKeyGenerator() {
} }
@ -251,13 +252,26 @@ public Void call() throws Exception {
validator.start(); validator.start();
LOG.info("Data validation is enabled."); LOG.info("Data validation is enabled.");
} }
Thread progressbar = getProgressBarThread();
Supplier<Long> currentValue;
long maxValue;
currentValue = () -> numberOfKeysAdded.get();
maxValue = numOfVolumes *
numOfBuckets *
numOfKeys;
progressbar = new ProgressBar(System.out, maxValue, currentValue);
LOG.info("Starting progress bar Thread."); LOG.info("Starting progress bar Thread.");
progressbar.start(); progressbar.start();
processor.shutdown(); processor.shutdown();
processor.awaitTermination(Integer.MAX_VALUE, TimeUnit.MILLISECONDS); processor.awaitTermination(Integer.MAX_VALUE, TimeUnit.MILLISECONDS);
completed = true;
progressbar.join(); progressbar.shutdown();
if (validateWrites) { if (validateWrites) {
validator.join(); validator.join();
} }
@ -280,22 +294,6 @@ private void addShutdownHook() {
Runtime.getRuntime().addShutdownHook( Runtime.getRuntime().addShutdownHook(
new Thread(() -> printStats(System.out))); new Thread(() -> printStats(System.out)));
} }
private Thread getProgressBarThread() {
Supplier<Long> currentValue;
long maxValue;
currentValue = () -> numberOfKeysAdded.get();
maxValue = (long) numOfVolumes *
numOfBuckets *
numOfKeys;
Thread progressBarThread = new Thread(
new ProgressBar(System.out, currentValue, maxValue));
progressBarThread.setName("ProgressBar");
return progressBarThread;
}
/** /**
* Prints stats of {@link Freon} run to the PrintStream. * Prints stats of {@link Freon} run to the PrintStream.
* *
@ -896,73 +894,6 @@ public String[] getTenQuantileKeyWriteTime() {
} }
} }
private class ProgressBar implements Runnable {
private static final long REFRESH_INTERVAL = 1000L;
private PrintStream stream;
private Supplier<Long> currentValue;
private long maxValue;
ProgressBar(PrintStream stream, Supplier<Long> currentValue,
long maxValue) {
this.stream = stream;
this.currentValue = currentValue;
this.maxValue = maxValue;
}
@Override
public void run() {
try {
stream.println();
long value;
while ((value = currentValue.get()) < maxValue) {
print(value);
if (completed) {
break;
}
Thread.sleep(REFRESH_INTERVAL);
}
if (exception) {
stream.println();
stream.println("Incomplete termination, " +
"check log for exception.");
} else {
print(maxValue);
}
stream.println();
} catch (InterruptedException e) {
}
}
/**
* Given current value prints the progress bar.
*
* @param value
*/
private void print(long value) {
stream.print('\r');
double percent = 100.0 * value / maxValue;
StringBuilder sb = new StringBuilder();
sb.append(" " + String.format("%.2f", percent) + "% |");
for (int i = 0; i <= percent; i++) {
sb.append('█');
}
for (int j = 0; j < 100 - percent; j++) {
sb.append(' ');
}
sb.append("| ");
sb.append(value + "/" + maxValue);
long timeInSec = TimeUnit.SECONDS.convert(
System.nanoTime() - startTime, TimeUnit.NANOSECONDS);
String timeToPrint = String.format("%d:%02d:%02d", timeInSec / 3600,
(timeInSec % 3600) / 60, timeInSec % 60);
sb.append(" Time: " + timeToPrint);
stream.print(sb);
}
}
/** /**
* Validates the write done in ozone cluster. * Validates the write done in ozone cluster.
*/ */

View File

@ -22,12 +22,15 @@
import org.mockito.junit.MockitoJUnitRunner; import org.mockito.junit.MockitoJUnitRunner;
import java.io.PrintStream; import java.io.PrintStream;
import java.util.function.Function; import java.util.concurrent.atomic.AtomicLong;
import java.util.function.Supplier; import java.util.function.Supplier;
import java.util.stream.IntStream; import java.util.stream.LongStream;
import static org.mockito.Mockito.*; import static org.mockito.Mockito.*;
/**
* Using Mockito runner.
*/
@RunWith(MockitoJUnitRunner.class) @RunWith(MockitoJUnitRunner.class)
/** /**
* Tests for the Progressbar class for Freon. * Tests for the Progressbar class for Freon.
@ -35,78 +38,36 @@
public class TestProgressBar { public class TestProgressBar {
private PrintStream stream; private PrintStream stream;
private AtomicLong numberOfKeysAdded;
private Supplier<Long> currentValue;
@Before @Before
public void setupMock() { public void setupMock() {
numberOfKeysAdded = new AtomicLong(0L);
currentValue = () -> numberOfKeysAdded.get();
stream = mock(PrintStream.class); stream = mock(PrintStream.class);
} }
@Test @Test
public void testWithRunnable() { public void testWithRunnable() {
int maxValue = 10; Long maxValue = 10L;
ProgressBar progressbar = new ProgressBar(stream, maxValue); ProgressBar progressbar = new ProgressBar(stream, maxValue, currentValue);
Runnable task = () -> { Runnable task = () -> {
IntStream.range(0, maxValue).forEach( LongStream.range(0, maxValue).forEach(
counter -> { counter -> {
progressbar.incrementProgress(); numberOfKeysAdded.getAndIncrement();
} }
); );
}; };
progressbar.start(task); progressbar.start();
task.run();
progressbar.shutdown();
verify(stream, atLeastOnce()).print(anyChar()); verify(stream, atLeastOnce()).print(anyChar());
verify(stream, atLeastOnce()).print(anyString()); verify(stream, atLeastOnce()).print(anyString());
} }
@Test
public void testWithSupplier() {
int maxValue = 10;
ProgressBar progressbar = new ProgressBar(stream, maxValue);
Supplier<Long> tasks = () -> {
IntStream.range(0, maxValue).forEach(
counter -> {
progressbar.incrementProgress();
}
);
return 1L; //return the result of the dummy task
};
progressbar.start(tasks);
verify(stream, atLeastOnce()).print(anyChar());
verify(stream, atLeastOnce()).print(anyString());
}
@Test
public void testWithFunction() {
int maxValue = 10;
Long result;
ProgressBar progressbar = new ProgressBar(stream, maxValue);
Function<Long, String> task = (Long l) -> {
IntStream.range(0, maxValue).forEach(
counter -> {
progressbar.incrementProgress();
}
);
return "dummy result"; //return the result of the dummy task
};
progressbar.start(1L, task);
verify(stream, atLeastOnce()).print(anyChar());
verify(stream, atLeastOnce()).print(anyString());
}
} }