You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by cm...@apache.org on 2015/03/26 21:28:47 UTC
hadoop git commit: HDFS-7410. Support CreateFlags with append() to
support hsync() for appending streams (Vinayakumar B via Colin P. McCabe)
Repository: hadoop
Updated Branches:
refs/heads/trunk 87130bf6b -> 61df1b27a
HDFS-7410. Support CreateFlags with append() to support hsync() for appending streams (Vinayakumar B via Colin P. McCabe)
Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/61df1b27
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/61df1b27
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/61df1b27
Branch: refs/heads/trunk
Commit: 61df1b27a797efd094328c7d9141b9e157e01bf4
Parents: 87130bf
Author: Colin Patrick Mccabe <cm...@cloudera.com>
Authored: Thu Mar 26 13:21:09 2015 -0700
Committer: Colin Patrick Mccabe <cm...@cloudera.com>
Committed: Thu Mar 26 13:21:09 2015 -0700
----------------------------------------------------------------------
hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt | 3 +++
.../java/org/apache/hadoop/hdfs/DFSClient.java | 8 ++++----
.../org/apache/hadoop/hdfs/DFSOutputStream.java | 13 ++++++++-----
.../hadoop/hdfs/server/datanode/TestHSync.java | 19 ++++++++++++++++++-
4 files changed, 33 insertions(+), 10 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/hadoop/blob/61df1b27/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index e16348a..dff8bd2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -806,6 +806,9 @@ Release 2.7.0 - UNRELEASED
HDFS-7976. Update NFS user guide for mount option "sync" to minimize or
avoid reordered writes. (brandonli)
+ HDFS-7410. Support CreateFlags with append() to support hsync() for
+ appending streams (Vinayakumar B via Colin P. McCabe)
+
OPTIMIZATIONS
HDFS-7454. Reduce memory footprint for AclEntries in NameNode.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/61df1b27/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
index 5d67eed..29bb604 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
@@ -1827,10 +1827,10 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
try {
LastBlockWithStatus blkWithStatus = namenode.append(src, clientName,
new EnumSetWritable<>(flag, CreateFlag.class));
- return DFSOutputStream.newStreamForAppend(this, src,
- flag.contains(CreateFlag.NEW_BLOCK),
- buffersize, progress, blkWithStatus.getLastBlock(),
- blkWithStatus.getFileStatus(), dfsClientConf.createChecksum(), favoredNodes);
+ return DFSOutputStream.newStreamForAppend(this, src, flag, buffersize,
+ progress, blkWithStatus.getLastBlock(),
+ blkWithStatus.getFileStatus(), dfsClientConf.createChecksum(),
+ favoredNodes);
} catch(RemoteException re) {
throw re.unwrapRemoteException(AccessControlException.class,
FileNotFoundException.class,
http://git-wip-us.apache.org/repos/asf/hadoop/blob/61df1b27/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
index ee3e6f6..933d8e6 100755
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
@@ -278,11 +278,14 @@ public class DFSOutputStream extends FSOutputSummer
}
/** Construct a new output stream for append. */
- private DFSOutputStream(DFSClient dfsClient, String src, boolean toNewBlock,
- Progressable progress, LocatedBlock lastBlock, HdfsFileStatus stat,
- DataChecksum checksum) throws IOException {
+ private DFSOutputStream(DFSClient dfsClient, String src,
+ EnumSet<CreateFlag> flags, Progressable progress, LocatedBlock lastBlock,
+ HdfsFileStatus stat, DataChecksum checksum) throws IOException {
this(dfsClient, src, progress, stat, checksum);
initialFileSize = stat.getLen(); // length of file when opened
+ this.shouldSyncBlock = flags.contains(CreateFlag.SYNC_BLOCK);
+
+ boolean toNewBlock = flags.contains(CreateFlag.NEW_BLOCK);
this.fileEncryptionInfo = stat.getFileEncryptionInfo();
@@ -338,13 +341,13 @@ public class DFSOutputStream extends FSOutputSummer
}
static DFSOutputStream newStreamForAppend(DFSClient dfsClient, String src,
- boolean toNewBlock, int bufferSize, Progressable progress,
+ EnumSet<CreateFlag> flags, int bufferSize, Progressable progress,
LocatedBlock lastBlock, HdfsFileStatus stat, DataChecksum checksum,
String[] favoredNodes) throws IOException {
TraceScope scope =
dfsClient.getPathTraceScope("newStreamForAppend", src);
try {
- final DFSOutputStream out = new DFSOutputStream(dfsClient, src, toNewBlock,
+ final DFSOutputStream out = new DFSOutputStream(dfsClient, src, flags,
progress, lastBlock, stat, checksum);
if (favoredNodes != null && favoredNodes.length != 0) {
out.streamer.setFavoredNodes(favoredNodes);
http://git-wip-us.apache.org/repos/asf/hadoop/blob/61df1b27/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestHSync.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestHSync.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestHSync.java
index b293075..10f371b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestHSync.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestHSync.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.hdfs.server.datanode;
import static org.apache.hadoop.test.MetricsAsserts.assertCounter;
import static org.apache.hadoop.test.MetricsAsserts.getMetrics;
+import java.io.IOException;
import java.util.EnumSet;
import java.util.Random;
@@ -30,6 +31,7 @@ import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.hdfs.AppendTestUtil;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.io.RandomDatum;
@@ -51,15 +53,30 @@ public class TestHSync {
/** Test basic hsync cases */
@Test
public void testHSync() throws Exception {
+ testHSyncOperation(false);
+ }
+
+ @Test
+ public void testHSyncWithAppend() throws Exception {
+ testHSyncOperation(true);
+ }
+
+ private void testHSyncOperation(boolean testWithAppend) throws IOException {
Configuration conf = new HdfsConfiguration();
MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).build();
- final FileSystem fs = cluster.getFileSystem();
+ final DistributedFileSystem fs = cluster.getFileSystem();
final Path p = new Path("/testHSync/foo");
final int len = 1 << 16;
FSDataOutputStream out = fs.create(p, FsPermission.getDefault(),
EnumSet.of(CreateFlag.CREATE, CreateFlag.OVERWRITE, CreateFlag.SYNC_BLOCK),
4096, (short) 1, len, null);
+ if (testWithAppend) {
+ // re-open the file with append call
+ out.close();
+ out = fs.append(p, EnumSet.of(CreateFlag.APPEND, CreateFlag.SYNC_BLOCK),
+ 4096, null);
+ }
out.hflush();
// hflush does not sync
checkSyncMetric(cluster, 0);