فهرست منبع

HADOOP-18668. Path capability probe for truncate is only honored by RawLocalFileSystem (#5492)

Viraj Jasani 2 سال پیش
والد
کامیت
0dbe1d3284

+ 1 - 0
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/DfsPathCapabilities.java

@@ -53,6 +53,7 @@ public final class DfsPathCapabilities {
     case CommonPathCapabilities.FS_SNAPSHOTS:
     case CommonPathCapabilities.FS_STORAGEPOLICY:
     case CommonPathCapabilities.FS_XATTRS:
+    case CommonPathCapabilities.FS_TRUNCATE:
       return Optional.of(true);
     case CommonPathCapabilities.FS_SYMLINKS:
       return Optional.of(FileSystem.areSymlinksEnabled());

+ 1 - 0
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/client/HttpFSFileSystem.java

@@ -1646,6 +1646,7 @@ public class HttpFSFileSystem extends FileSystem
     case CommonPathCapabilities.FS_SNAPSHOTS:
     case CommonPathCapabilities.FS_STORAGEPOLICY:
     case CommonPathCapabilities.FS_XATTRS:
+    case CommonPathCapabilities.FS_TRUNCATE:
       return true;
     case CommonPathCapabilities.FS_SYMLINKS:
       return false;

+ 9 - 0
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/client/BaseTestHttpFSWith.java

@@ -22,6 +22,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.BlockLocation;
 import org.apache.hadoop.fs.BlockStoragePolicySpi;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
+import org.apache.hadoop.fs.CommonPathCapabilities;
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileChecksum;
@@ -302,9 +303,17 @@ public abstract class BaseTestHttpFSWith extends HFSTestCase {
       AppendTestUtil.checkFullFile(fs, file, newLength, data, file.toString());
 
       fs.close();
+      assertPathCapabilityForTruncate(file);
     }
   }
 
+  private void assertPathCapabilityForTruncate(Path file) throws Exception {
+    FileSystem fs = this.getHttpFSFileSystem();
+    assertTrue("HttpFS/WebHdfs/SWebHdfs support truncate",
+        fs.hasPathCapability(file, CommonPathCapabilities.FS_TRUNCATE));
+    fs.close();
+  }
+
   private void testConcat() throws Exception {
     Configuration config = getProxiedFSConf();
     config.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, 1024);

+ 3 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemWithTruncate.java

@@ -22,6 +22,7 @@ import java.io.IOException;
 
 import java.util.function.Supplier;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonPathCapabilities;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystemTestHelper;
@@ -103,6 +104,8 @@ public class TestViewFileSystemWithTruncate {
     out.writeBytes("drtatedasfdasfgdfas");
     out.close();
     int newLength = 10;
+    assertTrue("ViewFS supports truncate",
+        fsView.hasPathCapability(filePath, CommonPathCapabilities.FS_TRUNCATE));
     boolean isReady = fsView.truncate(filePath, newLength);
     if (!isReady) {
       GenericTestUtils.waitFor(new Supplier<Boolean>() {

+ 18 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestViewDistributedFileSystem.java

@@ -19,6 +19,7 @@ package org.apache.hadoop.hdfs;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
+import org.apache.hadoop.fs.CommonPathCapabilities;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Options;
 import org.apache.hadoop.fs.Path;
@@ -191,4 +192,21 @@ public class TestViewDistributedFileSystem extends TestDistributedFileSystem{
       }
     }
   }
+
+  @Test
+  public void testPathCapabilities() throws IOException {
+    Configuration conf = getTestConfiguration();
+    try (MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(0).build()) {
+      URI defaultUri = URI.create(conf.get(CommonConfigurationKeys.FS_DEFAULT_NAME_KEY));
+      conf.set("fs.viewfs.mounttable." + defaultUri.getHost() + ".linkFallback",
+          defaultUri.toString());
+      try (ViewDistributedFileSystem fileSystem = (ViewDistributedFileSystem) FileSystem.get(
+          conf)) {
+        final Path testFile = new Path("/test");
+        assertTrue("ViewDfs supports truncate",
+            fileSystem.hasPathCapability(testFile, CommonPathCapabilities.FS_TRUNCATE));
+      }
+    }
+  }
+
 }

+ 7 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java

@@ -33,6 +33,7 @@ import static org.junit.Assert.fail;
 import java.io.IOException;
 import java.util.concurrent.ThreadLocalRandom;
 
+import org.apache.hadoop.fs.CommonPathCapabilities;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil;
 import org.apache.hadoop.hdfs.server.datanode.DataNodeFaultInjector;
 import org.apache.hadoop.ipc.RemoteException;
@@ -143,6 +144,8 @@ public class TestFileTruncate {
         writeContents(contents, fileLength, p);
 
         int newLength = fileLength - toTruncate;
+        assertTrue("DFS supports truncate",
+            fs.hasPathCapability(p, CommonPathCapabilities.FS_TRUNCATE));
         boolean isReady = fs.truncate(p, newLength);
         LOG.info("fileLength=" + fileLength + ", newLength=" + newLength
             + ", toTruncate=" + toTruncate + ", isReady=" + isReady);
@@ -176,6 +179,8 @@ public class TestFileTruncate {
 
     for(int n = data.length; n > 0; ) {
       final int newLength = ThreadLocalRandom.current().nextInt(n);
+      assertTrue("DFS supports truncate",
+          fs.hasPathCapability(p, CommonPathCapabilities.FS_TRUNCATE));
       final boolean isReady = fs.truncate(p, newLength);
       LOG.info("newLength=" + newLength + ", isReady=" + isReady);
       assertEquals("File must be closed for truncating at the block boundary",
@@ -209,6 +214,8 @@ public class TestFileTruncate {
     final int newLength = data.length - 1;
     assert newLength % BLOCK_SIZE != 0 :
         " newLength must not be multiple of BLOCK_SIZE";
+    assertTrue("DFS supports truncate",
+        fs.hasPathCapability(p, CommonPathCapabilities.FS_TRUNCATE));
     final boolean isReady = fs.truncate(p, newLength);
     LOG.info("newLength=" + newLength + ", isReady=" + isReady);
     assertEquals("File must be closed for truncating at the block boundary",

+ 3 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsWithRestCsrfPreventionFilter.java

@@ -29,6 +29,7 @@ import java.net.URI;
 import java.util.Arrays;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonPathCapabilities;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.DFSTestUtil;
@@ -160,6 +161,8 @@ public class TestWebHdfsWithRestCsrfPreventionFilter {
     if (nnRestCsrf && !clientRestCsrf) {
       expectException();
     }
+    assertTrue("WebHdfs supports truncate",
+        webhdfs.hasPathCapability(FILE, CommonPathCapabilities.FS_TRUNCATE));
     assertTrue(webhdfs.truncate(FILE, 0L));
   }