|
@@ -23,12 +23,14 @@ import static org.junit.Assert.assertTrue;
|
|
|
|
|
|
import java.io.IOException;
|
|
|
import java.io.InterruptedIOException;
|
|
|
+import java.util.EnumSet;
|
|
|
|
|
|
import org.apache.commons.logging.impl.Log4JLogger;
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.fs.FSDataInputStream;
|
|
|
import org.apache.hadoop.fs.FSDataOutputStream;
|
|
|
import org.apache.hadoop.fs.Path;
|
|
|
+import org.apache.hadoop.hdfs.client.HdfsDataOutputStream.SyncFlag;
|
|
|
import org.apache.hadoop.hdfs.server.datanode.DataNode;
|
|
|
import org.apache.log4j.Level;
|
|
|
import org.junit.Test;
|
|
@@ -43,16 +45,21 @@ public class TestHFlush {
|
|
|
|
|
|
private final String fName = "hflushtest.dat";
|
|
|
|
|
|
- /** The test uses {@link #doTheJob(Configuration, String, long, short)
|
|
|
+ /**
|
|
|
+ * The test uses
|
|
|
+ * {@link #doTheJob(Configuration, String, long, short, boolean, EnumSet)}
|
|
|
* to write a file with a standard block size
|
|
|
*/
|
|
|
@Test
|
|
|
public void hFlush_01() throws IOException {
|
|
|
- doTheJob(new HdfsConfiguration(), fName, AppendTestUtil.BLOCK_SIZE, (short)2);
|
|
|
+ doTheJob(new HdfsConfiguration(), fName, AppendTestUtil.BLOCK_SIZE,
|
|
|
+ (short) 2, false, EnumSet.noneOf(SyncFlag.class));
|
|
|
}
|
|
|
|
|
|
- /** The test uses {@link #doTheJob(Configuration, String, long, short)
|
|
|
- * to write a file with a custom block size so the writes will be
|
|
|
+ /**
|
|
|
+ * The test uses
|
|
|
+ * {@link #doTheJob(Configuration, String, long, short, boolean, EnumSet)}
|
|
|
+ * to write a file with a custom block size so the writes will be
|
|
|
* happening across block' boundaries
|
|
|
*/
|
|
|
@Test
|
|
@@ -64,14 +71,17 @@ public class TestHFlush {
|
|
|
conf.setInt(DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY, customPerChecksumSize);
|
|
|
conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, customBlockSize);
|
|
|
|
|
|
- doTheJob(conf, fName, customBlockSize, (short)2);
|
|
|
+ doTheJob(conf, fName, customBlockSize, (short) 2, false,
|
|
|
+ EnumSet.noneOf(SyncFlag.class));
|
|
|
}
|
|
|
|
|
|
- /** The test uses {@link #doTheJob(Configuration, String, long, short)
|
|
|
- * to write a file with a custom block size so the writes will be
|
|
|
+ /**
|
|
|
+ * The test uses
|
|
|
+ * {@link #doTheJob(Configuration, String, long, short, boolean, EnumSet)}
|
|
|
+ * to write a file with a custom block size so the writes will be
|
|
|
* happening across block's and checksum' boundaries
|
|
|
*/
|
|
|
- @Test
|
|
|
+ @Test
|
|
|
public void hFlush_03() throws IOException {
|
|
|
Configuration conf = new HdfsConfiguration();
|
|
|
int customPerChecksumSize = 400;
|
|
@@ -80,22 +90,106 @@ public class TestHFlush {
|
|
|
conf.setInt(DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY, customPerChecksumSize);
|
|
|
conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, customBlockSize);
|
|
|
|
|
|
- doTheJob(conf, fName, customBlockSize, (short)2);
|
|
|
+ doTheJob(conf, fName, customBlockSize, (short) 2, false,
|
|
|
+ EnumSet.noneOf(SyncFlag.class));
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Test hsync (with updating block length in NameNode) while no data is
|
|
|
+ * actually written yet
|
|
|
+ */
|
|
|
+ @Test
|
|
|
+ public void hSyncUpdateLength_00() throws IOException {
|
|
|
+ Configuration conf = new HdfsConfiguration();
|
|
|
+ MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(
|
|
|
+ 2).build();
|
|
|
+ DistributedFileSystem fileSystem =
|
|
|
+ (DistributedFileSystem)cluster.getFileSystem();
|
|
|
+
|
|
|
+ try {
|
|
|
+ Path path = new Path(fName);
|
|
|
+ FSDataOutputStream stm = fileSystem.create(path, true, 4096, (short) 2,
|
|
|
+ AppendTestUtil.BLOCK_SIZE);
|
|
|
+ System.out.println("Created file " + path.toString());
|
|
|
+ ((DFSOutputStream) stm.getWrappedStream()).hsync(EnumSet
|
|
|
+ .of(SyncFlag.UPDATE_LENGTH));
|
|
|
+ long currentFileLength = fileSystem.getFileStatus(path).getLen();
|
|
|
+ assertEquals(0L, currentFileLength);
|
|
|
+ stm.close();
|
|
|
+ } finally {
|
|
|
+ fileSystem.close();
|
|
|
+ cluster.shutdown();
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * The test calls
|
|
|
+ * {@link #doTheJob(Configuration, String, long, short, boolean, EnumSet)}
|
|
|
+ * while requiring the semantic of {@link SyncFlag#UPDATE_LENGTH}.
|
|
|
+ */
|
|
|
+ @Test
|
|
|
+ public void hSyncUpdateLength_01() throws IOException {
|
|
|
+ doTheJob(new HdfsConfiguration(), fName, AppendTestUtil.BLOCK_SIZE,
|
|
|
+ (short) 2, true, EnumSet.of(SyncFlag.UPDATE_LENGTH));
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
- The method starts new cluster with defined Configuration;
|
|
|
- creates a file with specified block_size and writes 10 equal sections in it;
|
|
|
- it also calls hflush() after each write and throws an IOException in case of
|
|
|
- an error.
|
|
|
- @param conf cluster configuration
|
|
|
- @param fileName of the file to be created and processed as required
|
|
|
- @param block_size value to be used for the file's creation
|
|
|
- @param replicas is the number of replicas
|
|
|
- @throws IOException in case of any errors
|
|
|
+ * The test calls
|
|
|
+ * {@link #doTheJob(Configuration, String, long, short, boolean, EnumSet)}
|
|
|
+ * while requiring the semantic of {@link SyncFlag#UPDATE_LENGTH}.
|
|
|
+ * Similar with {@link #hFlush_02()} , it writes a file with a custom block
|
|
|
+ * size so the writes will be happening across block' boundaries
|
|
|
+ */
|
|
|
+ @Test
|
|
|
+ public void hSyncUpdateLength_02() throws IOException {
|
|
|
+ Configuration conf = new HdfsConfiguration();
|
|
|
+ int customPerChecksumSize = 512;
|
|
|
+ int customBlockSize = customPerChecksumSize * 3;
|
|
|
+ // Modify defaul filesystem settings
|
|
|
+ conf.setInt(DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY, customPerChecksumSize);
|
|
|
+ conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, customBlockSize);
|
|
|
+
|
|
|
+ doTheJob(conf, fName, customBlockSize, (short) 2, true,
|
|
|
+ EnumSet.of(SyncFlag.UPDATE_LENGTH));
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * The test calls
|
|
|
+ * {@link #doTheJob(Configuration, String, long, short, boolean, EnumSet)}
|
|
|
+ * while requiring the semantic of {@link SyncFlag#UPDATE_LENGTH}.
|
|
|
+ * Similar with {@link #hFlush_03()} , it writes a file with a custom block
|
|
|
+ * size so the writes will be happening across block's and checksum'
|
|
|
+ * boundaries.
|
|
|
+ */
|
|
|
+ @Test
|
|
|
+ public void hSyncUpdateLength_03() throws IOException {
|
|
|
+ Configuration conf = new HdfsConfiguration();
|
|
|
+ int customPerChecksumSize = 400;
|
|
|
+ int customBlockSize = customPerChecksumSize * 3;
|
|
|
+ // Modify defaul filesystem settings
|
|
|
+ conf.setInt(DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY, customPerChecksumSize);
|
|
|
+ conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, customBlockSize);
|
|
|
+
|
|
|
+ doTheJob(conf, fName, customBlockSize, (short) 2, true,
|
|
|
+ EnumSet.of(SyncFlag.UPDATE_LENGTH));
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * The method starts new cluster with defined Configuration; creates a file
|
|
|
+ * with specified block_size and writes 10 equal sections in it; it also calls
|
|
|
+ * hflush/hsync after each write and throws an IOException in case of an error.
|
|
|
+ *
|
|
|
+ * @param conf cluster configuration
|
|
|
+ * @param fileName of the file to be created and processed as required
|
|
|
+ * @param block_size value to be used for the file's creation
|
|
|
+ * @param replicas is the number of replicas
|
|
|
+ * @param isSync hsync or hflush
|
|
|
+ * @param syncFlags specify the semantic of the sync/flush
|
|
|
+ * @throws IOException in case of any errors
|
|
|
*/
|
|
|
public static void doTheJob(Configuration conf, final String fileName,
|
|
|
- long block_size, short replicas) throws IOException {
|
|
|
+ long block_size, short replicas, boolean isSync,
|
|
|
+ EnumSet<SyncFlag> syncFlags) throws IOException {
|
|
|
byte[] fileContent;
|
|
|
final int SECTIONS = 10;
|
|
|
|
|
@@ -119,8 +213,21 @@ public class TestHFlush {
|
|
|
System.out.println("Writing " + (tenth * i) + " to " + (tenth * (i+1)) + " section to file " + fileName);
|
|
|
// write to the file
|
|
|
stm.write(fileContent, tenth * i, tenth);
|
|
|
- // Wait while hflush() pushes all packets through built pipeline
|
|
|
- ((DFSOutputStream)stm.getWrappedStream()).hflush();
|
|
|
+
|
|
|
+ // Wait while hflush/hsync pushes all packets through built pipeline
|
|
|
+ if (isSync) {
|
|
|
+ ((DFSOutputStream)stm.getWrappedStream()).hsync(syncFlags);
|
|
|
+ } else {
|
|
|
+ ((DFSOutputStream)stm.getWrappedStream()).hflush();
|
|
|
+ }
|
|
|
+
|
|
|
+ // Check file length if updatelength is required
|
|
|
+ if (isSync && syncFlags.contains(SyncFlag.UPDATE_LENGTH)) {
|
|
|
+ long currentFileLength = fileSystem.getFileStatus(path).getLen();
|
|
|
+ assertEquals(
|
|
|
+ "File size doesn't match for hsync/hflush with updating the length",
|
|
|
+ tenth * (i + 1), currentFileLength);
|
|
|
+ }
|
|
|
byte [] toRead = new byte[tenth];
|
|
|
byte [] expected = new byte[tenth];
|
|
|
System.arraycopy(fileContent, tenth * i, expected, 0, tenth);
|
|
@@ -139,8 +246,6 @@ public class TestHFlush {
|
|
|
|
|
|
assertEquals("File size doesn't match ", AppendTestUtil.FILE_SIZE, fileSystem.getFileStatus(path).getLen());
|
|
|
AppendTestUtil.checkFullFile(fileSystem, path, fileContent.length, fileContent, "hflush()");
|
|
|
- } catch (Exception e) {
|
|
|
- e.printStackTrace();
|
|
|
} finally {
|
|
|
fileSystem.close();
|
|
|
cluster.shutdown();
|