Return-Path: X-Original-To: archive-asf-public-internal@cust-asf2.ponee.io Delivered-To: archive-asf-public-internal@cust-asf2.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by cust-asf2.ponee.io (Postfix) with ESMTP id 1C243200C83 for ; Sun, 14 May 2017 06:41:18 +0200 (CEST) Received: by cust-asf.ponee.io (Postfix) id 1A8E0160BC6; Sun, 14 May 2017 04:41:18 +0000 (UTC) Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id 12F78160BBB for ; Sun, 14 May 2017 06:41:16 +0200 (CEST) Received: (qmail 33918 invoked by uid 500); 14 May 2017 04:41:16 -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 33909 invoked by uid 99); 14 May 2017 04:41:15 -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; Sun, 14 May 2017 04:41:15 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id D64E0DFBC8; Sun, 14 May 2017 04:41:15 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: wang@apache.org To: common-commits@hadoop.apache.org Message-Id: <9c336da91b464802bc03613d5145f322@git.apache.org> X-Mailer: ASF-Git Admin Mailer Subject: hadoop git commit: HDFS-11644. Support for querying outputstream capabilities. Contributed by Manoj Govindassamy. Date: Sun, 14 May 2017 04:41:15 +0000 (UTC) archived-at: Sun, 14 May 2017 04:41:18 -0000 Repository: hadoop Updated Branches: refs/heads/branch-2 aeca471ec -> 18b89e902 HDFS-11644. Support for querying outputstream capabilities. Contributed by Manoj Govindassamy. Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/18b89e90 Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/18b89e90 Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/18b89e90 Branch: refs/heads/branch-2 Commit: 18b89e902bb9cb55b002b35d227e993ddb40c4b5 Parents: aeca471 Author: Andrew Wang Authored: Sat May 13 21:41:11 2017 -0700 Committer: Andrew Wang Committed: Sat May 13 21:41:11 2017 -0700 ---------------------------------------------------------------------- .../apache/hadoop/fs/FSDataOutputStream.java | 10 ++- .../apache/hadoop/fs/StreamCapabilities.java | 67 ++++++++++++++++++++ .../org/apache/hadoop/hdfs/DFSOutputStream.java | 15 ++++- .../apache/hadoop/hdfs/TestDFSOutputStream.java | 25 ++++++++ .../fs/azure/SyncableDataOutputStream.java | 12 +++- 5 files changed, 126 insertions(+), 3 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/hadoop/blob/18b89e90/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSDataOutputStream.java ---------------------------------------------------------------------- diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSDataOutputStream.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSDataOutputStream.java index 9a59b1d..cbf0460 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSDataOutputStream.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSDataOutputStream.java @@ -31,7 +31,7 @@ import org.apache.hadoop.classification.InterfaceStability; @InterfaceAudience.Public @InterfaceStability.Stable public class FSDataOutputStream extends DataOutputStream - implements Syncable, CanSetDropBehind { + implements Syncable, CanSetDropBehind, StreamCapabilities { private final OutputStream wrappedStream; private static class PositionCache extends FilterOutputStream { @@ -116,6 +116,14 @@ public class FSDataOutputStream extends DataOutputStream return wrappedStream; } + @Override + public boolean hasCapability(String capability) { + if (wrappedStream instanceof StreamCapabilities) { + return ((StreamCapabilities) wrappedStream).hasCapability(capability); + } + return false; + } + @Override // Syncable @Deprecated public void sync() throws IOException { http://git-wip-us.apache.org/repos/asf/hadoop/blob/18b89e90/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/StreamCapabilities.java ---------------------------------------------------------------------- diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/StreamCapabilities.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/StreamCapabilities.java new file mode 100644 index 0000000..65aa679 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/StreamCapabilities.java @@ -0,0 +1,67 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; + +/** + * Interface to query streams for supported capabilities. + */ +@InterfaceAudience.Public +@InterfaceStability.Evolving +public interface StreamCapabilities { + /** + * Capabilities that a stream can support and be queried for. + */ + enum StreamCapability { + /** + * Stream hflush capability to flush out the data in client's buffer. + * Streams with this capability implement {@link Syncable} and support + * {@link Syncable#hflush()}. + */ + HFLUSH("hflush"), + + /** + * Stream hsync capability to flush out the data in client's buffer and + * the disk device. Streams with this capability implement {@link Syncable} + * and support {@link Syncable#hsync()}. + */ + HSYNC("hsync"); + + private final String capability; + + StreamCapability(String value) { + this.capability = value; + } + + public final String getValue() { + return capability; + } + } + + /** + * Query the stream for a specific capability. + * + * @param capability string to query the stream support for. + * @return True if the stream supports capability. + */ + boolean hasCapability(String capability); +} + http://git-wip-us.apache.org/repos/asf/hadoop/blob/18b89e90/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java ---------------------------------------------------------------------- diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java index 09d3143..d608f34 100755 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java @@ -17,6 +17,9 @@ */ package org.apache.hadoop.hdfs; +import static org.apache.hadoop.fs.StreamCapabilities.StreamCapability.HFLUSH; +import static org.apache.hadoop.fs.StreamCapabilities.StreamCapability.HSYNC; + import java.io.FileNotFoundException; import java.io.IOException; import java.io.InterruptedIOException; @@ -33,6 +36,7 @@ import org.apache.hadoop.fs.FSOutputSummer; import org.apache.hadoop.fs.FileAlreadyExistsException; import org.apache.hadoop.fs.FileEncryptionInfo; import org.apache.hadoop.fs.ParentNotDirectoryException; +import org.apache.hadoop.fs.StreamCapabilities; import org.apache.hadoop.fs.Syncable; import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys; @@ -88,7 +92,7 @@ import com.google.common.base.Preconditions; ****************************************************************/ @InterfaceAudience.Private public class DFSOutputStream extends FSOutputSummer - implements Syncable, CanSetDropBehind { + implements Syncable, CanSetDropBehind, StreamCapabilities { static final Logger LOG = LoggerFactory.getLogger(DFSOutputStream.class); /** * Number of times to retry creating a file when there are transient @@ -554,6 +558,15 @@ public class DFSOutputStream extends FSOutputSummer } } + @Override + public boolean hasCapability(String capability) { + if (capability.equalsIgnoreCase(HSYNC.getValue()) || + capability.equalsIgnoreCase((HFLUSH.getValue()))) { + return true; + } + return false; + } + /** * Flush/Sync buffered data to DataNodes. * http://git-wip-us.apache.org/repos/asf/hadoop/blob/18b89e90/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSOutputStream.java ---------------------------------------------------------------------- diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSOutputStream.java index 52e3bb4..f281a3b 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSOutputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSOutputStream.java @@ -17,9 +17,11 @@ */ package org.apache.hadoop.hdfs; +import java.io.ByteArrayInputStream; import java.io.DataOutputStream; import java.io.File; import java.io.IOException; +import java.io.InputStream; import java.lang.reflect.Field; import java.lang.reflect.InvocationTargetException; import java.lang.reflect.Method; @@ -32,8 +34,10 @@ import java.util.Random; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.CreateFlag; import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FsTracer; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.StreamCapabilities.StreamCapability; import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.hdfs.DataStreamer.LastExceptionInStreamer; import org.apache.hadoop.hdfs.client.impl.DfsClientConf; @@ -48,6 +52,7 @@ import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager; import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor; import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager; import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage; +import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.test.PathUtils; import org.apache.htrace.core.SpanId; @@ -55,6 +60,8 @@ import org.junit.AfterClass; import org.junit.Assert; import org.junit.BeforeClass; import org.junit.Test; + +import static org.junit.Assert.assertTrue; import static org.mockito.Matchers.anyBoolean; import static org.mockito.Matchers.anyLong; import org.mockito.Mockito; @@ -346,6 +353,24 @@ public class TestDFSOutputStream { verify(spyClient, times(1)).endFileLease(anyLong()); } + @Test + public void testStreamFlush() throws Exception { + FileSystem fs = cluster.getFileSystem(); + FSDataOutputStream os = fs.create(new Path("/normal-file")); + // Verify output stream supports hsync() and hflush(). + assertTrue("DFSOutputStream should support hflush()!", + os.hasCapability(StreamCapability.HFLUSH.getValue())); + assertTrue("DFSOutputStream should support hsync()!", + os.hasCapability(StreamCapability.HSYNC.getValue())); + byte[] bytes = new byte[1024]; + InputStream is = new ByteArrayInputStream(bytes); + IOUtils.copyBytes(is, os, bytes.length); + os.hflush(); + IOUtils.copyBytes(is, os, bytes.length); + os.hsync(); + os.close(); + } + @AfterClass public static void tearDown() { if (cluster != null) { http://git-wip-us.apache.org/repos/asf/hadoop/blob/18b89e90/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/SyncableDataOutputStream.java ---------------------------------------------------------------------- diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/SyncableDataOutputStream.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/SyncableDataOutputStream.java index 58ac326..b377f95 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/SyncableDataOutputStream.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/SyncableDataOutputStream.java @@ -22,6 +22,7 @@ import java.io.DataOutputStream; import java.io.IOException; import java.io.OutputStream; +import org.apache.hadoop.fs.StreamCapabilities; import org.apache.hadoop.fs.Syncable; /** @@ -30,13 +31,22 @@ import org.apache.hadoop.fs.Syncable; * wrapped stream passed in to the constructor. This is required * for HBase when wrapping a PageBlobOutputStream used as a write-ahead log. */ -public class SyncableDataOutputStream extends DataOutputStream implements Syncable { +public class SyncableDataOutputStream extends DataOutputStream + implements Syncable, StreamCapabilities { public SyncableDataOutputStream(OutputStream out) { super(out); } @Override + public boolean hasCapability(String capability) { + if (out instanceof StreamCapabilities) { + return ((StreamCapabilities) out).hasCapability(capability); + } + return false; + } + + @Override @Deprecated public void sync() throws IOException { hflush(); --------------------------------------------------------------------- To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org For additional commands, e-mail: common-commits-help@hadoop.apache.org