|
@@ -20,45 +20,40 @@ package org.apache.hadoop.hdfs;
|
|
import java.io.IOException;
|
|
import java.io.IOException;
|
|
import java.util.ArrayList;
|
|
import java.util.ArrayList;
|
|
import java.util.concurrent.CountDownLatch;
|
|
import java.util.concurrent.CountDownLatch;
|
|
|
|
+import java.util.concurrent.TimeUnit;
|
|
import java.util.concurrent.atomic.AtomicReference;
|
|
import java.util.concurrent.atomic.AtomicReference;
|
|
|
|
|
|
-import org.apache.commons.logging.LogFactory;
|
|
|
|
-import org.apache.commons.logging.impl.Log4JLogger;
|
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
|
+import org.apache.hadoop.conf.Configured;
|
|
import org.apache.hadoop.fs.CommonConfigurationKeys;
|
|
import org.apache.hadoop.fs.CommonConfigurationKeys;
|
|
import org.apache.hadoop.fs.FSDataOutputStream;
|
|
import org.apache.hadoop.fs.FSDataOutputStream;
|
|
import org.apache.hadoop.fs.FileSystem;
|
|
import org.apache.hadoop.fs.FileSystem;
|
|
import org.apache.hadoop.fs.Path;
|
|
import org.apache.hadoop.fs.Path;
|
|
-import org.apache.hadoop.hdfs.server.datanode.DataNode;
|
|
|
|
-import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
|
|
|
|
-import org.apache.hadoop.hdfs.server.namenode.LeaseManager;
|
|
|
|
-import org.apache.hadoop.hdfs.server.namenode.NameNode;
|
|
|
|
-import org.apache.hadoop.hdfs.server.protocol.InterDatanodeProtocol;
|
|
|
|
-import org.apache.log4j.Level;
|
|
|
|
|
|
+import org.apache.hadoop.metrics2.util.Quantile;
|
|
|
|
+import org.apache.hadoop.metrics2.util.SampleQuantiles;
|
|
|
|
+import org.apache.hadoop.util.Tool;
|
|
|
|
+import org.apache.hadoop.util.ToolRunner;
|
|
import org.junit.Test;
|
|
import org.junit.Test;
|
|
|
|
|
|
|
|
+import com.google.common.base.Stopwatch;
|
|
|
|
+
|
|
/**
|
|
/**
|
|
* This class tests hflushing concurrently from many threads.
|
|
* This class tests hflushing concurrently from many threads.
|
|
*/
|
|
*/
|
|
public class TestMultiThreadedHflush {
|
|
public class TestMultiThreadedHflush {
|
|
static final int blockSize = 1024*1024;
|
|
static final int blockSize = 1024*1024;
|
|
- static final int numBlocks = 10;
|
|
|
|
- static final int fileSize = numBlocks * blockSize + 1;
|
|
|
|
|
|
|
|
private static final int NUM_THREADS = 10;
|
|
private static final int NUM_THREADS = 10;
|
|
private static final int WRITE_SIZE = 517;
|
|
private static final int WRITE_SIZE = 517;
|
|
private static final int NUM_WRITES_PER_THREAD = 1000;
|
|
private static final int NUM_WRITES_PER_THREAD = 1000;
|
|
|
|
|
|
private byte[] toWrite = null;
|
|
private byte[] toWrite = null;
|
|
-
|
|
|
|
- {
|
|
|
|
- ((Log4JLogger)NameNode.stateChangeLog).getLogger().setLevel(Level.ALL);
|
|
|
|
- ((Log4JLogger)LeaseManager.LOG).getLogger().setLevel(Level.ALL);
|
|
|
|
- ((Log4JLogger)LogFactory.getLog(FSNamesystem.class)).getLogger().setLevel(Level.ALL);
|
|
|
|
- ((Log4JLogger)DataNode.LOG).getLogger().setLevel(Level.ALL);
|
|
|
|
- ((Log4JLogger)DFSClient.LOG).getLogger().setLevel(Level.ALL);
|
|
|
|
- ((Log4JLogger)InterDatanodeProtocol.LOG).getLogger().setLevel(Level.ALL);
|
|
|
|
- }
|
|
|
|
|
|
+
|
|
|
|
+ private final SampleQuantiles quantiles = new SampleQuantiles(
|
|
|
|
+ new Quantile[] {
|
|
|
|
+ new Quantile(0.50, 0.050),
|
|
|
|
+ new Quantile(0.75, 0.025), new Quantile(0.90, 0.010),
|
|
|
|
+ new Quantile(0.95, 0.005), new Quantile(0.99, 0.001) });
|
|
|
|
|
|
/*
|
|
/*
|
|
* creates a file but does not close it
|
|
* creates a file but does not close it
|
|
@@ -104,8 +99,11 @@ public class TestMultiThreadedHflush {
|
|
}
|
|
}
|
|
|
|
|
|
private void doAWrite() throws IOException {
|
|
private void doAWrite() throws IOException {
|
|
|
|
+ Stopwatch sw = new Stopwatch().start();
|
|
stm.write(toWrite);
|
|
stm.write(toWrite);
|
|
stm.hflush();
|
|
stm.hflush();
|
|
|
|
+ long micros = sw.elapsedTime(TimeUnit.MICROSECONDS);
|
|
|
|
+ quantiles.insert(micros);
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
@@ -115,14 +113,28 @@ public class TestMultiThreadedHflush {
|
|
* They all finish before the file is closed.
|
|
* They all finish before the file is closed.
|
|
*/
|
|
*/
|
|
@Test
|
|
@Test
|
|
- public void testMultipleHflushers() throws Exception {
|
|
|
|
|
|
+ public void testMultipleHflushersRepl1() throws Exception {
|
|
|
|
+ doTestMultipleHflushers(1);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ @Test
|
|
|
|
+ public void testMultipleHflushersRepl3() throws Exception {
|
|
|
|
+ doTestMultipleHflushers(3);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ private void doTestMultipleHflushers(int repl) throws Exception {
|
|
Configuration conf = new Configuration();
|
|
Configuration conf = new Configuration();
|
|
- MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).build();
|
|
|
|
|
|
+ MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
|
|
|
|
+ .numDataNodes(repl)
|
|
|
|
+ .build();
|
|
|
|
|
|
FileSystem fs = cluster.getFileSystem();
|
|
FileSystem fs = cluster.getFileSystem();
|
|
Path p = new Path("/multiple-hflushers.dat");
|
|
Path p = new Path("/multiple-hflushers.dat");
|
|
try {
|
|
try {
|
|
- doMultithreadedWrites(conf, p, NUM_THREADS, WRITE_SIZE, NUM_WRITES_PER_THREAD);
|
|
|
|
|
|
+ doMultithreadedWrites(conf, p, NUM_THREADS, WRITE_SIZE,
|
|
|
|
+ NUM_WRITES_PER_THREAD, repl);
|
|
|
|
+ System.out.println("Latency quantiles (in microseconds):\n" +
|
|
|
|
+ quantiles);
|
|
} finally {
|
|
} finally {
|
|
fs.close();
|
|
fs.close();
|
|
cluster.shutdown();
|
|
cluster.shutdown();
|
|
@@ -200,13 +212,13 @@ public class TestMultiThreadedHflush {
|
|
}
|
|
}
|
|
|
|
|
|
public void doMultithreadedWrites(
|
|
public void doMultithreadedWrites(
|
|
- Configuration conf, Path p, int numThreads, int bufferSize, int numWrites)
|
|
|
|
- throws Exception {
|
|
|
|
|
|
+ Configuration conf, Path p, int numThreads, int bufferSize, int numWrites,
|
|
|
|
+ int replication) throws Exception {
|
|
initBuffer(bufferSize);
|
|
initBuffer(bufferSize);
|
|
|
|
|
|
// create a new file.
|
|
// create a new file.
|
|
FileSystem fs = p.getFileSystem(conf);
|
|
FileSystem fs = p.getFileSystem(conf);
|
|
- FSDataOutputStream stm = createFile(fs, p, 1);
|
|
|
|
|
|
+ FSDataOutputStream stm = createFile(fs, p, replication);
|
|
System.out.println("Created file simpleFlush.dat");
|
|
System.out.println("Created file simpleFlush.dat");
|
|
|
|
|
|
// There have been a couple issues with flushing empty buffers, so do
|
|
// There have been a couple issues with flushing empty buffers, so do
|
|
@@ -240,20 +252,41 @@ public class TestMultiThreadedHflush {
|
|
}
|
|
}
|
|
|
|
|
|
public static void main(String args[]) throws Exception {
|
|
public static void main(String args[]) throws Exception {
|
|
- if (args.length != 1) {
|
|
|
|
- System.err.println(
|
|
|
|
- "usage: " + TestMultiThreadedHflush.class.getSimpleName() +
|
|
|
|
- " <path to test file> ");
|
|
|
|
- System.exit(1);
|
|
|
|
|
|
+ System.exit(ToolRunner.run(new CLIBenchmark(), args));
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ private static class CLIBenchmark extends Configured implements Tool {
|
|
|
|
+ public int run(String args[]) throws Exception {
|
|
|
|
+ if (args.length != 1) {
|
|
|
|
+ System.err.println(
|
|
|
|
+ "usage: " + TestMultiThreadedHflush.class.getSimpleName() +
|
|
|
|
+ " <path to test file> ");
|
|
|
|
+ System.err.println(
|
|
|
|
+ "Configurations settable by -D options:\n" +
|
|
|
|
+ " num.threads [default 10] - how many threads to run\n" +
|
|
|
|
+ " write.size [default 511] - bytes per write\n" +
|
|
|
|
+ " num.writes [default 50000] - how many writes to perform");
|
|
|
|
+ System.exit(1);
|
|
|
|
+ }
|
|
|
|
+ TestMultiThreadedHflush test = new TestMultiThreadedHflush();
|
|
|
|
+ Configuration conf = getConf();
|
|
|
|
+ Path p = new Path(args[0]);
|
|
|
|
+
|
|
|
|
+ int numThreads = conf.getInt("num.threads", 10);
|
|
|
|
+ int writeSize = conf.getInt("write.size", 511);
|
|
|
|
+ int numWrites = conf.getInt("num.writes", 50000);
|
|
|
|
+ int replication = conf.getInt(DFSConfigKeys.DFS_REPLICATION_KEY,
|
|
|
|
+ DFSConfigKeys.DFS_REPLICATION_DEFAULT);
|
|
|
|
+
|
|
|
|
+ Stopwatch sw = new Stopwatch().start();
|
|
|
|
+ test.doMultithreadedWrites(conf, p, numThreads, writeSize, numWrites,
|
|
|
|
+ replication);
|
|
|
|
+ sw.stop();
|
|
|
|
+
|
|
|
|
+ System.out.println("Finished in " + sw.elapsedMillis() + "ms");
|
|
|
|
+ System.out.println("Latency quantiles (in microseconds):\n" +
|
|
|
|
+ test.quantiles);
|
|
|
|
+ return 0;
|
|
}
|
|
}
|
|
- TestMultiThreadedHflush test = new TestMultiThreadedHflush();
|
|
|
|
- Configuration conf = new Configuration();
|
|
|
|
- Path p = new Path(args[0]);
|
|
|
|
- long st = System.nanoTime();
|
|
|
|
- test.doMultithreadedWrites(conf, p, 10, 511, 50000);
|
|
|
|
- long et = System.nanoTime();
|
|
|
|
-
|
|
|
|
- System.out.println("Finished in " + ((et - st) / 1000000) + "ms");
|
|
|
|
}
|
|
}
|
|
-
|
|
|
|
}
|
|
}
|