From common-commits-return-90310-archive-asf-public=cust-asf.ponee.io@hadoop.apache.org Mon Nov 5 21:42:58 2018 Return-Path: X-Original-To: archive-asf-public@cust-asf.ponee.io Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by mx-eu-01.ponee.io (Postfix) with SMTP id 424A4180670 for ; Mon, 5 Nov 2018 21:42:58 +0100 (CET) Received: (qmail 16511 invoked by uid 500); 5 Nov 2018 20:42:57 -0000 Mailing-List: contact common-commits-help@hadoop.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Delivered-To: mailing list common-commits@hadoop.apache.org Received: (qmail 16493 invoked by uid 99); 5 Nov 2018 20:42:57 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Mon, 05 Nov 2018 20:42:57 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 2E048DFF66; Mon, 5 Nov 2018 20:42:57 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: aengineer@apache.org To: common-commits@hadoop.apache.org Date: Mon, 05 Nov 2018 20:42:57 -0000 Message-Id: <8e590f61404a407d812f6fee0a37e693@git.apache.org> X-Mailer: ASF-Git Admin Mailer Subject: [01/50] [abbrv] hadoop git commit: HDFS-14027. DFSStripedOutputStream should implement both hsync methods. Repository: hadoop Updated Branches: refs/heads/HDDS-4 2115256ab -> 7119be30b 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/db7e6368 Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/db7e6368 Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/db7e6368 Branch: refs/heads/HDDS-4 Commit: db7e636824a36b90ba1c8e9b2fba1162771700fe Parents: 496f0ff Author: Xiao Chen Authored: Mon Oct 29 19:05:52 2018 -0700 Committer: Xiao Chen Committed: Mon Oct 29 19:06:15 2018 -0700 ---------------------------------------------------------------------- .../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/db7e6368/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 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/db7e6368/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