You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-issues@hadoop.apache.org by GitBox <gi...@apache.org> on 2020/08/21 03:02:13 UTC

[GitHub] [hadoop] yangagile opened a new pull request #2235: HDFS-15484 Add new method batchRename for DistributedFileSystem and W…

yangagile opened a new pull request #2235:
URL: https://github.com/apache/hadoop/pull/2235


   …ebHdfsFileSystem
   
   ## NOTICE
   
   Please create an issue in ASF JIRA before opening a pull request,
   and you need to set the title of the pull request which starts with
   the corresponding JIRA issue number. (e.g. HADOOP-XXXXX. Fix a typo in YYY.)
   For more details, please see https://cwiki.apache.org/confluence/display/HADOOP/How+To+Contribute
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [hadoop] yangagile commented on a change in pull request #2235: HDFS-15484 Add new method batchRename for DistributedFileSystem and W…

Posted by GitBox <gi...@apache.org>.
yangagile commented on a change in pull request #2235:
URL: https://github.com/apache/hadoop/pull/2235#discussion_r475380079



##########
File path: hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBatchRename.java
##########
@@ -0,0 +1,259 @@
+/**
+ * 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.hdfs;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.BatchOperations;
+import org.apache.hadoop.fs.InvalidPathException;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.protocol.BatchOpsException;
+import org.apache.hadoop.hdfs.web.WebHdfsConstants;
+import org.apache.hadoop.hdfs.web.WebHdfsFileSystem;
+import org.apache.hadoop.hdfs.web.WebHdfsTestUtil;
+import org.apache.hadoop.test.LambdaTestUtils;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+public class TestBatchRename {
+  private static MiniDFSCluster cluster;
+  private static Configuration conf;
+  private static DistributedFileSystem dfs;
+  private static WebHdfsFileSystem webHdfs;
+  private Path root = new Path("/test/batchrename/");
+
+  @BeforeClass
+  public static void beforeClass() throws Exception {
+    conf = new HdfsConfiguration();
+    cluster = new MiniDFSCluster.Builder(conf)
+        .numDataNodes(1)
+        .build();
+    dfs = cluster.getFileSystem();
+
+    webHdfs = WebHdfsTestUtil.getWebHdfsFileSystem(conf,
+        WebHdfsConstants.WEBHDFS_SCHEME);
+  }
+
+  @AfterClass
+  public static void afterClass() {
+    if (cluster != null) {
+      cluster.shutdown();
+    }
+  }
+
+  @Test
+  public void testhasPathCapability() throws Exception {
+    assertTrue("DistributedFileSystem should has batch rename capbility",
+        dfs.hasPathCapability(root, "fs.capability.batch.rename"));
+  }
+
+  private List<String> generateBatchFiles(
+      int totalNum, int createNum, final Path dir, String tag)
+      throws IOException {
+    List<String> files = new ArrayList<>();
+    for (int i = 0; i < totalNum; i++) {
+      Path p = new Path(dir, tag + "_" + i);
+      if (createNum-- > 0) {
+        DFSTestUtil.createFile(dfs, p, 10, (short) 1, 0);
+        assertTrue(dfs.exists(p));

Review comment:
       Done, thanks!




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [hadoop] yangagile commented on a change in pull request #2235: HDFS-15484 Add new method batchRename for DistributedFileSystem and W…

Posted by GitBox <gi...@apache.org>.
yangagile commented on a change in pull request #2235:
URL: https://github.com/apache/hadoop/pull/2235#discussion_r475395592



##########
File path: hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java
##########
@@ -732,6 +732,14 @@ protected Response put(
         return Response.ok().type(MediaType.APPLICATION_OCTET_STREAM).build();
       }
     }
+    case BATCH_RENAME:
+    {
+      validateOpParams(op, destination);
+      final EnumSet<Options.Rename> s = renameOptions.getValue();
+      cp.batchRename(fullpath.split(":"), destination.getValue().split(":"),

Review comment:
       The ":" is invalid character for hadoop absolute path, please refer to the method DFSUtilClient.isValidName().
   




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [hadoop] steveloughran commented on a change in pull request #2235: HDFS-15484 Add new method batchRename for DistributedFileSystem and W…

Posted by GitBox <gi...@apache.org>.
steveloughran commented on a change in pull request #2235:
URL: https://github.com/apache/hadoop/pull/2235#discussion_r475080490



##########
File path: hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/BatchOpsException.java
##########
@@ -0,0 +1,87 @@
+/**
+ * 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.hdfs.protocol;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+import java.io.IOException;
+
+/**
+ * Thrown when break during a batch operation .
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public final class BatchOpsException extends IOException {
+  private static final long serialVersionUID = 1L;
+  private static final String TAG_INDEX = "index";
+  private static final String TAG_TOTAL = "total";
+  private static final String TAG_REASON = "reason";
+
+  /**
+   * Used by RemoteException to instantiate an BatchOpsException.
+   */
+  public BatchOpsException(String msg) {
+    super(msg);
+  }
+
+  public BatchOpsException(long index, long total, Throwable cause) {
+    this(index, total,
+        cause.getClass().getName() + ": " + cause.getMessage());
+  }
+
+  public BatchOpsException(long index, long total,
+                           String cause) {
+    super("Batch operation break! " +

Review comment:
       not sure about "break!"

##########
File path: hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBatchRename.java
##########
@@ -0,0 +1,259 @@
+/**
+ * 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.hdfs;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.BatchOperations;
+import org.apache.hadoop.fs.InvalidPathException;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.protocol.BatchOpsException;
+import org.apache.hadoop.hdfs.web.WebHdfsConstants;
+import org.apache.hadoop.hdfs.web.WebHdfsFileSystem;
+import org.apache.hadoop.hdfs.web.WebHdfsTestUtil;
+import org.apache.hadoop.test.LambdaTestUtils;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+public class TestBatchRename {
+  private static MiniDFSCluster cluster;
+  private static Configuration conf;
+  private static DistributedFileSystem dfs;
+  private static WebHdfsFileSystem webHdfs;
+  private Path root = new Path("/test/batchrename/");
+
+  @BeforeClass
+  public static void beforeClass() throws Exception {
+    conf = new HdfsConfiguration();
+    cluster = new MiniDFSCluster.Builder(conf)
+        .numDataNodes(1)
+        .build();
+    dfs = cluster.getFileSystem();
+
+    webHdfs = WebHdfsTestUtil.getWebHdfsFileSystem(conf,
+        WebHdfsConstants.WEBHDFS_SCHEME);
+  }
+
+  @AfterClass
+  public static void afterClass() {
+    if (cluster != null) {
+      cluster.shutdown();
+    }
+  }
+
+  @Test
+  public void testhasPathCapability() throws Exception {
+    assertTrue("DistributedFileSystem should has batch rename capbility",
+        dfs.hasPathCapability(root, "fs.capability.batch.rename"));
+  }
+
+  private List<String> generateBatchFiles(
+      int totalNum, int createNum, final Path dir, String tag)
+      throws IOException {
+    List<String> files = new ArrayList<>();
+    for (int i = 0; i < totalNum; i++) {
+      Path p = new Path(dir, tag + "_" + i);
+      if (createNum-- > 0) {
+        DFSTestUtil.createFile(dfs, p, 10, (short) 1, 0);
+        assertTrue(dfs.exists(p));
+      } else {
+        assertFalse(dfs.exists(p));
+      }
+      files.add(p.toString());
+    }
+    return files;
+  }
+
+  private void testBatchRename(BatchOperations batchFS) throws Exception {
+    Path testDir = new Path(root, "testBatchRename");
+    assertTrue(dfs.mkdirs(testDir));
+
+    List<String> srcs = generateBatchFiles(
+        2, 2, testDir, "src");
+    List<String> dsts =generateBatchFiles(
+        2, 0, testDir, "dst");
+
+    batchFS.batchRename(
+        srcs.toArray(new String[srcs.size()]),
+        dsts.toArray(new String[dsts.size()]));
+
+    for (String f : srcs) {
+      assertFalse(dfs.exists(new Path(f)));
+    }
+    for (String f : dsts) {
+      assertTrue(dfs.exists(new Path(f)));
+      dfs.delete(new Path(f), true);
+    }
+  }
+
+  @Test
+  public void testBatchRaname() throws Exception {
+    testBatchRename(dfs);
+    testBatchRename(webHdfs);
+  }
+
+  private void testInvalidInput(BatchOperations batchFS) throws Exception {
+    List<String> srcs = new ArrayList<>();
+    srcs.add("/testInvalidInput_Mismatch");
+    List<String> dsts = new ArrayList<>();
+    LambdaTestUtils.intercept(InvalidPathException.class,
+        "mismatch batch path",
+        () -> batchFS.batchRename(
+            srcs.toArray(new String[srcs.size()]),
+            dsts.toArray(new String[dsts.size()])));
+  }
+
+  @Test
+  public void testInvalidInput() throws Exception {
+    testInvalidInput(dfs);
+    testInvalidInput(webHdfs);
+  }
+
+   // rename /src_1:/src_2(not existing) to /dst_1:/dst_2
+  private void testPartialSuccess1(BatchOperations batchFS) throws Exception {
+    Path testDir = new Path(root, "partial_success");
+    assertTrue(dfs.mkdirs(testDir));
+
+    List<String> srcs =  generateBatchFiles(
+        2, 1, testDir, "src");
+    List<String> dsts = generateBatchFiles(
+        2, 0, testDir, "dst");
+    try {
+      batchFS.batchRename(
+          srcs.toArray(new String[srcs.size()]),
+          dsts.toArray(new String[dsts.size()]));
+    } catch (BatchOpsException e) {
+      long index = e.getIndex();
+      assertEquals(1, index);
+      long total = e.getTotal();
+      assertEquals(2, total);
+
+      String reason = e.getReason();
+      assertTrue(reason.contains("FileNotFoundException"));
+
+      for (int i = 0; i < index; i++) {
+        Path p = new Path(testDir, "src_" + i);
+        assertFalse(dfs.exists(p));
+      }
+      for (int i = 0; i < index; i++) {
+        Path p = new Path(testDir, "dst_" + i);
+        assertTrue(dfs.exists(p));
+        dfs.delete(p, true);
+      }
+    }
+  }
+
+   // rename src_1:src_1/subdir to /dst_1:/dst_2
+  private void testPartialSuccess2(BatchOperations batchFS) throws Exception {
+    Path testDir = new Path(root, "partial_success");
+    List<String> srcs = new ArrayList<>();
+    Path src1 = new Path(testDir, "src_1");
+    assertTrue(dfs.mkdirs(src1));
+    srcs.add(src1.toString());
+    Path src1Subdir = new Path(src1, "subdir");
+    assertTrue(dfs.mkdirs(src1Subdir));
+    srcs.add(src1Subdir.toString());
+
+    List<String> dsts = generateBatchFiles(
+        2, 0, testDir, "dst");
+    try {
+      batchFS.batchRename(
+          srcs.toArray(new String[srcs.size()]),
+          dsts.toArray(new String[dsts.size()]));
+    } catch (BatchOpsException e) {
+      long index = e.getIndex();
+      assertEquals(1, index);
+      long total = e.getTotal();
+      assertEquals(2, total);
+      String reason = e.getReason();
+      assertTrue(reason.contains("FileNotFoundException"));
+      for (int i = 0; i < index; i++) {
+        Path p = new Path(testDir, "src_" + i);
+        assertFalse(dfs.exists(p));
+      }
+      for (int i = 0; i < index; i++) {
+        Path p = new Path(testDir, "dst_" + i);
+        assertTrue(dfs.exists(p));
+        dfs.delete(p, true);
+      }
+    }
+  }
+
+  // rename src_1:src_2 /dst_1:/dst_1
+  private void testPartialSuccess3(BatchOperations batchFS) throws Exception {
+    Path testDir = new Path(root, "partial_success_3");
+    List<String> srcs =  generateBatchFiles(
+        2, 2, testDir, "src");
+    List<String> dsts = generateBatchFiles(
+        1, 0, testDir, "dst");
+    dsts.add(dsts.get(0));
+
+    try {
+      batchFS.batchRename(
+          srcs.toArray(new String[srcs.size()]),
+          dsts.toArray(new String[dsts.size()]));
+    } catch (BatchOpsException e) {
+      long index = e.getIndex();
+      assertEquals(1, index);

Review comment:
       all tests to add a message about what a failure means. Imagine you have a yetus test run failure and are trying to debug from the log only
   

##########
File path: hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
##########
@@ -1609,6 +1610,26 @@ public void rename(String src, String dst, Options.Rename... options)
     }
   }
 
+  /**
+   * Rename a batch files or directories.
+   * @see ClientProtocol#batchRename(String[] , String[], Options.Rename...)
+   */
+  public void batchRename(String[] srcs, String[] dsts,

Review comment:
       better as a list of <src, dest> pairs, so it's obvious about the mapping. Add javadocs

##########
File path: hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBatchRename.java
##########
@@ -0,0 +1,259 @@
+/**
+ * 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.hdfs;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.BatchOperations;
+import org.apache.hadoop.fs.InvalidPathException;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.protocol.BatchOpsException;
+import org.apache.hadoop.hdfs.web.WebHdfsConstants;
+import org.apache.hadoop.hdfs.web.WebHdfsFileSystem;
+import org.apache.hadoop.hdfs.web.WebHdfsTestUtil;
+import org.apache.hadoop.test.LambdaTestUtils;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+public class TestBatchRename {

Review comment:
       extend AbstractHadoopTestBasefor its setp (thread naming, timeout)
   

##########
File path: hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonPathCapabilities.java
##########
@@ -139,4 +139,11 @@ private CommonPathCapabilities() {
   public static final String FS_MULTIPART_UPLOADER =
       "fs.capability.multipart.uploader";
 
+  /**
+   * Does the store support multipart uploading?

Review comment:
       fix

##########
File path: hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/BatchOpsException.java
##########
@@ -0,0 +1,87 @@
+/**
+ * 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.hdfs.protocol;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+import java.io.IOException;
+
+/**
+ * Thrown when break during a batch operation .
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public final class BatchOpsException extends IOException {
+  private static final long serialVersionUID = 1L;
+  private static final String TAG_INDEX = "index";
+  private static final String TAG_TOTAL = "total";
+  private static final String TAG_REASON = "reason";
+
+  /**
+   * Used by RemoteException to instantiate an BatchOpsException.
+   */
+  public BatchOpsException(String msg) {
+    super(msg);
+  }
+
+  public BatchOpsException(long index, long total, Throwable cause) {
+    this(index, total,
+        cause.getClass().getName() + ": " + cause.getMessage());

Review comment:
       cause.toString(); message may be null
   

##########
File path: hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/BatchOpsException.java
##########
@@ -0,0 +1,87 @@
+/**
+ * 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.hdfs.protocol;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+import java.io.IOException;
+
+/**
+ * Thrown when break during a batch operation .
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public final class BatchOpsException extends IOException {
+  private static final long serialVersionUID = 1L;

Review comment:
       needs a real serial version ID; your IDE can help there
   

##########
File path: hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBatchRename.java
##########
@@ -0,0 +1,259 @@
+/**
+ * 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.hdfs;
+
+import org.apache.hadoop.conf.Configuration;

Review comment:
       sparate import ordering as usual

##########
File path: hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBatchRename.java
##########
@@ -0,0 +1,259 @@
+/**
+ * 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.hdfs;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.BatchOperations;
+import org.apache.hadoop.fs.InvalidPathException;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.protocol.BatchOpsException;
+import org.apache.hadoop.hdfs.web.WebHdfsConstants;
+import org.apache.hadoop.hdfs.web.WebHdfsFileSystem;
+import org.apache.hadoop.hdfs.web.WebHdfsTestUtil;
+import org.apache.hadoop.test.LambdaTestUtils;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+public class TestBatchRename {
+  private static MiniDFSCluster cluster;
+  private static Configuration conf;
+  private static DistributedFileSystem dfs;
+  private static WebHdfsFileSystem webHdfs;
+  private Path root = new Path("/test/batchrename/");
+
+  @BeforeClass
+  public static void beforeClass() throws Exception {
+    conf = new HdfsConfiguration();
+    cluster = new MiniDFSCluster.Builder(conf)
+        .numDataNodes(1)
+        .build();
+    dfs = cluster.getFileSystem();
+
+    webHdfs = WebHdfsTestUtil.getWebHdfsFileSystem(conf,
+        WebHdfsConstants.WEBHDFS_SCHEME);
+  }
+
+  @AfterClass
+  public static void afterClass() {
+    if (cluster != null) {
+      cluster.shutdown();
+    }
+  }
+
+  @Test
+  public void testhasPathCapability() throws Exception {
+    assertTrue("DistributedFileSystem should has batch rename capbility",
+        dfs.hasPathCapability(root, "fs.capability.batch.rename"));

Review comment:
       refer to constant
   

##########
File path: hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBatchRename.java
##########
@@ -0,0 +1,259 @@
+/**
+ * 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.hdfs;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.BatchOperations;
+import org.apache.hadoop.fs.InvalidPathException;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.protocol.BatchOpsException;
+import org.apache.hadoop.hdfs.web.WebHdfsConstants;
+import org.apache.hadoop.hdfs.web.WebHdfsFileSystem;
+import org.apache.hadoop.hdfs.web.WebHdfsTestUtil;
+import org.apache.hadoop.test.LambdaTestUtils;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+public class TestBatchRename {
+  private static MiniDFSCluster cluster;
+  private static Configuration conf;
+  private static DistributedFileSystem dfs;
+  private static WebHdfsFileSystem webHdfs;
+  private Path root = new Path("/test/batchrename/");
+
+  @BeforeClass
+  public static void beforeClass() throws Exception {
+    conf = new HdfsConfiguration();
+    cluster = new MiniDFSCluster.Builder(conf)
+        .numDataNodes(1)
+        .build();
+    dfs = cluster.getFileSystem();
+
+    webHdfs = WebHdfsTestUtil.getWebHdfsFileSystem(conf,
+        WebHdfsConstants.WEBHDFS_SCHEME);
+  }
+
+  @AfterClass
+  public static void afterClass() {
+    if (cluster != null) {
+      cluster.shutdown();
+    }
+  }
+
+  @Test
+  public void testhasPathCapability() throws Exception {
+    assertTrue("DistributedFileSystem should has batch rename capbility",
+        dfs.hasPathCapability(root, "fs.capability.batch.rename"));
+  }
+
+  private List<String> generateBatchFiles(
+      int totalNum, int createNum, final Path dir, String tag)
+      throws IOException {
+    List<String> files = new ArrayList<>();
+    for (int i = 0; i < totalNum; i++) {
+      Path p = new Path(dir, tag + "_" + i);
+      if (createNum-- > 0) {
+        DFSTestUtil.createFile(dfs, p, 10, (short) 1, 0);
+        assertTrue(dfs.exists(p));

Review comment:
       ContractTestUtils assertFile() (or similar)
   

##########
File path: hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/BatchOperations.java
##########
@@ -0,0 +1,43 @@
+/*
+ * 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;
+
+import java.io.IOException;
+
+/**
+ * Interface filesystems MAY implement to offer a batched operations.
+ */
+
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
+public interface BatchOperations {

Review comment:
       BatchRename

##########
File path: hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/BatchOperations.java
##########
@@ -0,0 +1,43 @@
+/*
+ * 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;
+
+import java.io.IOException;
+
+/**
+ * Interface filesystems MAY implement to offer a batched operations.
+ */
+
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
+public interface BatchOperations {
+
+  /**
+   * Batched rename API that rename a batch of files.
+   *
+   * @param srcs source file list.
+   * @param dsts target file list.
+   * @throws IOException failure exception.
+   */
+   void batchRename(String[] srcs, String[] dsts, Options.Rename... options)

Review comment:
       Prefer list or array of pairs. We don't have any pair type in hadoop here and can't use commons-lang as we don't want that in our public API. Maybe we should add one to org.apache.hadoop.common.utils and use it here amongst other places. I could certainly use it (I may be able to add this to HADOOP-16830 for you to pick up)
   
   Return a future<RenameResult> where we define RenameResult as something (class/interface) which implements IOStatisticsSource.

##########
File path: hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java
##########
@@ -732,6 +732,14 @@ protected Response put(
         return Response.ok().type(MediaType.APPLICATION_OCTET_STREAM).build();
       }
     }
+    case BATCH_RENAME:
+    {
+      validateOpParams(op, destination);
+      final EnumSet<Options.Rename> s = renameOptions.getValue();
+      cp.batchRename(fullpath.split(":"), destination.getValue().split(":"),

Review comment:
       what will be done here if there's a ":" in paths
   




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [hadoop] yangagile commented on a change in pull request #2235: HDFS-15484 Add new method batchRename for DistributedFileSystem and W…

Posted by GitBox <gi...@apache.org>.
yangagile commented on a change in pull request #2235:
URL: https://github.com/apache/hadoop/pull/2235#discussion_r475386886



##########
File path: hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/BatchOperations.java
##########
@@ -0,0 +1,43 @@
+/*
+ * 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;
+
+import java.io.IOException;
+
+/**
+ * Interface filesystems MAY implement to offer a batched operations.
+ */
+
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
+public interface BatchOperations {

Review comment:
       I change to BatchRename, I thought we may add other batch method in this interface in the future.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [hadoop] hadoop-yetus removed a comment on pull request #2235: HDFS-15484. Add new method batchRename for DistributedFileSystem and W…

Posted by GitBox <gi...@apache.org>.
hadoop-yetus removed a comment on pull request #2235:
URL: https://github.com/apache/hadoop/pull/2235#issuecomment-679105788


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m 17s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  No case conflicting files found.  |
   | +0 :ok: |  buf  |   0m  1s |  buf was not available.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |  test4tests  |   0m  1s |  The patch appears to include 1 new or modified test files.  |
   ||| _ trunk Compile Tests _ |
   | +0 :ok: |  mvndep  |   3m 22s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |  30m 30s |  trunk passed  |
   | +1 :green_heart: |  compile  |  24m 15s |  trunk passed with JDK Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1  |
   | +1 :green_heart: |  compile  |  21m 10s |  trunk passed with JDK Private Build-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01  |
   | +1 :green_heart: |  checkstyle  |   3m 29s |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   5m 25s |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  23m 27s |  branch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   3m 27s |  trunk passed with JDK Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1  |
   | +1 :green_heart: |  javadoc  |   5m 16s |  trunk passed with JDK Private Build-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01  |
   | +0 :ok: |  spotbugs  |   1m 26s |  Used deprecated FindBugs config; considering switching to SpotBugs.  |
   | +1 :green_heart: |  findbugs  |   9m 27s |  trunk passed  |
   | -0 :warning: |  patch  |   1m 47s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 28s |  Maven dependency ordering for patch  |
   | -1 :x: |  mvninstall  |   0m 22s |  hadoop-hdfs-rbf in the patch failed.  |
   | -1 :x: |  compile  |   5m 20s |  root in the patch failed with JDK Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1.  |
   | -1 :x: |  cc  |   5m 20s |  root in the patch failed with JDK Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1.  |
   | -1 :x: |  javac  |   5m 20s |  root in the patch failed with JDK Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1.  |
   | -1 :x: |  compile  |   4m 47s |  root in the patch failed with JDK Private Build-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01.  |
   | -1 :x: |  cc  |   4m 47s |  root in the patch failed with JDK Private Build-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01.  |
   | -1 :x: |  javac  |   4m 47s |  root in the patch failed with JDK Private Build-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01.  |
   | -0 :warning: |  checkstyle  |   2m 36s |  root: The patch generated 6 new + 597 unchanged - 2 fixed = 603 total (was 599)  |
   | -1 :x: |  mvnsite  |   0m 26s |  hadoop-hdfs-rbf in the patch failed.  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  shadedclient  |  15m 41s |  patch has no errors when building and testing our client artifacts.  |
   | -1 :x: |  javadoc  |   0m 35s |  hadoop-hdfs-project_hadoop-hdfs-client-jdkUbuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1 with JDK Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1 generated 1 new + 0 unchanged - 0 fixed = 1 total (was 0)  |
   | +1 :green_heart: |  javadoc  |   4m  8s |  the patch passed with JDK Private Build-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01  |
   | -1 :x: |  findbugs  |   0m 26s |  hadoop-hdfs-rbf in the patch failed.  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   9m  6s |  hadoop-common in the patch passed.  |
   | +1 :green_heart: |  unit  |   2m  2s |  hadoop-hdfs-client in the patch passed.  |
   | -1 :x: |  unit  |  96m  5s |  hadoop-hdfs in the patch passed.  |
   | -1 :x: |  unit  |   0m 35s |  hadoop-hdfs-rbf in the patch failed.  |
   | +1 :green_heart: |  asflicense  |   0m 46s |  The patch does not generate ASF License warnings.  |
   |  |   | 285m 46s |   |
   
   
   | Reason | Tests |
   |-------:|:------|
   | Failed junit tests | hadoop.fs.contract.hdfs.TestHDFSContractMultipartUploader |
   |   | hadoop.hdfs.TestViewDistributedFileSystem |
   |   | hadoop.hdfs.server.sps.TestExternalStoragePolicySatisfier |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.40 ServerAPI=1.40 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2235/3/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/2235 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient findbugs checkstyle cc buflint bufcompat |
   | uname | Linux 42464361b85d 4.15.0-58-generic #64-Ubuntu SMP Tue Aug 6 11:12:41 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / 83fd15b412c |
   | Default Java | Private Build-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01 |
   | mvninstall | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2235/3/artifact/out/patch-mvninstall-hadoop-hdfs-project_hadoop-hdfs-rbf.txt |
   | compile | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2235/3/artifact/out/patch-compile-root-jdkUbuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1.txt |
   | cc | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2235/3/artifact/out/patch-compile-root-jdkUbuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1.txt |
   | javac | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2235/3/artifact/out/patch-compile-root-jdkUbuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1.txt |
   | compile | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2235/3/artifact/out/patch-compile-root-jdkPrivateBuild-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01.txt |
   | cc | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2235/3/artifact/out/patch-compile-root-jdkPrivateBuild-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01.txt |
   | javac | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2235/3/artifact/out/patch-compile-root-jdkPrivateBuild-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01.txt |
   | checkstyle | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2235/3/artifact/out/diff-checkstyle-root.txt |
   | mvnsite | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2235/3/artifact/out/patch-mvnsite-hadoop-hdfs-project_hadoop-hdfs-rbf.txt |
   | javadoc | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2235/3/artifact/out/diff-javadoc-javadoc-hadoop-hdfs-project_hadoop-hdfs-client-jdkUbuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1.txt |
   | findbugs | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2235/3/artifact/out/patch-findbugs-hadoop-hdfs-project_hadoop-hdfs-rbf.txt |
   | unit | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2235/3/artifact/out/patch-unit-hadoop-hdfs-project_hadoop-hdfs.txt |
   | unit | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2235/3/artifact/out/patch-unit-hadoop-hdfs-project_hadoop-hdfs-rbf.txt |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2235/3/testReport/ |
   | Max. process+thread count | 4613 (vs. ulimit of 5500) |
   | modules | C: hadoop-common-project/hadoop-common hadoop-hdfs-project/hadoop-hdfs-client hadoop-hdfs-project/hadoop-hdfs hadoop-hdfs-project/hadoop-hdfs-rbf U: . |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2235/3/console |
   | versions | git=2.17.1 maven=3.6.0 findbugs=4.0.6 |
   | Powered by | Apache Yetus 0.13.0-SNAPSHOT https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [hadoop] yangagile commented on a change in pull request #2235: HDFS-15484 Add new method batchRename for DistributedFileSystem and W…

Posted by GitBox <gi...@apache.org>.
yangagile commented on a change in pull request #2235:
URL: https://github.com/apache/hadoop/pull/2235#discussion_r475393456



##########
File path: hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/BatchOperations.java
##########
@@ -0,0 +1,43 @@
+/*
+ * 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;
+
+import java.io.IOException;
+
+/**
+ * Interface filesystems MAY implement to offer a batched operations.
+ */
+
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
+public interface BatchOperations {
+
+  /**
+   * Batched rename API that rename a batch of files.
+   *
+   * @param srcs source file list.
+   * @param dsts target file list.
+   * @throws IOException failure exception.
+   */
+   void batchRename(String[] srcs, String[] dsts, Options.Rename... options)

Review comment:
       OK, I'll integrate this later.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [hadoop] yangagile commented on a change in pull request #2235: HDFS-15484 Add new method batchRename for DistributedFileSystem and W…

Posted by GitBox <gi...@apache.org>.
yangagile commented on a change in pull request #2235:
URL: https://github.com/apache/hadoop/pull/2235#discussion_r475385904



##########
File path: hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBatchRename.java
##########
@@ -0,0 +1,259 @@
+/**
+ * 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.hdfs;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.BatchOperations;
+import org.apache.hadoop.fs.InvalidPathException;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.protocol.BatchOpsException;
+import org.apache.hadoop.hdfs.web.WebHdfsConstants;
+import org.apache.hadoop.hdfs.web.WebHdfsFileSystem;
+import org.apache.hadoop.hdfs.web.WebHdfsTestUtil;
+import org.apache.hadoop.test.LambdaTestUtils;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+public class TestBatchRename {
+  private static MiniDFSCluster cluster;
+  private static Configuration conf;
+  private static DistributedFileSystem dfs;
+  private static WebHdfsFileSystem webHdfs;
+  private Path root = new Path("/test/batchrename/");
+
+  @BeforeClass
+  public static void beforeClass() throws Exception {
+    conf = new HdfsConfiguration();
+    cluster = new MiniDFSCluster.Builder(conf)
+        .numDataNodes(1)
+        .build();
+    dfs = cluster.getFileSystem();
+
+    webHdfs = WebHdfsTestUtil.getWebHdfsFileSystem(conf,
+        WebHdfsConstants.WEBHDFS_SCHEME);
+  }
+
+  @AfterClass
+  public static void afterClass() {
+    if (cluster != null) {
+      cluster.shutdown();
+    }
+  }
+
+  @Test
+  public void testhasPathCapability() throws Exception {
+    assertTrue("DistributedFileSystem should has batch rename capbility",
+        dfs.hasPathCapability(root, "fs.capability.batch.rename"));
+  }
+
+  private List<String> generateBatchFiles(
+      int totalNum, int createNum, final Path dir, String tag)
+      throws IOException {
+    List<String> files = new ArrayList<>();
+    for (int i = 0; i < totalNum; i++) {
+      Path p = new Path(dir, tag + "_" + i);
+      if (createNum-- > 0) {
+        DFSTestUtil.createFile(dfs, p, 10, (short) 1, 0);
+        assertTrue(dfs.exists(p));
+      } else {
+        assertFalse(dfs.exists(p));
+      }
+      files.add(p.toString());
+    }
+    return files;
+  }
+
+  private void testBatchRename(BatchOperations batchFS) throws Exception {
+    Path testDir = new Path(root, "testBatchRename");
+    assertTrue(dfs.mkdirs(testDir));
+
+    List<String> srcs = generateBatchFiles(
+        2, 2, testDir, "src");
+    List<String> dsts =generateBatchFiles(
+        2, 0, testDir, "dst");
+
+    batchFS.batchRename(
+        srcs.toArray(new String[srcs.size()]),
+        dsts.toArray(new String[dsts.size()]));
+
+    for (String f : srcs) {
+      assertFalse(dfs.exists(new Path(f)));
+    }
+    for (String f : dsts) {
+      assertTrue(dfs.exists(new Path(f)));
+      dfs.delete(new Path(f), true);
+    }
+  }
+
+  @Test
+  public void testBatchRaname() throws Exception {
+    testBatchRename(dfs);
+    testBatchRename(webHdfs);
+  }
+
+  private void testInvalidInput(BatchOperations batchFS) throws Exception {
+    List<String> srcs = new ArrayList<>();
+    srcs.add("/testInvalidInput_Mismatch");
+    List<String> dsts = new ArrayList<>();
+    LambdaTestUtils.intercept(InvalidPathException.class,
+        "mismatch batch path",
+        () -> batchFS.batchRename(
+            srcs.toArray(new String[srcs.size()]),
+            dsts.toArray(new String[dsts.size()])));
+  }
+
+  @Test
+  public void testInvalidInput() throws Exception {
+    testInvalidInput(dfs);
+    testInvalidInput(webHdfs);
+  }
+
+   // rename /src_1:/src_2(not existing) to /dst_1:/dst_2
+  private void testPartialSuccess1(BatchOperations batchFS) throws Exception {
+    Path testDir = new Path(root, "partial_success");
+    assertTrue(dfs.mkdirs(testDir));
+
+    List<String> srcs =  generateBatchFiles(
+        2, 1, testDir, "src");
+    List<String> dsts = generateBatchFiles(
+        2, 0, testDir, "dst");
+    try {
+      batchFS.batchRename(
+          srcs.toArray(new String[srcs.size()]),
+          dsts.toArray(new String[dsts.size()]));
+    } catch (BatchOpsException e) {
+      long index = e.getIndex();
+      assertEquals(1, index);
+      long total = e.getTotal();
+      assertEquals(2, total);
+
+      String reason = e.getReason();
+      assertTrue(reason.contains("FileNotFoundException"));
+
+      for (int i = 0; i < index; i++) {
+        Path p = new Path(testDir, "src_" + i);
+        assertFalse(dfs.exists(p));
+      }
+      for (int i = 0; i < index; i++) {
+        Path p = new Path(testDir, "dst_" + i);
+        assertTrue(dfs.exists(p));
+        dfs.delete(p, true);
+      }
+    }
+  }
+
+   // rename src_1:src_1/subdir to /dst_1:/dst_2
+  private void testPartialSuccess2(BatchOperations batchFS) throws Exception {
+    Path testDir = new Path(root, "partial_success");
+    List<String> srcs = new ArrayList<>();
+    Path src1 = new Path(testDir, "src_1");
+    assertTrue(dfs.mkdirs(src1));
+    srcs.add(src1.toString());
+    Path src1Subdir = new Path(src1, "subdir");
+    assertTrue(dfs.mkdirs(src1Subdir));
+    srcs.add(src1Subdir.toString());
+
+    List<String> dsts = generateBatchFiles(
+        2, 0, testDir, "dst");
+    try {
+      batchFS.batchRename(
+          srcs.toArray(new String[srcs.size()]),
+          dsts.toArray(new String[dsts.size()]));
+    } catch (BatchOpsException e) {
+      long index = e.getIndex();
+      assertEquals(1, index);
+      long total = e.getTotal();
+      assertEquals(2, total);
+      String reason = e.getReason();
+      assertTrue(reason.contains("FileNotFoundException"));
+      for (int i = 0; i < index; i++) {
+        Path p = new Path(testDir, "src_" + i);
+        assertFalse(dfs.exists(p));
+      }
+      for (int i = 0; i < index; i++) {
+        Path p = new Path(testDir, "dst_" + i);
+        assertTrue(dfs.exists(p));
+        dfs.delete(p, true);
+      }
+    }
+  }
+
+  // rename src_1:src_2 /dst_1:/dst_1
+  private void testPartialSuccess3(BatchOperations batchFS) throws Exception {
+    Path testDir = new Path(root, "partial_success_3");
+    List<String> srcs =  generateBatchFiles(
+        2, 2, testDir, "src");
+    List<String> dsts = generateBatchFiles(
+        1, 0, testDir, "dst");
+    dsts.add(dsts.get(0));
+
+    try {
+      batchFS.batchRename(
+          srcs.toArray(new String[srcs.size()]),
+          dsts.toArray(new String[dsts.size()]));
+    } catch (BatchOpsException e) {
+      long index = e.getIndex();
+      assertEquals(1, index);

Review comment:
       Done.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [hadoop] yangagile commented on a change in pull request #2235: HDFS-15484 Add new method batchRename for DistributedFileSystem and W…

Posted by GitBox <gi...@apache.org>.
yangagile commented on a change in pull request #2235:
URL: https://github.com/apache/hadoop/pull/2235#discussion_r475329083



##########
File path: hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
##########
@@ -1609,6 +1610,26 @@ public void rename(String src, String dst, Options.Rename... options)
     }
   }
 
+  /**
+   * Rename a batch files or directories.
+   * @see ClientProtocol#batchRename(String[] , String[], Options.Rename...)
+   */
+  public void batchRename(String[] srcs, String[] dsts,

Review comment:
       OK.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [hadoop] yangagile commented on a change in pull request #2235: HDFS-15484 Add new method batchRename for DistributedFileSystem and W…

Posted by GitBox <gi...@apache.org>.
yangagile commented on a change in pull request #2235:
URL: https://github.com/apache/hadoop/pull/2235#discussion_r475375334



##########
File path: hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonPathCapabilities.java
##########
@@ -139,4 +139,11 @@ private CommonPathCapabilities() {
   public static final String FS_MULTIPART_UPLOADER =
       "fs.capability.multipart.uploader";
 
+  /**
+   * Does the store support multipart uploading?

Review comment:
       Done




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [hadoop] yangagile commented on a change in pull request #2235: HDFS-15484 Add new method batchRename for DistributedFileSystem and W…

Posted by GitBox <gi...@apache.org>.
yangagile commented on a change in pull request #2235:
URL: https://github.com/apache/hadoop/pull/2235#discussion_r475374974



##########
File path: hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/BatchOpsException.java
##########
@@ -0,0 +1,87 @@
+/**
+ * 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.hdfs.protocol;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+import java.io.IOException;
+
+/**
+ * Thrown when break during a batch operation .
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public final class BatchOpsException extends IOException {
+  private static final long serialVersionUID = 1L;
+  private static final String TAG_INDEX = "index";
+  private static final String TAG_TOTAL = "total";
+  private static final String TAG_REASON = "reason";
+
+  /**
+   * Used by RemoteException to instantiate an BatchOpsException.
+   */
+  public BatchOpsException(String msg) {
+    super(msg);
+  }
+
+  public BatchOpsException(long index, long total, Throwable cause) {
+    this(index, total,
+        cause.getClass().getName() + ": " + cause.getMessage());

Review comment:
       add a more check.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [hadoop] yangagile commented on a change in pull request #2235: HDFS-15484 Add new method batchRename for DistributedFileSystem and W…

Posted by GitBox <gi...@apache.org>.
yangagile commented on a change in pull request #2235:
URL: https://github.com/apache/hadoop/pull/2235#discussion_r475373293



##########
File path: hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/BatchOpsException.java
##########
@@ -0,0 +1,87 @@
+/**
+ * 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.hdfs.protocol;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+import java.io.IOException;
+
+/**
+ * Thrown when break during a batch operation .
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public final class BatchOpsException extends IOException {
+  private static final long serialVersionUID = 1L;

Review comment:
       Done, Thanks!




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [hadoop] yangagile commented on pull request #2235: HDFS-15484 Add new method batchRename for DistributedFileSystem and W…

Posted by GitBox <gi...@apache.org>.
yangagile commented on pull request #2235:
URL: https://github.com/apache/hadoop/pull/2235#issuecomment-678966359


   > I've discussed some of what I'd like in https://issues.apache.org/jira/browse/HDFS-15484?focusedCommentId=17162752&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-17162752
   > 
   > * matching hadoop common JIRA to make clear this is going near the FS APIs
   > * something in the fileysystem spec to cover the new interface, define its semantics, etc. In particular, we need all things which are "obvious" to be written down, because often it turns out they aren't obvious at all.
   > * tests which really set out to break things. Writing the spec will help you think of them
   > 
   > Some ideas for tests
   > 
   > * renaming root
   > * rename to root
   > * rename to self
   > * path under self
   > * path above self
   > * two sources to same dest
   > * chained rename
   > * swapping paths
   > 
   > API wise, this could be our chance to fix rename properly, that is: I should be able to use this for a single rename((src, dest), opts) and have it do what I want. And as discussed, I Want something which works well with object stores
   > 
   > * use a builder to let apps specify options (see openFile()) and use the same base builder classes
   > * Return a future of the outcome. If we can get the HADOOP-16830 IOStatistics patch in first, the outome returned can be declared as it something which implement IOStatisticSource. This matters to me, as I want to know the costs of rename operations.
   > 
   > I think we should also add a rename option about atomicity; for a single rename() this would be that the rename itself is atomic. For a batch with size > 1, this means "the entire set of renames are atomic".
   > 
   > FileContext and ViewFS will also need to pass this through. Sorry.
   > 
   > One thing we could do here is actually provide a base implementation which iterates through the list/array of (src, dest) paths. This would let us add a non-atomic implementation to all filesystems/filecontexts. That would be very nice as it really would let me switch to using this API wherever we used rename(), such as distcp and MR committers.
   > 
   > rename() is that the trickiest of all FS API calls to get right. I don't think we fully understand what right is. certainly if I was asked about the nuances (src = file, dest = dir) and (src = dir, dest=dir) I'm not confident I could give an answer which is consistent for both POSIX and HDFS. This is our opportunity to make some progress here!
   > 
   > I know, this is going to add more work. But it is time.
   
   Thanks @steveloughran for the detailed introduction.
   Yes, we should do the right things, and implemnt step by step.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [hadoop] yangagile commented on a change in pull request #2235: HDFS-15484 Add new method batchRename for DistributedFileSystem and W…

Posted by GitBox <gi...@apache.org>.
yangagile commented on a change in pull request #2235:
URL: https://github.com/apache/hadoop/pull/2235#discussion_r475369420



##########
File path: hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/BatchOpsException.java
##########
@@ -0,0 +1,87 @@
+/**
+ * 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.hdfs.protocol;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+import java.io.IOException;
+
+/**
+ * Thrown when break during a batch operation .
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public final class BatchOpsException extends IOException {
+  private static final long serialVersionUID = 1L;
+  private static final String TAG_INDEX = "index";
+  private static final String TAG_TOTAL = "total";
+  private static final String TAG_REASON = "reason";
+
+  /**
+   * Used by RemoteException to instantiate an BatchOpsException.
+   */
+  public BatchOpsException(String msg) {
+    super(msg);
+  }
+
+  public BatchOpsException(long index, long total, Throwable cause) {
+    this(index, total,
+        cause.getClass().getName() + ": " + cause.getMessage());
+  }
+
+  public BatchOpsException(long index, long total,
+                           String cause) {
+    super("Batch operation break! " +

Review comment:
       change to "Batch operation partial success"




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [hadoop] hadoop-yetus removed a comment on pull request #2235: HDFS-15484. Add new method batchRename for DistributedFileSystem and W…

Posted by GitBox <gi...@apache.org>.
hadoop-yetus removed a comment on pull request #2235:
URL: https://github.com/apache/hadoop/pull/2235#issuecomment-679201173


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 30s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  No case conflicting files found.  |
   | +0 :ok: |  buf  |   0m  0s |  buf was not available.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  The patch appears to include 1 new or modified test files.  |
   ||| _ trunk Compile Tests _ |
   | +0 :ok: |  mvndep  |   3m 19s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |  26m 15s |  trunk passed  |
   | +1 :green_heart: |  compile  |  20m 28s |  trunk passed with JDK Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1  |
   | +1 :green_heart: |  compile  |  21m  8s |  trunk passed with JDK Private Build-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01  |
   | +1 :green_heart: |  checkstyle  |   3m 52s |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   5m 30s |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  27m 56s |  branch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   3m 29s |  trunk passed with JDK Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1  |
   | +1 :green_heart: |  javadoc  |   5m 39s |  trunk passed with JDK Private Build-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01  |
   | +0 :ok: |  spotbugs  |   1m 38s |  Used deprecated FindBugs config; considering switching to SpotBugs.  |
   | +1 :green_heart: |  findbugs  |  11m 39s |  trunk passed  |
   | -0 :warning: |  patch  |   2m  1s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 30s |  Maven dependency ordering for patch  |
   | -1 :x: |  mvninstall  |   0m 26s |  hadoop-hdfs-rbf in the patch failed.  |
   | -1 :x: |  compile  |   7m  7s |  root in the patch failed with JDK Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1.  |
   | -1 :x: |  cc  |   7m  7s |  root in the patch failed with JDK Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1.  |
   | -1 :x: |  javac  |   7m  7s |  root in the patch failed with JDK Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1.  |
   | -1 :x: |  compile  |   6m 21s |  root in the patch failed with JDK Private Build-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01.  |
   | -1 :x: |  cc  |   6m 21s |  root in the patch failed with JDK Private Build-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01.  |
   | -1 :x: |  javac  |   6m 21s |  root in the patch failed with JDK Private Build-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01.  |
   | -0 :warning: |  checkstyle  |   3m 39s |  root: The patch generated 4 new + 595 unchanged - 4 fixed = 599 total (was 599)  |
   | -1 :x: |  mvnsite  |   0m 28s |  hadoop-hdfs-rbf in the patch failed.  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  shadedclient  |  18m 34s |  patch has no errors when building and testing our client artifacts.  |
   | -1 :x: |  javadoc  |   0m 40s |  hadoop-hdfs-project_hadoop-hdfs-client-jdkUbuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1 with JDK Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1 generated 1 new + 0 unchanged - 0 fixed = 1 total (was 0)  |
   | +1 :green_heart: |  javadoc  |   4m 43s |  the patch passed with JDK Private Build-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01  |
   | -1 :x: |  findbugs  |   0m 27s |  hadoop-hdfs-rbf in the patch failed.  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   9m 21s |  hadoop-common in the patch passed.  |
   | +1 :green_heart: |  unit  |   2m  6s |  hadoop-hdfs-client in the patch passed.  |
   | -1 :x: |  unit  | 100m 44s |  hadoop-hdfs in the patch passed.  |
   | -1 :x: |  unit  |   0m 38s |  hadoop-hdfs-rbf in the patch failed.  |
   | +1 :green_heart: |  asflicense  |   0m 43s |  The patch does not generate ASF License warnings.  |
   |  |   | 300m 17s |   |
   
   
   | Reason | Tests |
   |-------:|:------|
   | Failed junit tests | hadoop.hdfs.server.blockmanagement.TestUnderReplicatedBlocks |
   |   | hadoop.hdfs.server.namenode.TestNameNodeRetryCacheMetrics |
   |   | hadoop.fs.contract.hdfs.TestHDFSContractMultipartUploader |
   |   | hadoop.hdfs.server.balancer.TestBalancer |
   |   | hadoop.hdfs.TestReadStripedFileWithMissingBlocks |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.40 ServerAPI=1.40 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2235/4/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/2235 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient findbugs checkstyle cc buflint bufcompat |
   | uname | Linux 11c4a9412986 4.15.0-112-generic #113-Ubuntu SMP Thu Jul 9 23:41:39 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / 5c927eb550f |
   | Default Java | Private Build-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01 |
   | mvninstall | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2235/4/artifact/out/patch-mvninstall-hadoop-hdfs-project_hadoop-hdfs-rbf.txt |
   | compile | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2235/4/artifact/out/patch-compile-root-jdkUbuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1.txt |
   | cc | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2235/4/artifact/out/patch-compile-root-jdkUbuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1.txt |
   | javac | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2235/4/artifact/out/patch-compile-root-jdkUbuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1.txt |
   | compile | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2235/4/artifact/out/patch-compile-root-jdkPrivateBuild-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01.txt |
   | cc | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2235/4/artifact/out/patch-compile-root-jdkPrivateBuild-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01.txt |
   | javac | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2235/4/artifact/out/patch-compile-root-jdkPrivateBuild-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01.txt |
   | checkstyle | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2235/4/artifact/out/diff-checkstyle-root.txt |
   | mvnsite | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2235/4/artifact/out/patch-mvnsite-hadoop-hdfs-project_hadoop-hdfs-rbf.txt |
   | javadoc | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2235/4/artifact/out/diff-javadoc-javadoc-hadoop-hdfs-project_hadoop-hdfs-client-jdkUbuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1.txt |
   | findbugs | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2235/4/artifact/out/patch-findbugs-hadoop-hdfs-project_hadoop-hdfs-rbf.txt |
   | unit | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2235/4/artifact/out/patch-unit-hadoop-hdfs-project_hadoop-hdfs.txt |
   | unit | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2235/4/artifact/out/patch-unit-hadoop-hdfs-project_hadoop-hdfs-rbf.txt |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2235/4/testReport/ |
   | Max. process+thread count | 4093 (vs. ulimit of 5500) |
   | modules | C: hadoop-common-project/hadoop-common hadoop-hdfs-project/hadoop-hdfs-client hadoop-hdfs-project/hadoop-hdfs hadoop-hdfs-project/hadoop-hdfs-rbf U: . |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2235/4/console |
   | versions | git=2.17.1 maven=3.6.0 findbugs=4.0.6 |
   | Powered by | Apache Yetus 0.13.0-SNAPSHOT https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [hadoop] hadoop-yetus commented on pull request #2235: HDFS-15484. Add new method batchRename for DistributedFileSystem and W…

Posted by GitBox <gi...@apache.org>.
hadoop-yetus commented on pull request #2235:
URL: https://github.com/apache/hadoop/pull/2235#issuecomment-679201173


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 30s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  No case conflicting files found.  |
   | +0 :ok: |  buf  |   0m  0s |  buf was not available.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  The patch appears to include 1 new or modified test files.  |
   ||| _ trunk Compile Tests _ |
   | +0 :ok: |  mvndep  |   3m 19s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |  26m 15s |  trunk passed  |
   | +1 :green_heart: |  compile  |  20m 28s |  trunk passed with JDK Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1  |
   | +1 :green_heart: |  compile  |  21m  8s |  trunk passed with JDK Private Build-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01  |
   | +1 :green_heart: |  checkstyle  |   3m 52s |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   5m 30s |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  27m 56s |  branch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   3m 29s |  trunk passed with JDK Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1  |
   | +1 :green_heart: |  javadoc  |   5m 39s |  trunk passed with JDK Private Build-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01  |
   | +0 :ok: |  spotbugs  |   1m 38s |  Used deprecated FindBugs config; considering switching to SpotBugs.  |
   | +1 :green_heart: |  findbugs  |  11m 39s |  trunk passed  |
   | -0 :warning: |  patch  |   2m  1s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 30s |  Maven dependency ordering for patch  |
   | -1 :x: |  mvninstall  |   0m 26s |  hadoop-hdfs-rbf in the patch failed.  |
   | -1 :x: |  compile  |   7m  7s |  root in the patch failed with JDK Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1.  |
   | -1 :x: |  cc  |   7m  7s |  root in the patch failed with JDK Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1.  |
   | -1 :x: |  javac  |   7m  7s |  root in the patch failed with JDK Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1.  |
   | -1 :x: |  compile  |   6m 21s |  root in the patch failed with JDK Private Build-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01.  |
   | -1 :x: |  cc  |   6m 21s |  root in the patch failed with JDK Private Build-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01.  |
   | -1 :x: |  javac  |   6m 21s |  root in the patch failed with JDK Private Build-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01.  |
   | -0 :warning: |  checkstyle  |   3m 39s |  root: The patch generated 4 new + 595 unchanged - 4 fixed = 599 total (was 599)  |
   | -1 :x: |  mvnsite  |   0m 28s |  hadoop-hdfs-rbf in the patch failed.  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  shadedclient  |  18m 34s |  patch has no errors when building and testing our client artifacts.  |
   | -1 :x: |  javadoc  |   0m 40s |  hadoop-hdfs-project_hadoop-hdfs-client-jdkUbuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1 with JDK Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1 generated 1 new + 0 unchanged - 0 fixed = 1 total (was 0)  |
   | +1 :green_heart: |  javadoc  |   4m 43s |  the patch passed with JDK Private Build-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01  |
   | -1 :x: |  findbugs  |   0m 27s |  hadoop-hdfs-rbf in the patch failed.  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   9m 21s |  hadoop-common in the patch passed.  |
   | +1 :green_heart: |  unit  |   2m  6s |  hadoop-hdfs-client in the patch passed.  |
   | -1 :x: |  unit  | 100m 44s |  hadoop-hdfs in the patch passed.  |
   | -1 :x: |  unit  |   0m 38s |  hadoop-hdfs-rbf in the patch failed.  |
   | +1 :green_heart: |  asflicense  |   0m 43s |  The patch does not generate ASF License warnings.  |
   |  |   | 300m 17s |   |
   
   
   | Reason | Tests |
   |-------:|:------|
   | Failed junit tests | hadoop.hdfs.server.blockmanagement.TestUnderReplicatedBlocks |
   |   | hadoop.hdfs.server.namenode.TestNameNodeRetryCacheMetrics |
   |   | hadoop.fs.contract.hdfs.TestHDFSContractMultipartUploader |
   |   | hadoop.hdfs.server.balancer.TestBalancer |
   |   | hadoop.hdfs.TestReadStripedFileWithMissingBlocks |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.40 ServerAPI=1.40 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2235/4/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/2235 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient findbugs checkstyle cc buflint bufcompat |
   | uname | Linux 11c4a9412986 4.15.0-112-generic #113-Ubuntu SMP Thu Jul 9 23:41:39 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / 5c927eb550f |
   | Default Java | Private Build-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01 |
   | mvninstall | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2235/4/artifact/out/patch-mvninstall-hadoop-hdfs-project_hadoop-hdfs-rbf.txt |
   | compile | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2235/4/artifact/out/patch-compile-root-jdkUbuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1.txt |
   | cc | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2235/4/artifact/out/patch-compile-root-jdkUbuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1.txt |
   | javac | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2235/4/artifact/out/patch-compile-root-jdkUbuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1.txt |
   | compile | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2235/4/artifact/out/patch-compile-root-jdkPrivateBuild-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01.txt |
   | cc | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2235/4/artifact/out/patch-compile-root-jdkPrivateBuild-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01.txt |
   | javac | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2235/4/artifact/out/patch-compile-root-jdkPrivateBuild-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01.txt |
   | checkstyle | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2235/4/artifact/out/diff-checkstyle-root.txt |
   | mvnsite | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2235/4/artifact/out/patch-mvnsite-hadoop-hdfs-project_hadoop-hdfs-rbf.txt |
   | javadoc | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2235/4/artifact/out/diff-javadoc-javadoc-hadoop-hdfs-project_hadoop-hdfs-client-jdkUbuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1.txt |
   | findbugs | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2235/4/artifact/out/patch-findbugs-hadoop-hdfs-project_hadoop-hdfs-rbf.txt |
   | unit | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2235/4/artifact/out/patch-unit-hadoop-hdfs-project_hadoop-hdfs.txt |
   | unit | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2235/4/artifact/out/patch-unit-hadoop-hdfs-project_hadoop-hdfs-rbf.txt |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2235/4/testReport/ |
   | Max. process+thread count | 4093 (vs. ulimit of 5500) |
   | modules | C: hadoop-common-project/hadoop-common hadoop-hdfs-project/hadoop-hdfs-client hadoop-hdfs-project/hadoop-hdfs hadoop-hdfs-project/hadoop-hdfs-rbf U: . |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2235/4/console |
   | versions | git=2.17.1 maven=3.6.0 findbugs=4.0.6 |
   | Powered by | Apache Yetus 0.13.0-SNAPSHOT https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [hadoop] hadoop-yetus removed a comment on pull request #2235: HDFS-15484. Add new method batchRename for DistributedFileSystem and W…

Posted by GitBox <gi...@apache.org>.
hadoop-yetus removed a comment on pull request #2235:
URL: https://github.com/apache/hadoop/pull/2235#issuecomment-678133552


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |  33m 54s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  No case conflicting files found.  |
   | +0 :ok: |  buf  |   0m  0s |  buf was not available.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  The patch appears to include 1 new or modified test files.  |
   ||| _ trunk Compile Tests _ |
   | +0 :ok: |  mvndep  |   3m 17s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |  28m  4s |  trunk passed  |
   | +1 :green_heart: |  compile  |  21m 15s |  trunk passed with JDK Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1  |
   | +1 :green_heart: |  compile  |  17m 59s |  trunk passed with JDK Private Build-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01  |
   | +1 :green_heart: |  checkstyle  |   3m  2s |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   4m 30s |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  23m 32s |  branch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   2m 53s |  trunk passed with JDK Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1  |
   | +1 :green_heart: |  javadoc  |   4m 32s |  trunk passed with JDK Private Build-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01  |
   | +0 :ok: |  spotbugs  |   1m 20s |  Used deprecated FindBugs config; considering switching to SpotBugs.  |
   | +1 :green_heart: |  findbugs  |   9m 19s |  trunk passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 23s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m 19s |  the patch passed  |
   | +1 :green_heart: |  compile  |  20m 46s |  the patch passed with JDK Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1  |
   | -1 :x: |  cc  |  20m 46s |  root-jdkUbuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1 with JDK Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1 generated 27 new + 135 unchanged - 27 fixed = 162 total (was 162)  |
   | +1 :green_heart: |  javac  |  20m 46s |  the patch passed  |
   | +1 :green_heart: |  compile  |  18m 33s |  the patch passed with JDK Private Build-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01  |
   | -1 :x: |  cc  |  18m 33s |  root-jdkPrivateBuild-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01 with JDK Private Build-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01 generated 19 new + 143 unchanged - 19 fixed = 162 total (was 162)  |
   | +1 :green_heart: |  javac  |  18m 33s |  the patch passed  |
   | -0 :warning: |  checkstyle  |   3m 15s |  root: The patch generated 3 new + 598 unchanged - 2 fixed = 601 total (was 600)  |
   | +1 :green_heart: |  mvnsite  |   4m 37s |  the patch passed  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  shadedclient  |  15m 40s |  patch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   2m 50s |  the patch passed with JDK Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1  |
   | +1 :green_heart: |  javadoc  |   4m 41s |  the patch passed with JDK Private Build-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01  |
   | +1 :green_heart: |  findbugs  |   9m 59s |  the patch passed  |
   ||| _ Other Tests _ |
   | -1 :x: |  unit  |   9m 19s |  hadoop-common in the patch passed.  |
   | +1 :green_heart: |  unit  |   2m 10s |  hadoop-hdfs-client in the patch passed.  |
   | -1 :x: |  unit  | 112m 18s |  hadoop-hdfs in the patch passed.  |
   | +1 :green_heart: |  unit  |  10m 21s |  hadoop-hdfs-rbf in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 55s |  The patch does not generate ASF License warnings.  |
   |  |   | 368m 47s |   |
   
   
   | Reason | Tests |
   |-------:|:------|
   | Failed junit tests | hadoop.security.TestRaceWhenRelogin |
   |   | hadoop.hdfs.server.namenode.TestNameNodeMXBean |
   |   | hadoop.hdfs.server.datanode.TestDataNodeUUID |
   |   | hadoop.hdfs.server.sps.TestExternalStoragePolicySatisfier |
   |   | hadoop.hdfs.server.datanode.TestBPOfferService |
   |   | hadoop.hdfs.server.blockmanagement.TestBlockTokenWithDFSStriped |
   |   | hadoop.hdfs.TestDFSInotifyEventInputStreamKerberized |
   |   | hadoop.fs.contract.hdfs.TestHDFSContractMultipartUploader |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.40 ServerAPI=1.40 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2235/1/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/2235 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient findbugs checkstyle cc buflint bufcompat |
   | uname | Linux 6db479ee01f1 4.15.0-112-generic #113-Ubuntu SMP Thu Jul 9 23:41:39 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / f734455e5d7 |
   | Default Java | Private Build-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01 |
   | cc | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2235/1/artifact/out/diff-compile-cc-root-jdkUbuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1.txt |
   | cc | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2235/1/artifact/out/diff-compile-cc-root-jdkPrivateBuild-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01.txt |
   | checkstyle | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2235/1/artifact/out/diff-checkstyle-root.txt |
   | unit | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2235/1/artifact/out/patch-unit-hadoop-common-project_hadoop-common.txt |
   | unit | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2235/1/artifact/out/patch-unit-hadoop-hdfs-project_hadoop-hdfs.txt |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2235/1/testReport/ |
   | Max. process+thread count | 3230 (vs. ulimit of 5500) |
   | modules | C: hadoop-common-project/hadoop-common hadoop-hdfs-project/hadoop-hdfs-client hadoop-hdfs-project/hadoop-hdfs hadoop-hdfs-project/hadoop-hdfs-rbf U: . |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2235/1/console |
   | versions | git=2.17.1 maven=3.6.0 findbugs=4.0.6 |
   | Powered by | Apache Yetus 0.13.0-SNAPSHOT https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [hadoop] hadoop-yetus commented on pull request #2235: HDFS-15484 Add new method batchRename for DistributedFileSystem and W…

Posted by GitBox <gi...@apache.org>.
hadoop-yetus commented on pull request #2235:
URL: https://github.com/apache/hadoop/pull/2235#issuecomment-679024058


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 31s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  No case conflicting files found.  |
   | +0 :ok: |  buf  |   0m  1s |  buf was not available.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  The patch appears to include 1 new or modified test files.  |
   ||| _ trunk Compile Tests _ |
   | +0 :ok: |  mvndep  |   3m 58s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |  31m 18s |  trunk passed  |
   | +1 :green_heart: |  compile  |  25m 27s |  trunk passed with JDK Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1  |
   | +1 :green_heart: |  compile  |  22m 14s |  trunk passed with JDK Private Build-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01  |
   | +1 :green_heart: |  checkstyle  |   3m 40s |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   5m 31s |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  27m 40s |  branch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   3m 30s |  trunk passed with JDK Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1  |
   | +1 :green_heart: |  javadoc  |   5m 14s |  trunk passed with JDK Private Build-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01  |
   | +0 :ok: |  spotbugs  |   1m 44s |  Used deprecated FindBugs config; considering switching to SpotBugs.  |
   | +1 :green_heart: |  findbugs  |  11m  7s |  trunk passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 29s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m 54s |  the patch passed  |
   | +1 :green_heart: |  compile  |  21m 49s |  the patch passed with JDK Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1  |
   | -1 :x: |  cc  |  21m 49s |  root-jdkUbuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1 with JDK Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1 generated 31 new + 131 unchanged - 31 fixed = 162 total (was 162)  |
   | +1 :green_heart: |  javac  |  21m 49s |  the patch passed  |
   | +1 :green_heart: |  compile  |  16m 55s |  the patch passed with JDK Private Build-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01  |
   | -1 :x: |  cc  |  16m 55s |  root-jdkPrivateBuild-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01 with JDK Private Build-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01 generated 37 new + 125 unchanged - 37 fixed = 162 total (was 162)  |
   | +1 :green_heart: |  javac  |  16m 55s |  the patch passed  |
   | -0 :warning: |  checkstyle  |   2m 55s |  root: The patch generated 3 new + 597 unchanged - 2 fixed = 600 total (was 599)  |
   | +1 :green_heart: |  mvnsite  |   4m 58s |  the patch passed  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  shadedclient  |  14m 22s |  patch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   3m 20s |  the patch passed with JDK Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1  |
   | +1 :green_heart: |  javadoc  |   5m 12s |  the patch passed with JDK Private Build-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01  |
   | +1 :green_heart: |  findbugs  |   9m 54s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   9m 16s |  hadoop-common in the patch passed.  |
   | +1 :green_heart: |  unit  |   2m 20s |  hadoop-hdfs-client in the patch passed.  |
   | -1 :x: |  unit  | 100m 57s |  hadoop-hdfs in the patch passed.  |
   | +1 :green_heart: |  unit  |  10m 41s |  hadoop-hdfs-rbf in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   1m  1s |  The patch does not generate ASF License warnings.  |
   |  |   | 344m 49s |   |
   
   
   | Reason | Tests |
   |-------:|:------|
   | Failed junit tests | hadoop.hdfs.server.namenode.ha.TestEditLogTailer |
   |   | hadoop.hdfs.server.blockmanagement.TestBlockTokenWithDFSStriped |
   |   | hadoop.fs.contract.hdfs.TestHDFSContractMultipartUploader |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.40 ServerAPI=1.40 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2235/2/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/2235 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient findbugs checkstyle cc buflint bufcompat |
   | uname | Linux 8b0f2abfe434 4.15.0-58-generic #64-Ubuntu SMP Tue Aug 6 11:12:41 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / 83fd15b412c |
   | Default Java | Private Build-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01 |
   | cc | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2235/2/artifact/out/diff-compile-cc-root-jdkUbuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1.txt |
   | cc | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2235/2/artifact/out/diff-compile-cc-root-jdkPrivateBuild-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01.txt |
   | checkstyle | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2235/2/artifact/out/diff-checkstyle-root.txt |
   | unit | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2235/2/artifact/out/patch-unit-hadoop-hdfs-project_hadoop-hdfs.txt |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2235/2/testReport/ |
   | Max. process+thread count | 4357 (vs. ulimit of 5500) |
   | modules | C: hadoop-common-project/hadoop-common hadoop-hdfs-project/hadoop-hdfs-client hadoop-hdfs-project/hadoop-hdfs hadoop-hdfs-project/hadoop-hdfs-rbf U: . |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2235/2/console |
   | versions | git=2.17.1 maven=3.6.0 findbugs=4.0.6 |
   | Powered by | Apache Yetus 0.13.0-SNAPSHOT https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [hadoop] hadoop-yetus commented on pull request #2235: HDFS-15484 Add new method batchRename for DistributedFileSystem and W…

Posted by GitBox <gi...@apache.org>.
hadoop-yetus commented on pull request #2235:
URL: https://github.com/apache/hadoop/pull/2235#issuecomment-678133552


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |  33m 54s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  No case conflicting files found.  |
   | +0 :ok: |  buf  |   0m  0s |  buf was not available.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  The patch appears to include 1 new or modified test files.  |
   ||| _ trunk Compile Tests _ |
   | +0 :ok: |  mvndep  |   3m 17s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |  28m  4s |  trunk passed  |
   | +1 :green_heart: |  compile  |  21m 15s |  trunk passed with JDK Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1  |
   | +1 :green_heart: |  compile  |  17m 59s |  trunk passed with JDK Private Build-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01  |
   | +1 :green_heart: |  checkstyle  |   3m  2s |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   4m 30s |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  23m 32s |  branch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   2m 53s |  trunk passed with JDK Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1  |
   | +1 :green_heart: |  javadoc  |   4m 32s |  trunk passed with JDK Private Build-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01  |
   | +0 :ok: |  spotbugs  |   1m 20s |  Used deprecated FindBugs config; considering switching to SpotBugs.  |
   | +1 :green_heart: |  findbugs  |   9m 19s |  trunk passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 23s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m 19s |  the patch passed  |
   | +1 :green_heart: |  compile  |  20m 46s |  the patch passed with JDK Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1  |
   | -1 :x: |  cc  |  20m 46s |  root-jdkUbuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1 with JDK Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1 generated 27 new + 135 unchanged - 27 fixed = 162 total (was 162)  |
   | +1 :green_heart: |  javac  |  20m 46s |  the patch passed  |
   | +1 :green_heart: |  compile  |  18m 33s |  the patch passed with JDK Private Build-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01  |
   | -1 :x: |  cc  |  18m 33s |  root-jdkPrivateBuild-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01 with JDK Private Build-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01 generated 19 new + 143 unchanged - 19 fixed = 162 total (was 162)  |
   | +1 :green_heart: |  javac  |  18m 33s |  the patch passed  |
   | -0 :warning: |  checkstyle  |   3m 15s |  root: The patch generated 3 new + 598 unchanged - 2 fixed = 601 total (was 600)  |
   | +1 :green_heart: |  mvnsite  |   4m 37s |  the patch passed  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  shadedclient  |  15m 40s |  patch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   2m 50s |  the patch passed with JDK Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1  |
   | +1 :green_heart: |  javadoc  |   4m 41s |  the patch passed with JDK Private Build-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01  |
   | +1 :green_heart: |  findbugs  |   9m 59s |  the patch passed  |
   ||| _ Other Tests _ |
   | -1 :x: |  unit  |   9m 19s |  hadoop-common in the patch passed.  |
   | +1 :green_heart: |  unit  |   2m 10s |  hadoop-hdfs-client in the patch passed.  |
   | -1 :x: |  unit  | 112m 18s |  hadoop-hdfs in the patch passed.  |
   | +1 :green_heart: |  unit  |  10m 21s |  hadoop-hdfs-rbf in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 55s |  The patch does not generate ASF License warnings.  |
   |  |   | 368m 47s |   |
   
   
   | Reason | Tests |
   |-------:|:------|
   | Failed junit tests | hadoop.security.TestRaceWhenRelogin |
   |   | hadoop.hdfs.server.namenode.TestNameNodeMXBean |
   |   | hadoop.hdfs.server.datanode.TestDataNodeUUID |
   |   | hadoop.hdfs.server.sps.TestExternalStoragePolicySatisfier |
   |   | hadoop.hdfs.server.datanode.TestBPOfferService |
   |   | hadoop.hdfs.server.blockmanagement.TestBlockTokenWithDFSStriped |
   |   | hadoop.hdfs.TestDFSInotifyEventInputStreamKerberized |
   |   | hadoop.fs.contract.hdfs.TestHDFSContractMultipartUploader |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.40 ServerAPI=1.40 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2235/1/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/2235 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient findbugs checkstyle cc buflint bufcompat |
   | uname | Linux 6db479ee01f1 4.15.0-112-generic #113-Ubuntu SMP Thu Jul 9 23:41:39 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / f734455e5d7 |
   | Default Java | Private Build-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01 |
   | cc | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2235/1/artifact/out/diff-compile-cc-root-jdkUbuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1.txt |
   | cc | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2235/1/artifact/out/diff-compile-cc-root-jdkPrivateBuild-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01.txt |
   | checkstyle | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2235/1/artifact/out/diff-checkstyle-root.txt |
   | unit | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2235/1/artifact/out/patch-unit-hadoop-common-project_hadoop-common.txt |
   | unit | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2235/1/artifact/out/patch-unit-hadoop-hdfs-project_hadoop-hdfs.txt |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2235/1/testReport/ |
   | Max. process+thread count | 3230 (vs. ulimit of 5500) |
   | modules | C: hadoop-common-project/hadoop-common hadoop-hdfs-project/hadoop-hdfs-client hadoop-hdfs-project/hadoop-hdfs hadoop-hdfs-project/hadoop-hdfs-rbf U: . |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2235/1/console |
   | versions | git=2.17.1 maven=3.6.0 findbugs=4.0.6 |
   | Powered by | Apache Yetus 0.13.0-SNAPSHOT https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [hadoop] hadoop-yetus commented on pull request #2235: HDFS-15484. Add new method batchRename for DistributedFileSystem and W…

Posted by GitBox <gi...@apache.org>.
hadoop-yetus commented on pull request #2235:
URL: https://github.com/apache/hadoop/pull/2235#issuecomment-679105788


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m 17s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  No case conflicting files found.  |
   | +0 :ok: |  buf  |   0m  1s |  buf was not available.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |  test4tests  |   0m  1s |  The patch appears to include 1 new or modified test files.  |
   ||| _ trunk Compile Tests _ |
   | +0 :ok: |  mvndep  |   3m 22s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |  30m 30s |  trunk passed  |
   | +1 :green_heart: |  compile  |  24m 15s |  trunk passed with JDK Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1  |
   | +1 :green_heart: |  compile  |  21m 10s |  trunk passed with JDK Private Build-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01  |
   | +1 :green_heart: |  checkstyle  |   3m 29s |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   5m 25s |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  23m 27s |  branch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   3m 27s |  trunk passed with JDK Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1  |
   | +1 :green_heart: |  javadoc  |   5m 16s |  trunk passed with JDK Private Build-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01  |
   | +0 :ok: |  spotbugs  |   1m 26s |  Used deprecated FindBugs config; considering switching to SpotBugs.  |
   | +1 :green_heart: |  findbugs  |   9m 27s |  trunk passed  |
   | -0 :warning: |  patch  |   1m 47s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 28s |  Maven dependency ordering for patch  |
   | -1 :x: |  mvninstall  |   0m 22s |  hadoop-hdfs-rbf in the patch failed.  |
   | -1 :x: |  compile  |   5m 20s |  root in the patch failed with JDK Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1.  |
   | -1 :x: |  cc  |   5m 20s |  root in the patch failed with JDK Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1.  |
   | -1 :x: |  javac  |   5m 20s |  root in the patch failed with JDK Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1.  |
   | -1 :x: |  compile  |   4m 47s |  root in the patch failed with JDK Private Build-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01.  |
   | -1 :x: |  cc  |   4m 47s |  root in the patch failed with JDK Private Build-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01.  |
   | -1 :x: |  javac  |   4m 47s |  root in the patch failed with JDK Private Build-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01.  |
   | -0 :warning: |  checkstyle  |   2m 36s |  root: The patch generated 6 new + 597 unchanged - 2 fixed = 603 total (was 599)  |
   | -1 :x: |  mvnsite  |   0m 26s |  hadoop-hdfs-rbf in the patch failed.  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  shadedclient  |  15m 41s |  patch has no errors when building and testing our client artifacts.  |
   | -1 :x: |  javadoc  |   0m 35s |  hadoop-hdfs-project_hadoop-hdfs-client-jdkUbuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1 with JDK Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1 generated 1 new + 0 unchanged - 0 fixed = 1 total (was 0)  |
   | +1 :green_heart: |  javadoc  |   4m  8s |  the patch passed with JDK Private Build-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01  |
   | -1 :x: |  findbugs  |   0m 26s |  hadoop-hdfs-rbf in the patch failed.  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   9m  6s |  hadoop-common in the patch passed.  |
   | +1 :green_heart: |  unit  |   2m  2s |  hadoop-hdfs-client in the patch passed.  |
   | -1 :x: |  unit  |  96m  5s |  hadoop-hdfs in the patch passed.  |
   | -1 :x: |  unit  |   0m 35s |  hadoop-hdfs-rbf in the patch failed.  |
   | +1 :green_heart: |  asflicense  |   0m 46s |  The patch does not generate ASF License warnings.  |
   |  |   | 285m 46s |   |
   
   
   | Reason | Tests |
   |-------:|:------|
   | Failed junit tests | hadoop.fs.contract.hdfs.TestHDFSContractMultipartUploader |
   |   | hadoop.hdfs.TestViewDistributedFileSystem |
   |   | hadoop.hdfs.server.sps.TestExternalStoragePolicySatisfier |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.40 ServerAPI=1.40 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2235/3/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/2235 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient findbugs checkstyle cc buflint bufcompat |
   | uname | Linux 42464361b85d 4.15.0-58-generic #64-Ubuntu SMP Tue Aug 6 11:12:41 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / 83fd15b412c |
   | Default Java | Private Build-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01 |
   | mvninstall | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2235/3/artifact/out/patch-mvninstall-hadoop-hdfs-project_hadoop-hdfs-rbf.txt |
   | compile | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2235/3/artifact/out/patch-compile-root-jdkUbuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1.txt |
   | cc | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2235/3/artifact/out/patch-compile-root-jdkUbuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1.txt |
   | javac | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2235/3/artifact/out/patch-compile-root-jdkUbuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1.txt |
   | compile | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2235/3/artifact/out/patch-compile-root-jdkPrivateBuild-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01.txt |
   | cc | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2235/3/artifact/out/patch-compile-root-jdkPrivateBuild-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01.txt |
   | javac | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2235/3/artifact/out/patch-compile-root-jdkPrivateBuild-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01.txt |
   | checkstyle | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2235/3/artifact/out/diff-checkstyle-root.txt |
   | mvnsite | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2235/3/artifact/out/patch-mvnsite-hadoop-hdfs-project_hadoop-hdfs-rbf.txt |
   | javadoc | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2235/3/artifact/out/diff-javadoc-javadoc-hadoop-hdfs-project_hadoop-hdfs-client-jdkUbuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1.txt |
   | findbugs | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2235/3/artifact/out/patch-findbugs-hadoop-hdfs-project_hadoop-hdfs-rbf.txt |
   | unit | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2235/3/artifact/out/patch-unit-hadoop-hdfs-project_hadoop-hdfs.txt |
   | unit | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2235/3/artifact/out/patch-unit-hadoop-hdfs-project_hadoop-hdfs-rbf.txt |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2235/3/testReport/ |
   | Max. process+thread count | 4613 (vs. ulimit of 5500) |
   | modules | C: hadoop-common-project/hadoop-common hadoop-hdfs-project/hadoop-hdfs-client hadoop-hdfs-project/hadoop-hdfs hadoop-hdfs-project/hadoop-hdfs-rbf U: . |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2235/3/console |
   | versions | git=2.17.1 maven=3.6.0 findbugs=4.0.6 |
   | Powered by | Apache Yetus 0.13.0-SNAPSHOT https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [hadoop] steveloughran commented on pull request #2235: HDFS-15484. Add new method batchRename for DistributedFileSystem and W…

Posted by GitBox <gi...@apache.org>.
steveloughran commented on pull request #2235:
URL: https://github.com/apache/hadoop/pull/2235#issuecomment-684941268


   Updated #743 with a better (still WiP) attempt to factor out rename/3 in FileSystem and let implementations use efficiently. Once in we can just make rename/3 public there and see what subset of the (new) test we will need. 
   
   * have a look @ the comments on the commits to see my thoughts on next steps
   * and the rename.md file is where I want to put all the rename logic.
   
   While we do this work in parallel (and me in my spare time :), you can create a rename.md file there too, just with your section....we can resolve the conflict when the time comes.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [hadoop] hadoop-yetus commented on pull request #2235: HDFS-15484. Add new method batchRename for DistributedFileSystem and W…

Posted by GitBox <gi...@apache.org>.
hadoop-yetus commented on pull request #2235:
URL: https://github.com/apache/hadoop/pull/2235#issuecomment-683852266


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |  28m 49s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  No case conflicting files found.  |
   | +0 :ok: |  buf  |   0m  1s |  buf was not available.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  The patch appears to include 1 new or modified test files.  |
   ||| _ trunk Compile Tests _ |
   | +0 :ok: |  mvndep  |   3m 21s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |  25m 58s |  trunk passed  |
   | +1 :green_heart: |  compile  |  19m 41s |  trunk passed with JDK Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1  |
   | +1 :green_heart: |  compile  |  17m  0s |  trunk passed with JDK Private Build-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01  |
   | +1 :green_heart: |  checkstyle  |   3m  0s |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   4m 58s |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  22m 43s |  branch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   3m 26s |  trunk passed with JDK Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1  |
   | +1 :green_heart: |  javadoc  |   5m 10s |  trunk passed with JDK Private Build-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01  |
   | +0 :ok: |  spotbugs  |   1m 24s |  Used deprecated FindBugs config; considering switching to SpotBugs.  |
   | +1 :green_heart: |  findbugs  |   9m 33s |  trunk passed  |
   | -0 :warning: |  patch  |   1m 46s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 27s |  Maven dependency ordering for patch  |
   | -1 :x: |  mvninstall  |   0m 22s |  hadoop-hdfs-rbf in the patch failed.  |
   | -1 :x: |  compile  |   5m 25s |  root in the patch failed with JDK Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1.  |
   | -1 :x: |  cc  |   5m 25s |  root in the patch failed with JDK Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1.  |
   | -1 :x: |  javac  |   5m 25s |  root in the patch failed with JDK Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1.  |
   | -1 :x: |  compile  |   4m 53s |  root in the patch failed with JDK Private Build-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01.  |
   | -1 :x: |  cc  |   4m 53s |  root in the patch failed with JDK Private Build-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01.  |
   | -1 :x: |  javac  |   4m 53s |  root in the patch failed with JDK Private Build-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01.  |
   | -0 :warning: |  checkstyle  |   2m 41s |  root: The patch generated 13 new + 672 unchanged - 4 fixed = 685 total (was 676)  |
   | -1 :x: |  mvnsite  |   0m 25s |  hadoop-hdfs-rbf in the patch failed.  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  shadedclient  |  13m 42s |  patch has no errors when building and testing our client artifacts.  |
   | -1 :x: |  javadoc  |   0m 36s |  hadoop-hdfs-project_hadoop-hdfs-client-jdkUbuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1 with JDK Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1 generated 1 new + 0 unchanged - 0 fixed = 1 total (was 0)  |
   | -1 :x: |  javadoc  |   1m 19s |  hadoop-common-project_hadoop-common-jdkPrivateBuild-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01 with JDK Private Build-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01 generated 6 new + 1 unchanged - 0 fixed = 7 total (was 1)  |
   | -1 :x: |  findbugs  |   0m 27s |  hadoop-hdfs-rbf in the patch failed.  |
   ||| _ Other Tests _ |
   | -1 :x: |  unit  |   9m  6s |  hadoop-common in the patch passed.  |
   | +1 :green_heart: |  unit  |   2m  5s |  hadoop-hdfs-client in the patch passed.  |
   | -1 :x: |  unit  | 108m  7s |  hadoop-hdfs in the patch passed.  |
   | -1 :x: |  unit  |   0m 36s |  hadoop-hdfs-rbf in the patch failed.  |
   | -1 :x: |  asflicense  |   0m 45s |  The patch generated 3 ASF License warnings.  |
   |  |   | 309m 16s |   |
   
   
   | Reason | Tests |
   |-------:|:------|
   | Failed junit tests | hadoop.fs.TestFilterFileSystem |
   |   | hadoop.fs.TestHarFileSystem |
   |   | hadoop.hdfs.TestFileChecksumCompositeCrc |
   |   | hadoop.hdfs.server.namenode.TestFSEditLogLoader |
   |   | hadoop.hdfs.server.sps.TestExternalStoragePolicySatisfier |
   |   | hadoop.hdfs.server.namenode.TestCacheDirectives |
   |   | hadoop.hdfs.server.namenode.snapshot.TestSnapshotDiffReport |
   |   | hadoop.hdfs.TestFileChecksum |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.40 ServerAPI=1.40 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2235/5/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/2235 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient findbugs checkstyle cc buflint bufcompat |
   | uname | Linux 7f6c2ee9c03a 4.15.0-58-generic #64-Ubuntu SMP Tue Aug 6 11:12:41 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / 60de592a883 |
   | Default Java | Private Build-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01 |
   | mvninstall | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2235/5/artifact/out/patch-mvninstall-hadoop-hdfs-project_hadoop-hdfs-rbf.txt |
   | compile | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2235/5/artifact/out/patch-compile-root-jdkUbuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1.txt |
   | cc | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2235/5/artifact/out/patch-compile-root-jdkUbuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1.txt |
   | javac | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2235/5/artifact/out/patch-compile-root-jdkUbuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1.txt |
   | compile | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2235/5/artifact/out/patch-compile-root-jdkPrivateBuild-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01.txt |
   | cc | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2235/5/artifact/out/patch-compile-root-jdkPrivateBuild-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01.txt |
   | javac | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2235/5/artifact/out/patch-compile-root-jdkPrivateBuild-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01.txt |
   | checkstyle | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2235/5/artifact/out/diff-checkstyle-root.txt |
   | mvnsite | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2235/5/artifact/out/patch-mvnsite-hadoop-hdfs-project_hadoop-hdfs-rbf.txt |
   | javadoc | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2235/5/artifact/out/diff-javadoc-javadoc-hadoop-hdfs-project_hadoop-hdfs-client-jdkUbuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1.txt |
   | javadoc | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2235/5/artifact/out/diff-javadoc-javadoc-hadoop-common-project_hadoop-common-jdkPrivateBuild-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01.txt |
   | findbugs | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2235/5/artifact/out/patch-findbugs-hadoop-hdfs-project_hadoop-hdfs-rbf.txt |
   | unit | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2235/5/artifact/out/patch-unit-hadoop-common-project_hadoop-common.txt |
   | unit | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2235/5/artifact/out/patch-unit-hadoop-hdfs-project_hadoop-hdfs.txt |
   | unit | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2235/5/artifact/out/patch-unit-hadoop-hdfs-project_hadoop-hdfs-rbf.txt |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2235/5/testReport/ |
   | asflicense | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2235/5/artifact/out/patch-asflicense-problems.txt |
   | Max. process+thread count | 5057 (vs. ulimit of 5500) |
   | modules | C: hadoop-common-project/hadoop-common hadoop-hdfs-project/hadoop-hdfs-client hadoop-hdfs-project/hadoop-hdfs hadoop-hdfs-project/hadoop-hdfs-rbf U: . |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2235/5/console |
   | versions | git=2.17.1 maven=3.6.0 findbugs=4.0.6 |
   | Powered by | Apache Yetus 0.13.0-SNAPSHOT https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [hadoop] hadoop-yetus removed a comment on pull request #2235: HDFS-15484. Add new method batchRename for DistributedFileSystem and W…

Posted by GitBox <gi...@apache.org>.
hadoop-yetus removed a comment on pull request #2235:
URL: https://github.com/apache/hadoop/pull/2235#issuecomment-679024058


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 31s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  No case conflicting files found.  |
   | +0 :ok: |  buf  |   0m  1s |  buf was not available.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  The patch appears to include 1 new or modified test files.  |
   ||| _ trunk Compile Tests _ |
   | +0 :ok: |  mvndep  |   3m 58s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |  31m 18s |  trunk passed  |
   | +1 :green_heart: |  compile  |  25m 27s |  trunk passed with JDK Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1  |
   | +1 :green_heart: |  compile  |  22m 14s |  trunk passed with JDK Private Build-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01  |
   | +1 :green_heart: |  checkstyle  |   3m 40s |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   5m 31s |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  27m 40s |  branch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   3m 30s |  trunk passed with JDK Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1  |
   | +1 :green_heart: |  javadoc  |   5m 14s |  trunk passed with JDK Private Build-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01  |
   | +0 :ok: |  spotbugs  |   1m 44s |  Used deprecated FindBugs config; considering switching to SpotBugs.  |
   | +1 :green_heart: |  findbugs  |  11m  7s |  trunk passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 29s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m 54s |  the patch passed  |
   | +1 :green_heart: |  compile  |  21m 49s |  the patch passed with JDK Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1  |
   | -1 :x: |  cc  |  21m 49s |  root-jdkUbuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1 with JDK Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1 generated 31 new + 131 unchanged - 31 fixed = 162 total (was 162)  |
   | +1 :green_heart: |  javac  |  21m 49s |  the patch passed  |
   | +1 :green_heart: |  compile  |  16m 55s |  the patch passed with JDK Private Build-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01  |
   | -1 :x: |  cc  |  16m 55s |  root-jdkPrivateBuild-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01 with JDK Private Build-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01 generated 37 new + 125 unchanged - 37 fixed = 162 total (was 162)  |
   | +1 :green_heart: |  javac  |  16m 55s |  the patch passed  |
   | -0 :warning: |  checkstyle  |   2m 55s |  root: The patch generated 3 new + 597 unchanged - 2 fixed = 600 total (was 599)  |
   | +1 :green_heart: |  mvnsite  |   4m 58s |  the patch passed  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  shadedclient  |  14m 22s |  patch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   3m 20s |  the patch passed with JDK Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1  |
   | +1 :green_heart: |  javadoc  |   5m 12s |  the patch passed with JDK Private Build-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01  |
   | +1 :green_heart: |  findbugs  |   9m 54s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   9m 16s |  hadoop-common in the patch passed.  |
   | +1 :green_heart: |  unit  |   2m 20s |  hadoop-hdfs-client in the patch passed.  |
   | -1 :x: |  unit  | 100m 57s |  hadoop-hdfs in the patch passed.  |
   | +1 :green_heart: |  unit  |  10m 41s |  hadoop-hdfs-rbf in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   1m  1s |  The patch does not generate ASF License warnings.  |
   |  |   | 344m 49s |   |
   
   
   | Reason | Tests |
   |-------:|:------|
   | Failed junit tests | hadoop.hdfs.server.namenode.ha.TestEditLogTailer |
   |   | hadoop.hdfs.server.blockmanagement.TestBlockTokenWithDFSStriped |
   |   | hadoop.fs.contract.hdfs.TestHDFSContractMultipartUploader |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.40 ServerAPI=1.40 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2235/2/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/2235 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient findbugs checkstyle cc buflint bufcompat |
   | uname | Linux 8b0f2abfe434 4.15.0-58-generic #64-Ubuntu SMP Tue Aug 6 11:12:41 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / 83fd15b412c |
   | Default Java | Private Build-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01 |
   | cc | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2235/2/artifact/out/diff-compile-cc-root-jdkUbuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1.txt |
   | cc | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2235/2/artifact/out/diff-compile-cc-root-jdkPrivateBuild-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01.txt |
   | checkstyle | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2235/2/artifact/out/diff-checkstyle-root.txt |
   | unit | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2235/2/artifact/out/patch-unit-hadoop-hdfs-project_hadoop-hdfs.txt |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2235/2/testReport/ |
   | Max. process+thread count | 4357 (vs. ulimit of 5500) |
   | modules | C: hadoop-common-project/hadoop-common hadoop-hdfs-project/hadoop-hdfs-client hadoop-hdfs-project/hadoop-hdfs hadoop-hdfs-project/hadoop-hdfs-rbf U: . |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2235/2/console |
   | versions | git=2.17.1 maven=3.6.0 findbugs=4.0.6 |
   | Powered by | Apache Yetus 0.13.0-SNAPSHOT https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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