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 br...@apache.org on 2016/10/21 11:20:20 UTC

[1/3] hadoop git commit: HADOOP-13236. truncate will fail when we use viewfilesystem. Contributed by Brahma Reddy Battula

Repository: hadoop
Updated Branches:
  refs/heads/branch-2 440a6326d -> 2ab149f0e
  refs/heads/branch-2.7 897cf09b4 -> d05e737b8
  refs/heads/branch-2.8 cfac82a44 -> 5754772bd


HADOOP-13236. truncate will fail when we use viewfilesystem. Contributed by Brahma Reddy Battula

(cherry picked from commit f872c6bc0390415f13e95b99749b0b1a690991b7)


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

Branch: refs/heads/branch-2.7
Commit: d05e737b84b4e7ef54cf0ded1abe5aee610c80ac
Parents: 897cf09
Author: Brahma Reddy Battula <br...@apache.org>
Authored: Thu Oct 20 21:47:17 2016 +0530
Committer: Brahma Reddy Battula <br...@apache.org>
Committed: Fri Oct 21 06:21:46 2016 +0530

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt |   3 +
 .../hadoop/fs/viewfs/ChRootedFileSystem.java    |   5 +
 .../apache/hadoop/fs/viewfs/ViewFileSystem.java |   2 +-
 .../viewfs/TestViewFileSystemWithTruncate.java  | 124 +++++++++++++++++++
 4 files changed, 133 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/d05e737b/hadoop-common-project/hadoop-common/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt
index ffbc311..f11933b 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -71,6 +71,9 @@ Release 2.7.4 - UNRELEASED
     HADOOP-13535. Add jetty6 acceptor startup issue workaround to branch-2.
     (Min Shen).
 
+    HADOOP-13236. truncate will fail when we use viewfilesystem.
+    (Brahma Reddy Battula)
+
 Release 2.7.3 - 2016-08-25
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d05e737b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ChRootedFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ChRootedFileSystem.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ChRootedFileSystem.java
index 18e2391..a754752 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ChRootedFileSystem.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ChRootedFileSystem.java
@@ -354,6 +354,11 @@ class ChRootedFileSystem extends FilterFileSystem {
   }
 
   @Override
