|
@@ -26,8 +26,10 @@ import static org.junit.Assert.*;
|
|
|
import java.io.Closeable;
|
|
|
import java.io.IOException;
|
|
|
import java.lang.management.ManagementFactory;
|
|
|
+import java.util.concurrent.atomic.AtomicInteger;
|
|
|
import java.util.List;
|
|
|
|
|
|
+import com.google.common.base.Supplier;
|
|
|
import com.google.common.collect.Lists;
|
|
|
|
|
|
import org.apache.commons.logging.Log;
|
|
@@ -47,6 +49,7 @@ import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
|
|
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
|
|
|
import org.apache.hadoop.io.IOUtils;
|
|
|
import org.apache.hadoop.metrics2.MetricsRecordBuilder;
|
|
|
+import org.apache.hadoop.test.GenericTestUtils;
|
|
|
import org.apache.hadoop.util.Time;
|
|
|
import org.junit.Test;
|
|
|
import org.mockito.Mockito;
|
|
@@ -255,38 +258,43 @@ public class TestDataNodeMetrics {
|
|
|
* and reading causes totalReadTime to move.
|
|
|
* @throws Exception
|
|
|
*/
|
|
|
- @Test
|
|
|
+ @Test(timeout=60000)
|
|
|
public void testDataNodeTimeSpend() throws Exception {
|
|
|
Configuration conf = new HdfsConfiguration();
|
|
|
SimulatedFSDataset.setFactory(conf);
|
|
|
MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).build();
|
|
|
try {
|
|
|
- FileSystem fs = cluster.getFileSystem();
|
|
|
+ final FileSystem fs = cluster.getFileSystem();
|
|
|
List<DataNode> datanodes = cluster.getDataNodes();
|
|
|
assertEquals(datanodes.size(), 1);
|
|
|
- DataNode datanode = datanodes.get(0);
|
|
|
+ final DataNode datanode = datanodes.get(0);
|
|
|
MetricsRecordBuilder rb = getMetrics(datanode.getMetrics().name());
|
|
|
final long LONG_FILE_LEN = 1024 * 1024 * 10;
|
|
|
|
|
|
- long startWriteValue = getLongCounter("TotalWriteTime", rb);
|
|
|
- long startReadValue = getLongCounter("TotalReadTime", rb);
|
|
|
-
|
|
|
- for (int x =0; x < 50; x++) {
|
|
|
- DFSTestUtil.createFile(fs, new Path("/time.txt."+ x),
|
|
|
- LONG_FILE_LEN, (short) 1, Time.monotonicNow());
|
|
|
- }
|
|
|
-
|
|
|
- for (int x =0; x < 50; x++) {
|
|
|
- DFSTestUtil.readFile(fs, new Path("/time.txt." + x));
|
|
|
- }
|
|
|
+ final long startWriteValue = getLongCounter("TotalWriteTime", rb);
|
|
|
+ final long startReadValue = getLongCounter("TotalReadTime", rb);
|
|
|
+ final AtomicInteger x = new AtomicInteger(0);
|
|
|
|
|
|
- MetricsRecordBuilder rbNew = getMetrics(datanode.getMetrics().name());
|
|
|
- long endWriteValue = getLongCounter("TotalWriteTime", rbNew);
|
|
|
- long endReadValue = getLongCounter("TotalReadTime", rbNew);
|
|
|
// Lets Metric system update latest metrics
|
|
|
- Thread.sleep(100);
|
|
|
- assertTrue(endReadValue > startReadValue);
|
|
|
- assertTrue(endWriteValue > startWriteValue);
|
|
|
+ GenericTestUtils.waitFor(new Supplier<Boolean>() {
|
|
|
+ @Override
|
|
|
+ public Boolean get() {
|
|
|
+ x.getAndIncrement();
|
|
|
+ try {
|
|
|
+ DFSTestUtil.createFile(fs, new Path("/time.txt." + x.get()),
|
|
|
+ LONG_FILE_LEN, (short) 1, Time.monotonicNow());
|
|
|
+ DFSTestUtil.readFile(fs, new Path("/time.txt." + x.get()));
|
|
|
+ } catch (IOException ioe) {
|
|
|
+ LOG.error("Caught IOException while ingesting DN metrics", ioe);
|
|
|
+ return false;
|
|
|
+ }
|
|
|
+ MetricsRecordBuilder rbNew = getMetrics(datanode.getMetrics().name());
|
|
|
+ final long endWriteValue = getLongCounter("TotalWriteTime", rbNew);
|
|
|
+ final long endReadValue = getLongCounter("TotalReadTime", rbNew);
|
|
|
+ return endWriteValue > startWriteValue
|
|
|
+ && endReadValue > startReadValue;
|
|
|
+ }
|
|
|
+ }, 30, 30000);
|
|
|
} finally {
|
|
|
if (cluster != null) {
|
|
|
cluster.shutdown();
|