From 399645ebc162371a63eb81840e01ed52261f43bc Mon Sep 17 00:00:00 2001 From: Xiao Chen Date: Mon, 29 Oct 2018 19:05:52 -0700 Subject: [PATCH] HDFS-14027. DFSStripedOutputStream should implement both hsync methods. (cherry picked from commit db7e636824a36b90ba1c8e9b2fba1162771700fe) Conflicts: hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java --- .../hadoop/hdfs/DFSStripedOutputStream.java | 12 +++++++ .../hdfs/TestDFSStripedOutputStream.java | 36 ++++++++++++------- 2 files changed, 35 insertions(+), 13 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java index ed875bbc153..df9770e38f3 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java @@ -24,6 +24,7 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.fs.CreateFlag; import org.apache.hadoop.fs.StreamCapabilities; import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys; +import org.apache.hadoop.hdfs.client.HdfsDataOutputStream.SyncFlag; import org.apache.hadoop.hdfs.protocol.ClientProtocol; import org.apache.hadoop.hdfs.protocol.DatanodeID; import org.apache.hadoop.hdfs.protocol.DatanodeInfo; @@ -956,11 +957,22 @@ public class DFSStripedOutputStream extends DFSOutputStream @Override public void hflush() { // not supported yet + LOG.debug("DFSStripedOutputStream does not support hflush. " + + "Caller should check StreamCapabilities before calling."); } @Override public void hsync() { // not supported yet + LOG.debug("DFSStripedOutputStream does not support hsync. " + + "Caller should check StreamCapabilities before calling."); + } + + @Override + public void hsync(EnumSet syncFlags) { + // not supported yet + LOG.debug("DFSStripedOutputStream does not support hsync {}. " + + "Caller should check StreamCapabilities before calling.", syncFlags); } @Override diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java index 3714542411d..473557bf275 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java @@ -18,11 +18,13 @@ package org.apache.hadoop.hdfs; import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; import java.io.ByteArrayInputStream; import java.io.IOException; import java.io.InputStream; import java.util.ArrayList; +import java.util.EnumSet; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -30,6 +32,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.StreamCapabilities.StreamCapability; +import org.apache.hadoop.hdfs.client.HdfsDataOutputStream.SyncFlag; import org.apache.hadoop.hdfs.protocol.DatanodeInfo; import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy; import org.apache.hadoop.io.IOUtils; @@ -195,19 +198,26 @@ public class TestDFSStripedOutputStream { public void testStreamFlush() throws Exception { final byte[] bytes = StripedFileTestUtil.generateBytes(blockSize * dataBlocks * 3 + cellSize * dataBlocks + cellSize + 123); - FSDataOutputStream os = fs.create(new Path("/ec-file-1")); - assertFalse("DFSStripedOutputStream should not have hflush() " + - "capability yet!", os.hasCapability( - StreamCapability.HFLUSH.getValue())); - assertFalse("DFSStripedOutputStream should not have hsync() " + - "capability yet!", os.hasCapability( - StreamCapability.HSYNC.getValue())); - InputStream is = new ByteArrayInputStream(bytes); - IOUtils.copyBytes(is, os, bytes.length); - os.hflush(); - IOUtils.copyBytes(is, os, bytes.length); - os.hsync(); - os.close(); + try (FSDataOutputStream os = fs.create(new Path("/ec-file-1"))) { + assertFalse( + "DFSStripedOutputStream should not have hflush() capability yet!", + os.hasCapability(StreamCapability.HFLUSH.getValue())); + assertFalse( + "DFSStripedOutputStream should not have hsync() capability yet!", + os.hasCapability(StreamCapability.HSYNC.getValue())); + try (InputStream is = new ByteArrayInputStream(bytes)) { + IOUtils.copyBytes(is, os, bytes.length); + os.hflush(); + IOUtils.copyBytes(is, os, bytes.length); + os.hsync(); + IOUtils.copyBytes(is, os, bytes.length); + } + assertTrue("stream is not a DFSStripedOutputStream", + os.getWrappedStream() instanceof DFSStripedOutputStream); + final DFSStripedOutputStream dfssos = + (DFSStripedOutputStream) os.getWrappedStream(); + dfssos.hsync(EnumSet.of(SyncFlag.UPDATE_LENGTH)); + } } private void testOneFile(String src, int writeBytes) throws Exception {