+  public boolean truncate(Path path, long newLength) throws IOException {
+    return super.truncate(fullPath(path), newLength);
+  }
+
+  @Override
   public List<String> listXAttrs(Path path) throws IOException {
     return super.listXAttrs(fullPath(path));
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d05e737b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFileSystem.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFileSystem.java
index 43fe23f..4eabb66 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFileSystem.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFileSystem.java
@@ -504,7 +504,7 @@ public class ViewFileSystem extends FileSystem {
       throws IOException {
     InodeTree.ResolveResult<FileSystem> res =
         fsState.resolve(getUriPath(f), true);
-    return res.targetFileSystem.truncate(f, newLength);
+    return res.targetFileSystem.truncate(res.remainingPath, newLength);
   }
   
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d05e737b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemWithTruncate.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemWithTruncate.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemWithTruncate.java
new file mode 100644
index 0000000..621f82e
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemWithTruncate.java
@@ -0,0 +1,124 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.viewfs;
+
+import java.io.IOException;
+
+import com.google.common.base.Supplier;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FileSystemTestHelper;
+import org.apache.hadoop.fs.FsConstants;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.MiniDFSNNTopology;
+import org.apache.hadoop.test.GenericTestUtils;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import static org.junit.Assert.*;
+
+/**
+ * Verify truncate through ViewFileSystem functionality.
+ *
+ */
+public class TestViewFileSystemWithTruncate {
+  private static MiniDFSCluster cluster;
+  private static Configuration clusterConf = new Configuration();
+  private static FileSystem fHdfs;
+  private FileSystem fsView;
+  private Configuration fsViewConf;
+  private FileSystem fsTarget;
+  private Path targetTestRoot, mountOnNn1;
+  private FileSystemTestHelper fileSystemTestHelper =
+      new FileSystemTestHelper("/tmp/TestViewFileSystemWithXAttrs");
+
+  @BeforeClass
+  public static void clusterSetupAtBeginning() throws IOException {
+    cluster = new MiniDFSCluster.Builder(clusterConf)
+        .nnTopology(MiniDFSNNTopology.simpleFederatedTopology(2))
+        .numDataNodes(2).build();
+    cluster.waitClusterUp();
+
+    fHdfs = cluster.getFileSystem(0);
+  }
+
+  @AfterClass
+  public static void clusterShutdownAtEnd() throws Exception {
+    if (cluster != null) {
+      cluster.shutdown();
+    }
+  }
+
+  @Before
+  public void setUp() throws Exception {
+    fsTarget = fHdfs;
+    targetTestRoot = fileSystemTestHelper.getAbsoluteTestRootPath(fsTarget);
+
+    fsTarget.delete(targetTestRoot, true);
+    fsTarget.mkdirs(targetTestRoot);
+
+    fsViewConf = ViewFileSystemTestSetup.createConfig();
+    setupMountPoints();
+    fsView = FileSystem.get(FsConstants.VIEWFS_URI, fsViewConf);
+  }
+
+  private void setupMountPoints() {
+    mountOnNn1 = new Path("/mountOnNn1");
+    ConfigUtil
+        .addLink(fsViewConf, mountOnNn1.toString(), targetTestRoot.toUri());
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    fsTarget.delete(fileSystemTestHelper.getTestRootPath(fsTarget), true);
+  }
+
+  @Test(timeout = 30000)
+  public void testTruncateWithViewFileSystem()
+      throws Exception {
+    Path filePath = new Path(mountOnNn1 + "/ttest");
+    final Path hdfFilepath = new Path(
+        "/tmp/TestViewFileSystemWithXAttrs/ttest");
+    FSDataOutputStream out = fsView.create(filePath);
+    out.writeBytes("drtatedasfdasfgdfas");
+    out.close();
+    int newLength = 10;
+    boolean isReady = fsView.truncate(filePath, newLength);
+    if (!isReady) {
+      GenericTestUtils.waitFor(new Supplier<Boolean>() {
+        @Override
+        public Boolean get() {
+          try {
+            return cluster.getFileSystem(0).isFileClosed(hdfFilepath);
+          } catch (IOException e) {
+            return false;
+          }
+        }
+      }, 100, 60 * 1000);
+    }
+    // file length should be 10 after truncate
+    assertEquals(newLength, fsView.getFileStatus(filePath).getLen());
+  }
+
+}


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


[3/3] hadoop git commit: HADOOP-13236. truncate will fail when we use viewfilesystem. Contributed by Brahma Reddy Battula

Posted by br...@apache.org.
HADOOP-13236. truncate will fail when we use viewfilesystem. Contributed by Brahma Reddy Battula

(cherry picked from commit f872c6bc0390415f13e95b99749b0b1a690991b7)


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

Branch: refs/heads/branch-2.8
Commit: 5754772bd69f0cce0f3670f5eae714f8299d1ab9
Parents: cfac82a
Author: Brahma Reddy Battula <br...@apache.org>
Authored: Thu Oct 20 21:47:17 2016 +0530
Committer: Brahma Reddy Battula <br...@apache.org>
Committed: Fri Oct 21 16:22:42 2016 +0530

----------------------------------------------------------------------
 .../hadoop/fs/viewfs/ChRootedFileSystem.java    |   5 +
 .../apache/hadoop/fs/viewfs/ViewFileSystem.java |   2 +-
 .../viewfs/TestViewFileSystemWithTruncate.java  | 124 +++++++++++++++++++
 3 files changed, 130 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/5754772b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ChRootedFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ChRootedFileSystem.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ChRootedFileSystem.java
index 46e8f2d..943846d 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ChRootedFileSystem.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ChRootedFileSystem.java
@@ -354,6 +354,11 @@ class ChRootedFileSystem extends FilterFileSystem {
   }
 
   @Override
+  public boolean truncate(Path path, long newLength) throws IOException {
+    return super.truncate(fullPath(path), newLength);
+  }
+
+  @Override
   public List<String> listXAttrs(Path path) throws IOException {
     return super.listXAttrs(fullPath(path));
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5754772b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFileSystem.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFileSystem.java
index 37893d7..0f090a5 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFileSystem.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFileSystem.java
@@ -505,7 +505,7 @@ public class ViewFileSystem extends FileSystem {
       throws IOException {
     InodeTree.ResolveResult<FileSystem> res =
         fsState.resolve(getUriPath(f), true);
-    return res.targetFileSystem.truncate(f, newLength);
+    return res.targetFileSystem.truncate(res.remainingPath, newLength);
   }
   
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5754772b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemWithTruncate.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemWithTruncate.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemWithTruncate.java
new file mode 100644
index 0000000..621f82e
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemWithTruncate.java
@@ -0,0 +1,124 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.viewfs;
+
+import java.io.IOException;
+
+import com.google.common.base.Supplier;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FileSystemTestHelper;
+import org.apache.hadoop.fs.FsConstants;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.MiniDFSNNTopology;
+import org.apache.hadoop.test.GenericTestUtils;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import static org.junit.Assert.*;
+
+/**
+ * Verify truncate through ViewFileSystem functionality.
+ *
+ */
+public class TestViewFileSystemWithTruncate {
+  private static MiniDFSCluster cluster;
+  private static Configuration clusterConf = new Configuration();
+  private static FileSystem fHdfs;
+  private FileSystem fsView;
+  private Configuration fsViewConf;
+  private FileSystem fsTarget;
+  private Path targetTestRoot, mountOnNn1;
+  private FileSystemTestHelper fileSystemTestHelper =
+      new FileSystemTestHelper("/tmp/TestViewFileSystemWithXAttrs");
+
+  @BeforeClass
+  public static void clusterSetupAtBeginning() throws IOException {
+    cluster = new MiniDFSCluster.Builder(clusterConf)
+        .nnTopology(MiniDFSNNTopology.simpleFederatedTopology(2))
+        .numDataNodes(2).build();
+    cluster.waitClusterUp();
+
+    fHdfs = cluster.getFileSystem(0);
+  }
+
+  @AfterClass
+  public static void clusterShutdownAtEnd() throws Exception {
+    if (cluster != null) {
+      cluster.shutdown();
+    }
+  }
+
+  @Before
+  public void setUp() throws Exception {
+    fsTarget = fHdfs;
+    targetTestRoot = fileSystemTestHelper.getAbsoluteTestRootPath(fsTarget);
+
+    fsTarget.delete(targetTestRoot, true);
+    fsTarget.mkdirs(targetTestRoot);
+
+    fsViewConf = ViewFileSystemTestSetup.createConfig();
+    setupMountPoints();
+    fsView = FileSystem.get(FsConstants.VIEWFS_URI, fsViewConf);
+  }
+
+  private void setupMountPoints() {
+    mountOnNn1 = new Path("/mountOnNn1");
+    ConfigUtil
+        .addLink(fsViewConf, mountOnNn1.toString(), targetTestRoot.toUri());
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    fsTarget.delete(fileSystemTestHelper.getTestRootPath(fsTarget), true);
+  }
+
+  @Test(timeout = 30000)
+  public void testTruncateWithViewFileSystem()
+      throws Exception {
+    Path filePath = new Path(mountOnNn1 + "/ttest");
+    final Path hdfFilepath = new Path(
+        "/tmp/TestViewFileSystemWithXAttrs/ttest");
+    FSDataOutputStream out = fsView.create(filePath);
+    out.writeBytes("drtatedasfdasfgdfas");
+    out.close();
+    int newLength = 10;
+    boolean isReady = fsView.truncate(filePath, newLength);
+    if (!isReady) {
+      GenericTestUtils.waitFor(new Supplier<Boolean>() {
+        @Override
+        public Boolean get() {
+          try {
+            return cluster.getFileSystem(0).isFileClosed(hdfFilepath);
+          } catch (IOException e) {
+            return false;
+          }
+        }
+      }, 100, 60 * 1000);
+    }
+    // file length should be 10 after truncate
+    assertEquals(newLength, fsView.getFileStatus(filePath).getLen());
+  }
+
+}


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


[2/3] hadoop git commit: HADOOP-13236. truncate will fail when we use viewfilesystem. Contributed by Brahma Reddy Battula

Posted by br...@apache.org.
HADOOP-13236. truncate will fail when we use viewfilesystem. Contributed by Brahma Reddy Battula

(cherry picked from commit f872c6bc0390415f13e95b99749b0b1a690991b7)


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

Branch: refs/heads/branch-2
Commit: 2ab149f0e40fded3a9e5e9aa11f7003d0de53644
Parents: 440a632
Author: Brahma Reddy Battula <br...@apache.org>
Authored: Thu Oct 20 21:47:17 2016 +0530
Committer: Brahma Reddy Battula <br...@apache.org>
Committed: Fri Oct 21 16:19:18 2016 +0530

----------------------------------------------------------------------
 .../hadoop/fs/viewfs/ChRootedFileSystem.java    |   5 +
 .../apache/hadoop/fs/viewfs/ViewFileSystem.java |   2 +-
 .../viewfs/TestViewFileSystemWithTruncate.java  | 124 +++++++++++++++++++
 3 files changed, 130 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/2ab149f0/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ChRootedFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ChRootedFileSystem.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ChRootedFileSystem.java
index 46e8f2d..943846d 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ChRootedFileSystem.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ChRootedFileSystem.java
@@ -354,6 +354,11 @@ class ChRootedFileSystem extends FilterFileSystem {
   }
 
   @Override
+  public boolean truncate(Path path, long newLength) throws IOException {
+    return super.truncate(fullPath(path), newLength);
+  }
+
+  @Override
   public List<String> listXAttrs(Path path) throws IOException {
     return super.listXAttrs(fullPath(path));
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2ab149f0/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFileSystem.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFileSystem.java
index 37893d7..0f090a5 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFileSystem.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFileSystem.java
@@ -505,7 +505,7 @@ public class ViewFileSystem extends FileSystem {
       throws IOException {
     InodeTree.ResolveResult<FileSystem> res =
         fsState.resolve(getUriPath(f), true);
-    return res.targetFileSystem.truncate(f, newLength);
+    return res.targetFileSystem.truncate(res.remainingPath, newLength);
   }
   
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2ab149f0/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemWithTruncate.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemWithTruncate.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemWithTruncate.java
new file mode 100644
index 0000000..621f82e
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemWithTruncate.java
@@ -0,0 +1,124 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.viewfs;
+
+import java.io.IOException;
+
+import com.google.common.base.Supplier;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FileSystemTestHelper;
+import org.apache.hadoop.fs.FsConstants;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.MiniDFSNNTopology;
+import org.apache.hadoop.test.GenericTestUtils;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import static org.junit.Assert.*;
+
+/**
+ * Verify truncate through ViewFileSystem functionality.
+ *
+ */
+public class TestViewFileSystemWithTruncate {
+  private static MiniDFSCluster cluster;
+  private static Configuration clusterConf = new Configuration();
+  private static FileSystem fHdfs;
+  private FileSystem fsView;
+  private Configuration fsViewConf;
+  private FileSystem fsTarget;
+  private Path targetTestRoot, mountOnNn1;
+  private FileSystemTestHelper fileSystemTestHelper =
+      new FileSystemTestHelper("/tmp/TestViewFileSystemWithXAttrs");
+
+  @BeforeClass
+  public static void clusterSetupAtBeginning() throws IOException {
+    cluster = new MiniDFSCluster.Builder(clusterConf)
+        .nnTopology(MiniDFSNNTopology.simpleFederatedTopology(2))
+        .numDataNodes(2).build();
+    cluster.waitClusterUp();
+
+    fHdfs = cluster.getFileSystem(0);
+  }
+
+  @AfterClass
+  public static void clusterShutdownAtEnd() throws Exception {
+    if (cluster != null) {
+      cluster.shutdown();
+    }
+  }
+
+  @Before
+  public void setUp() throws Exception {
+    fsTarget = fHdfs;
+    targetTestRoot = fileSystemTestHelper.getAbsoluteTestRootPath(fsTarget);
+
+    fsTarget.delete(targetTestRoot, true);
+    fsTarget.mkdirs(targetTestRoot);
+
+    fsViewConf = ViewFileSystemTestSetup.createConfig();
+    setupMountPoints();
+    fsView = FileSystem.get(FsConstants.VIEWFS_URI, fsViewConf);
+  }
+
+  private void setupMountPoints() {
+    mountOnNn1 = new Path("/mountOnNn1");
+    ConfigUtil
+        .addLink(fsViewConf, mountOnNn1.toString(), targetTestRoot.toUri());
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    fsTarget.delete(fileSystemTestHelper.getTestRootPath(fsTarget), true);
+  }
+
+  @Test(timeout = 30000)
+  public void testTruncateWithViewFileSystem()
+      throws Exception {
+    Path filePath = new Path(mountOnNn1 + "/ttest");
+    final Path hdfFilepath = new Path(
+        "/tmp/TestViewFileSystemWithXAttrs/ttest");
+    FSDataOutputStream out = fsView.create(filePath);
+    out.writeBytes("drtatedasfdasfgdfas");
+    out.close();
+    int newLength = 10;
+    boolean isReady = fsView.truncate(filePath, newLength);
+    if (!isReady) {
+      GenericTestUtils.waitFor(new Supplier<Boolean>() {
+        @Override
+        public Boolean get() {
+          try {
+            return cluster.getFileSystem(0).isFileClosed(hdfFilepath);
+          } catch (IOException e) {
+            return false;
+          }
+        }
+      }, 100, 60 * 1000);
+    }
+    // file length should be 10 after truncate
+    assertEquals(newLength, fsView.getFileStatus(filePath).getLen());
+  }
+
+}


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