hadoop-common-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From bra...@apache.org
Subject [49/50] [abbrv] hadoop git commit: HDFS-14027. DFSStripedOutputStream should implement both hsync methods.
Date Tue, 30 Oct 2018 06:03:19 GMT
HDFS-14027. DFSStripedOutputStream should implement both hsync methods.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/4f89eb33
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/4f89eb33
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/4f89eb33

Branch: refs/heads/HDFS-13891
Commit: 4f89eb33af445a374cbb6120fd78c765d61a560c
Parents: f541d83
Author: Xiao Chen <xiao@apache.org>
Authored: Mon Oct 29 19:05:52 2018 -0700
Committer: Brahma Reddy Battula <brahma@apache.org>
Committed: Tue Oct 30 11:31:17 2018 +0530

----------------------------------------------------------------------
 .../hadoop/hdfs/DFSStripedOutputStream.java     | 12 +++++++
 .../hadoop/hdfs/TestDFSStripedOutputStream.java | 36 +++++++++++++-------
 2 files changed, 35 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/4f89eb33/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java
----------------------------------------------------------------------
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 ed875bb..df9770e 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<SyncFlag> syncFlags) {
+    // not supported yet
+    LOG.debug("DFSStripedOutputStream does not support hsync {}. "
+        + "Caller should check StreamCapabilities before calling.", syncFlags);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4f89eb33/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java
----------------------------------------------------------------------
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 865a736..092aa0a 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,12 +18,14 @@
 package org.apache.hadoop.hdfs;
 
 import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
 import java.io.ByteArrayInputStream;
 import java.io.IOException;
 import java.io.InputStream;
 import java.util.ArrayList;
+import java.util.EnumSet;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -31,6 +33,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;
@@ -196,19 +199,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 {


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


Mime
View raw message