|
@@ -552,6 +552,21 @@ public class DiskBalancer {
|
|
|
* @return FsDatasetSpi
|
|
|
*/
|
|
|
FsDatasetSpi getDataset();
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Returns time when this plan started executing.
|
|
|
+ *
|
|
|
+ * @return Start time in milliseconds.
|
|
|
+ */
|
|
|
+ long getStartTime();
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Number of seconds elapsed.
|
|
|
+ *
|
|
|
+ * @return time in seconds
|
|
|
+ */
|
|
|
+ long getElapsedSeconds();
|
|
|
+
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -622,6 +637,8 @@ public class DiskBalancer {
|
|
|
private long maxDiskErrors;
|
|
|
private int poolIndex;
|
|
|
private AtomicBoolean shouldRun;
|
|
|
+ private long startTime;
|
|
|
+ private long secondsElapsed;
|
|
|
|
|
|
/**
|
|
|
* Constructs diskBalancerMover.
|
|
@@ -897,6 +914,9 @@ public class DiskBalancer {
|
|
|
FsVolumeSpi source = pair.getSource();
|
|
|
FsVolumeSpi dest = pair.getDest();
|
|
|
List<FsVolumeSpi.BlockIterator> poolIters = new LinkedList<>();
|
|
|
+ startTime = Time.now();
|
|
|
+ item.setStartTime(startTime);
|
|
|
+ secondsElapsed = 0;
|
|
|
|
|
|
if (source.isTransientStorage() || dest.isTransientStorage()) {
|
|
|
return;
|
|
@@ -937,7 +957,7 @@ public class DiskBalancer {
|
|
|
if (block == null) {
|
|
|
this.setExitFlag();
|
|
|
LOG.error("No source blocks, exiting the copy. Source: {}, " +
|
|
|
- "dest:{}", source.getBasePath(), dest.getBasePath());
|
|
|
+ "dest:{}", source.getBasePath(), dest.getBasePath());
|
|
|
continue;
|
|
|
}
|
|
|
|
|
@@ -973,9 +993,6 @@ public class DiskBalancer {
|
|
|
block.getNumBytes(), source.getBasePath(),
|
|
|
dest.getBasePath());
|
|
|
|
|
|
- item.incCopiedSoFar(block.getNumBytes());
|
|
|
- item.incBlocksCopied();
|
|
|
-
|
|
|
// Check for the max throughput constraint.
|
|
|
// We sleep here to keep the promise that we will not
|
|
|
// copy more than Max MB/sec. we sleep enough time
|
|
@@ -984,6 +1001,14 @@ public class DiskBalancer {
|
|
|
// we exit via Thread Interrupted exception.
|
|
|
Thread.sleep(computeDelay(block.getNumBytes(), timeUsed, item));
|
|
|
|
|
|
+ // We delay updating the info to avoid confusing the user.
|
|
|
+ // This way we report the copy only if it is under the
|
|
|
+ // throughput threshold.
|
|
|
+ item.incCopiedSoFar(block.getNumBytes());
|
|
|
+ item.incBlocksCopied();
|
|
|
+ secondsElapsed = TimeUnit.MILLISECONDS.toSeconds(Time.now() -
|
|
|
+ startTime);
|
|
|
+ item.setSecondsElapsed(secondsElapsed);
|
|
|
} catch (IOException ex) {
|
|
|
LOG.error("Exception while trying to copy blocks. error: {}", ex);
|
|
|
item.incErrorCount();
|
|
@@ -1009,5 +1034,25 @@ public class DiskBalancer {
|
|
|
public FsDatasetSpi getDataset() {
|
|
|
return dataset;
|
|
|
}
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Returns time when this plan started executing.
|
|
|
+ *
|
|
|
+ * @return Start time in milliseconds.
|
|
|
+ */
|
|
|
+ @Override
|
|
|
+ public long getStartTime() {
|
|
|
+ return startTime;
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Number of seconds elapsed.
|
|
|
+ *
|
|
|
+ * @return time in seconds
|
|
|
+ */
|
|
|
+ @Override
|
|
|
+ public long getElapsedSeconds() {
|
|
|
+ return secondsElapsed;
|
|
|
+ }
|
|
|
}
|
|
|
}
|