[ https://issues.apache.org/jira/browse/HDFS-17381?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17884335#comment-17884335 ]
ASF GitHub Bot commented on HDFS-17381: --------------------------------------- steveloughran commented on code in PR #6551: URL: https://github.com/apache/hadoop/pull/6551#discussion_r1773612613 ########## hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileUtil.java: ########## @@ -2108,4 +2108,21 @@ public static void maybeIgnoreMissingDirectory(FileSystem fs, LOG.info("Ignoring missing directory {}", path); LOG.debug("Directory missing", e); } + + /** + * Return true if the FS implements {@link WithErasureCoding} and + * supports EC_POLICY option in {@link Options.OpenFileOptions} Review Comment: 1. add a . at the end of this sentence 2. add a second sentence saying that a message is logged when not available ########## hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java: ########## @@ -380,6 +380,8 @@ public FSDataInputStream open(PathHandle fd, int bufferSize) @Override public String getErasureCodingPolicyName(FileStatus fileStatus) { + if (!(fileStatus instanceof HdfsFileStatus)) Review Comment: has to have curly braces round. Not for us, but for our successors in maintaining this code ########## hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/mapred/RetriableFileCopyCommand.java: ########## @@ -208,8 +208,8 @@ private long copyToFile(Path targetPath, FileSystem targetFS, String ecPolicyName = null; if (preserveEC && sourceStatus.isErasureCoded() - && checkFSSupportsEC(sourceStatus.getPath(), sourceFS) - && checkFSSupportsEC(targetPath, targetFS)) { + && checkFSSupportsEC(sourceFS,sourceStatus.getPath()) Review Comment: nit: space after , ########## hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileUtil.java: ########## @@ -2108,4 +2108,21 @@ public static void maybeIgnoreMissingDirectory(FileSystem fs, LOG.info("Ignoring missing directory {}", path); LOG.debug("Directory missing", e); } + + /** + * Return true if the FS implements {@link WithErasureCoding} and + * supports EC_POLICY option in {@link Options.OpenFileOptions} + * @param fs filesystem + * @param path path + * @return true if the Filesystem supports EC + * @throws IOException if there is a failure in hasPathCapability call + */ + public static boolean checkFSSupportsEC(FileSystem fs, Path path) throws IOException { + if (fs instanceof WithErasureCoding && + fs.hasPathCapability(path, Options.OpenFileOptions.FS_OPTION_OPENFILE_EC_POLICY)) { + return true; + } + LOG.warn("FS with scheme {} does not support EC", fs.getScheme()); Review Comment: * prefer "Erasure Coding" to "EC" * do you think we should print the full path? > Distcp of EC files should not be limited to DFS. > ------------------------------------------------ > > Key: HDFS-17381 > URL: https://issues.apache.org/jira/browse/HDFS-17381 > Project: Hadoop HDFS > Issue Type: Bug > Components: distcp > Reporter: Sadanand Shenoy > Assignee: Sadanand Shenoy > Priority: Major > Labels: pull-request-available > > Currently EC file support in distcp is limited to DFS and the code checks if > the given FS instance is DistributedFileSystem, In Ozone, EC is supported > now, and this limitation can be removed and a general contract for any > filesystem that supports EC files should be allowed by implementing few > interfaces/methods. -- This message was sent by Atlassian Jira (v8.20.10#820010) --------------------------------------------------------------------- To unsubscribe, e-mail: hdfs-issues-unsubscr...@hadoop.apache.org For additional commands, e-mail: hdfs-issues-h...@hadoop.apache.org