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/12/15 06:18:43 UTC

[GitHub] [hadoop] bilaharith opened a new pull request #2548: DRAFT PR: Implementing ListStatusRemoteIterator

bilaharith opened a new pull request #2548:
URL: https://github.com/apache/hadoop/pull/2548


   This is a draft PR not ready fo review.


----------------------------------------------------------------
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] bilaharith commented on a change in pull request #2548: DRAFT PR: Implementing ListStatusRemoteIterator

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



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ListStatusRemoteIterator.java
##########
@@ -0,0 +1,129 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs.services;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.ListIterator;
+import java.util.NoSuchElementException;
+import java.util.Queue;
+import java.util.concurrent.CompletableFuture;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore;
+
+public class ListStatusRemoteIterator implements RemoteIterator<FileStatus> {
+
+  private static final Logger LOG = LoggerFactory
+      .getLogger(ListStatusRemoteIterator.class);
+
+  private static final boolean FETCH_ALL_FALSE = false;
+
+  private final Path path;
+  private final AzureBlobFileSystemStore abfsStore;
+  private final Queue<ListIterator<FileStatus>> iteratorsQueue =
+      new LinkedList<>();
+
+  private boolean firstRead = true;
+  private String continuation;
+  private ListIterator<FileStatus> currIterator;
+  private IOException ioException;
+
+  public ListStatusRemoteIterator(final Path path,
+      final AzureBlobFileSystemStore abfsStore) throws IOException {
+    this.path = path;
+    this.abfsStore = abfsStore;
+    fetchAllAsync();
+    updateCurrentIterator();
+  }
+
+  @Override
+  public boolean hasNext() throws IOException {
+    if (currIterator.hasNext()) {
+      return true;
+    }
+    updateCurrentIterator();
+    return currIterator.hasNext();
+  }
+
+  @Override
+  public FileStatus next() throws IOException {
+    if (!this.hasNext()) {
+      throw new NoSuchElementException();
+    }
+    return currIterator.next();
+  }
+
+  private void updateCurrentIterator() throws IOException {
+    synchronized (this) {
+      while (!isIterationComplete() && iteratorsQueue.isEmpty()) {
+        try {
+          this.wait();
+        } catch (InterruptedException e) {
+          Thread.currentThread().interrupt();
+          LOG.error("Thread got interrupted: {}", e);
+        }
+      }
+      if (!iteratorsQueue.isEmpty()) {
+        currIterator = iteratorsQueue.poll();
+      } else if (ioException != null) {
+        throw ioException;
+      }
+    }
+  }
+
+  private void fetchAllAsync() {
+    CompletableFuture.supplyAsync(() -> {
+      while (!isIterationComplete()) {
+        List<FileStatus> fileStatuses = new ArrayList<>();
+        try {
+          continuation = abfsStore
+              .listStatus(path, null, fileStatuses, FETCH_ALL_FALSE,
+                  continuation);
+        } catch (IOException e) {
+          ioException = e;
+          return null;
+        } finally {
+          if (firstRead) {
+            firstRead = false;
+          }
+        }
+        if (fileStatuses != null && !fileStatuses.isEmpty()) {

Review comment:
       Not doing, since the same has to be in the same synchronous block fetching the first page




----------------------------------------------------------------
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 #2548: HADOOP-17475. ABFS: Implementing ListStatusRemoteIterator

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


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |   1m 23s |  |  Docker mode activated.  |
   |||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  |  No case conflicting files found.  |
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |   |   0m  0s | [test4tests](test4tests) |  The patch appears to include 1 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |  33m 45s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |   0m 38s |  |  trunk passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04  |
   | +1 :green_heart: |  compile  |   0m 34s |  |  trunk passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01  |
   | +1 :green_heart: |  checkstyle  |   0m 28s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   0m 40s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  17m  0s |  |  branch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 30s |  |  trunk passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04  |
   | +1 :green_heart: |  javadoc  |   0m 27s |  |  trunk passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01  |
   | +0 :ok: |  spotbugs  |   1m  5s |  |  Used deprecated FindBugs config; considering switching to SpotBugs.  |
   | +1 :green_heart: |  findbugs  |   1m  1s |  |  trunk passed  |
   | -0 :warning: |  patch  |   1m 21s |  |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   |||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   0m 32s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 31s |  |  the patch passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04  |
   | +1 :green_heart: |  javac  |   0m 31s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 26s |  |  the patch passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01  |
   | +1 :green_heart: |  javac  |   0m 26s |  |  the patch passed  |
   | -0 :warning: |  checkstyle  |   0m 17s | [/diff-checkstyle-hadoop-tools_hadoop-azure.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/15/artifact/out/diff-checkstyle-hadoop-tools_hadoop-azure.txt) |  hadoop-tools/hadoop-azure: The patch generated 7 new + 4 unchanged - 0 fixed = 11 total (was 4)  |
   | +1 :green_heart: |  mvnsite  |   0m 30s |  |  the patch passed  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  shadedclient  |  15m 17s |  |  patch has no errors when building and testing our client artifacts.  |
   | -1 :x: |  javadoc  |   0m 25s | [/diff-javadoc-javadoc-hadoop-tools_hadoop-azure-jdkUbuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/15/artifact/out/diff-javadoc-javadoc-hadoop-tools_hadoop-azure-jdkUbuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04.txt) |  hadoop-tools_hadoop-azure-jdkUbuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04 with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04 generated 3 new + 15 unchanged - 2 fixed = 18 total (was 17)  |
   | -1 :x: |  javadoc  |   0m 24s | [/diff-javadoc-javadoc-hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/15/artifact/out/diff-javadoc-javadoc-hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01.txt) |  hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01 with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01 generated 3 new + 15 unchanged - 2 fixed = 18 total (was 17)  |
   | -1 :x: |  findbugs  |   1m  7s | [/new-findbugs-hadoop-tools_hadoop-azure.html](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/15/artifact/out/new-findbugs-hadoop-tools_hadoop-azure.html) |  hadoop-tools/hadoop-azure generated 2 new + 0 unchanged - 0 fixed = 2 total (was 0)  |
   |||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   2m  1s |  |  hadoop-azure in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 34s |  |  The patch does not generate ASF License warnings.  |
   |  |   |  80m 36s |  |  |
   
   
   | Reason | Tests |
   |-------:|:------|
   | FindBugs | module:hadoop-tools/hadoop-azure |
   |  |  Inconsistent synchronization of org.apache.hadoop.fs.azurebfs.services.AbfsListStatusRemoteIterator.continuation; locked 50% of time  Unsynchronized access at AbfsListStatusRemoteIterator.java:50% of time  Unsynchronized access at AbfsListStatusRemoteIterator.java:[line 145] |
   |  |  Inconsistent synchronization of org.apache.hadoop.fs.azurebfs.services.AbfsListStatusRemoteIterator.isIterationComplete; locked 50% of time  Unsynchronized access at AbfsListStatusRemoteIterator.java:50% of time  Unsynchronized access at AbfsListStatusRemoteIterator.java:[line 82] |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/15/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/2548 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient findbugs checkstyle |
   | uname | Linux bf7511d4d1c8 4.15.0-65-generic #74-Ubuntu SMP Tue Sep 17 17:06:04 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / 06a5d3437f6 |
   | Default Java | Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01 |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/15/testReport/ |
   | Max. process+thread count | 669 (vs. ulimit of 5500) |
   | modules | C: hadoop-tools/hadoop-azure U: hadoop-tools/hadoop-azure |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/15/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] bilaharith commented on pull request #2548: HADOOP-17475. ABFS: Implementing ListStatusRemoteIterator

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


   HNS-OAuth-IteratorOff
   ========================
   [INFO] Results:
   [INFO] 
   [INFO] Tests run: 90, Failures: 0, Errors: 0, Skipped: 0
   [INFO] Results:
   [INFO] 
   [WARNING] Tests run: 507, Failures: 0, Errors: 0, Skipped: 68
   [INFO] Results:
   [INFO] 
   [WARNING] Tests run: 256, Failures: 0, Errors: 0, Skipped: 48
   
   HNS-OAuth
   ========================
   [INFO] Results:
   [INFO] 
   [INFO] Tests run: 90, Failures: 0, Errors: 0, Skipped: 0
   [INFO] Results:
   [INFO] 
   [WARNING] Tests run: 507, Failures: 0, Errors: 0, Skipped: 68
   [INFO] Results:
   [INFO] 
   [WARNING] Tests run: 256, Failures: 0, Errors: 0, Skipped: 48
   
   HNS-SharedKey
   ========================[INFO] Results:
   [INFO] 
   [INFO] Tests run: 90, Failures: 0, Errors: 0, Skipped: 0
   [INFO] Results:
   [INFO] 
   [WARNING] Tests run: 507, Failures: 0, Errors: 0, Skipped: 24
   [INFO] Results:
   [INFO] 
   [WARNING] Tests run: 256, Failures: 0, Errors: 0, Skipped: 40
   
   NonHNS-SharedKey
   ========================
   [INFO] Results:
   [INFO] 
   [INFO] Tests run: 90, Failures: 0, Errors: 0, Skipped: 0
   [INFO] Results:
   [INFO] 
   [WARNING] Tests run: 507, Failures: 0, Errors: 0, Skipped: 249
   [INFO] Results:
   [INFO] 
   [WARNING] Tests run: 256, Failures: 0, Errors: 0, Skipped: 40
   


----------------------------------------------------------------
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] bilaharith commented on pull request #2548: HADOOP-17475. ABFS: Implementing ListStatusRemoteIterator

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


   HNS-OAuth-IteratorOff
   ========================
   [INFO] Results:
   [INFO] 
   [INFO] Tests run: 90, Failures: 0, Errors: 0, Skipped: 0
   [INFO] Results:
   [INFO] 
   [WARNING] Tests run: 507, Failures: 0, Errors: 0, Skipped: 68
   [INFO] Results:
   [INFO] 
   [WARNING] Tests run: 256, Failures: 0, Errors: 0, Skipped: 48
   
   HNS-OAuth
   ========================
   [INFO] Results:
   [INFO] 
   [INFO] Tests run: 90, Failures: 0, Errors: 0, Skipped: 0
   [INFO] Results:
   [INFO] 
   [WARNING] Tests run: 507, Failures: 0, Errors: 0, Skipped: 68
   [INFO] Results:
   [INFO] 
   [WARNING] Tests run: 256, Failures: 0, Errors: 0, Skipped: 48
   
   HNS-SharedKey
   ========================[INFO] Results:
   [INFO] 
   [INFO] Tests run: 90, Failures: 0, Errors: 0, Skipped: 0
   [INFO] Results:
   [INFO] 
   [WARNING] Tests run: 507, Failures: 0, Errors: 0, Skipped: 24
   [INFO] Results:
   [INFO] 
   [WARNING] Tests run: 256, Failures: 0, Errors: 0, Skipped: 40
   
   NonHNS-SharedKey
   ========================
   [INFO] Results:
   [INFO] 
   [INFO] Tests run: 90, Failures: 0, Errors: 0, Skipped: 0
   [INFO] Results:
   [INFO] 
   [WARNING] Tests run: 507, Failures: 0, Errors: 0, Skipped: 249
   [INFO] Results:
   [INFO] 
   [WARNING] Tests run: 256, Failures: 0, Errors: 0, Skipped: 40
   


----------------------------------------------------------------
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] snvijaya commented on a change in pull request #2548: DRAFT PR: Implementing ListStatusRemoteIterator

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



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ListStatusRemoteIterator.java
##########
@@ -0,0 +1,149 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs.services;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.CompletableFuture;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore;
+
+public class ListStatusRemoteIterator implements RemoteIterator<FileStatus> {
+
+  private static final Logger LOG = LoggerFactory
+      .getLogger(ListStatusRemoteIterator.class);
+
+  private static final boolean FETCH_ALL_FALSE = false;
+  private static final int MAX_QUEUE_SIZE = 10;
+
+  private final Path path;
+  private final AzureBlobFileSystemStore abfsStore;
+  private final ArrayBlockingQueue<Iterator<FileStatus>> iteratorsQueue;
+  private final Object asyncOpLock = new Object();
+
+  private boolean firstBatch = true;
+  private boolean isAsyncInProgress = false;
+  private String continuation;
+  private Iterator<FileStatus> currIterator;
+  private IOException ioException;
+
+  public ListStatusRemoteIterator(final Path path,
+      final AzureBlobFileSystemStore abfsStore) {
+    this.path = path;
+    this.abfsStore = abfsStore;
+    iteratorsQueue = new ArrayBlockingQueue<>(MAX_QUEUE_SIZE);
+    currIterator = Collections.emptyIterator();
+    fetchBatchesAsync();
+  }
+
+  @Override
+  public boolean hasNext() throws IOException {
+    if (currIterator.hasNext()) {
+      return true;
+    }
+    updateCurrentIterator();
+    return currIterator.hasNext();
+  }
+
+  @Override
+  public FileStatus next() throws IOException {
+    if (!this.hasNext()) {
+      throw new NoSuchElementException();
+    }
+    return currIterator.next();
+  }
+
+  private void updateCurrentIterator() throws IOException {
+    fetchBatchesAsync();
+    synchronized (this) {
+      if (iteratorsQueue.isEmpty()) {
+        if (ioException != null) {
+          throw ioException;
+        }
+        if (isListingComplete()) {
+          return;
+        }
+      }
+    }
+    try {
+      currIterator = iteratorsQueue.take();
+      if (!currIterator.hasNext() && !isListingComplete()) {
+        updateCurrentIterator();
+      }
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();

Review comment:
       We are here because the thread already got interrupted ? why call interrupt ?




----------------------------------------------------------------
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] vinaysbadami commented on a change in pull request #2548: DRAFT PR: Implementing ListStatusRemoteIterator

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



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ListStatusRemoteIterator.java
##########
@@ -0,0 +1,149 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs.services;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.CompletableFuture;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore;
+
+public class ListStatusRemoteIterator implements RemoteIterator<FileStatus> {
+
+  private static final Logger LOG = LoggerFactory
+      .getLogger(ListStatusRemoteIterator.class);
+
+  private static final boolean FETCH_ALL_FALSE = false;
+  private static final int MAX_QUEUE_SIZE = 10;
+
+  private final Path path;
+  private final AzureBlobFileSystemStore abfsStore;
+  private final ArrayBlockingQueue<Iterator<FileStatus>> iteratorsQueue;
+  private final Object asyncOpLock = new Object();
+
+  private boolean firstBatch = true;
+  private boolean isAsyncInProgress = false;
+  private String continuation;
+  private Iterator<FileStatus> currIterator;
+  private IOException ioException;
+
+  public ListStatusRemoteIterator(final Path path,
+      final AzureBlobFileSystemStore abfsStore) {
+    this.path = path;
+    this.abfsStore = abfsStore;
+    iteratorsQueue = new ArrayBlockingQueue<>(MAX_QUEUE_SIZE);
+    currIterator = Collections.emptyIterator();
+    fetchBatchesAsync();
+  }
+
+  @Override
+  public boolean hasNext() throws IOException {
+    if (currIterator.hasNext()) {
+      return true;
+    }
+    updateCurrentIterator();
+    return currIterator.hasNext();
+  }
+
+  @Override
+  public FileStatus next() throws IOException {
+    if (!this.hasNext()) {
+      throw new NoSuchElementException();
+    }
+    return currIterator.next();
+  }
+
+  private void updateCurrentIterator() throws IOException {
+    fetchBatchesAsync();
+    synchronized (this) {
+      if (iteratorsQueue.isEmpty()) {
+        if (ioException != null) {

Review comment:
       I thought we agreed that we will throw exception at correct place?




----------------------------------------------------------------
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 #2548: HADOOP-17475. ABFS: Implementing ListStatusRemoteIterator

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






----------------------------------------------------------------
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 #2548: DRAFT PR: Implementing ListStatusRemoteIterator

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


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |   1m 30s |  |  Docker mode activated.  |
   |||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  |  No case conflicting files found.  |
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |   |   0m  0s | [test4tests](test4tests) |  The patch appears to include 1 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |  33m 53s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |   0m 36s |  |  trunk passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04  |
   | +1 :green_heart: |  compile  |   0m 34s |  |  trunk passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01  |
   | +1 :green_heart: |  checkstyle  |   0m 26s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   0m 37s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  16m 22s |  |  branch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 29s |  |  trunk passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04  |
   | +1 :green_heart: |  javadoc  |   0m 29s |  |  trunk passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01  |
   | +0 :ok: |  spotbugs  |   1m  1s |  |  Used deprecated FindBugs config; considering switching to SpotBugs.  |
   | +1 :green_heart: |  findbugs  |   0m 58s |  |  trunk passed  |
   |||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   0m 30s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 29s |  |  the patch passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04  |
   | +1 :green_heart: |  javac  |   0m 29s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 26s |  |  the patch passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01  |
   | +1 :green_heart: |  javac  |   0m 26s |  |  the patch passed  |
   | +1 :green_heart: |  checkstyle  |   0m 17s |  |  the patch passed  |
   | +1 :green_heart: |  mvnsite  |   0m 29s |  |  the patch passed  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  shadedclient  |  14m 59s |  |  patch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 26s |  |  the patch passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04  |
   | +1 :green_heart: |  javadoc  |   0m 25s |  |  the patch passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01  |
   | -1 :x: |  findbugs  |   0m 59s | [/new-findbugs-hadoop-tools_hadoop-azure.html](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/7/artifact/out/new-findbugs-hadoop-tools_hadoop-azure.html) |  hadoop-tools/hadoop-azure generated 4 new + 0 unchanged - 0 fixed = 4 total (was 0)  |
   |||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   1m 27s |  |  hadoop-azure in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 34s |  |  The patch does not generate ASF License warnings.  |
   |  |   |  79m 10s |  |  |
   
   
   | Reason | Tests |
   |-------:|:------|
   | FindBugs | module:hadoop-tools/hadoop-azure |
   |  |  Possible doublecheck on org.apache.hadoop.fs.azurebfs.services.ListStatusRemoteIterator.isAsyncInProgress in org.apache.hadoop.fs.azurebfs.services.ListStatusRemoteIterator.asyncOp()  At ListStatusRemoteIterator.java:org.apache.hadoop.fs.azurebfs.services.ListStatusRemoteIterator.asyncOp()  At ListStatusRemoteIterator.java:[lines 115-119] |
   |  |  Inconsistent synchronization of org.apache.hadoop.fs.azurebfs.services.ListStatusRemoteIterator.continuation; locked 40% of time  Unsynchronized access at ListStatusRemoteIterator.java:40% of time  Unsynchronized access at ListStatusRemoteIterator.java:[line 107] |
   |  |  Inconsistent synchronization of org.apache.hadoop.fs.azurebfs.services.ListStatusRemoteIterator.firstBatch; locked 66% of time  Unsynchronized access at ListStatusRemoteIterator.java:66% of time  Unsynchronized access at ListStatusRemoteIterator.java:[line 107] |
   |  |  Inconsistent synchronization of org.apache.hadoop.fs.azurebfs.services.ListStatusRemoteIterator.ioException; locked 66% of time  Unsynchronized access at ListStatusRemoteIterator.java:66% of time  Unsynchronized access at ListStatusRemoteIterator.java:[line 129] |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/7/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/2548 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient findbugs checkstyle |
   | uname | Linux ad17f0141ddd 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 / ec22850dbef |
   | Default Java | Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01 |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/7/testReport/ |
   | Max. process+thread count | 733 (vs. ulimit of 5500) |
   | modules | C: hadoop-tools/hadoop-azure U: hadoop-tools/hadoop-azure |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/7/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] bilaharith commented on pull request #2548: DRAFT PR: Implementing ListStatusRemoteIterator

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


   > I fear you are the same mistake we did in the S3A codebase: giving all the helper classes a reference back to the ABFS Store class, so making them too intermingled. It is unsustainable. See: https://github.com/steveloughran/engineering-proposals/blob/trunk/refactoring-s3a.md
   > 
   > Proposed
   > 
   > * Use a specific listing callback which the store can directly/indirectly implement: org.apache.hadoop.fs.s3a.impl.ListingOperationCallbacks.
   > * This will also help you write unit tests against a stub implementation without having to use reflection and manipulating access modifiers in a way that is over-complex and brittle.
   > * Use IOStatistics API to track duration of calls
   > * and serve this up. It is really interesting to know how long lists take
   > * add a close() operator to cancel queue & wait for completion.
   > 
   > the other incremental listX calls (listFileStatus, listLocatedStatus) are all similar and should be done in the same uber-JIRA. listLocatedStatus is used in LocatedFileStatusFetcher, which is used during MR and spark file scanning. listFileStatus should be used more for deep tree scans.
   > 
   > I'd also prefer if you use a shared thread pool of that ABFS store instance
   > 
   > * stops many, many list iterators overloading things
   > * may offer faster startup
   > * custom thread names help identify origin of less-helpful stack traces
   > * could have a gauge on the instance to measure pool load
   > * filesystem.close() can interrupt the pool to shut things down.
   > 
   > Have a look @ org.apache.hadoop.fs.s3a.Listing in trunk to see what's been done there. There's a lot you don't need (S3Guard reconciliation), and it currently only prefetches the next page of results. But it does include the stats collection, a restricted callback to the FS, use of org.apache.hadoop.util.functional.RemoteIterators for functional-programming style use of RemoteIterator classes. Take a look at it's {{org.apache.hadoop.util.functional.RemoteIterators#foreach}} method too.
   > 
   > I know, I'm giving extra homework. But I'll only keep asking for these, so better to start now.
   
   I will raise the JIRA as suggested. For now keeping the PR as a draft.


----------------------------------------------------------------
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] vinaysbadami commented on a change in pull request #2548: HADOOP-17475. ABFS: Implementing ListStatusRemoteIterator

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



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsListStatusRemoteIterator.java
##########
@@ -0,0 +1,159 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs.services;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.CompletableFuture;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.RemoteIterator;
+
+public class AbfsListStatusRemoteIterator implements RemoteIterator<FileStatus> {
+
+  private static final Logger LOG = LoggerFactory
+      .getLogger(AbfsListStatusRemoteIterator.class);
+
+  private static final boolean FETCH_ALL_FALSE = false;
+  private static final int MAX_QUEUE_SIZE = 10;
+
+  private final FileStatus fileStatus;
+  private final ListingSupport listingSupport;
+  private final ArrayBlockingQueue<Object> iteratorsQueue;
+  private final Object asyncOpLock = new Object();

Review comment:
       can we avoid this completely?




----------------------------------------------------------------
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] bilaharith commented on pull request #2548: HADOOP-17475. ABFS: Implementing ListStatusRemoteIterator

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


   Hi @steveloughran 
   I have created a separate JIRA for IOStatistics collection and linked the same with HADOOP-17475. We will be picking the same afterwards.


----------------------------------------------------------------
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] bilaharith commented on a change in pull request #2548: DRAFT PR: Implementing ListStatusRemoteIterator

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



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ListStatusRemoteIterator.java
##########
@@ -0,0 +1,149 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs.services;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.CompletableFuture;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore;
+
+public class ListStatusRemoteIterator implements RemoteIterator<FileStatus> {
+
+  private static final Logger LOG = LoggerFactory
+      .getLogger(ListStatusRemoteIterator.class);
+
+  private static final boolean FETCH_ALL_FALSE = false;
+  private static final int MAX_QUEUE_SIZE = 10;
+
+  private final Path path;
+  private final AzureBlobFileSystemStore abfsStore;
+  private final ArrayBlockingQueue<Iterator<FileStatus>> iteratorsQueue;
+  private final Object asyncOpLock = new Object();
+
+  private boolean firstBatch = true;
+  private boolean isAsyncInProgress = false;
+  private String continuation;
+  private Iterator<FileStatus> currIterator;
+  private IOException ioException;
+
+  public ListStatusRemoteIterator(final Path path,
+      final AzureBlobFileSystemStore abfsStore) {
+    this.path = path;
+    this.abfsStore = abfsStore;
+    iteratorsQueue = new ArrayBlockingQueue<>(MAX_QUEUE_SIZE);
+    currIterator = Collections.emptyIterator();
+    fetchBatchesAsync();
+  }
+
+  @Override
+  public boolean hasNext() throws IOException {
+    if (currIterator.hasNext()) {
+      return true;
+    }
+    updateCurrentIterator();

Review comment:
       If there is new iterators present in the queue it updates the currentIterator to the one from the queue. Otherwise currentIterator is not updated and the hasNext will be called on the existing iterator which will return false.




----------------------------------------------------------------
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] bilaharith commented on a change in pull request #2548: DRAFT PR: Implementing ListStatusRemoteIterator

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



##########
File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemListStatusIterator.java
##########
@@ -0,0 +1,190 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs;
+
+import java.io.IOException;
+import java.lang.reflect.Field;
+import java.lang.reflect.Modifier;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+
+import org.assertj.core.api.Assertions;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+import org.apache.hadoop.fs.azurebfs.services.ListStatusRemoteIterator;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+
+/**
+ * Test listStatus operation.
+ */
+public class ITestAzureBlobFileSystemListStatusIterator
+    extends AbstractAbfsIntegrationTest {
+
+  private static final int TEST_FILES_NUMBER = 1000;
+
+  public ITestAzureBlobFileSystemListStatusIterator() throws Exception {
+    super();
+  }
+
+  @Test
+  public void testListPath() throws Exception {
+    final AzureBlobFileSystem fs = getFileSystem();
+    String rootPath = "testRoot1";
+    final List<String> fileNames = createFiles(TEST_FILES_NUMBER, rootPath,
+        "testListPath");
+    AzureBlobFileSystemStore abfsStore = getAbfsStore(fs);
+    abfsStore.getAbfsConfiguration().setListMaxResults(10);
+    RemoteIterator<FileStatus> fsIt = fs.listStatusIterator(new Path(rootPath));
+    int itrCount = 0;
+    while (fsIt.hasNext()) {
+      FileStatus fileStatus = fsIt.next();
+      String pathStr = fileStatus.getPath().toString();
+      fileNames.remove(pathStr);
+      itrCount++;
+    }
+    assertEquals(TEST_FILES_NUMBER, itrCount);
+    assertEquals(0, fileNames.size());
+  }
+
+  @Test
+  public void testNextWhenNoMoreElementsPresent() throws Exception {
+    final AzureBlobFileSystem fs = getFileSystem();
+    String rootPathStr = "testRoot2";
+    Path rootPath = new Path(rootPathStr);
+    getFileSystem().mkdirs(rootPath);
+    RemoteIterator<FileStatus> fsItr = fs.listStatusIterator(rootPath);
+    fsItr = Mockito.spy(fsItr);
+    Mockito.doReturn(false).when(fsItr).hasNext();
+
+    RemoteIterator<FileStatus> finalFsItr = fsItr;
+    Assertions.assertThatThrownBy(() -> finalFsItr.next()).describedAs(
+        "next() should throw NoSuchElementException if hasNext() return "
+            + "false").isInstanceOf(NoSuchElementException.class);
+  }
+
+  @Test
+  public void testHasNextForEmptyDir() throws Exception {
+    final AzureBlobFileSystem fs = getFileSystem();
+    String rootPathStr = "testRoot3";
+    Path rootPath = new Path(rootPathStr);
+    getFileSystem().mkdirs(rootPath);
+    RemoteIterator<FileStatus> fsItr = fs.listStatusIterator(rootPath);
+    Assertions.assertThat(fsItr.hasNext())
+        .describedAs("hasNext returns false for empty directory").isFalse();
+  }
+
+  @Test
+  public void testHasNextForFile() throws Exception {
+    final AzureBlobFileSystem fs = getFileSystem();
+    String rootPathStr = "testRoot4";
+    Path rootPath = new Path(rootPathStr);
+    getFileSystem().create(rootPath);
+    RemoteIterator<FileStatus> fsItr = fs.listStatusIterator(rootPath);
+    Assertions.assertThat(fsItr.hasNext())
+        .describedAs("hasNext returns true for file").isTrue();
+  }
+
+  @Test
+  public void testHasNextForIOException() throws Exception {
+    final AzureBlobFileSystem fs = getFileSystem();
+    String rootPathStr = "testRoot5";
+    Path rootPath = new Path(rootPathStr);
+    getFileSystem().mkdirs(rootPath);
+    ListStatusRemoteIterator fsItr = (ListStatusRemoteIterator) fs
+        .listStatusIterator(rootPath);
+    Thread.sleep(1000);
+
+    String exceptionMessage = "test exception";
+    setPrivateField(fsItr, ListStatusRemoteIterator.class, "ioException",
+        new IOException(exceptionMessage));
+    setPrivateFinalField(fsItr, ListStatusRemoteIterator.class,

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] bilaharith commented on pull request #2548: HADOOP-17475. ABFS: Implementing ListStatusRemoteIterator

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


   > very close to getting in. other than some minor checkstyle/javadoc complaints, that findbugs needs to be made to stop complaining. Usually it is indicating a real risk of a problem, so I'd like to see what can be done about it -rather than just edit the XML file to turn off the check
   
   There is one one findbugs warning remaining which is of medium priority. The same can be ignored since at line 147 the continuation token returned by the ListingOperation ouside the synchroniced lock since the same involve an http call.


----------------------------------------------------------------
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 #2548: DRAFT PR: Implementing ListStatusRemoteIterator

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


   we should talk about this in 2021. For now 
   
   * see #2553 for IOStatistics collection *including in remote iterators*, and a class *RemoteIterators* to help you wrap them
   * look @ mukund's work HADOOP-17400 including the issue of when to report failures
   
   
   I think it makes sense to have an over all "optimise abfs incremental listings" JIRA and create issues underneath, as a lot is unified.


----------------------------------------------------------------
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] bilaharith commented on a change in pull request #2548: HADOOP-17475. ABFS: Implementing ListStatusRemoteIterator

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



##########
File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsListStatusIterator.java
##########
@@ -0,0 +1,339 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+
+import org.assertj.core.api.Assertions;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.fs.azurebfs.services.AbfsListStatusRemoteIterator;
+import org.apache.hadoop.fs.azurebfs.services.ListingSupport;
+
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.anyBoolean;
+import static org.mockito.ArgumentMatchers.anyList;
+import static org.mockito.ArgumentMatchers.nullable;
+import static org.mockito.Mockito.verify;
+
+/**
+ * Test ListStatusRemoteIterator operation.
+ */
+public class ITestAbfsListStatusIterator extends AbstractAbfsIntegrationTest {
+
+  private static final int TEST_FILES_NUMBER = 1000;
+
+  public ITestAbfsListStatusIterator() throws Exception {
+    super();
+  }
+
+  @Test
+  public void testListStatusRemoteIterator() throws Exception {
+    Path testDir = createTestDirectory();
+    setPageSize(10);
+    final List<String> fileNames = createFilesUnderDirectory(TEST_FILES_NUMBER,
+        testDir, "testListPath");
+
+    ListingSupport listngSupport = Mockito.spy(getFileSystem().getAbfsStore());
+    RemoteIterator<FileStatus> fsItr = new AbfsListStatusRemoteIterator(
+        getFileSystem().getFileStatus(testDir), listngSupport);
+    Assertions.assertThat(fsItr)
+        .describedAs("RemoteIterator should be instance of "
+            + "AbfsListStatusRemoteIterator by default")
+        .isInstanceOf(AbfsListStatusRemoteIterator.class);
+    int itrCount = 0;
+    while (fsItr.hasNext()) {
+      FileStatus fileStatus = fsItr.next();
+      String pathStr = fileStatus.getPath().toString();
+      fileNames.remove(pathStr);
+      itrCount++;
+    }
+    Assertions.assertThat(itrCount)
+        .describedAs("Number of iterations should be equal to the files "
+            + "created")
+        .isEqualTo(TEST_FILES_NUMBER);
+    Assertions.assertThat(fileNames.size())
+        .describedAs("After removing every iterm found from the iterator, "
+            + "there should be no more elements in the fileNames")
+        .isEqualTo(0);
+    verify(listngSupport, Mockito.atLeast(100))
+        .listStatus(any(Path.class), nullable(String.class),
+            anyList(), anyBoolean(),
+            nullable(String.class));
+  }
+
+  @Test
+  public void testListStatusRemoteIteratorWithoutHasNext() throws Exception {
+    Path testDir = createTestDirectory();
+    setPageSize(10);
+    final List<String> fileNames = createFilesUnderDirectory(TEST_FILES_NUMBER,
+        testDir, "testListPath");
+
+    ListingSupport listngSupport = Mockito.spy(getFileSystem().getAbfsStore());
+    RemoteIterator<FileStatus> fsItr = new AbfsListStatusRemoteIterator(
+        getFileSystem().getFileStatus(testDir), listngSupport);
+    Assertions.assertThat(fsItr)
+        .describedAs("RemoteIterator should be instance of "
+            + "AbfsListStatusRemoteIterator by default")
+        .isInstanceOf(AbfsListStatusRemoteIterator.class);
+    int itrCount = 0;
+    for (int i = 0; i < TEST_FILES_NUMBER; i++) {
+      FileStatus fileStatus = fsItr.next();
+      String pathStr = fileStatus.getPath().toString();
+      fileNames.remove(pathStr);
+      itrCount++;
+    }
+    Assertions.assertThatThrownBy(() -> fsItr.next())
+        .describedAs(
+            "next() should throw NoSuchElementException since next has been "
+                + "called " + TEST_FILES_NUMBER + " times")
+        .isInstanceOf(NoSuchElementException.class);
+    Assertions.assertThat(itrCount)
+        .describedAs("Number of iterations should be equal to the files "
+            + "created")
+        .isEqualTo(TEST_FILES_NUMBER);
+    Assertions.assertThat(fileNames.size())
+        .describedAs("After removing every iterm found from the iterator, "
+            + "there should be no more elements in the fileNames")
+        .isEqualTo(0);
+    verify(listngSupport, Mockito.atLeast(100))
+        .listStatus(any(Path.class), nullable(String.class),
+            anyList(), anyBoolean(),
+            nullable(String.class));
+  }
+
+  @Test
+  public void testWithAbfsIteratorDisabled() throws Exception {
+    Path testDir = createTestDirectory();
+    setPageSize(10);
+    setEnableAbfsIterator(false);
+    final List<String> fileNames = createFilesUnderDirectory(TEST_FILES_NUMBER,
+        testDir, "testListPath");
+
+    RemoteIterator<FileStatus> fsItr =
+        getFileSystem().listStatusIterator(testDir);
+    Assertions.assertThat(fsItr)
+        .describedAs("RemoteIterator should not be instance of "
+            + "AbfsListStatusRemoteIterator when it is disabled")
+        .isNotInstanceOf(AbfsListStatusRemoteIterator.class);
+    int itrCount = 0;
+    while (fsItr.hasNext()) {
+      FileStatus fileStatus = fsItr.next();
+      String pathStr = fileStatus.getPath().toString();
+      fileNames.remove(pathStr);
+      itrCount++;
+    }
+    Assertions.assertThat(itrCount)
+        .describedAs("Number of iterations should be equal to the files "
+            + "created")
+        .isEqualTo(TEST_FILES_NUMBER);
+    Assertions.assertThat(fileNames.size())
+        .describedAs("After removing every iterm found from the iterator, "
+            + "there should be no more elements in the fileNames")
+        .isEqualTo(0);
+  }
+
+  @Test
+  public void testWithAbfsIteratorDisabledWithutHasNext() throws Exception {
+    Path testDir = createTestDirectory();
+    setPageSize(10);
+    setEnableAbfsIterator(false);
+    final List<String> fileNames = createFilesUnderDirectory(TEST_FILES_NUMBER,
+        testDir, "testListPath");
+
+    RemoteIterator<FileStatus> fsItr =
+        getFileSystem().listStatusIterator(testDir);
+    Assertions.assertThat(fsItr)
+        .describedAs("RemoteIterator should not be instance of "
+            + "AbfsListStatusRemoteIterator when it is disabled")
+        .isNotInstanceOf(AbfsListStatusRemoteIterator.class);
+    int itrCount = 0;
+    for (int i = 0; i < TEST_FILES_NUMBER; i++) {
+      FileStatus fileStatus = fsItr.next();
+      String pathStr = fileStatus.getPath().toString();
+      fileNames.remove(pathStr);
+      itrCount++;
+    }
+    Assertions.assertThatThrownBy(() -> fsItr.next())
+        .describedAs(
+            "next() should throw NoSuchElementException since next has been "
+                + "called " + TEST_FILES_NUMBER + " times")
+        .isInstanceOf(NoSuchElementException.class);
+    Assertions.assertThat(itrCount)
+        .describedAs("Number of iterations should be equal to the files "
+            + "created")
+        .isEqualTo(TEST_FILES_NUMBER);
+    Assertions.assertThat(fileNames.size())
+        .describedAs("After removing every iterm found from the iterator, "
+            + "there should be no more elements in the fileNames")
+        .isEqualTo(0);
+  }
+
+  @Test
+  public void testNextWhenNoMoreElementsPresent() throws Exception {
+    Path testDir = createTestDirectory();
+    setPageSize(10);
+    RemoteIterator fsItr =
+        new AbfsListStatusRemoteIterator(getFileSystem().getFileStatus(testDir),
+            getFileSystem().getAbfsStore());
+    fsItr = Mockito.spy(fsItr);
+    Mockito.doReturn(false).when(fsItr).hasNext();
+
+    RemoteIterator<FileStatus> finalFsItr = fsItr;
+    Assertions.assertThatThrownBy(() -> finalFsItr.next())
+        .describedAs(
+        "next() should throw NoSuchElementException if hasNext() return "
+            + "false")
+        .isInstanceOf(NoSuchElementException.class);
+  }
+
+  @Test
+  public void testHasNextForEmptyDir() throws Exception {
+    Path testDir = createTestDirectory();
+    setPageSize(10);
+    RemoteIterator<FileStatus> fsItr = getFileSystem()
+        .listStatusIterator(testDir);
+    Assertions.assertThat(fsItr.hasNext())
+        .describedAs("hasNext returns false for empty directory")
+        .isFalse();
+  }
+
+  @Test
+  public void testHasNextForFile() throws Exception {
+    final AzureBlobFileSystem fs = getFileSystem();
+    String testFileName = "testFile";
+    Path testFile = new Path(testFileName);
+    getFileSystem().create(testFile);
+    setPageSize(10);
+    RemoteIterator<FileStatus> fsItr = fs.listStatusIterator(testFile);
+    Assertions.assertThat(fsItr.hasNext())
+        .describedAs("hasNext returns true for file").isTrue();
+    Assertions.assertThat(fsItr.next().getPath().toString())
+        .describedAs("next returns the file itself")
+        .endsWith(testFileName);
+  }
+
+  @Test
+  public void testIOException() throws Exception {
+    Path testDir = createTestDirectory();
+    setPageSize(10);
+    getFileSystem().mkdirs(testDir);
+
+    String exceptionMessage = "test exception";
+    ListingSupport lsSupport =getMockListingSupport(exceptionMessage);
+    RemoteIterator fsItr =
+        new AbfsListStatusRemoteIterator(getFileSystem().getFileStatus(testDir),
+        lsSupport);
+
+    Assertions.assertThatThrownBy(() -> fsItr.next())
+        .describedAs(
+        "When ioException is not null and queue is empty exception should be "
+            + "thrown")
+        .isInstanceOf(IOException.class)
+        .hasMessage(exceptionMessage);
+  }
+
+  @Test
+  public void testNonExistingPath() throws Throwable {
+    Path nonExistingDir = new Path("nonExistingPath");
+    Assertions.assertThatThrownBy(
+        () -> getFileSystem().listStatusIterator(nonExistingDir)).describedAs(
+        "test the listStatusIterator call on a path which is not "
+            + "present should result in FileNotFoundException")
+        .isInstanceOf(FileNotFoundException.class);
+  }
+
+  private ListingSupport getMockListingSupport(String exceptionMessage) {
+    return new ListingSupport() {
+      @Override
+      public FileStatus[] listStatus(Path path) throws IOException {
+        return null;
+      }
+
+      @Override
+      public FileStatus[] listStatus(Path path, String startFrom)
+          throws IOException {
+        return null;
+      }
+
+      @Override
+      public String listStatus(Path path, String startFrom,
+          List<FileStatus> fileStatuses, boolean fetchAll, String continuation)
+          throws IOException {
+        throw new IOException(exceptionMessage);
+      }
+    };
+  }
+
+  private Path createTestDirectory() throws IOException {
+    String testDirectoryName = "testDirectory" + System.currentTimeMillis();
+    Path testDirectory = new Path(testDirectoryName);
+    getFileSystem().mkdirs(testDirectory);
+    return testDirectory;
+  }
+
+  private void setEnableAbfsIterator(boolean shouldEnable) throws IOException {
+    AzureBlobFileSystemStore abfsStore = getAbfsStore(getFileSystem());
+    abfsStore.getAbfsConfiguration().setEnableAbfsListIterator(shouldEnable);
+  }
+
+  private void setPageSize(int pageSize) throws IOException {
+    AzureBlobFileSystemStore abfsStore = getAbfsStore(getFileSystem());
+    abfsStore.getAbfsConfiguration().setListMaxResults(pageSize);
+  }
+
+  private List<String> createFilesUnderDirectory(int numFiles, Path rootPath,
+      String filenamePrefix)
+      throws ExecutionException, InterruptedException, IOException {
+    final List<Future<Void>> tasks = new ArrayList<>();
+    final List<String> fileNames = new ArrayList<>();
+    ExecutorService es = Executors.newFixedThreadPool(10);
+    for (int i = 0; i < numFiles; i++) {
+      final Path filePath = new Path(rootPath, filenamePrefix + i);
+      Callable<Void> callable = new Callable<Void>() {

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] bilaharith commented on a change in pull request #2548: HADOOP-17475. ABFS: Implementing ListStatusRemoteIterator

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



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsListStatusRemoteIterator.java
##########
@@ -0,0 +1,151 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs.services;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.CompletableFuture;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.RemoteIterator;
+
+public class AbfsListStatusRemoteIterator implements RemoteIterator<FileStatus> {
+
+  private static final Logger LOG = LoggerFactory
+      .getLogger(AbfsListStatusRemoteIterator.class);
+
+  private static final boolean FETCH_ALL_FALSE = false;
+  private static final int MAX_QUEUE_SIZE = 10;
+
+  private final FileStatus fileStatus;
+  private final ListingSupport listingSupport;
+  private final ArrayBlockingQueue<Iterator<FileStatus>> iteratorsQueue;
+  private final Object asyncOpLock = new Object();
+
+  private volatile boolean isAsyncInProgress = false;
+  private boolean firstBatch = true;
+  private String continuation;
+  private Iterator<FileStatus> currIterator;
+  private IOException ioException;
+
+  public AbfsListStatusRemoteIterator(final FileStatus fileStatus,
+      final ListingSupport listingSupport) {
+    this.fileStatus = fileStatus;
+    this.listingSupport = listingSupport;
+    iteratorsQueue = new ArrayBlockingQueue<>(MAX_QUEUE_SIZE);
+    currIterator = Collections.emptyIterator();
+    fetchBatchesAsync();
+  }
+
+  @Override
+  public boolean hasNext() throws IOException {
+    if (currIterator.hasNext()) {
+      return true;
+    }
+    updateCurrentIterator();
+    return currIterator.hasNext();
+  }
+
+  @Override
+  public FileStatus next() throws IOException {
+    if (!this.hasNext()) {
+      throw new NoSuchElementException();
+    }
+    return currIterator.next();
+  }
+
+  private void updateCurrentIterator() throws IOException {
+    fetchBatchesAsync();
+    synchronized (this) {
+      if (iteratorsQueue.isEmpty()) {
+        if (ioException != null) {
+          throw ioException;
+        }
+        if (isListingComplete()) {
+          return;
+        }
+      }
+    }
+    try {
+      currIterator = iteratorsQueue.take();
+      if (!currIterator.hasNext() && !isListingComplete()) {
+        updateCurrentIterator();
+      }
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      LOG.error("Thread got interrupted: {}", e);
+    }
+  }
+
+  private synchronized boolean isListingComplete() {
+    return !firstBatch && (continuation == null || continuation.isEmpty());
+  }
+
+  private void fetchBatchesAsync() {
+    if (isAsyncInProgress) {
+      return;
+    }
+    synchronized (asyncOpLock) {
+      if (isAsyncInProgress) {
+        return;
+      }
+      isAsyncInProgress = true;
+    }
+    CompletableFuture.runAsync(() -> asyncOp());
+  }
+
+  private void asyncOp() {
+    try {
+      while (!isListingComplete() && iteratorsQueue.size() <= MAX_QUEUE_SIZE) {
+        addNextBatchIteratorToQueue();
+      }
+    } catch (IOException e) {
+      ioException = e;
+      iteratorsQueue.offer(Collections.emptyIterator());
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      LOG.error("Thread got interrupted: {}", e);
+    } finally {
+      synchronized (asyncOpLock) {
+        isAsyncInProgress = false;
+      }
+    }
+  }
+
+  private synchronized void addNextBatchIteratorToQueue()

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] vinaysbadami commented on a change in pull request #2548: DRAFT PR: Implementing ListStatusRemoteIterator

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



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ListStatusRemoteIterator.java
##########
@@ -0,0 +1,129 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs.services;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.ListIterator;
+import java.util.NoSuchElementException;
+import java.util.Queue;
+import java.util.concurrent.CompletableFuture;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore;
+
+public class ListStatusRemoteIterator implements RemoteIterator<FileStatus> {
+
+  private static final Logger LOG = LoggerFactory
+      .getLogger(ListStatusRemoteIterator.class);
+
+  private static final boolean FETCH_ALL_FALSE = false;
+
+  private final Path path;
+  private final AzureBlobFileSystemStore abfsStore;
+  private final Queue<ListIterator<FileStatus>> iteratorsQueue =

Review comment:
       ArrayBlockingQueue

##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ListStatusRemoteIterator.java
##########
@@ -0,0 +1,129 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs.services;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.ListIterator;
+import java.util.NoSuchElementException;
+import java.util.Queue;
+import java.util.concurrent.CompletableFuture;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore;
+
+public class ListStatusRemoteIterator implements RemoteIterator<FileStatus> {
+
+  private static final Logger LOG = LoggerFactory
+      .getLogger(ListStatusRemoteIterator.class);
+
+  private static final boolean FETCH_ALL_FALSE = false;
+
+  private final Path path;
+  private final AzureBlobFileSystemStore abfsStore;
+  private final Queue<ListIterator<FileStatus>> iteratorsQueue =
+      new LinkedList<>();
+
+  private boolean firstRead = true;
+  private String continuation;
+  private ListIterator<FileStatus> currIterator;
+  private IOException ioException;
+
+  public ListStatusRemoteIterator(final Path path,
+      final AzureBlobFileSystemStore abfsStore) throws IOException {
+    this.path = path;
+    this.abfsStore = abfsStore;

Review comment:
       ioExcetion and currentIterator to null

##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ListStatusRemoteIterator.java
##########
@@ -0,0 +1,129 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs.services;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.ListIterator;
+import java.util.NoSuchElementException;
+import java.util.Queue;
+import java.util.concurrent.CompletableFuture;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore;
+
+public class ListStatusRemoteIterator implements RemoteIterator<FileStatus> {
+
+  private static final Logger LOG = LoggerFactory
+      .getLogger(ListStatusRemoteIterator.class);
+
+  private static final boolean FETCH_ALL_FALSE = false;
+
+  private final Path path;
+  private final AzureBlobFileSystemStore abfsStore;
+  private final Queue<ListIterator<FileStatus>> iteratorsQueue =
+      new LinkedList<>();
+
+  private boolean firstRead = true;
+  private String continuation;
+  private ListIterator<FileStatus> currIterator;
+  private IOException ioException;
+
+  public ListStatusRemoteIterator(final Path path,
+      final AzureBlobFileSystemStore abfsStore) throws IOException {
+    this.path = path;
+    this.abfsStore = abfsStore;
+    fetchAllAsync();
+    updateCurrentIterator();
+  }
+
+  @Override
+  public boolean hasNext() throws IOException {
+    if (currIterator.hasNext()) {
+      return true;
+    }
+    updateCurrentIterator();
+    return currIterator.hasNext();
+  }
+
+  @Override
+  public FileStatus next() throws IOException {
+    if (!this.hasNext()) {
+      throw new NoSuchElementException();
+    }
+    return currIterator.next();
+  }
+
+  private void updateCurrentIterator() throws IOException {
+    synchronized (this) {
+      while (!isIterationComplete() && iteratorsQueue.isEmpty()) {
+        try {
+          this.wait();
+        } catch (InterruptedException e) {
+          Thread.currentThread().interrupt();
+          LOG.error("Thread got interrupted: {}", e);
+        }
+      }
+      if (!iteratorsQueue.isEmpty()) {
+        currIterator = iteratorsQueue.poll();
+      } else if (ioException != null) {
+        throw ioException;
+      }
+    }
+  }
+
+  private void fetchAllAsync() {
+    CompletableFuture.supplyAsync(() -> {
+      while (!isIterationComplete()) {
+        List<FileStatus> fileStatuses = new ArrayList<>();
+        try {
+          continuation = abfsStore
+              .listStatus(path, null, fileStatuses, FETCH_ALL_FALSE,
+                  continuation);
+        } catch (IOException e) {
+          ioException = e;
+          return null;
+        } finally {
+          if (firstRead) {
+            firstRead = false;
+          }
+        }
+        if (fileStatuses != null && !fileStatuses.isEmpty()) {

Review comment:
       Move to line 106

##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ListStatusRemoteIterator.java
##########
@@ -0,0 +1,129 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs.services;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.ListIterator;
+import java.util.NoSuchElementException;
+import java.util.Queue;
+import java.util.concurrent.CompletableFuture;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore;
+
+public class ListStatusRemoteIterator implements RemoteIterator<FileStatus> {
+
+  private static final Logger LOG = LoggerFactory
+      .getLogger(ListStatusRemoteIterator.class);
+
+  private static final boolean FETCH_ALL_FALSE = false;
+
+  private final Path path;
+  private final AzureBlobFileSystemStore abfsStore;
+  private final Queue<ListIterator<FileStatus>> iteratorsQueue =
+      new LinkedList<>();
+
+  private boolean firstRead = true;
+  private String continuation;
+  private ListIterator<FileStatus> currIterator;
+  private IOException ioException;
+
+  public ListStatusRemoteIterator(final Path path,
+      final AzureBlobFileSystemStore abfsStore) throws IOException {
+    this.path = path;
+    this.abfsStore = abfsStore;
+    fetchAllAsync();
+    updateCurrentIterator();
+  }
+
+  @Override
+  public boolean hasNext() throws IOException {
+    if (currIterator.hasNext()) {
+      return true;
+    }
+    updateCurrentIterator();
+    return currIterator.hasNext();
+  }
+
+  @Override
+  public FileStatus next() throws IOException {
+    if (!this.hasNext()) {
+      throw new NoSuchElementException();
+    }
+    return currIterator.next();
+  }
+
+  private void updateCurrentIterator() throws IOException {
+    synchronized (this) {
+      while (!isIterationComplete() && iteratorsQueue.isEmpty()) {
+        try {
+          this.wait();
+        } catch (InterruptedException e) {
+          Thread.currentThread().interrupt();
+          LOG.error("Thread got interrupted: {}", e);
+        }
+      }
+      if (!iteratorsQueue.isEmpty()) {
+        currIterator = iteratorsQueue.poll();
+      } else if (ioException != null) {
+        throw ioException;
+      }
+    }
+  }
+
+  private void fetchAllAsync() {
+    CompletableFuture.supplyAsync(() -> {
+      while (!isIterationComplete()) {
+        List<FileStatus> fileStatuses = new ArrayList<>();
+        try {
+          continuation = abfsStore
+              .listStatus(path, null, fileStatuses, FETCH_ALL_FALSE,
+                  continuation);
+        } catch (IOException e) {
+          ioException = e;
+          return null;
+        } finally {
+          if (firstRead) {
+            firstRead = false;
+          }
+        }
+        if (fileStatuses != null && !fileStatuses.isEmpty()) {
+          iteratorsQueue.add(fileStatuses.listIterator());
+          synchronized (this) {

Review comment:
       we just need to add to queue

##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java
##########
@@ -862,16 +871,16 @@ public FileStatus getFileStatus(final Path path) throws IOException {
             startFrom);
 
     final String relativePath = getRelativePath(path);
-    String continuation = null;
 
-    // generate continuation token if a valid startFrom is provided.
-    if (startFrom != null && !startFrom.isEmpty()) {
-      continuation = getIsNamespaceEnabled()
-              ? generateContinuationTokenForXns(startFrom)
-              : generateContinuationTokenForNonXns(relativePath, startFrom);
+    if(continuation==null ||continuation.length()<1) {

Review comment:
       spacing etc
   continuation.IsEmpty

##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ListStatusRemoteIterator.java
##########
@@ -0,0 +1,129 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs.services;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.ListIterator;
+import java.util.NoSuchElementException;
+import java.util.Queue;
+import java.util.concurrent.CompletableFuture;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore;
+
+public class ListStatusRemoteIterator implements RemoteIterator<FileStatus> {
+
+  private static final Logger LOG = LoggerFactory
+      .getLogger(ListStatusRemoteIterator.class);
+
+  private static final boolean FETCH_ALL_FALSE = false;
+
+  private final Path path;
+  private final AzureBlobFileSystemStore abfsStore;
+  private final Queue<ListIterator<FileStatus>> iteratorsQueue =
+      new LinkedList<>();
+
+  private boolean firstRead = true;
+  private String continuation;
+  private ListIterator<FileStatus> currIterator;
+  private IOException ioException;
+
+  public ListStatusRemoteIterator(final Path path,
+      final AzureBlobFileSystemStore abfsStore) throws IOException {
+    this.path = path;
+    this.abfsStore = abfsStore;
+    fetchAllAsync();
+    updateCurrentIterator();

Review comment:
       Let us avoid blocking in the ctor

##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ListStatusRemoteIterator.java
##########
@@ -0,0 +1,129 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs.services;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.ListIterator;
+import java.util.NoSuchElementException;
+import java.util.Queue;
+import java.util.concurrent.CompletableFuture;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore;
+
+public class ListStatusRemoteIterator implements RemoteIterator<FileStatus> {
+
+  private static final Logger LOG = LoggerFactory
+      .getLogger(ListStatusRemoteIterator.class);
+
+  private static final boolean FETCH_ALL_FALSE = false;
+
+  private final Path path;
+  private final AzureBlobFileSystemStore abfsStore;
+  private final Queue<ListIterator<FileStatus>> iteratorsQueue =
+      new LinkedList<>();
+
+  private boolean firstRead = true;
+  private String continuation;
+  private ListIterator<FileStatus> currIterator;
+  private IOException ioException;
+
+  public ListStatusRemoteIterator(final Path path,
+      final AzureBlobFileSystemStore abfsStore) throws IOException {
+    this.path = path;
+    this.abfsStore = abfsStore;
+    fetchAllAsync();
+    updateCurrentIterator();
+  }
+
+  @Override
+  public boolean hasNext() throws IOException {
+    if (currIterator.hasNext()) {

Review comment:
       currentIterator && 




----------------------------------------------------------------
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] bilaharith commented on pull request #2548: HADOOP-17475. ABFS: Implementing ListStatusRemoteIterator

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


   > merged to trunk; cp'd to branch-3.3 and will push up if a recompile works. I am not testing that ---can you do that and if there are problems we can do a followup?
   
   Sure


----------------------------------------------------------------
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] vinaysbadami commented on a change in pull request #2548: HADOOP-17475. ABFS: Implementing ListStatusRemoteIterator

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



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsListStatusRemoteIterator.java
##########
@@ -0,0 +1,156 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs.services;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.CompletableFuture;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.RemoteIterator;
+
+public class AbfsListStatusRemoteIterator implements RemoteIterator<FileStatus> {
+
+  private static final Logger LOG = LoggerFactory
+      .getLogger(AbfsListStatusRemoteIterator.class);
+
+  private static final boolean FETCH_ALL_FALSE = false;
+  private static final int MAX_QUEUE_SIZE = 10;
+
+  private final FileStatus fileStatus;
+  private final ListingSupport listingSupport;
+  private final ArrayBlockingQueue<Object> iteratorsQueue;
+
+  private volatile boolean isAsyncInProgress = false;
+  private boolean isIterationComplete = false;
+  private String continuation;
+  private Iterator<FileStatus> currIterator;
+
+  public AbfsListStatusRemoteIterator(final FileStatus fileStatus,
+      final ListingSupport listingSupport) {
+    this.fileStatus = fileStatus;
+    this.listingSupport = listingSupport;
+    iteratorsQueue = new ArrayBlockingQueue<>(MAX_QUEUE_SIZE);
+    currIterator = Collections.emptyIterator();
+    fetchBatchesAsync();
+  }
+
+  @Override
+  public boolean hasNext() throws IOException {
+    if (currIterator.hasNext()) {
+      return true;
+    }
+    updateCurrentIterator();
+    return currIterator.hasNext();
+  }
+
+  @Override
+  public FileStatus next() throws IOException {
+    if (!this.hasNext()) {
+      throw new NoSuchElementException();
+    }
+    return currIterator.next();
+  }
+
+  private void updateCurrentIterator() throws IOException {
+    do {
+      currIterator = getNextIterator();
+    } while (currIterator != null && !currIterator.hasNext()
+        && !isIterationComplete);
+  }
+
+  private Iterator<FileStatus> getNextIterator() throws IOException {
+    fetchBatchesAsync();
+    synchronized (this) {
+      if (iteratorsQueue.isEmpty() && isIterationComplete) {
+          return Collections.emptyIterator();
+      }
+    }
+    try {
+      Object obj = iteratorsQueue.take();
+      if(obj instanceof Iterator){
+        return (Iterator<FileStatus>) obj;
+      }
+      throw (IOException) obj;
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      LOG.error("Thread got interrupted: {}", e);
+      return Collections.emptyIterator();
+    }
+  }
+
+  private void fetchBatchesAsync() {
+    if (isAsyncInProgress) {
+      return;
+    }
+    synchronized (this) {
+      if (isAsyncInProgress) {
+        return;
+      }
+      isAsyncInProgress = true;
+    }
+    CompletableFuture.runAsync(() -> asyncOp());
+  }
+
+  private void asyncOp() {
+    try {
+      while (!isIterationComplete && iteratorsQueue.size() <= MAX_QUEUE_SIZE) {
+        addNextBatchIteratorToQueue();
+      }
+    } catch (IOException e) {
+      try {
+        iteratorsQueue.put(e);
+      } catch (InterruptedException interruptedException) {
+        Thread.currentThread().interrupt();
+        LOG.error("Thread got interrupted: {}", interruptedException);
+      }
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      LOG.error("Thread got interrupted: {}", e);
+    } finally {
+      synchronized (this  ) {
+        isAsyncInProgress = false;
+      }
+    }
+  }
+
+  private void addNextBatchIteratorToQueue()
+      throws IOException, InterruptedException {
+    List<FileStatus> fileStatuses = new ArrayList<>();
+    continuation = listingSupport
+        .listStatus(fileStatus.getPath(), null, fileStatuses, FETCH_ALL_FALSE,
+            continuation);
+    iteratorsQueue.put(fileStatuses.iterator());
+    synchronized (this) {
+      if (continuation == null || continuation.isEmpty()) {
+        isIterationComplete = true;
+        iteratorsQueue.put(Collections.emptyIterator());

Review comment:
       not within sync block




----------------------------------------------------------------
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 #2548: DRAFT PR: Implementing ListStatusRemoteIterator

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


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |   1m 53s |  |  Docker mode activated.  |
   |||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  |  No case conflicting files found.  |
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |   |   0m  0s | [test4tests](test4tests) |  The patch appears to include 1 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | -1 :x: |  mvninstall  |   0m 23s | [/branch-mvninstall-root.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/4/artifact/out/branch-mvninstall-root.txt) |  root in trunk failed.  |
   | -1 :x: |  compile  |   0m 23s | [/branch-compile-hadoop-tools_hadoop-azure-jdkUbuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/4/artifact/out/branch-compile-hadoop-tools_hadoop-azure-jdkUbuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04.txt) |  hadoop-azure in trunk failed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04.  |
   | -1 :x: |  compile  |   0m 23s | [/branch-compile-hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/4/artifact/out/branch-compile-hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01.txt) |  hadoop-azure in trunk failed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01.  |
   | -0 :warning: |  checkstyle  |   0m 22s | [/buildtool-branch-checkstyle-hadoop-tools_hadoop-azure.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/4/artifact/out/buildtool-branch-checkstyle-hadoop-tools_hadoop-azure.txt) |  The patch fails to run checkstyle in hadoop-azure  |
   | -1 :x: |  mvnsite  |   0m 24s | [/branch-mvnsite-hadoop-tools_hadoop-azure.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/4/artifact/out/branch-mvnsite-hadoop-tools_hadoop-azure.txt) |  hadoop-azure in trunk failed.  |
   | +1 :green_heart: |  shadedclient  |   1m 11s |  |  branch has no errors when building and testing our client artifacts.  |
   | -1 :x: |  javadoc  |   0m 23s | [/branch-javadoc-hadoop-tools_hadoop-azure-jdkUbuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/4/artifact/out/branch-javadoc-hadoop-tools_hadoop-azure-jdkUbuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04.txt) |  hadoop-azure in trunk failed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04.  |
   | -1 :x: |  javadoc  |   0m 23s | [/branch-javadoc-hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/4/artifact/out/branch-javadoc-hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01.txt) |  hadoop-azure in trunk failed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01.  |
   | +0 :ok: |  spotbugs  |   2m 22s |  |  Used deprecated FindBugs config; considering switching to SpotBugs.  |
   | -1 :x: |  findbugs  |   0m 24s | [/branch-findbugs-hadoop-tools_hadoop-azure.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/4/artifact/out/branch-findbugs-hadoop-tools_hadoop-azure.txt) |  hadoop-azure in trunk failed.  |
   |||| _ Patch Compile Tests _ |
   | -1 :x: |  mvninstall  |   0m 22s | [/patch-mvninstall-hadoop-tools_hadoop-azure.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/4/artifact/out/patch-mvninstall-hadoop-tools_hadoop-azure.txt) |  hadoop-azure in the patch failed.  |
   | -1 :x: |  compile  |   0m 23s | [/patch-compile-hadoop-tools_hadoop-azure-jdkUbuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/4/artifact/out/patch-compile-hadoop-tools_hadoop-azure-jdkUbuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04.txt) |  hadoop-azure in the patch failed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04.  |
   | -1 :x: |  javac  |   0m 23s | [/patch-compile-hadoop-tools_hadoop-azure-jdkUbuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/4/artifact/out/patch-compile-hadoop-tools_hadoop-azure-jdkUbuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04.txt) |  hadoop-azure in the patch failed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04.  |
   | -1 :x: |  compile  |   0m 24s | [/patch-compile-hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/4/artifact/out/patch-compile-hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01.txt) |  hadoop-azure in the patch failed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01.  |
   | -1 :x: |  javac  |   0m 24s | [/patch-compile-hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/4/artifact/out/patch-compile-hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01.txt) |  hadoop-azure in the patch failed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01.  |
   | -0 :warning: |  checkstyle  |   0m 21s | [/buildtool-patch-checkstyle-hadoop-tools_hadoop-azure.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/4/artifact/out/buildtool-patch-checkstyle-hadoop-tools_hadoop-azure.txt) |  The patch fails to run checkstyle in hadoop-azure  |
   | -1 :x: |  mvnsite  |   0m 23s | [/patch-mvnsite-hadoop-tools_hadoop-azure.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/4/artifact/out/patch-mvnsite-hadoop-tools_hadoop-azure.txt) |  hadoop-azure in the patch failed.  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  shadedclient  |   0m 22s |  |  patch has no errors when building and testing our client artifacts.  |
   | -1 :x: |  javadoc  |   0m 24s | [/patch-javadoc-hadoop-tools_hadoop-azure-jdkUbuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/4/artifact/out/patch-javadoc-hadoop-tools_hadoop-azure-jdkUbuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04.txt) |  hadoop-azure in the patch failed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04.  |
   | -1 :x: |  javadoc  |   0m 25s | [/patch-javadoc-hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/4/artifact/out/patch-javadoc-hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01.txt) |  hadoop-azure in the patch failed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01.  |
   | -1 :x: |  findbugs  |   0m 25s | [/patch-findbugs-hadoop-tools_hadoop-azure.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/4/artifact/out/patch-findbugs-hadoop-tools_hadoop-azure.txt) |  hadoop-azure in the patch failed.  |
   |||| _ Other Tests _ |
   | -1 :x: |  unit  |   0m 24s | [/patch-unit-hadoop-tools_hadoop-azure.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/4/artifact/out/patch-unit-hadoop-tools_hadoop-azure.txt) |  hadoop-azure in the patch failed.  |
   | +0 :ok: |  asflicense  |   0m 23s |  |  ASF License check generated no output?  |
   |  |   |  13m 15s |  |  |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/4/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/2548 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient findbugs checkstyle |
   | uname | Linux 502273929e35 4.15.0-65-generic #74-Ubuntu SMP Tue Sep 17 17:06:04 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / 513f1995adc |
   | Default Java | Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01 |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/4/testReport/ |
   | Max. process+thread count | 10 (vs. ulimit of 5500) |
   | modules | C: hadoop-tools/hadoop-azure U: hadoop-tools/hadoop-azure |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/4/console |
   | versions | git=2.17.1 maven=3.6.0 |
   | 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] bilaharith commented on a change in pull request #2548: HADOOP-17475. ABFS: Implementing ListStatusRemoteIterator

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



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java
##########
@@ -982,6 +985,19 @@ public boolean exists(Path f) throws IOException {
     return super.exists(f);
   }
 
+  @Override
+  public RemoteIterator<FileStatus> listStatusIterator(Path path)
+      throws IOException {
+    LOG.debug("AzureBlobFileSystem.listStatusIterator path : {}", path);
+    if (abfsStore.getAbfsConfiguration().enableAbfsListIterator()) {

Review comment:
       Done
   https://issues.apache.org/jira/browse/HADOOP-17512




----------------------------------------------------------------
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 #2548: HADOOP-17475. ABFS: Implementing ListStatusRemoteIterator

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


   > There is one one findbugs warning remaining which is of medium priority. The same can be ignored since at line 147 the continuation token returned by the ListingOperation ouside the synchroniced lock since the same involve an http call.
   
   well, it needs to be dealt with either by fixing or findbugs.xml. Define "medium priority" here


----------------------------------------------------------------
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] vinaysbadami commented on a change in pull request #2548: HADOOP-17475. ABFS: Implementing ListStatusRemoteIterator

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



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsListStatusRemoteIterator.java
##########
@@ -0,0 +1,159 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs.services;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.CompletableFuture;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.RemoteIterator;
+
+public class AbfsListStatusRemoteIterator implements RemoteIterator<FileStatus> {
+
+  private static final Logger LOG = LoggerFactory
+      .getLogger(AbfsListStatusRemoteIterator.class);
+
+  private static final boolean FETCH_ALL_FALSE = false;
+  private static final int MAX_QUEUE_SIZE = 10;
+
+  private final FileStatus fileStatus;
+  private final ListingSupport listingSupport;
+  private final ArrayBlockingQueue<Object> iteratorsQueue;
+  private final Object asyncOpLock = new Object();
+
+  private volatile boolean isAsyncInProgress = false;
+  private boolean isIterationComplete = false;
+  private String continuation;
+  private Iterator<FileStatus> currIterator;
+
+  public AbfsListStatusRemoteIterator(final FileStatus fileStatus,
+      final ListingSupport listingSupport) {
+    this.fileStatus = fileStatus;
+    this.listingSupport = listingSupport;
+    iteratorsQueue = new ArrayBlockingQueue<>(MAX_QUEUE_SIZE);
+    currIterator = Collections.emptyIterator();
+    fetchBatchesAsync();
+  }
+
+  @Override
+  public boolean hasNext() throws IOException {
+    if (currIterator.hasNext()) {
+      return true;
+    }
+    updateCurrentIterator();
+    if (currIterator == null) {
+      return false;
+    }
+    return currIterator.hasNext();
+  }
+
+  @Override
+  public FileStatus next() throws IOException {
+    if (!this.hasNext()) {
+      throw new NoSuchElementException();
+    }
+    return currIterator.next();
+  }
+
+  private void updateCurrentIterator() throws IOException {
+    do {
+      currIterator = getNextIterator();
+    } while (currIterator != null && !currIterator.hasNext()
+        && !isIterationComplete);
+  }
+
+  private Iterator<FileStatus> getNextIterator() throws IOException {
+    fetchBatchesAsync();
+    synchronized (this) {
+      if (iteratorsQueue.isEmpty() && isIterationComplete) {
+          return null;
+      }
+    }
+    try {
+      Object obj = iteratorsQueue.take();
+      if(obj instanceof Iterator){
+        return (Iterator<FileStatus>) obj;
+      }
+      throw (IOException) obj;
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      LOG.error("Thread got interrupted: {}", e);
+      return null;
+    }
+  }
+
+  private void fetchBatchesAsync() {
+    if (isAsyncInProgress) {
+      return;
+    }
+    synchronized (asyncOpLock) {
+      if (isAsyncInProgress) {
+        return;
+      }
+      isAsyncInProgress = true;
+    }
+    CompletableFuture.runAsync(() -> asyncOp());
+  }
+
+  private void asyncOp() {
+    try {
+      while (!isIterationComplete && iteratorsQueue.size() <= MAX_QUEUE_SIZE) {
+        addNextBatchIteratorToQueue();
+      }
+    } catch (IOException e) {
+      try {
+        iteratorsQueue.put(e);
+      } catch (InterruptedException interruptedException) {
+        Thread.currentThread().interrupt();
+        LOG.error("Thread got interrupted: {}", interruptedException);
+      }
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      LOG.error("Thread got interrupted: {}", e);
+    } finally {
+      synchronized (asyncOpLock) {
+        isAsyncInProgress = false;
+      }
+    }
+  }
+
+  private void addNextBatchIteratorToQueue()
+      throws IOException, InterruptedException {
+    List<FileStatus> fileStatuses = new ArrayList<>();
+    continuation = listingSupport
+        .listStatus(fileStatus.getPath(), null, fileStatuses, FETCH_ALL_FALSE,
+            continuation);
+    synchronized (this) {
+      if (continuation == null || continuation.isEmpty()) {
+        isIterationComplete = true;
+      }
+      iteratorsQueue.put(fileStatuses.iterator());

Review comment:
       can this be done outside the sync block? If put blocks u are holding the lock.




----------------------------------------------------------------
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 #2548: DRAFT PR: Implementing ListStatusRemoteIterator

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


   1. What is is the JIRA ID?
   2. As discussed, you need an uber-JIRA to cover the whole set of list optimisations you can do, including an overall goal.
   
   I would recommend something like "ABFS listing to support asynchronous prefetch and optimise for incremental listing of large directories and deep/wide directory trees"
   
   That is: if that hurts performance of listing empty directories, or calling the listX calls against files, that is acceptable.


----------------------------------------------------------------
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] vinaysbadami commented on a change in pull request #2548: HADOOP-17475. ABFS: Implementing ListStatusRemoteIterator

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



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsListStatusRemoteIterator.java
##########
@@ -0,0 +1,151 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs.services;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.CompletableFuture;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.RemoteIterator;
+
+public class AbfsListStatusRemoteIterator implements RemoteIterator<FileStatus> {
+
+  private static final Logger LOG = LoggerFactory
+      .getLogger(AbfsListStatusRemoteIterator.class);
+
+  private static final boolean FETCH_ALL_FALSE = false;
+  private static final int MAX_QUEUE_SIZE = 10;
+
+  private final FileStatus fileStatus;
+  private final ListingSupport listingSupport;
+  private final ArrayBlockingQueue<Iterator<FileStatus>> iteratorsQueue;
+  private final Object asyncOpLock = new Object();
+
+  private volatile boolean isAsyncInProgress = false;
+  private boolean firstBatch = true;
+  private String continuation;
+  private Iterator<FileStatus> currIterator;
+  private IOException ioException;
+
+  public AbfsListStatusRemoteIterator(final FileStatus fileStatus,
+      final ListingSupport listingSupport) {
+    this.fileStatus = fileStatus;
+    this.listingSupport = listingSupport;
+    iteratorsQueue = new ArrayBlockingQueue<>(MAX_QUEUE_SIZE);
+    currIterator = Collections.emptyIterator();
+    fetchBatchesAsync();
+  }
+
+  @Override
+  public boolean hasNext() throws IOException {
+    if (currIterator.hasNext()) {
+      return true;
+    }
+    updateCurrentIterator();
+    return currIterator.hasNext();
+  }
+
+  @Override
+  public FileStatus next() throws IOException {
+    if (!this.hasNext()) {
+      throw new NoSuchElementException();
+    }
+    return currIterator.next();
+  }
+
+  private void updateCurrentIterator() throws IOException {
+    fetchBatchesAsync();
+    synchronized (this) {
+      if (iteratorsQueue.isEmpty()) {
+        if (ioException != null) {
+          throw ioException;
+        }
+        if (isListingComplete()) {
+          return;
+        }
+      }
+    }
+    try {
+      currIterator = iteratorsQueue.take();
+      if (!currIterator.hasNext() && !isListingComplete()) {
+        updateCurrentIterator();
+      }
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      LOG.error("Thread got interrupted: {}", e);
+    }
+  }
+
+  private synchronized boolean isListingComplete() {
+    return !firstBatch && (continuation == null || continuation.isEmpty());
+  }
+
+  private void fetchBatchesAsync() {
+    if (isAsyncInProgress) {
+      return;
+    }
+    synchronized (asyncOpLock) {
+      if (isAsyncInProgress) {
+        return;
+      }
+      isAsyncInProgress = true;
+    }
+    CompletableFuture.runAsync(() -> asyncOp());
+  }
+
+  private void asyncOp() {
+    try {
+      while (!isListingComplete() && iteratorsQueue.size() <= MAX_QUEUE_SIZE) {
+        addNextBatchIteratorToQueue();
+      }
+    } catch (IOException e) {
+      ioException = e;
+      iteratorsQueue.offer(Collections.emptyIterator());
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      LOG.error("Thread got interrupted: {}", e);
+    } finally {
+      synchronized (asyncOpLock) {
+        isAsyncInProgress = false;
+      }
+    }
+  }
+
+  private synchronized void addNextBatchIteratorToQueue()

Review comment:
       why synchronized? this will cause contention on line 83. u are holding the lock across a listing call to server.

##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsListStatusRemoteIterator.java
##########
@@ -0,0 +1,151 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs.services;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.CompletableFuture;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.RemoteIterator;
+
+public class AbfsListStatusRemoteIterator implements RemoteIterator<FileStatus> {
+
+  private static final Logger LOG = LoggerFactory
+      .getLogger(AbfsListStatusRemoteIterator.class);
+
+  private static final boolean FETCH_ALL_FALSE = false;
+  private static final int MAX_QUEUE_SIZE = 10;
+
+  private final FileStatus fileStatus;
+  private final ListingSupport listingSupport;
+  private final ArrayBlockingQueue<Iterator<FileStatus>> iteratorsQueue;
+  private final Object asyncOpLock = new Object();
+
+  private volatile boolean isAsyncInProgress = false;
+  private boolean firstBatch = true;
+  private String continuation;
+  private Iterator<FileStatus> currIterator;
+  private IOException ioException;
+
+  public AbfsListStatusRemoteIterator(final FileStatus fileStatus,
+      final ListingSupport listingSupport) {
+    this.fileStatus = fileStatus;
+    this.listingSupport = listingSupport;
+    iteratorsQueue = new ArrayBlockingQueue<>(MAX_QUEUE_SIZE);
+    currIterator = Collections.emptyIterator();
+    fetchBatchesAsync();
+  }
+
+  @Override
+  public boolean hasNext() throws IOException {
+    if (currIterator.hasNext()) {
+      return true;
+    }
+    updateCurrentIterator();
+    return currIterator.hasNext();
+  }
+
+  @Override
+  public FileStatus next() throws IOException {
+    if (!this.hasNext()) {
+      throw new NoSuchElementException();
+    }
+    return currIterator.next();
+  }
+
+  private void updateCurrentIterator() throws IOException {
+    fetchBatchesAsync();
+    synchronized (this) {
+      if (iteratorsQueue.isEmpty()) {
+        if (ioException != null) {
+          throw ioException;
+        }
+        if (isListingComplete()) {
+          return;
+        }
+      }
+    }
+    try {
+      currIterator = iteratorsQueue.take();
+      if (!currIterator.hasNext() && !isListingComplete()) {
+        updateCurrentIterator();

Review comment:
       why recursion?

##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsListStatusRemoteIterator.java
##########
@@ -0,0 +1,151 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs.services;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.CompletableFuture;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.RemoteIterator;
+
+public class AbfsListStatusRemoteIterator implements RemoteIterator<FileStatus> {
+
+  private static final Logger LOG = LoggerFactory
+      .getLogger(AbfsListStatusRemoteIterator.class);
+
+  private static final boolean FETCH_ALL_FALSE = false;
+  private static final int MAX_QUEUE_SIZE = 10;
+
+  private final FileStatus fileStatus;
+  private final ListingSupport listingSupport;
+  private final ArrayBlockingQueue<Iterator<FileStatus>> iteratorsQueue;
+  private final Object asyncOpLock = new Object();
+
+  private volatile boolean isAsyncInProgress = false;
+  private boolean firstBatch = true;

Review comment:
       set continuationtoken to empty. why do we need firstbatch?




----------------------------------------------------------------
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] vinaysbadami commented on a change in pull request #2548: DRAFT PR: Implementing ListStatusRemoteIterator

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



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ListStatusRemoteIterator.java
##########
@@ -0,0 +1,149 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs.services;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.CompletableFuture;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore;
+
+public class ListStatusRemoteIterator implements RemoteIterator<FileStatus> {
+
+  private static final Logger LOG = LoggerFactory
+      .getLogger(ListStatusRemoteIterator.class);
+
+  private static final boolean FETCH_ALL_FALSE = false;
+  private static final int MAX_QUEUE_SIZE = 10;
+
+  private final Path path;
+  private final AzureBlobFileSystemStore abfsStore;
+  private final ArrayBlockingQueue<Iterator<FileStatus>> iteratorsQueue;
+  private final Object asyncOpLock = new Object();
+
+  private boolean firstBatch = true;
+  private boolean isAsyncInProgress = false;
+  private String continuation;
+  private Iterator<FileStatus> currIterator;
+  private IOException ioException;
+
+  public ListStatusRemoteIterator(final Path path,
+      final AzureBlobFileSystemStore abfsStore) {
+    this.path = path;
+    this.abfsStore = abfsStore;
+    iteratorsQueue = new ArrayBlockingQueue<>(MAX_QUEUE_SIZE);
+    currIterator = Collections.emptyIterator();
+    fetchBatchesAsync();
+  }
+
+  @Override
+  public boolean hasNext() throws IOException {
+    if (currIterator.hasNext()) {
+      return true;
+    }
+    updateCurrentIterator();
+    return currIterator.hasNext();
+  }
+
+  @Override
+  public FileStatus next() throws IOException {
+    if (!this.hasNext()) {
+      throw new NoSuchElementException();
+    }
+    return currIterator.next();
+  }
+
+  private void updateCurrentIterator() throws IOException {
+    fetchBatchesAsync();
+    synchronized (this) {
+      if (iteratorsQueue.isEmpty()) {
+        if (ioException != null) {
+          throw ioException;
+        }
+        if (isListingComplete()) {
+          return;
+        }
+      }
+    }
+    try {
+      currIterator = iteratorsQueue.take();
+      if (!currIterator.hasNext() && !isListingComplete()) {
+        updateCurrentIterator();
+      }
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      LOG.error("Thread got interrupted: {}", e);
+    }
+  }
+
+  private boolean isListingComplete() {
+    return !firstBatch && (continuation == null || continuation.isEmpty());
+  }
+
+  private void fetchBatchesAsync() {
+    CompletableFuture.runAsync(() -> asyncOp());
+  }
+
+  private void asyncOp() {
+    if (isAsyncInProgress) {
+      return;
+    }
+    synchronized (asyncOpLock) {
+      if (isAsyncInProgress) {
+        return;
+      }
+      isAsyncInProgress = true;
+    }
+    try {
+      while (!isListingComplete() && iteratorsQueue.size() <= MAX_QUEUE_SIZE) {
+        addNextBatchIteratorToQueue();
+      }
+    } catch (IOException e) {
+      ioException = e;
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      LOG.error("Thread got interrupted: {}", e);
+    } finally {
+      isAsyncInProgress = false;

Review comment:
       needs to be done under lock

##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ListStatusRemoteIterator.java
##########
@@ -0,0 +1,149 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs.services;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.CompletableFuture;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore;
+
+public class ListStatusRemoteIterator implements RemoteIterator<FileStatus> {
+
+  private static final Logger LOG = LoggerFactory
+      .getLogger(ListStatusRemoteIterator.class);
+
+  private static final boolean FETCH_ALL_FALSE = false;
+  private static final int MAX_QUEUE_SIZE = 10;
+
+  private final Path path;
+  private final AzureBlobFileSystemStore abfsStore;
+  private final ArrayBlockingQueue<Iterator<FileStatus>> iteratorsQueue;
+  private final Object asyncOpLock = new Object();
+
+  private boolean firstBatch = true;
+  private boolean isAsyncInProgress = false;
+  private String continuation;
+  private Iterator<FileStatus> currIterator;
+  private IOException ioException;
+
+  public ListStatusRemoteIterator(final Path path,
+      final AzureBlobFileSystemStore abfsStore) {
+    this.path = path;
+    this.abfsStore = abfsStore;
+    iteratorsQueue = new ArrayBlockingQueue<>(MAX_QUEUE_SIZE);
+    currIterator = Collections.emptyIterator();
+    fetchBatchesAsync();
+  }
+
+  @Override
+  public boolean hasNext() throws IOException {
+    if (currIterator.hasNext()) {
+      return true;
+    }
+    updateCurrentIterator();
+    return currIterator.hasNext();
+  }
+
+  @Override
+  public FileStatus next() throws IOException {
+    if (!this.hasNext()) {
+      throw new NoSuchElementException();
+    }
+    return currIterator.next();
+  }
+
+  private void updateCurrentIterator() throws IOException {
+    fetchBatchesAsync();
+    synchronized (this) {
+      if (iteratorsQueue.isEmpty()) {
+        if (ioException != null) {

Review comment:
       I thought we agreed that we will throw exception at correct place?




----------------------------------------------------------------
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] bilaharith commented on pull request #2548: HADOOP-17475. ABFS: Implementing ListStatusRemoteIterator

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


   > very close to getting in. other than some minor checkstyle/javadoc complaints, that findbugs needs to be made to stop complaining. Usually it is indicating a real risk of a problem, so I'd like to see what can be done about it -rather than just edit the XML file to turn off the check
   
   There is one one findbugs warning remaining which is of medium priority. The same can be ignored since at line 147 the continuation token returned by the ListingOperation ouside the synchroniced lock since the same involve an http call.


----------------------------------------------------------------
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] bilaharith commented on a change in pull request #2548: DRAFT PR: Implementing ListStatusRemoteIterator

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



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ListStatusRemoteIterator.java
##########
@@ -0,0 +1,149 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs.services;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.CompletableFuture;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore;
+
+public class ListStatusRemoteIterator implements RemoteIterator<FileStatus> {
+
+  private static final Logger LOG = LoggerFactory
+      .getLogger(ListStatusRemoteIterator.class);
+
+  private static final boolean FETCH_ALL_FALSE = false;
+  private static final int MAX_QUEUE_SIZE = 10;
+
+  private final Path path;
+  private final AzureBlobFileSystemStore abfsStore;
+  private final ArrayBlockingQueue<Iterator<FileStatus>> iteratorsQueue;
+  private final Object asyncOpLock = new Object();
+
+  private boolean firstBatch = true;
+  private boolean isAsyncInProgress = false;
+  private String continuation;
+  private Iterator<FileStatus> currIterator;
+  private IOException ioException;
+
+  public ListStatusRemoteIterator(final Path path,
+      final AzureBlobFileSystemStore abfsStore) {
+    this.path = path;
+    this.abfsStore = abfsStore;
+    iteratorsQueue = new ArrayBlockingQueue<>(MAX_QUEUE_SIZE);
+    currIterator = Collections.emptyIterator();
+    fetchBatchesAsync();
+  }
+
+  @Override
+  public boolean hasNext() throws IOException {
+    if (currIterator.hasNext()) {
+      return true;
+    }
+    updateCurrentIterator();
+    return currIterator.hasNext();
+  }
+
+  @Override
+  public FileStatus next() throws IOException {
+    if (!this.hasNext()) {
+      throw new NoSuchElementException();
+    }
+    return currIterator.next();
+  }
+
+  private void updateCurrentIterator() throws IOException {
+    fetchBatchesAsync();
+    synchronized (this) {
+      if (iteratorsQueue.isEmpty()) {
+        if (ioException != null) {
+          throw ioException;
+        }
+        if (isListingComplete()) {
+          return;
+        }
+      }
+    }
+    try {
+      currIterator = iteratorsQueue.take();
+      if (!currIterator.hasNext() && !isListingComplete()) {
+        updateCurrentIterator();
+      }
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();

Review comment:
       https://stackoverflow.com/questions/4906799/why-invoke-thread-currentthread-interrupt-in-a-catch-interruptexception-block




----------------------------------------------------------------
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] bilaharith commented on a change in pull request #2548: DRAFT PR: Implementing ListStatusRemoteIterator

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



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ListStatusRemoteIterator.java
##########
@@ -0,0 +1,129 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs.services;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.ListIterator;
+import java.util.NoSuchElementException;
+import java.util.Queue;
+import java.util.concurrent.CompletableFuture;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore;
+
+public class ListStatusRemoteIterator implements RemoteIterator<FileStatus> {
+
+  private static final Logger LOG = LoggerFactory
+      .getLogger(ListStatusRemoteIterator.class);
+
+  private static final boolean FETCH_ALL_FALSE = false;
+
+  private final Path path;
+  private final AzureBlobFileSystemStore abfsStore;
+  private final Queue<ListIterator<FileStatus>> iteratorsQueue =

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] steveloughran commented on a change in pull request #2548: DRAFT PR: Implementing ListStatusRemoteIterator

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



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ListStatusRemoteIterator.java
##########
@@ -0,0 +1,149 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs.services;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.CompletableFuture;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore;
+
+public class ListStatusRemoteIterator implements RemoteIterator<FileStatus> {
+
+  private static final Logger LOG = LoggerFactory
+      .getLogger(ListStatusRemoteIterator.class);
+
+  private static final boolean FETCH_ALL_FALSE = false;
+  private static final int MAX_QUEUE_SIZE = 10;
+
+  private final Path path;
+  private final AzureBlobFileSystemStore abfsStore;
+  private final ArrayBlockingQueue<Iterator<FileStatus>> iteratorsQueue;
+  private final Object asyncOpLock = new Object();
+
+  private boolean firstBatch = true;
+  private boolean isAsyncInProgress = false;
+  private String continuation;
+  private Iterator<FileStatus> currIterator;
+  private IOException ioException;
+
+  public ListStatusRemoteIterator(final Path path,

Review comment:
       use interface for operations, as with org.apache.hadoop.fs.s3a.impl.ListingOperationCallbacks

##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ListStatusRemoteIterator.java
##########
@@ -0,0 +1,149 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs.services;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.CompletableFuture;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore;
+
+public class ListStatusRemoteIterator implements RemoteIterator<FileStatus> {
+
+  private static final Logger LOG = LoggerFactory
+      .getLogger(ListStatusRemoteIterator.class);
+
+  private static final boolean FETCH_ALL_FALSE = false;
+  private static final int MAX_QUEUE_SIZE = 10;
+
+  private final Path path;
+  private final AzureBlobFileSystemStore abfsStore;
+  private final ArrayBlockingQueue<Iterator<FileStatus>> iteratorsQueue;
+  private final Object asyncOpLock = new Object();
+
+  private boolean firstBatch = true;
+  private boolean isAsyncInProgress = false;
+  private String continuation;
+  private Iterator<FileStatus> currIterator;
+  private IOException ioException;
+
+  public ListStatusRemoteIterator(final Path path,
+      final AzureBlobFileSystemStore abfsStore) {
+    this.path = path;
+    this.abfsStore = abfsStore;
+    iteratorsQueue = new ArrayBlockingQueue<>(MAX_QUEUE_SIZE);
+    currIterator = Collections.emptyIterator();
+    fetchBatchesAsync();
+  }
+
+  @Override
+  public boolean hasNext() throws IOException {
+    if (currIterator.hasNext()) {
+      return true;
+    }
+    updateCurrentIterator();
+    return currIterator.hasNext();
+  }
+
+  @Override
+  public FileStatus next() throws IOException {
+    if (!this.hasNext()) {
+      throw new NoSuchElementException();
+    }
+    return currIterator.next();
+  }
+
+  private void updateCurrentIterator() throws IOException {
+    fetchBatchesAsync();
+    synchronized (this) {
+      if (iteratorsQueue.isEmpty()) {
+        if (ioException != null) {
+          throw ioException;
+        }
+        if (isListingComplete()) {
+          return;
+        }
+      }
+    }
+    try {
+      currIterator = iteratorsQueue.take();
+      if (!currIterator.hasNext() && !isListingComplete()) {
+        updateCurrentIterator();
+      }
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      LOG.error("Thread got interrupted: {}", e);
+    }
+  }
+
+  private boolean isListingComplete() {
+    return !firstBatch && (continuation == null || continuation.isEmpty());
+  }
+
+  private void fetchBatchesAsync() {
+    CompletableFuture.runAsync(() -> asyncOp());

Review comment:
       should only be scheduled if there isn't one already in progress

##########
File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemListStatusIterator.java
##########
@@ -0,0 +1,190 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs;
+
+import java.io.IOException;
+import java.lang.reflect.Field;
+import java.lang.reflect.Modifier;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+
+import org.assertj.core.api.Assertions;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+import org.apache.hadoop.fs.azurebfs.services.ListStatusRemoteIterator;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+
+/**
+ * Test listStatus operation.
+ */
+public class ITestAzureBlobFileSystemListStatusIterator
+    extends AbstractAbfsIntegrationTest {
+
+  private static final int TEST_FILES_NUMBER = 1000;
+
+  public ITestAzureBlobFileSystemListStatusIterator() throws Exception {
+    super();
+  }
+
+  @Test
+  public void testListPath() throws Exception {
+    final AzureBlobFileSystem fs = getFileSystem();
+    String rootPath = "testRoot1";
+    final List<String> fileNames = createFiles(TEST_FILES_NUMBER, rootPath,
+        "testListPath");
+    AzureBlobFileSystemStore abfsStore = getAbfsStore(fs);
+    abfsStore.getAbfsConfiguration().setListMaxResults(10);
+    RemoteIterator<FileStatus> fsIt = fs.listStatusIterator(new Path(rootPath));
+    int itrCount = 0;
+    while (fsIt.hasNext()) {

Review comment:
       also need to test a loop of it.next() until NoMoreElementsException is raised

##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java
##########
@@ -37,6 +37,8 @@
 import java.util.concurrent.Executors;
 import java.util.concurrent.Future;
 
+import org.apache.hadoop.fs.RemoteIterator;

Review comment:
       needs to go into the non-shaded bit of hadoop imports.

##########
File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemListStatusIterator.java
##########
@@ -0,0 +1,190 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs;
+
+import java.io.IOException;
+import java.lang.reflect.Field;
+import java.lang.reflect.Modifier;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+
+import org.assertj.core.api.Assertions;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+import org.apache.hadoop.fs.azurebfs.services.ListStatusRemoteIterator;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+
+/**
+ * Test listStatus operation.
+ */
+public class ITestAzureBlobFileSystemListStatusIterator
+    extends AbstractAbfsIntegrationTest {
+
+  private static final int TEST_FILES_NUMBER = 1000;
+
+  public ITestAzureBlobFileSystemListStatusIterator() throws Exception {
+    super();
+  }
+
+  @Test
+  public void testListPath() throws Exception {
+    final AzureBlobFileSystem fs = getFileSystem();
+    String rootPath = "testRoot1";
+    final List<String> fileNames = createFiles(TEST_FILES_NUMBER, rootPath,
+        "testListPath");
+    AzureBlobFileSystemStore abfsStore = getAbfsStore(fs);
+    abfsStore.getAbfsConfiguration().setListMaxResults(10);
+    RemoteIterator<FileStatus> fsIt = fs.listStatusIterator(new Path(rootPath));
+    int itrCount = 0;
+    while (fsIt.hasNext()) {
+      FileStatus fileStatus = fsIt.next();
+      String pathStr = fileStatus.getPath().toString();
+      fileNames.remove(pathStr);
+      itrCount++;
+    }
+    assertEquals(TEST_FILES_NUMBER, itrCount);
+    assertEquals(0, fileNames.size());
+  }
+
+  @Test
+  public void testNextWhenNoMoreElementsPresent() throws Exception {
+    final AzureBlobFileSystem fs = getFileSystem();
+    String rootPathStr = "testRoot2";
+    Path rootPath = new Path(rootPathStr);
+    getFileSystem().mkdirs(rootPath);
+    RemoteIterator<FileStatus> fsItr = fs.listStatusIterator(rootPath);
+    fsItr = Mockito.spy(fsItr);
+    Mockito.doReturn(false).when(fsItr).hasNext();
+
+    RemoteIterator<FileStatus> finalFsItr = fsItr;
+    Assertions.assertThatThrownBy(() -> finalFsItr.next()).describedAs(
+        "next() should throw NoSuchElementException if hasNext() return "
+            + "false").isInstanceOf(NoSuchElementException.class);
+  }
+
+  @Test
+  public void testHasNextForEmptyDir() throws Exception {
+    final AzureBlobFileSystem fs = getFileSystem();
+    String rootPathStr = "testRoot3";
+    Path rootPath = new Path(rootPathStr);
+    getFileSystem().mkdirs(rootPath);
+    RemoteIterator<FileStatus> fsItr = fs.listStatusIterator(rootPath);
+    Assertions.assertThat(fsItr.hasNext())
+        .describedAs("hasNext returns false for empty directory").isFalse();
+  }
+
+  @Test
+  public void testHasNextForFile() throws Exception {
+    final AzureBlobFileSystem fs = getFileSystem();
+    String rootPathStr = "testRoot4";
+    Path rootPath = new Path(rootPathStr);
+    getFileSystem().create(rootPath);
+    RemoteIterator<FileStatus> fsItr = fs.listStatusIterator(rootPath);
+    Assertions.assertThat(fsItr.hasNext())
+        .describedAs("hasNext returns true for file").isTrue();
+  }
+
+  @Test
+  public void testHasNextForIOException() throws Exception {
+    final AzureBlobFileSystem fs = getFileSystem();
+    String rootPathStr = "testRoot5";
+    Path rootPath = new Path(rootPathStr);
+    getFileSystem().mkdirs(rootPath);
+    ListStatusRemoteIterator fsItr = (ListStatusRemoteIterator) fs
+        .listStatusIterator(rootPath);
+    Thread.sleep(1000);
+
+    String exceptionMessage = "test exception";
+    setPrivateField(fsItr, ListStatusRemoteIterator.class, "ioException",

Review comment:
       this is an abuse of internals. If you use a callback for the listing operations, you can explicitly raise the IOE instead.

##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ListStatusRemoteIterator.java
##########
@@ -0,0 +1,149 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs.services;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.CompletableFuture;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore;
+
+public class ListStatusRemoteIterator implements RemoteIterator<FileStatus> {
+
+  private static final Logger LOG = LoggerFactory
+      .getLogger(ListStatusRemoteIterator.class);
+
+  private static final boolean FETCH_ALL_FALSE = false;
+  private static final int MAX_QUEUE_SIZE = 10;
+
+  private final Path path;
+  private final AzureBlobFileSystemStore abfsStore;
+  private final ArrayBlockingQueue<Iterator<FileStatus>> iteratorsQueue;
+  private final Object asyncOpLock = new Object();
+
+  private boolean firstBatch = true;
+  private boolean isAsyncInProgress = false;
+  private String continuation;
+  private Iterator<FileStatus> currIterator;
+  private IOException ioException;
+
+  public ListStatusRemoteIterator(final Path path,
+      final AzureBlobFileSystemStore abfsStore) {
+    this.path = path;
+    this.abfsStore = abfsStore;
+    iteratorsQueue = new ArrayBlockingQueue<>(MAX_QUEUE_SIZE);
+    currIterator = Collections.emptyIterator();
+    fetchBatchesAsync();
+  }
+
+  @Override
+  public boolean hasNext() throws IOException {
+    if (currIterator.hasNext()) {
+      return true;
+    }
+    updateCurrentIterator();

Review comment:
       if, at the end of the loop I keep call hasNext() repeatedly, what happens?

##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ListStatusRemoteIterator.java
##########
@@ -0,0 +1,149 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs.services;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.CompletableFuture;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore;
+
+public class ListStatusRemoteIterator implements RemoteIterator<FileStatus> {
+
+  private static final Logger LOG = LoggerFactory
+      .getLogger(ListStatusRemoteIterator.class);
+
+  private static final boolean FETCH_ALL_FALSE = false;
+  private static final int MAX_QUEUE_SIZE = 10;
+
+  private final Path path;
+  private final AzureBlobFileSystemStore abfsStore;
+  private final ArrayBlockingQueue<Iterator<FileStatus>> iteratorsQueue;
+  private final Object asyncOpLock = new Object();
+
+  private boolean firstBatch = true;
+  private boolean isAsyncInProgress = false;

Review comment:
       these are going to have to be Atomic, aren't they?

##########
File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemListStatusIterator.java
##########
@@ -0,0 +1,190 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs;
+
+import java.io.IOException;
+import java.lang.reflect.Field;
+import java.lang.reflect.Modifier;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+
+import org.assertj.core.api.Assertions;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+import org.apache.hadoop.fs.azurebfs.services.ListStatusRemoteIterator;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+
+/**
+ * Test listStatus operation.
+ */
+public class ITestAzureBlobFileSystemListStatusIterator
+    extends AbstractAbfsIntegrationTest {
+
+  private static final int TEST_FILES_NUMBER = 1000;
+
+  public ITestAzureBlobFileSystemListStatusIterator() throws Exception {
+    super();
+  }
+
+  @Test
+  public void testListPath() throws Exception {
+    final AzureBlobFileSystem fs = getFileSystem();
+    String rootPath = "testRoot1";
+    final List<String> fileNames = createFiles(TEST_FILES_NUMBER, rootPath,
+        "testListPath");
+    AzureBlobFileSystemStore abfsStore = getAbfsStore(fs);
+    abfsStore.getAbfsConfiguration().setListMaxResults(10);
+    RemoteIterator<FileStatus> fsIt = fs.listStatusIterator(new Path(rootPath));
+    int itrCount = 0;
+    while (fsIt.hasNext()) {
+      FileStatus fileStatus = fsIt.next();
+      String pathStr = fileStatus.getPath().toString();
+      fileNames.remove(pathStr);
+      itrCount++;
+    }
+    assertEquals(TEST_FILES_NUMBER, itrCount);
+    assertEquals(0, fileNames.size());
+  }
+
+  @Test
+  public void testNextWhenNoMoreElementsPresent() throws Exception {
+    final AzureBlobFileSystem fs = getFileSystem();
+    String rootPathStr = "testRoot2";
+    Path rootPath = new Path(rootPathStr);
+    getFileSystem().mkdirs(rootPath);
+    RemoteIterator<FileStatus> fsItr = fs.listStatusIterator(rootPath);
+    fsItr = Mockito.spy(fsItr);
+    Mockito.doReturn(false).when(fsItr).hasNext();
+
+    RemoteIterator<FileStatus> finalFsItr = fsItr;
+    Assertions.assertThatThrownBy(() -> finalFsItr.next()).describedAs(
+        "next() should throw NoSuchElementException if hasNext() return "
+            + "false").isInstanceOf(NoSuchElementException.class);
+  }
+
+  @Test
+  public void testHasNextForEmptyDir() throws Exception {
+    final AzureBlobFileSystem fs = getFileSystem();
+    String rootPathStr = "testRoot3";
+    Path rootPath = new Path(rootPathStr);
+    getFileSystem().mkdirs(rootPath);
+    RemoteIterator<FileStatus> fsItr = fs.listStatusIterator(rootPath);
+    Assertions.assertThat(fsItr.hasNext())
+        .describedAs("hasNext returns false for empty directory").isFalse();
+  }
+
+  @Test
+  public void testHasNextForFile() throws Exception {
+    final AzureBlobFileSystem fs = getFileSystem();
+    String rootPathStr = "testRoot4";
+    Path rootPath = new Path(rootPathStr);
+    getFileSystem().create(rootPath);
+    RemoteIterator<FileStatus> fsItr = fs.listStatusIterator(rootPath);
+    Assertions.assertThat(fsItr.hasNext())
+        .describedAs("hasNext returns true for file").isTrue();
+  }
+
+  @Test
+  public void testHasNextForIOException() throws Exception {
+    final AzureBlobFileSystem fs = getFileSystem();
+    String rootPathStr = "testRoot5";
+    Path rootPath = new Path(rootPathStr);
+    getFileSystem().mkdirs(rootPath);
+    ListStatusRemoteIterator fsItr = (ListStatusRemoteIterator) fs
+        .listStatusIterator(rootPath);
+    Thread.sleep(1000);
+
+    String exceptionMessage = "test exception";
+    setPrivateField(fsItr, ListStatusRemoteIterator.class, "ioException",
+        new IOException(exceptionMessage));
+    setPrivateFinalField(fsItr, ListStatusRemoteIterator.class,
+        "iteratorsQueue", new ArrayBlockingQueue<Iterator>(1));
+
+    Assertions.assertThatThrownBy(() -> fsItr.hasNext()).describedAs(
+        "When ioException is not null and queue is empty exception should be "
+            + "thrown").isInstanceOf(IOException.class)
+        .hasMessage(exceptionMessage);
+  }
+
+  private void setPrivateField(Object obj, Class classObj, String fieldName,
+      Object value) throws NoSuchFieldException, IllegalAccessException {
+    Field field = classObj.getDeclaredField(fieldName);
+    field.setAccessible(true);
+    field.set(obj, value);
+  }
+
+  private void setPrivateFinalField(Object obj, Class classObj,
+      String fieldName, Object value)
+      throws NoSuchFieldException, IllegalAccessException {
+    Field field = classObj.getDeclaredField(fieldName);
+    field.setAccessible(true);
+    Field modifiersField = Field.class.getDeclaredField("modifiers");
+    modifiersField.setAccessible(true);
+    modifiersField.setInt(field, field.getModifiers() & ~Modifier.FINAL);
+    field.set(obj, value);
+  }
+
+  private List<String> createFiles(int numFiles, String rootPathStr,
+      String filenamePrefix)
+      throws ExecutionException, InterruptedException, IOException {
+    final List<Future<Void>> tasks = new ArrayList<>();
+    final List<String> fileNames = new ArrayList<>();
+    ExecutorService es = Executors.newFixedThreadPool(10);
+    final Path rootPath = new Path(rootPathStr);
+    for (int i = 0; i < numFiles; i++) {
+      final Path filePath = new Path(rootPath, filenamePrefix + i);
+      Callable<Void> callable = new Callable<Void>() {
+        @Override
+        public Void call() throws Exception {
+          getFileSystem().create(filePath);
+          fileNames.add(makeQualified(filePath).toString());
+          return null;
+        }
+      };
+      tasks.add(es.submit(callable));
+    }
+    for (Future<Void> task : tasks) {
+      task.get();
+    }
+    es.shutdownNow();
+    return fileNames;
+  }
+
+  private AzureBlobFileSystemStore getAbfsStore(FileSystem fs)

Review comment:
       this is why you need a list callback. This is low-level field abuse. Better to have a {{getAbfsStoreForTesting()}} call, at the very least




----------------------------------------------------------------
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 #2548: HADOOP-17475. ABFS: Implementing ListStatusRemoteIterator

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


   +1, Merging


----------------------------------------------------------------
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] bilaharith commented on a change in pull request #2548: HADOOP-17475. ABFS: Implementing ListStatusRemoteIterator

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



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java
##########
@@ -37,6 +37,8 @@
 import java.util.concurrent.Executors;
 import java.util.concurrent.Future;
 
+import org.apache.hadoop.fs.RemoteIterator;

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] bilaharith commented on a change in pull request #2548: HADOOP-17475. ABFS: Implementing ListStatusRemoteIterator

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



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java
##########
@@ -45,6 +45,8 @@
 import org.apache.commons.lang3.ArrayUtils;
 import org.apache.hadoop.fs.azurebfs.services.AbfsClient;
 import org.apache.hadoop.fs.azurebfs.services.AbfsClientThrottlingIntercept;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.fs.azurebfs.services.AbfsListStatusRemoteIterator;

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] steveloughran commented on pull request #2548: DRAFT PR: Implementing ListStatusRemoteIterator

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


   I need you to use `org.apache.hadoop.util.functional.RemoteIterators` as the wrapper iterators. These are only in trunk but will be backported with the rest of HADOOP-16380 after a few days of stabilisation.
   
   These iterators propagate the IOStatisticsSource interface, so when the innermost iterator collects cost/count of list calls, the stats will be visible to and collectable by callers.


----------------------------------------------------------------
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] bilaharith commented on a change in pull request #2548: HADOOP-17475. ABFS: Implementing ListStatusRemoteIterator

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



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsListStatusRemoteIterator.java
##########
@@ -0,0 +1,159 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs.services;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeUnit;
+import javax.activation.UnsupportedDataTypeException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.RemoteIterator;
+
+public class AbfsListStatusRemoteIterator
+    implements RemoteIterator<FileStatus> {
+
+  private static final Logger LOG = LoggerFactory
+      .getLogger(AbfsListStatusRemoteIterator.class);
+
+  private static final boolean FETCH_ALL_FALSE = false;
+  private static final int MAX_QUEUE_SIZE = 10;
+  private static final long POLL_WAIT_TIME_IN_MS = 250;
+
+  private final FileStatus fileStatus;
+  private final ListingSupport listingSupport;
+  private final ArrayBlockingQueue<Object> iteratorsQueue;
+
+  private volatile boolean isAsyncInProgress = false;
+  private boolean isIterationComplete = false;
+  private String continuation;
+  private Iterator<FileStatus> currIterator;
+
+  public AbfsListStatusRemoteIterator(final FileStatus fileStatus,
+      final ListingSupport listingSupport) {
+    this.fileStatus = fileStatus;
+    this.listingSupport = listingSupport;
+    iteratorsQueue = new ArrayBlockingQueue<>(MAX_QUEUE_SIZE);
+    currIterator = Collections.emptyIterator();
+    fetchBatchesAsync();
+  }
+
+  @Override
+  public boolean hasNext() throws IOException {
+    if (currIterator.hasNext()) {
+      return true;
+    }
+    currIterator = getNextIterator();
+    return currIterator.hasNext();
+  }
+
+  @Override
+  public FileStatus next() throws IOException {
+    if (!this.hasNext()) {
+      throw new NoSuchElementException();
+    }
+    return currIterator.next();
+  }
+
+  private Iterator<FileStatus> getNextIterator() throws IOException {
+    fetchBatchesAsync();
+    try {
+      Object obj = null;
+      while (obj == null
+          && (!isIterationComplete || !iteratorsQueue.isEmpty())) {
+        obj = iteratorsQueue.poll(POLL_WAIT_TIME_IN_MS, TimeUnit.MILLISECONDS);
+      }
+      if (obj == null) {
+        return Collections.emptyIterator();
+      } else if (obj instanceof Iterator) {

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] bilaharith commented on a change in pull request #2548: DRAFT PR: Implementing ListStatusRemoteIterator

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



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ListStatusRemoteIterator.java
##########
@@ -0,0 +1,149 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs.services;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.CompletableFuture;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore;
+
+public class ListStatusRemoteIterator implements RemoteIterator<FileStatus> {
+
+  private static final Logger LOG = LoggerFactory
+      .getLogger(ListStatusRemoteIterator.class);
+
+  private static final boolean FETCH_ALL_FALSE = false;
+  private static final int MAX_QUEUE_SIZE = 10;
+
+  private final Path path;
+  private final AzureBlobFileSystemStore abfsStore;
+  private final ArrayBlockingQueue<Iterator<FileStatus>> iteratorsQueue;
+  private final Object asyncOpLock = new Object();
+
+  private boolean firstBatch = true;
+  private boolean isAsyncInProgress = false;
+  private String continuation;
+  private Iterator<FileStatus> currIterator;
+  private IOException ioException;
+
+  public ListStatusRemoteIterator(final Path path,
+      final AzureBlobFileSystemStore abfsStore) {
+    this.path = path;
+    this.abfsStore = abfsStore;
+    iteratorsQueue = new ArrayBlockingQueue<>(MAX_QUEUE_SIZE);
+    currIterator = Collections.emptyIterator();
+    fetchBatchesAsync();
+  }
+
+  @Override
+  public boolean hasNext() throws IOException {
+    if (currIterator.hasNext()) {
+      return true;
+    }
+    updateCurrentIterator();
+    return currIterator.hasNext();
+  }
+
+  @Override
+  public FileStatus next() throws IOException {
+    if (!this.hasNext()) {
+      throw new NoSuchElementException();
+    }
+    return currIterator.next();
+  }
+
+  private void updateCurrentIterator() throws IOException {
+    fetchBatchesAsync();
+    synchronized (this) {
+      if (iteratorsQueue.isEmpty()) {
+        if (ioException != null) {
+          throw ioException;
+        }
+        if (isListingComplete()) {
+          return;
+        }
+      }
+    }
+    try {
+      currIterator = iteratorsQueue.take();
+      if (!currIterator.hasNext() && !isListingComplete()) {
+        updateCurrentIterator();
+      }
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      LOG.error("Thread got interrupted: {}", e);
+    }
+  }
+
+  private boolean isListingComplete() {
+    return !firstBatch && (continuation == null || continuation.isEmpty());
+  }
+
+  private void fetchBatchesAsync() {
+    CompletableFuture.runAsync(() -> asyncOp());

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] bilaharith commented on a change in pull request #2548: HADOOP-17475. ABFS: Implementing ListStatusRemoteIterator

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



##########
File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsListStatusIterator.java
##########
@@ -0,0 +1,339 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+
+import org.assertj.core.api.Assertions;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.fs.azurebfs.services.AbfsListStatusRemoteIterator;
+import org.apache.hadoop.fs.azurebfs.services.ListingSupport;
+
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.anyBoolean;
+import static org.mockito.ArgumentMatchers.anyList;
+import static org.mockito.ArgumentMatchers.nullable;
+import static org.mockito.Mockito.verify;
+
+/**
+ * Test ListStatusRemoteIterator operation.
+ */
+public class ITestAbfsListStatusIterator extends AbstractAbfsIntegrationTest {
+
+  private static final int TEST_FILES_NUMBER = 1000;
+
+  public ITestAbfsListStatusIterator() throws Exception {
+    super();
+  }
+
+  @Test
+  public void testListStatusRemoteIterator() throws Exception {
+    Path testDir = createTestDirectory();
+    setPageSize(10);
+    final List<String> fileNames = createFilesUnderDirectory(TEST_FILES_NUMBER,
+        testDir, "testListPath");
+
+    ListingSupport listngSupport = Mockito.spy(getFileSystem().getAbfsStore());

Review comment:
       Here I want to mock so that I can verify on the number of times few of the internal methods called.




----------------------------------------------------------------
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] vinaysbadami commented on a change in pull request #2548: HADOOP-17475. ABFS: Implementing ListStatusRemoteIterator

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



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsListStatusRemoteIterator.java
##########
@@ -0,0 +1,151 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs.services;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.CompletableFuture;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.RemoteIterator;
+
+public class AbfsListStatusRemoteIterator implements RemoteIterator<FileStatus> {
+
+  private static final Logger LOG = LoggerFactory
+      .getLogger(AbfsListStatusRemoteIterator.class);
+
+  private static final boolean FETCH_ALL_FALSE = false;
+  private static final int MAX_QUEUE_SIZE = 10;
+
+  private final FileStatus fileStatus;
+  private final ListingSupport listingSupport;
+  private final ArrayBlockingQueue<Iterator<FileStatus>> iteratorsQueue;
+  private final Object asyncOpLock = new Object();
+
+  private volatile boolean isAsyncInProgress = false;
+  private boolean firstBatch = true;
+  private String continuation;
+  private Iterator<FileStatus> currIterator;
+  private IOException ioException;
+
+  public AbfsListStatusRemoteIterator(final FileStatus fileStatus,
+      final ListingSupport listingSupport) {
+    this.fileStatus = fileStatus;
+    this.listingSupport = listingSupport;
+    iteratorsQueue = new ArrayBlockingQueue<>(MAX_QUEUE_SIZE);
+    currIterator = Collections.emptyIterator();
+    fetchBatchesAsync();
+  }
+
+  @Override
+  public boolean hasNext() throws IOException {
+    if (currIterator.hasNext()) {
+      return true;
+    }
+    updateCurrentIterator();
+    return currIterator.hasNext();
+  }
+
+  @Override
+  public FileStatus next() throws IOException {
+    if (!this.hasNext()) {
+      throw new NoSuchElementException();
+    }
+    return currIterator.next();
+  }
+
+  private void updateCurrentIterator() throws IOException {
+    fetchBatchesAsync();
+    synchronized (this) {
+      if (iteratorsQueue.isEmpty()) {
+        if (ioException != null) {
+          throw ioException;
+        }
+        if (isListingComplete()) {
+          return;
+        }
+      }
+    }
+    try {
+      currIterator = iteratorsQueue.take();
+      if (!currIterator.hasNext() && !isListingComplete()) {
+        updateCurrentIterator();
+      }
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      LOG.error("Thread got interrupted: {}", e);
+    }
+  }
+
+  private synchronized boolean isListingComplete() {
+    return !firstBatch && (continuation == null || continuation.isEmpty());
+  }
+
+  private void fetchBatchesAsync() {
+    if (isAsyncInProgress) {

Review comment:
       || ioException != null




----------------------------------------------------------------
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 #2548: DRAFT PR: Implementing ListStatusRemoteIterator

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


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |   1m 19s |  |  Docker mode activated.  |
   |||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  |  No case conflicting files found.  |
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |   |   0m  0s | [test4tests](test4tests) |  The patch appears to include 1 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |  33m 24s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |   0m 38s |  |  trunk passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04  |
   | +1 :green_heart: |  compile  |   0m 34s |  |  trunk passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01  |
   | +1 :green_heart: |  checkstyle  |   0m 28s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   0m 39s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  16m 26s |  |  branch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 31s |  |  trunk passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04  |
   | +1 :green_heart: |  javadoc  |   0m 29s |  |  trunk passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01  |
   | +0 :ok: |  spotbugs  |   0m 59s |  |  Used deprecated FindBugs config; considering switching to SpotBugs.  |
   | +1 :green_heart: |  findbugs  |   0m 56s |  |  trunk passed  |
   | -0 :warning: |  patch  |   1m 16s |  |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   |||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   0m 29s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 29s |  |  the patch passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04  |
   | +1 :green_heart: |  javac  |   0m 29s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 25s |  |  the patch passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01  |
   | +1 :green_heart: |  javac  |   0m 25s |  |  the patch passed  |
   | -0 :warning: |  checkstyle  |   0m 17s | [/diff-checkstyle-hadoop-tools_hadoop-azure.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/11/artifact/out/diff-checkstyle-hadoop-tools_hadoop-azure.txt) |  hadoop-tools/hadoop-azure: The patch generated 7 new + 4 unchanged - 0 fixed = 11 total (was 4)  |
   | +1 :green_heart: |  mvnsite  |   0m 28s |  |  the patch passed  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  shadedclient  |  15m  5s |  |  patch has no errors when building and testing our client artifacts.  |
   | -1 :x: |  javadoc  |   0m 25s | [/diff-javadoc-javadoc-hadoop-tools_hadoop-azure-jdkUbuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/11/artifact/out/diff-javadoc-javadoc-hadoop-tools_hadoop-azure-jdkUbuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04.txt) |  hadoop-tools_hadoop-azure-jdkUbuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04 with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04 generated 3 new + 15 unchanged - 2 fixed = 18 total (was 17)  |
   | -1 :x: |  javadoc  |   0m 24s | [/diff-javadoc-javadoc-hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/11/artifact/out/diff-javadoc-javadoc-hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01.txt) |  hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01 with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01 generated 3 new + 15 unchanged - 2 fixed = 18 total (was 17)  |
   | -1 :x: |  findbugs  |   1m  0s | [/new-findbugs-hadoop-tools_hadoop-azure.html](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/11/artifact/out/new-findbugs-hadoop-tools_hadoop-azure.html) |  hadoop-tools/hadoop-azure generated 2 new + 0 unchanged - 0 fixed = 2 total (was 0)  |
   |||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   1m 26s |  |  hadoop-azure in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 33s |  |  The patch does not generate ASF License warnings.  |
   |  |   |  78m 30s |  |  |
   
   
   | Reason | Tests |
   |-------:|:------|
   | FindBugs | module:hadoop-tools/hadoop-azure |
   |  |  Inconsistent synchronization of org.apache.hadoop.fs.azurebfs.services.AbfsListStatusRemoteIterator.ioException; locked 66% of time  Unsynchronized access at AbfsListStatusRemoteIterator.java:66% of time  Unsynchronized access at AbfsListStatusRemoteIterator.java:[line 128] |
   |  |  Exceptional return value of java.util.concurrent.ArrayBlockingQueue.offer(Object) ignored in org.apache.hadoop.fs.azurebfs.services.AbfsListStatusRemoteIterator.asyncOp()  At AbfsListStatusRemoteIterator.java:ignored in org.apache.hadoop.fs.azurebfs.services.AbfsListStatusRemoteIterator.asyncOp()  At AbfsListStatusRemoteIterator.java:[line 135] |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/11/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/2548 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient findbugs checkstyle |
   | uname | Linux 01c565f7c78d 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 / 630f8ddd2c7 |
   | Default Java | Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01 |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/11/testReport/ |
   | Max. process+thread count | 535 (vs. ulimit of 5500) |
   | modules | C: hadoop-tools/hadoop-azure U: hadoop-tools/hadoop-azure |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/11/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 a change in pull request #2548: HADOOP-17475. ABFS: Implementing ListStatusRemoteIterator

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



##########
File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsListStatusRemoteIterator.java
##########
@@ -0,0 +1,338 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+
+import org.assertj.core.api.Assertions;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.fs.azurebfs.services.AbfsListStatusRemoteIterator;
+import org.apache.hadoop.fs.azurebfs.services.ListingSupport;
+
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.anyBoolean;
+import static org.mockito.ArgumentMatchers.anyList;
+import static org.mockito.ArgumentMatchers.nullable;
+import static org.mockito.Mockito.verify;
+
+/**
+ * Test ListStatusRemoteIterator operation.
+ */
+public class ITestAbfsListStatusRemoteIterator extends AbstractAbfsIntegrationTest {
+
+  private static final int TEST_FILES_NUMBER = 1000;
+
+  public ITestAbfsListStatusRemoteIterator() throws Exception {
+    super();

Review comment:
       cut

##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java
##########
@@ -982,6 +985,19 @@ public boolean exists(Path f) throws IOException {
     return super.exists(f);
   }
 
+  @Override
+  public RemoteIterator<FileStatus> listStatusIterator(Path path)
+      throws IOException {
+    LOG.debug("AzureBlobFileSystem.listStatusIterator path : {}", path);
+    if (abfsStore.getAbfsConfiguration().enableAbfsListIterator()) {
+      AbfsListStatusRemoteIterator abfsLsItr =
+          new AbfsListStatusRemoteIterator(getFileStatus(path), abfsStore);
+      return RemoteIterators.typeCastingRemoteIterator(abfsLsItr);

Review comment:
       nice

##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsListStatusRemoteIterator.java
##########
@@ -0,0 +1,159 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs.services;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeUnit;
+import javax.activation.UnsupportedDataTypeException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.RemoteIterator;
+
+public class AbfsListStatusRemoteIterator
+    implements RemoteIterator<FileStatus> {
+
+  private static final Logger LOG = LoggerFactory
+      .getLogger(AbfsListStatusRemoteIterator.class);
+
+  private static final boolean FETCH_ALL_FALSE = false;
+  private static final int MAX_QUEUE_SIZE = 10;
+  private static final long POLL_WAIT_TIME_IN_MS = 250;
+
+  private final FileStatus fileStatus;
+  private final ListingSupport listingSupport;
+  private final ArrayBlockingQueue<Object> iteratorsQueue;
+
+  private volatile boolean isAsyncInProgress = false;
+  private boolean isIterationComplete = false;
+  private String continuation;
+  private Iterator<FileStatus> currIterator;
+
+  public AbfsListStatusRemoteIterator(final FileStatus fileStatus,
+      final ListingSupport listingSupport) {
+    this.fileStatus = fileStatus;
+    this.listingSupport = listingSupport;
+    iteratorsQueue = new ArrayBlockingQueue<>(MAX_QUEUE_SIZE);
+    currIterator = Collections.emptyIterator();
+    fetchBatchesAsync();
+  }
+
+  @Override
+  public boolean hasNext() throws IOException {
+    if (currIterator.hasNext()) {
+      return true;
+    }
+    currIterator = getNextIterator();
+    return currIterator.hasNext();
+  }
+
+  @Override
+  public FileStatus next() throws IOException {
+    if (!this.hasNext()) {
+      throw new NoSuchElementException();
+    }
+    return currIterator.next();
+  }
+
+  private Iterator<FileStatus> getNextIterator() throws IOException {
+    fetchBatchesAsync();
+    try {
+      Object obj = null;
+      while (obj == null
+          && (!isIterationComplete || !iteratorsQueue.isEmpty())) {
+        obj = iteratorsQueue.poll(POLL_WAIT_TIME_IN_MS, TimeUnit.MILLISECONDS);
+      }
+      if (obj == null) {
+        return Collections.emptyIterator();
+      } else if (obj instanceof Iterator) {
+        return (Iterator<FileStatus>) obj;
+      } else if (obj instanceof IOException) {
+        throw (IOException) obj;
+      } else {
+        throw new UnsupportedDataTypeException();
+      }
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      LOG.error("Thread got interrupted: {}", e);
+      throw new IOException(e);
+    }
+  }
+
+  private void fetchBatchesAsync() {
+    if (isAsyncInProgress || isIterationComplete) {
+      return;
+    }
+    synchronized (this) {
+      if (isAsyncInProgress || isIterationComplete) {
+        return;
+      }
+      isAsyncInProgress = true;
+    }
+    CompletableFuture.runAsync(() -> asyncOp());
+  }
+
+  private void asyncOp() {
+    try {
+      while (!isIterationComplete && iteratorsQueue.size() <= MAX_QUEUE_SIZE) {
+        addNextBatchIteratorToQueue();
+      }
+    } catch (IOException ioe) {
+      LOG.error("Fetching filestatuses failed", ioe);
+      try {
+        iteratorsQueue.put(ioe);
+      } catch (InterruptedException interruptedException) {
+        Thread.currentThread().interrupt();
+        LOG.error("Thread got interrupted: {}", interruptedException);
+      }
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      LOG.error("Thread got interrupted: {}", e);
+    } finally {
+      synchronized (this) {
+        isAsyncInProgress = false;
+      }
+    }
+  }
+
+  private void addNextBatchIteratorToQueue()
+      throws IOException, InterruptedException {
+    List<FileStatus> fileStatuses = new ArrayList<>();
+    continuation = listingSupport
+        .listStatus(fileStatus.getPath(), null, fileStatuses, FETCH_ALL_FALSE,
+            continuation);
+    if(!fileStatuses.isEmpty()) {

Review comment:
       nit: space after `if`

##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java
##########
@@ -982,6 +985,19 @@ public boolean exists(Path f) throws IOException {
     return super.exists(f);
   }
 
+  @Override
+  public RemoteIterator<FileStatus> listStatusIterator(Path path)
+      throws IOException {
+    LOG.debug("AzureBlobFileSystem.listStatusIterator path : {}", path);
+    if (abfsStore.getAbfsConfiguration().enableAbfsListIterator()) {

Review comment:
       OK, but it's something to plan to cut after a release has been out. So file a new JIRA about cutting the old one. 

##########
File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsListStatusRemoteIterator.java
##########
@@ -0,0 +1,338 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+
+import org.assertj.core.api.Assertions;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.fs.azurebfs.services.AbfsListStatusRemoteIterator;
+import org.apache.hadoop.fs.azurebfs.services.ListingSupport;
+
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.anyBoolean;
+import static org.mockito.ArgumentMatchers.anyList;
+import static org.mockito.ArgumentMatchers.nullable;
+import static org.mockito.Mockito.verify;
+
+/**
+ * Test ListStatusRemoteIterator operation.
+ */
+public class ITestAbfsListStatusRemoteIterator extends AbstractAbfsIntegrationTest {
+
+  private static final int TEST_FILES_NUMBER = 1000;
+
+  public ITestAbfsListStatusRemoteIterator() throws Exception {
+    super();
+  }
+
+  @Test
+  public void testAbfsIteratorWithHasNext() throws Exception {
+    Path testDir = createTestDirectory();
+    setPageSize(10);
+    final List<String> fileNames = createFilesUnderDirectory(TEST_FILES_NUMBER,
+        testDir, "testListPath");
+
+    ListingSupport listngSupport = Mockito.spy(getFileSystem().getAbfsStore());
+    RemoteIterator<FileStatus> fsItr = new AbfsListStatusRemoteIterator(
+        getFileSystem().getFileStatus(testDir), listngSupport);
+    Assertions.assertThat(fsItr)
+        .describedAs("RemoteIterator should be instance of "
+            + "AbfsListStatusRemoteIterator by default")
+        .isInstanceOf(AbfsListStatusRemoteIterator.class);
+    int itrCount = 0;
+    while (fsItr.hasNext()) {
+      FileStatus fileStatus = fsItr.next();
+      String pathStr = fileStatus.getPath().toString();
+      fileNames.remove(pathStr);
+      itrCount++;
+    }
+    Assertions.assertThat(itrCount)
+        .describedAs("Number of iterations should be equal to the files "
+            + "created")
+        .isEqualTo(TEST_FILES_NUMBER);
+    Assertions.assertThat(fileNames.size())
+        .describedAs("After removing every iterm found from the iterator, "
+            + "there should be no more elements in the fileNames")
+        .isEqualTo(0);
+    int minNumberOfInvokations = TEST_FILES_NUMBER / 10;
+    verify(listngSupport, Mockito.atLeast(minNumberOfInvokations))
+        .listStatus(any(Path.class), nullable(String.class),
+            anyList(), anyBoolean(),
+            nullable(String.class));
+  }
+
+  @Test
+  public void testAbfsIteratorWithoutHasNext() throws Exception {
+    Path testDir = createTestDirectory();
+    setPageSize(10);
+    final List<String> fileNames = createFilesUnderDirectory(TEST_FILES_NUMBER,
+        testDir, "testListPath");
+
+    ListingSupport listngSupport = Mockito.spy(getFileSystem().getAbfsStore());
+    RemoteIterator<FileStatus> fsItr = new AbfsListStatusRemoteIterator(
+        getFileSystem().getFileStatus(testDir), listngSupport);
+    Assertions.assertThat(fsItr)
+        .describedAs("RemoteIterator should be instance of "
+            + "AbfsListStatusRemoteIterator by default")
+        .isInstanceOf(AbfsListStatusRemoteIterator.class);
+    int itrCount = 0;
+    for (int i = 0; i < TEST_FILES_NUMBER; i++) {
+      FileStatus fileStatus = fsItr.next();
+      String pathStr = fileStatus.getPath().toString();
+      fileNames.remove(pathStr);
+      itrCount++;
+    }
+    Assertions.assertThatThrownBy(() -> fsItr.next())
+        .describedAs(
+            "next() should throw NoSuchElementException since next has been "
+                + "called " + TEST_FILES_NUMBER + " times")
+        .isInstanceOf(NoSuchElementException.class);
+    Assertions.assertThat(itrCount)
+        .describedAs("Number of iterations should be equal to the files "
+            + "created")
+        .isEqualTo(TEST_FILES_NUMBER);
+    Assertions.assertThat(fileNames.size())
+        .describedAs("After removing every iterm found from the iterator, "
+            + "there should be no more elements in the fileNames")
+        .isEqualTo(0);
+    int minNumberOfInvokations = TEST_FILES_NUMBER / 10;
+    verify(listngSupport, Mockito.atLeast(minNumberOfInvokations))
+        .listStatus(any(Path.class), nullable(String.class),
+            anyList(), anyBoolean(),
+            nullable(String.class));
+  }
+
+  @Test
+  public void testWithAbfsIteratorDisabled() throws Exception {
+    Path testDir = createTestDirectory();
+    setPageSize(10);
+    setEnableAbfsIterator(false);
+    final List<String> fileNames = createFilesUnderDirectory(TEST_FILES_NUMBER,
+        testDir, "testListPath");
+
+    RemoteIterator<FileStatus> fsItr =
+        getFileSystem().listStatusIterator(testDir);
+    Assertions.assertThat(fsItr)
+        .describedAs("RemoteIterator should not be instance of "
+            + "AbfsListStatusRemoteIterator when it is disabled")
+        .isNotInstanceOf(AbfsListStatusRemoteIterator.class);
+    int itrCount = 0;
+    while (fsItr.hasNext()) {
+      FileStatus fileStatus = fsItr.next();
+      String pathStr = fileStatus.getPath().toString();
+      fileNames.remove(pathStr);
+      itrCount++;
+    }
+    Assertions.assertThat(itrCount)
+        .describedAs("Number of iterations should be equal to the files "
+            + "created")
+        .isEqualTo(TEST_FILES_NUMBER);
+    Assertions.assertThat(fileNames.size())
+        .describedAs("After removing every iterm found from the iterator, "
+            + "there should be no more elements in the fileNames")
+        .isEqualTo(0);
+  }
+
+  @Test
+  public void testWithAbfsIteratorDisabledWithoutHasNext() throws Exception {
+    Path testDir = createTestDirectory();
+    setPageSize(10);
+    setEnableAbfsIterator(false);
+    final List<String> fileNames = createFilesUnderDirectory(TEST_FILES_NUMBER,
+        testDir, "testListPath");
+
+    RemoteIterator<FileStatus> fsItr =
+        getFileSystem().listStatusIterator(testDir);
+    Assertions.assertThat(fsItr)
+        .describedAs("RemoteIterator should not be instance of "
+            + "AbfsListStatusRemoteIterator when it is disabled")
+        .isNotInstanceOf(AbfsListStatusRemoteIterator.class);
+    int itrCount = 0;
+    for (int i = 0; i < TEST_FILES_NUMBER; i++) {
+      FileStatus fileStatus = fsItr.next();
+      String pathStr = fileStatus.getPath().toString();
+      fileNames.remove(pathStr);
+      itrCount++;
+    }
+    Assertions.assertThatThrownBy(() -> fsItr.next())
+        .describedAs(
+            "next() should throw NoSuchElementException since next has been "
+                + "called " + TEST_FILES_NUMBER + " times")
+        .isInstanceOf(NoSuchElementException.class);
+    Assertions.assertThat(itrCount)
+        .describedAs("Number of iterations should be equal to the files "
+            + "created")
+        .isEqualTo(TEST_FILES_NUMBER);
+    Assertions.assertThat(fileNames.size())
+        .describedAs("After removing every iterm found from the iterator, "
+            + "there should be no more elements in the fileNames")
+        .isEqualTo(0);
+  }
+
+  @Test
+  public void testNextWhenNoMoreElementsPresent() throws Exception {
+    Path testDir = createTestDirectory();
+    setPageSize(10);
+    RemoteIterator fsItr =
+        new AbfsListStatusRemoteIterator(getFileSystem().getFileStatus(testDir),
+            getFileSystem().getAbfsStore());
+    fsItr = Mockito.spy(fsItr);
+    Mockito.doReturn(false).when(fsItr).hasNext();
+
+    RemoteIterator<FileStatus> finalFsItr = fsItr;
+    Assertions.assertThatThrownBy(() -> finalFsItr.next())
+        .describedAs(
+        "next() should throw NoSuchElementException if hasNext() return "
+            + "false")
+        .isInstanceOf(NoSuchElementException.class);
+  }
+
+  @Test
+  public void testHasNextForEmptyDir() throws Exception {
+    Path testDir = createTestDirectory();
+    setPageSize(10);
+    RemoteIterator<FileStatus> fsItr = getFileSystem()
+        .listStatusIterator(testDir);
+    Assertions.assertThat(fsItr.hasNext())
+        .describedAs("hasNext returns false for empty directory")
+        .isFalse();
+  }
+
+  @Test
+  public void testHasNextForFile() throws Exception {
+    final AzureBlobFileSystem fs = getFileSystem();
+    String testFileName = "testFile";
+    Path testFile = new Path(testFileName);
+    getFileSystem().create(testFile);
+    setPageSize(10);
+    RemoteIterator<FileStatus> fsItr = fs.listStatusIterator(testFile);
+    Assertions.assertThat(fsItr.hasNext())
+        .describedAs("hasNext returns true for file").isTrue();
+    Assertions.assertThat(fsItr.next().getPath().toString())
+        .describedAs("next returns the file itself")
+        .endsWith(testFileName);
+  }
+
+  @Test
+  public void testIOException() throws Exception {
+    Path testDir = createTestDirectory();
+    setPageSize(10);
+    getFileSystem().mkdirs(testDir);
+
+    String exceptionMessage = "test exception";
+    ListingSupport lsSupport =getMockListingSupport(exceptionMessage);
+    RemoteIterator fsItr =
+        new AbfsListStatusRemoteIterator(getFileSystem().getFileStatus(testDir),
+        lsSupport);
+
+    Assertions.assertThatThrownBy(() -> fsItr.next())
+        .describedAs(
+        "When ioException is not null and queue is empty exception should be "
+            + "thrown")
+        .isInstanceOf(IOException.class)
+        .hasMessage(exceptionMessage);
+  }
+
+  @Test
+  public void testNonExistingPath() throws Throwable {
+    Path nonExistingDir = new Path("nonExistingPath");
+    Assertions.assertThatThrownBy(
+        () -> getFileSystem().listStatusIterator(nonExistingDir)).describedAs(
+        "test the listStatusIterator call on a path which is not "
+            + "present should result in FileNotFoundException")
+        .isInstanceOf(FileNotFoundException.class);
+  }
+
+  private ListingSupport getMockListingSupport(String exceptionMessage) {
+    return new ListingSupport() {
+      @Override
+      public FileStatus[] listStatus(Path path) throws IOException {
+        return null;
+      }
+
+      @Override
+      public FileStatus[] listStatus(Path path, String startFrom)
+          throws IOException {
+        return null;
+      }
+
+      @Override
+      public String listStatus(Path path, String startFrom,
+          List<FileStatus> fileStatuses, boolean fetchAll, String continuation)
+          throws IOException {
+        throw new IOException(exceptionMessage);
+      }
+    };
+  }
+
+  private Path createTestDirectory() throws IOException {
+    String testDirectoryName = "testDirectory" + System.currentTimeMillis();
+    Path testDirectory = new Path(testDirectoryName);
+    getFileSystem().mkdirs(testDirectory);
+    return testDirectory;
+  }
+
+  private void setEnableAbfsIterator(boolean shouldEnable) throws IOException {
+    AzureBlobFileSystemStore abfsStore = getAbfsStore(getFileSystem());
+    abfsStore.getAbfsConfiguration().setEnableAbfsListIterator(shouldEnable);
+  }
+
+  private void setPageSize(int pageSize) throws IOException {
+    AzureBlobFileSystemStore abfsStore = getAbfsStore(getFileSystem());
+    abfsStore.getAbfsConfiguration().setListMaxResults(pageSize);
+  }
+
+  private List<String> createFilesUnderDirectory(int numFiles, Path rootPath,
+      String filenamePrefix)
+      throws ExecutionException, InterruptedException, IOException {
+    final List<Future<Void>> tasks = new ArrayList<>();
+    final List<String> fileNames = new ArrayList<>();
+    ExecutorService es = Executors.newFixedThreadPool(10);
+    for (int i = 0; i < numFiles; i++) {
+      final Path filePath = new Path(rootPath, filenamePrefix + i);
+      Callable<Void> callable = () -> {
+        getFileSystem().create(filePath);
+        fileNames.add(makeQualified(filePath).toString());
+        return null;
+      };
+      tasks.add(es.submit(callable));
+    }
+    for (Future<Void> task : tasks) {
+      task.get();
+    }
+    es.shutdownNow();

Review comment:
       this should be in a finally block

##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsListStatusRemoteIterator.java
##########
@@ -0,0 +1,159 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs.services;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeUnit;
+import javax.activation.UnsupportedDataTypeException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.RemoteIterator;
+
+public class AbfsListStatusRemoteIterator
+    implements RemoteIterator<FileStatus> {
+
+  private static final Logger LOG = LoggerFactory
+      .getLogger(AbfsListStatusRemoteIterator.class);
+
+  private static final boolean FETCH_ALL_FALSE = false;
+  private static final int MAX_QUEUE_SIZE = 10;
+  private static final long POLL_WAIT_TIME_IN_MS = 250;
+
+  private final FileStatus fileStatus;
+  private final ListingSupport listingSupport;
+  private final ArrayBlockingQueue<Object> iteratorsQueue;
+
+  private volatile boolean isAsyncInProgress = false;
+  private boolean isIterationComplete = false;
+  private String continuation;
+  private Iterator<FileStatus> currIterator;
+
+  public AbfsListStatusRemoteIterator(final FileStatus fileStatus,
+      final ListingSupport listingSupport) {
+    this.fileStatus = fileStatus;
+    this.listingSupport = listingSupport;
+    iteratorsQueue = new ArrayBlockingQueue<>(MAX_QUEUE_SIZE);
+    currIterator = Collections.emptyIterator();
+    fetchBatchesAsync();
+  }
+
+  @Override
+  public boolean hasNext() throws IOException {
+    if (currIterator.hasNext()) {
+      return true;
+    }
+    currIterator = getNextIterator();
+    return currIterator.hasNext();
+  }
+
+  @Override
+  public FileStatus next() throws IOException {
+    if (!this.hasNext()) {
+      throw new NoSuchElementException();
+    }
+    return currIterator.next();
+  }
+
+  private Iterator<FileStatus> getNextIterator() throws IOException {
+    fetchBatchesAsync();
+    try {
+      Object obj = null;
+      while (obj == null
+          && (!isIterationComplete || !iteratorsQueue.isEmpty())) {
+        obj = iteratorsQueue.poll(POLL_WAIT_TIME_IN_MS, TimeUnit.MILLISECONDS);
+      }
+      if (obj == null) {
+        return Collections.emptyIterator();
+      } else if (obj instanceof Iterator) {

Review comment:
       I can't help thinking this a bit of an abuse of a queue to mix object types in there -but to do it in any other way within java would be pretty convoluted (new class, essentially), so let's go with what is here. Just need to make sure there's test coverage for the IOE path

##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ListingSupport.java
##########
@@ -0,0 +1,75 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs.services;
+
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+
+public interface ListingSupport {
+
+  /**
+   * @param path The list path.
+   * @return the entries in the path.
+   */
+  FileStatus[] listStatus(Path path) throws IOException;
+
+  /**
+   * @param path      Path the list path.
+   * @param startFrom The entry name that list results should start with.
+   *                  For example, if folder "/folder" contains four
+   *                  files: "afile", "bfile", "hfile", "ifile". Then
+   *                  listStatus(Path("/folder"), "hfile") will return
+   *                  "/folder/hfile" and "folder/ifile" Notice that if
+   *                  startFrom is a non-existent entry name, then the
+   *                  list response contains all entries after this
+   *                  non-existent entry in lexical order: listStatus
+   *                  (Path("/folder"), "cfile") will return
+   *                  "/folder/hfile" and "/folder/ifile".
+   * @return the entries in the path start from  "startFrom" in lexical order.
+   */
+  @InterfaceStability.Unstable

Review comment:
       add that at the actual interface, along with the @Private . Not that we'd expect anyone to use it




----------------------------------------------------------------
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 #2548: HADOOP-17475. ABFS: Implementing ListStatusRemoteIterator

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


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |  42m 17s |  |  Docker mode activated.  |
   |||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  |  No case conflicting files found.  |
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |   |   0m  0s | [test4tests](test4tests) |  The patch appears to include 1 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |  32m 24s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |   0m 38s |  |  trunk passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  compile  |   0m 33s |  |  trunk passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~20.04-b01  |
   | +1 :green_heart: |  checkstyle  |   0m 27s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   0m 39s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  14m 21s |  |  branch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 32s |  |  trunk passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javadoc  |   0m 30s |  |  trunk passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~20.04-b01  |
   | +0 :ok: |  spotbugs  |   1m  0s |  |  Used deprecated FindBugs config; considering switching to SpotBugs.  |
   | +1 :green_heart: |  findbugs  |   0m 58s |  |  trunk passed  |
   | -0 :warning: |  patch  |   1m 19s |  |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   |||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   0m 29s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 29s |  |  the patch passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javac  |   0m 29s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 26s |  |  the patch passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~20.04-b01  |
   | +1 :green_heart: |  javac  |   0m 26s |  |  the patch passed  |
   | -0 :warning: |  checkstyle  |   0m 18s | [/diff-checkstyle-hadoop-tools_hadoop-azure.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/16/artifact/out/diff-checkstyle-hadoop-tools_hadoop-azure.txt) |  hadoop-tools/hadoop-azure: The patch generated 1 new + 4 unchanged - 0 fixed = 5 total (was 4)  |
   | +1 :green_heart: |  mvnsite  |   0m 29s |  |  the patch passed  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  shadedclient  |  13m 10s |  |  patch has no errors when building and testing our client artifacts.  |
   | -1 :x: |  javadoc  |   0m 27s | [/diff-javadoc-javadoc-hadoop-tools_hadoop-azure-jdkUbuntu-11.0.9.1+1-Ubuntu-0ubuntu1.20.04.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/16/artifact/out/diff-javadoc-javadoc-hadoop-tools_hadoop-azure-jdkUbuntu-11.0.9.1+1-Ubuntu-0ubuntu1.20.04.txt) |  hadoop-tools_hadoop-azure-jdkUbuntu-11.0.9.1+1-Ubuntu-0ubuntu1.20.04 with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.20.04 generated 3 new + 15 unchanged - 2 fixed = 18 total (was 17)  |
   | -1 :x: |  javadoc  |   0m 25s | [/diff-javadoc-javadoc-hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_275-8u275-b01-0ubuntu1~20.04-b01.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/16/artifact/out/diff-javadoc-javadoc-hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_275-8u275-b01-0ubuntu1~20.04-b01.txt) |  hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_275-8u275-b01-0ubuntu1~20.04-b01 with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~20.04-b01 generated 3 new + 15 unchanged - 2 fixed = 18 total (was 17)  |
   | -1 :x: |  findbugs  |   0m 58s | [/new-findbugs-hadoop-tools_hadoop-azure.html](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/16/artifact/out/new-findbugs-hadoop-tools_hadoop-azure.html) |  hadoop-tools/hadoop-azure generated 1 new + 0 unchanged - 0 fixed = 1 total (was 0)  |
   |||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   1m 55s |  |  hadoop-azure in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 33s |  |  The patch does not generate ASF License warnings.  |
   |  |   | 115m 11s |  |  |
   
   
   | Reason | Tests |
   |-------:|:------|
   | FindBugs | module:hadoop-tools/hadoop-azure |
   |  |  Inconsistent synchronization of org.apache.hadoop.fs.azurebfs.services.AbfsListStatusRemoteIterator.continuation; locked 50% of time  Unsynchronized access at AbfsListStatusRemoteIterator.java:50% of time  Unsynchronized access at AbfsListStatusRemoteIterator.java:[line 147] |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/16/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/2548 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient findbugs checkstyle |
   | uname | Linux 91155a4390ce 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 / 80c7404b519 |
   | Default Java | Private Build-1.8.0_275-8u275-b01-0ubuntu1~20.04-b01 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.20.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_275-8u275-b01-0ubuntu1~20.04-b01 |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/16/testReport/ |
   | Max. process+thread count | 627 (vs. ulimit of 5500) |
   | modules | C: hadoop-tools/hadoop-azure U: hadoop-tools/hadoop-azure |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/16/console |
   | versions | git=2.25.1 maven=3.6.3 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] bilaharith commented on a change in pull request #2548: DRAFT PR: Implementing ListStatusRemoteIterator

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



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ListStatusRemoteIterator.java
##########
@@ -0,0 +1,149 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs.services;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.CompletableFuture;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore;
+
+public class ListStatusRemoteIterator implements RemoteIterator<FileStatus> {
+
+  private static final Logger LOG = LoggerFactory
+      .getLogger(ListStatusRemoteIterator.class);
+
+  private static final boolean FETCH_ALL_FALSE = false;
+  private static final int MAX_QUEUE_SIZE = 10;
+
+  private final Path path;
+  private final AzureBlobFileSystemStore abfsStore;
+  private final ArrayBlockingQueue<Iterator<FileStatus>> iteratorsQueue;
+  private final Object asyncOpLock = new Object();
+
+  private boolean firstBatch = true;
+  private boolean isAsyncInProgress = false;

Review comment:
       Not necessary since the writes are made in synchronised block. Also rather than the atomicity synchronisation is the need here.




----------------------------------------------------------------
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 #2548: HADOOP-17475. ABFS: Implementing ListStatusRemoteIterator

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


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |   1m 30s |  |  Docker mode activated.  |
   |||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  |  No case conflicting files found.  |
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |   |   0m  0s | [test4tests](test4tests) |  The patch appears to include 1 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |  33m 53s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |   0m 36s |  |  trunk passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04  |
   | +1 :green_heart: |  compile  |   0m 34s |  |  trunk passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01  |
   | +1 :green_heart: |  checkstyle  |   0m 26s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   0m 37s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  16m 22s |  |  branch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 29s |  |  trunk passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04  |
   | +1 :green_heart: |  javadoc  |   0m 29s |  |  trunk passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01  |
   | +0 :ok: |  spotbugs  |   1m  1s |  |  Used deprecated FindBugs config; considering switching to SpotBugs.  |
   | +1 :green_heart: |  findbugs  |   0m 58s |  |  trunk passed  |
   |||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   0m 30s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 29s |  |  the patch passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04  |
   | +1 :green_heart: |  javac  |   0m 29s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 26s |  |  the patch passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01  |
   | +1 :green_heart: |  javac  |   0m 26s |  |  the patch passed  |
   | +1 :green_heart: |  checkstyle  |   0m 17s |  |  the patch passed  |
   | +1 :green_heart: |  mvnsite  |   0m 29s |  |  the patch passed  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  shadedclient  |  14m 59s |  |  patch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 26s |  |  the patch passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04  |
   | +1 :green_heart: |  javadoc  |   0m 25s |  |  the patch passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01  |
   | -1 :x: |  findbugs  |   0m 59s | [/new-findbugs-hadoop-tools_hadoop-azure.html](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/7/artifact/out/new-findbugs-hadoop-tools_hadoop-azure.html) |  hadoop-tools/hadoop-azure generated 4 new + 0 unchanged - 0 fixed = 4 total (was 0)  |
   |||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   1m 27s |  |  hadoop-azure in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 34s |  |  The patch does not generate ASF License warnings.  |
   |  |   |  79m 10s |  |  |
   
   
   | Reason | Tests |
   |-------:|:------|
   | FindBugs | module:hadoop-tools/hadoop-azure |
   |  |  Possible doublecheck on org.apache.hadoop.fs.azurebfs.services.ListStatusRemoteIterator.isAsyncInProgress in org.apache.hadoop.fs.azurebfs.services.ListStatusRemoteIterator.asyncOp()  At ListStatusRemoteIterator.java:org.apache.hadoop.fs.azurebfs.services.ListStatusRemoteIterator.asyncOp()  At ListStatusRemoteIterator.java:[lines 115-119] |
   |  |  Inconsistent synchronization of org.apache.hadoop.fs.azurebfs.services.ListStatusRemoteIterator.continuation; locked 40% of time  Unsynchronized access at ListStatusRemoteIterator.java:40% of time  Unsynchronized access at ListStatusRemoteIterator.java:[line 107] |
   |  |  Inconsistent synchronization of org.apache.hadoop.fs.azurebfs.services.ListStatusRemoteIterator.firstBatch; locked 66% of time  Unsynchronized access at ListStatusRemoteIterator.java:66% of time  Unsynchronized access at ListStatusRemoteIterator.java:[line 107] |
   |  |  Inconsistent synchronization of org.apache.hadoop.fs.azurebfs.services.ListStatusRemoteIterator.ioException; locked 66% of time  Unsynchronized access at ListStatusRemoteIterator.java:66% of time  Unsynchronized access at ListStatusRemoteIterator.java:[line 129] |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/7/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/2548 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient findbugs checkstyle |
   | uname | Linux ad17f0141ddd 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 / ec22850dbef |
   | Default Java | Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01 |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/7/testReport/ |
   | Max. process+thread count | 733 (vs. ulimit of 5500) |
   | modules | C: hadoop-tools/hadoop-azure U: hadoop-tools/hadoop-azure |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/7/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 #2548: DRAFT PR: Implementing ListStatusRemoteIterator

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


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |   1m  5s |  |  Docker mode activated.  |
   |||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  |  No case conflicting files found.  |
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |   |   0m  0s | [test4tests](test4tests) |  The patch appears to include 1 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |  34m 29s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |   0m 36s |  |  trunk passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04  |
   | +1 :green_heart: |  compile  |   0m 33s |  |  trunk passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01  |
   | +1 :green_heart: |  checkstyle  |   0m 26s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   0m 38s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  16m 28s |  |  branch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 31s |  |  trunk passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04  |
   | +1 :green_heart: |  javadoc  |   0m 28s |  |  trunk passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01  |
   | +0 :ok: |  spotbugs  |   0m 58s |  |  Used deprecated FindBugs config; considering switching to SpotBugs.  |
   | +1 :green_heart: |  findbugs  |   0m 57s |  |  trunk passed  |
   |||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   0m 30s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 30s |  |  the patch passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04  |
   | -1 :x: |  javac  |   0m 30s | [/diff-compile-javac-hadoop-tools_hadoop-azure-jdkUbuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/6/artifact/out/diff-compile-javac-hadoop-tools_hadoop-azure-jdkUbuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04.txt) |  hadoop-tools_hadoop-azure-jdkUbuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04 with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04 generated 1 new + 14 unchanged - 1 fixed = 15 total (was 15)  |
   | +1 :green_heart: |  compile  |   0m 26s |  |  the patch passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01  |
   | -1 :x: |  javac  |   0m 26s | [/diff-compile-javac-hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/6/artifact/out/diff-compile-javac-hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01.txt) |  hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01 with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01 generated 1 new + 8 unchanged - 1 fixed = 9 total (was 9)  |
   | +1 :green_heart: |  checkstyle  |   0m 17s |  |  the patch passed  |
   | +1 :green_heart: |  mvnsite  |   0m 29s |  |  the patch passed  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  shadedclient  |  15m 37s |  |  patch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 24s |  |  the patch passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04  |
   | +1 :green_heart: |  javadoc  |   0m 23s |  |  the patch passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01  |
   | -1 :x: |  findbugs  |   1m  1s | [/new-findbugs-hadoop-tools_hadoop-azure.html](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/6/artifact/out/new-findbugs-hadoop-tools_hadoop-azure.html) |  hadoop-tools/hadoop-azure generated 4 new + 0 unchanged - 0 fixed = 4 total (was 0)  |
   |||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   1m 25s |  |  hadoop-azure in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 33s |  |  The patch does not generate ASF License warnings.  |
   |  |   |  79m 38s |  |  |
   
   
   | Reason | Tests |
   |-------:|:------|
   | FindBugs | module:hadoop-tools/hadoop-azure |
   |  |  Possible doublecheck on org.apache.hadoop.fs.azurebfs.services.ListStatusRemoteIterator.isAsyncInProgress in org.apache.hadoop.fs.azurebfs.services.ListStatusRemoteIterator.asyncOp()  At ListStatusRemoteIterator.java:org.apache.hadoop.fs.azurebfs.services.ListStatusRemoteIterator.asyncOp()  At ListStatusRemoteIterator.java:[lines 115-119] |
   |  |  Inconsistent synchronization of org.apache.hadoop.fs.azurebfs.services.ListStatusRemoteIterator.continuation; locked 40% of time  Unsynchronized access at ListStatusRemoteIterator.java:40% of time  Unsynchronized access at ListStatusRemoteIterator.java:[line 107] |
   |  |  Inconsistent synchronization of org.apache.hadoop.fs.azurebfs.services.ListStatusRemoteIterator.firstBatch; locked 66% of time  Unsynchronized access at ListStatusRemoteIterator.java:66% of time  Unsynchronized access at ListStatusRemoteIterator.java:[line 107] |
   |  |  Inconsistent synchronization of org.apache.hadoop.fs.azurebfs.services.ListStatusRemoteIterator.ioException; locked 66% of time  Unsynchronized access at ListStatusRemoteIterator.java:66% of time  Unsynchronized access at ListStatusRemoteIterator.java:[line 129] |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/6/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/2548 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient findbugs checkstyle |
   | uname | Linux 7a17bcb50465 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 / ec22850dbef |
   | Default Java | Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01 |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/6/testReport/ |
   | Max. process+thread count | 544 (vs. ulimit of 5500) |
   | modules | C: hadoop-tools/hadoop-azure U: hadoop-tools/hadoop-azure |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/6/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] vinaysbadami commented on a change in pull request #2548: DRAFT PR: Implementing ListStatusRemoteIterator

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



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java
##########
@@ -982,6 +984,14 @@ public boolean exists(Path f) throws IOException {
     return super.exists(f);
   }
 
+  @Override
+  public RemoteIterator<FileStatus> listStatusIterator(Path path)
+      throws IOException {
+    LOG.debug("AzureBlobFileSystem.listStatusIterator path : {}", path);

Review comment:
       do under config

##########
File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemListStatusIterator.java
##########
@@ -0,0 +1,190 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs;
+
+import java.io.IOException;
+import java.lang.reflect.Field;
+import java.lang.reflect.Modifier;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+
+import org.assertj.core.api.Assertions;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+import org.apache.hadoop.fs.azurebfs.services.ListStatusRemoteIterator;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+
+/**
+ * Test listStatus operation.
+ */
+public class ITestAzureBlobFileSystemListStatusIterator
+    extends AbstractAbfsIntegrationTest {
+
+  private static final int TEST_FILES_NUMBER = 1000;
+
+  public ITestAzureBlobFileSystemListStatusIterator() throws Exception {
+    super();
+  }
+
+  @Test
+  public void testListPath() throws Exception {
+    final AzureBlobFileSystem fs = getFileSystem();
+    String rootPath = "testRoot1";
+    final List<String> fileNames = createFiles(TEST_FILES_NUMBER, rootPath,
+        "testListPath");
+    AzureBlobFileSystemStore abfsStore = getAbfsStore(fs);
+    abfsStore.getAbfsConfiguration().setListMaxResults(10);
+    RemoteIterator<FileStatus> fsIt = fs.listStatusIterator(new Path(rootPath));
+    int itrCount = 0;
+    while (fsIt.hasNext()) {
+      FileStatus fileStatus = fsIt.next();
+      String pathStr = fileStatus.getPath().toString();
+      fileNames.remove(pathStr);
+      itrCount++;
+    }
+    assertEquals(TEST_FILES_NUMBER, itrCount);
+    assertEquals(0, fileNames.size());
+  }
+
+  @Test
+  public void testNextWhenNoMoreElementsPresent() throws Exception {
+    final AzureBlobFileSystem fs = getFileSystem();
+    String rootPathStr = "testRoot2";
+    Path rootPath = new Path(rootPathStr);
+    getFileSystem().mkdirs(rootPath);
+    RemoteIterator<FileStatus> fsItr = fs.listStatusIterator(rootPath);
+    fsItr = Mockito.spy(fsItr);
+    Mockito.doReturn(false).when(fsItr).hasNext();
+
+    RemoteIterator<FileStatus> finalFsItr = fsItr;
+    Assertions.assertThatThrownBy(() -> finalFsItr.next()).describedAs(
+        "next() should throw NoSuchElementException if hasNext() return "
+            + "false").isInstanceOf(NoSuchElementException.class);
+  }
+
+  @Test
+  public void testHasNextForEmptyDir() throws Exception {
+    final AzureBlobFileSystem fs = getFileSystem();
+    String rootPathStr = "testRoot3";
+    Path rootPath = new Path(rootPathStr);
+    getFileSystem().mkdirs(rootPath);
+    RemoteIterator<FileStatus> fsItr = fs.listStatusIterator(rootPath);
+    Assertions.assertThat(fsItr.hasNext())

Review comment:
       do next and ensure correct exception

##########
File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemListStatusIterator.java
##########
@@ -0,0 +1,190 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs;
+
+import java.io.IOException;
+import java.lang.reflect.Field;
+import java.lang.reflect.Modifier;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+
+import org.assertj.core.api.Assertions;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+import org.apache.hadoop.fs.azurebfs.services.ListStatusRemoteIterator;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+
+/**
+ * Test listStatus operation.
+ */
+public class ITestAzureBlobFileSystemListStatusIterator
+    extends AbstractAbfsIntegrationTest {
+
+  private static final int TEST_FILES_NUMBER = 1000;
+
+  public ITestAzureBlobFileSystemListStatusIterator() throws Exception {
+    super();
+  }
+
+  @Test
+  public void testListPath() throws Exception {
+    final AzureBlobFileSystem fs = getFileSystem();
+    String rootPath = "testRoot1";
+    final List<String> fileNames = createFiles(TEST_FILES_NUMBER, rootPath,
+        "testListPath");
+    AzureBlobFileSystemStore abfsStore = getAbfsStore(fs);
+    abfsStore.getAbfsConfiguration().setListMaxResults(10);
+    RemoteIterator<FileStatus> fsIt = fs.listStatusIterator(new Path(rootPath));
+    int itrCount = 0;
+    while (fsIt.hasNext()) {
+      FileStatus fileStatus = fsIt.next();
+      String pathStr = fileStatus.getPath().toString();
+      fileNames.remove(pathStr);
+      itrCount++;
+    }
+    assertEquals(TEST_FILES_NUMBER, itrCount);
+    assertEquals(0, fileNames.size());
+  }
+
+  @Test
+  public void testNextWhenNoMoreElementsPresent() throws Exception {
+    final AzureBlobFileSystem fs = getFileSystem();
+    String rootPathStr = "testRoot2";
+    Path rootPath = new Path(rootPathStr);
+    getFileSystem().mkdirs(rootPath);
+    RemoteIterator<FileStatus> fsItr = fs.listStatusIterator(rootPath);
+    fsItr = Mockito.spy(fsItr);
+    Mockito.doReturn(false).when(fsItr).hasNext();
+
+    RemoteIterator<FileStatus> finalFsItr = fsItr;
+    Assertions.assertThatThrownBy(() -> finalFsItr.next()).describedAs(
+        "next() should throw NoSuchElementException if hasNext() return "
+            + "false").isInstanceOf(NoSuchElementException.class);
+  }
+
+  @Test
+  public void testHasNextForEmptyDir() throws Exception {
+    final AzureBlobFileSystem fs = getFileSystem();
+    String rootPathStr = "testRoot3";
+    Path rootPath = new Path(rootPathStr);
+    getFileSystem().mkdirs(rootPath);
+    RemoteIterator<FileStatus> fsItr = fs.listStatusIterator(rootPath);
+    Assertions.assertThat(fsItr.hasNext())
+        .describedAs("hasNext returns false for empty directory").isFalse();
+  }
+
+  @Test
+  public void testHasNextForFile() throws Exception {
+    final AzureBlobFileSystem fs = getFileSystem();
+    String rootPathStr = "testRoot4";
+    Path rootPath = new Path(rootPathStr);
+    getFileSystem().create(rootPath);
+    RemoteIterator<FileStatus> fsItr = fs.listStatusIterator(rootPath);
+    Assertions.assertThat(fsItr.hasNext())
+        .describedAs("hasNext returns true for file").isTrue();
+  }
+
+  @Test
+  public void testHasNextForIOException() throws Exception {
+    final AzureBlobFileSystem fs = getFileSystem();
+    String rootPathStr = "testRoot5";
+    Path rootPath = new Path(rootPathStr);
+    getFileSystem().mkdirs(rootPath);
+    ListStatusRemoteIterator fsItr = (ListStatusRemoteIterator) fs
+        .listStatusIterator(rootPath);
+    Thread.sleep(1000);
+
+    String exceptionMessage = "test exception";

Review comment:
       make queue empty by iterating and then set the exception and do another hasNext 

##########
File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemListStatusIterator.java
##########
@@ -0,0 +1,190 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs;
+
+import java.io.IOException;
+import java.lang.reflect.Field;
+import java.lang.reflect.Modifier;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+
+import org.assertj.core.api.Assertions;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+import org.apache.hadoop.fs.azurebfs.services.ListStatusRemoteIterator;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+
+/**
+ * Test listStatus operation.
+ */
+public class ITestAzureBlobFileSystemListStatusIterator
+    extends AbstractAbfsIntegrationTest {
+
+  private static final int TEST_FILES_NUMBER = 1000;
+
+  public ITestAzureBlobFileSystemListStatusIterator() throws Exception {
+    super();
+  }
+
+  @Test
+  public void testListPath() throws Exception {
+    final AzureBlobFileSystem fs = getFileSystem();
+    String rootPath = "testRoot1";
+    final List<String> fileNames = createFiles(TEST_FILES_NUMBER, rootPath,
+        "testListPath");
+    AzureBlobFileSystemStore abfsStore = getAbfsStore(fs);
+    abfsStore.getAbfsConfiguration().setListMaxResults(10);
+    RemoteIterator<FileStatus> fsIt = fs.listStatusIterator(new Path(rootPath));
+    int itrCount = 0;
+    while (fsIt.hasNext()) {
+      FileStatus fileStatus = fsIt.next();
+      String pathStr = fileStatus.getPath().toString();
+      fileNames.remove(pathStr);
+      itrCount++;
+    }
+    assertEquals(TEST_FILES_NUMBER, itrCount);
+    assertEquals(0, fileNames.size());
+  }
+
+  @Test
+  public void testNextWhenNoMoreElementsPresent() throws Exception {
+    final AzureBlobFileSystem fs = getFileSystem();
+    String rootPathStr = "testRoot2";
+    Path rootPath = new Path(rootPathStr);
+    getFileSystem().mkdirs(rootPath);
+    RemoteIterator<FileStatus> fsItr = fs.listStatusIterator(rootPath);
+    fsItr = Mockito.spy(fsItr);
+    Mockito.doReturn(false).when(fsItr).hasNext();
+
+    RemoteIterator<FileStatus> finalFsItr = fsItr;
+    Assertions.assertThatThrownBy(() -> finalFsItr.next()).describedAs(
+        "next() should throw NoSuchElementException if hasNext() return "
+            + "false").isInstanceOf(NoSuchElementException.class);
+  }
+
+  @Test
+  public void testHasNextForEmptyDir() throws Exception {
+    final AzureBlobFileSystem fs = getFileSystem();
+    String rootPathStr = "testRoot3";
+    Path rootPath = new Path(rootPathStr);
+    getFileSystem().mkdirs(rootPath);
+    RemoteIterator<FileStatus> fsItr = fs.listStatusIterator(rootPath);
+    Assertions.assertThat(fsItr.hasNext())
+        .describedAs("hasNext returns false for empty directory").isFalse();
+  }
+
+  @Test
+  public void testHasNextForFile() throws Exception {
+    final AzureBlobFileSystem fs = getFileSystem();
+    String rootPathStr = "testRoot4";

Review comment:
       filename

##########
File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemListStatusIterator.java
##########
@@ -0,0 +1,190 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs;
+
+import java.io.IOException;
+import java.lang.reflect.Field;
+import java.lang.reflect.Modifier;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+
+import org.assertj.core.api.Assertions;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+import org.apache.hadoop.fs.azurebfs.services.ListStatusRemoteIterator;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+
+/**
+ * Test listStatus operation.
+ */
+public class ITestAzureBlobFileSystemListStatusIterator
+    extends AbstractAbfsIntegrationTest {
+
+  private static final int TEST_FILES_NUMBER = 1000;
+
+  public ITestAzureBlobFileSystemListStatusIterator() throws Exception {
+    super();
+  }
+
+  @Test
+  public void testListPath() throws Exception {
+    final AzureBlobFileSystem fs = getFileSystem();
+    String rootPath = "testRoot1";
+    final List<String> fileNames = createFiles(TEST_FILES_NUMBER, rootPath,
+        "testListPath");
+    AzureBlobFileSystemStore abfsStore = getAbfsStore(fs);
+    abfsStore.getAbfsConfiguration().setListMaxResults(10);
+    RemoteIterator<FileStatus> fsIt = fs.listStatusIterator(new Path(rootPath));
+    int itrCount = 0;
+    while (fsIt.hasNext()) {
+      FileStatus fileStatus = fsIt.next();
+      String pathStr = fileStatus.getPath().toString();
+      fileNames.remove(pathStr);
+      itrCount++;
+    }
+    assertEquals(TEST_FILES_NUMBER, itrCount);
+    assertEquals(0, fileNames.size());
+  }
+
+  @Test
+  public void testNextWhenNoMoreElementsPresent() throws Exception {
+    final AzureBlobFileSystem fs = getFileSystem();
+    String rootPathStr = "testRoot2";
+    Path rootPath = new Path(rootPathStr);
+    getFileSystem().mkdirs(rootPath);
+    RemoteIterator<FileStatus> fsItr = fs.listStatusIterator(rootPath);
+    fsItr = Mockito.spy(fsItr);
+    Mockito.doReturn(false).when(fsItr).hasNext();
+
+    RemoteIterator<FileStatus> finalFsItr = fsItr;
+    Assertions.assertThatThrownBy(() -> finalFsItr.next()).describedAs(
+        "next() should throw NoSuchElementException if hasNext() return "
+            + "false").isInstanceOf(NoSuchElementException.class);
+  }
+
+  @Test
+  public void testHasNextForEmptyDir() throws Exception {
+    final AzureBlobFileSystem fs = getFileSystem();
+    String rootPathStr = "testRoot3";
+    Path rootPath = new Path(rootPathStr);
+    getFileSystem().mkdirs(rootPath);
+    RemoteIterator<FileStatus> fsItr = fs.listStatusIterator(rootPath);
+    Assertions.assertThat(fsItr.hasNext())
+        .describedAs("hasNext returns false for empty directory").isFalse();
+  }
+
+  @Test
+  public void testHasNextForFile() throws Exception {
+    final AzureBlobFileSystem fs = getFileSystem();
+    String rootPathStr = "testRoot4";
+    Path rootPath = new Path(rootPathStr);
+    getFileSystem().create(rootPath);
+    RemoteIterator<FileStatus> fsItr = fs.listStatusIterator(rootPath);
+    Assertions.assertThat(fsItr.hasNext())
+        .describedAs("hasNext returns true for file").isTrue();
+  }
+
+  @Test
+  public void testHasNextForIOException() throws Exception {
+    final AzureBlobFileSystem fs = getFileSystem();
+    String rootPathStr = "testRoot5";
+    Path rootPath = new Path(rootPathStr);

Review comment:
       rename throughout

##########
File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemListStatusIterator.java
##########
@@ -0,0 +1,190 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs;
+
+import java.io.IOException;
+import java.lang.reflect.Field;
+import java.lang.reflect.Modifier;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+
+import org.assertj.core.api.Assertions;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+import org.apache.hadoop.fs.azurebfs.services.ListStatusRemoteIterator;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+
+/**
+ * Test listStatus operation.
+ */
+public class ITestAzureBlobFileSystemListStatusIterator
+    extends AbstractAbfsIntegrationTest {
+
+  private static final int TEST_FILES_NUMBER = 1000;
+
+  public ITestAzureBlobFileSystemListStatusIterator() throws Exception {
+    super();
+  }
+
+  @Test
+  public void testListPath() throws Exception {
+    final AzureBlobFileSystem fs = getFileSystem();
+    String rootPath = "testRoot1";
+    final List<String> fileNames = createFiles(TEST_FILES_NUMBER, rootPath,
+        "testListPath");
+    AzureBlobFileSystemStore abfsStore = getAbfsStore(fs);
+    abfsStore.getAbfsConfiguration().setListMaxResults(10);
+    RemoteIterator<FileStatus> fsIt = fs.listStatusIterator(new Path(rootPath));
+    int itrCount = 0;
+    while (fsIt.hasNext()) {
+      FileStatus fileStatus = fsIt.next();
+      String pathStr = fileStatus.getPath().toString();
+      fileNames.remove(pathStr);
+      itrCount++;
+    }
+    assertEquals(TEST_FILES_NUMBER, itrCount);
+    assertEquals(0, fileNames.size());
+  }
+
+  @Test
+  public void testNextWhenNoMoreElementsPresent() throws Exception {
+    final AzureBlobFileSystem fs = getFileSystem();
+    String rootPathStr = "testRoot2";
+    Path rootPath = new Path(rootPathStr);
+    getFileSystem().mkdirs(rootPath);
+    RemoteIterator<FileStatus> fsItr = fs.listStatusIterator(rootPath);
+    fsItr = Mockito.spy(fsItr);
+    Mockito.doReturn(false).when(fsItr).hasNext();
+
+    RemoteIterator<FileStatus> finalFsItr = fsItr;
+    Assertions.assertThatThrownBy(() -> finalFsItr.next()).describedAs(
+        "next() should throw NoSuchElementException if hasNext() return "
+            + "false").isInstanceOf(NoSuchElementException.class);
+  }
+
+  @Test
+  public void testHasNextForEmptyDir() throws Exception {
+    final AzureBlobFileSystem fs = getFileSystem();
+    String rootPathStr = "testRoot3";
+    Path rootPath = new Path(rootPathStr);
+    getFileSystem().mkdirs(rootPath);
+    RemoteIterator<FileStatus> fsItr = fs.listStatusIterator(rootPath);
+    Assertions.assertThat(fsItr.hasNext())
+        .describedAs("hasNext returns false for empty directory").isFalse();
+  }
+
+  @Test
+  public void testHasNextForFile() throws Exception {
+    final AzureBlobFileSystem fs = getFileSystem();
+    String rootPathStr = "testRoot4";
+    Path rootPath = new Path(rootPathStr);
+    getFileSystem().create(rootPath);
+    RemoteIterator<FileStatus> fsItr = fs.listStatusIterator(rootPath);
+    Assertions.assertThat(fsItr.hasNext())
+        .describedAs("hasNext returns true for file").isTrue();

Review comment:
       Next() and then ensure hasnext return false and NExt throws

##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ListStatusRemoteIterator.java
##########
@@ -0,0 +1,149 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs.services;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.CompletableFuture;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore;
+
+public class ListStatusRemoteIterator implements RemoteIterator<FileStatus> {
+
+  private static final Logger LOG = LoggerFactory
+      .getLogger(ListStatusRemoteIterator.class);
+
+  private static final boolean FETCH_ALL_FALSE = false;
+  private static final int MAX_QUEUE_SIZE = 10;
+
+  private final Path path;
+  private final AzureBlobFileSystemStore abfsStore;
+  private final ArrayBlockingQueue<Iterator<FileStatus>> iteratorsQueue;
+  private final Object asyncOpLock = new Object();
+
+  private boolean firstBatch = true;
+  private boolean isAsyncInProgress = false;
+  private String continuation;
+  private Iterator<FileStatus> currIterator;
+  private IOException ioException;
+
+  public ListStatusRemoteIterator(final Path path,
+      final AzureBlobFileSystemStore abfsStore) {
+    this.path = path;
+    this.abfsStore = abfsStore;
+    iteratorsQueue = new ArrayBlockingQueue<>(MAX_QUEUE_SIZE);
+    currIterator = Collections.emptyIterator();
+    fetchBatchesAsync();
+  }
+
+  @Override
+  public boolean hasNext() throws IOException {
+    if (currIterator.hasNext()) {
+      return true;
+    }
+    updateCurrentIterator();
+    return currIterator.hasNext();
+  }
+
+  @Override
+  public FileStatus next() throws IOException {
+    if (!this.hasNext()) {
+      throw new NoSuchElementException();
+    }
+    return currIterator.next();
+  }
+
+  private void updateCurrentIterator() throws IOException {
+    fetchBatchesAsync();
+    synchronized (this) {
+      if (iteratorsQueue.isEmpty()) {
+        if (ioException != null) {
+          throw ioException;
+        }
+        if (isListingComplete()) {
+          return;
+        }
+      }
+    }
+    try {
+      currIterator = iteratorsQueue.take();
+      if (!currIterator.hasNext() && !isListingComplete()) {
+        updateCurrentIterator();
+      }
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      LOG.error("Thread got interrupted: {}", e);
+    }
+  }
+
+  private boolean isListingComplete() {
+    return !firstBatch && (continuation == null || continuation.isEmpty());
+  }
+
+  private void fetchBatchesAsync() {
+    CompletableFuture.runAsync(() -> asyncOp());
+  }
+
+  private void asyncOp() {
+    if (isAsyncInProgress) {
+      return;
+    }
+    synchronized (asyncOpLock) {
+      if (isAsyncInProgress) {
+        return;
+      }
+      isAsyncInProgress = true;
+    }
+    try {
+      while (!isListingComplete() && iteratorsQueue.size() <= MAX_QUEUE_SIZE) {
+        addNextBatchIteratorToQueue();
+      }
+    } catch (IOException e) {
+      ioException = e;
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      LOG.error("Thread got interrupted: {}", e);

Review comment:
       is it right to catch interrupted exception?




----------------------------------------------------------------
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] vinaysbadami commented on a change in pull request #2548: HADOOP-17475. ABFS: Implementing ListStatusRemoteIterator

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



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsListStatusRemoteIterator.java
##########
@@ -0,0 +1,159 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs.services;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.CompletableFuture;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.RemoteIterator;
+
+public class AbfsListStatusRemoteIterator implements RemoteIterator<FileStatus> {
+
+  private static final Logger LOG = LoggerFactory
+      .getLogger(AbfsListStatusRemoteIterator.class);
+
+  private static final boolean FETCH_ALL_FALSE = false;
+  private static final int MAX_QUEUE_SIZE = 10;
+
+  private final FileStatus fileStatus;
+  private final ListingSupport listingSupport;
+  private final ArrayBlockingQueue<Object> iteratorsQueue;
+  private final Object asyncOpLock = new Object();
+
+  private volatile boolean isAsyncInProgress = false;
+  private boolean isIterationComplete = false;
+  private String continuation;
+  private Iterator<FileStatus> currIterator;
+
+  public AbfsListStatusRemoteIterator(final FileStatus fileStatus,
+      final ListingSupport listingSupport) {
+    this.fileStatus = fileStatus;
+    this.listingSupport = listingSupport;
+    iteratorsQueue = new ArrayBlockingQueue<>(MAX_QUEUE_SIZE);
+    currIterator = Collections.emptyIterator();
+    fetchBatchesAsync();
+  }
+
+  @Override
+  public boolean hasNext() throws IOException {
+    if (currIterator.hasNext()) {
+      return true;
+    }
+    updateCurrentIterator();
+    if (currIterator == null) {
+      return false;
+    }
+    return currIterator.hasNext();
+  }
+
+  @Override
+  public FileStatus next() throws IOException {
+    if (!this.hasNext()) {
+      throw new NoSuchElementException();
+    }
+    return currIterator.next();
+  }
+
+  private void updateCurrentIterator() throws IOException {
+    do {
+      currIterator = getNextIterator();
+    } while (currIterator != null && !currIterator.hasNext()
+        && !isIterationComplete);
+  }
+
+  private Iterator<FileStatus> getNextIterator() throws IOException {
+    fetchBatchesAsync();
+    synchronized (this) {
+      if (iteratorsQueue.isEmpty() && isIterationComplete) {
+          return null;

Review comment:
       null or empty iterator? same question for line 106.
   Standrad with collections is to return an empty collection/iterator - not a null object.




----------------------------------------------------------------
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 #2548: HADOOP-17475. ABFS: Implementing ListStatusRemoteIterator

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


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |   1m  4s |  |  Docker mode activated.  |
   |||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  |  No case conflicting files found.  |
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |   |   0m  0s | [test4tests](test4tests) |  The patch appears to include 1 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |  33m 11s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |   0m 37s |  |  trunk passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04  |
   | +1 :green_heart: |  compile  |   0m 34s |  |  trunk passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01  |
   | +1 :green_heart: |  checkstyle  |   0m 28s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   0m 39s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  16m 48s |  |  branch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 32s |  |  trunk passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04  |
   | +1 :green_heart: |  javadoc  |   0m 30s |  |  trunk passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01  |
   | +0 :ok: |  spotbugs  |   0m 58s |  |  Used deprecated FindBugs config; considering switching to SpotBugs.  |
   | +1 :green_heart: |  findbugs  |   0m 56s |  |  trunk passed  |
   | -0 :warning: |  patch  |   1m 16s |  |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   |||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   0m 30s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 29s |  |  the patch passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04  |
   | +1 :green_heart: |  javac  |   0m 29s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 25s |  |  the patch passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01  |
   | +1 :green_heart: |  javac  |   0m 25s |  |  the patch passed  |
   | -0 :warning: |  checkstyle  |   0m 18s | [/diff-checkstyle-hadoop-tools_hadoop-azure.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/13/artifact/out/diff-checkstyle-hadoop-tools_hadoop-azure.txt) |  hadoop-tools/hadoop-azure: The patch generated 6 new + 4 unchanged - 0 fixed = 10 total (was 4)  |
   | +1 :green_heart: |  mvnsite  |   0m 29s |  |  the patch passed  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  shadedclient  |  15m  8s |  |  patch has no errors when building and testing our client artifacts.  |
   | -1 :x: |  javadoc  |   0m 26s | [/diff-javadoc-javadoc-hadoop-tools_hadoop-azure-jdkUbuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/13/artifact/out/diff-javadoc-javadoc-hadoop-tools_hadoop-azure-jdkUbuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04.txt) |  hadoop-tools_hadoop-azure-jdkUbuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04 with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04 generated 3 new + 15 unchanged - 2 fixed = 18 total (was 17)  |
   | -1 :x: |  javadoc  |   0m 25s | [/diff-javadoc-javadoc-hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/13/artifact/out/diff-javadoc-javadoc-hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01.txt) |  hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01 with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01 generated 3 new + 15 unchanged - 2 fixed = 18 total (was 17)  |
   | -1 :x: |  findbugs  |   1m  2s | [/new-findbugs-hadoop-tools_hadoop-azure.html](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/13/artifact/out/new-findbugs-hadoop-tools_hadoop-azure.html) |  hadoop-tools/hadoop-azure generated 2 new + 0 unchanged - 0 fixed = 2 total (was 0)  |
   |||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   1m 56s |  |  hadoop-azure in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 33s |  |  The patch does not generate ASF License warnings.  |
   |  |   |  79m 17s |  |  |
   
   
   | Reason | Tests |
   |-------:|:------|
   | FindBugs | module:hadoop-tools/hadoop-azure |
   |  |  Inconsistent synchronization of org.apache.hadoop.fs.azurebfs.services.AbfsListStatusRemoteIterator.continuation; locked 50% of time  Unsynchronized access at AbfsListStatusRemoteIterator.java:50% of time  Unsynchronized access at AbfsListStatusRemoteIterator.java:[line 149] |
   |  |  Inconsistent synchronization of org.apache.hadoop.fs.azurebfs.services.AbfsListStatusRemoteIterator.isIterationComplete; locked 50% of time  Unsynchronized access at AbfsListStatusRemoteIterator.java:50% of time  Unsynchronized access at AbfsListStatusRemoteIterator.java:[line 86] |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/13/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/2548 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient findbugs checkstyle |
   | uname | Linux cf1f444f6be9 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 / d82009599a2 |
   | Default Java | Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01 |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/13/testReport/ |
   | Max. process+thread count | 536 (vs. ulimit of 5500) |
   | modules | C: hadoop-tools/hadoop-azure U: hadoop-tools/hadoop-azure |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/13/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 #2548: HADOOP-17475. ABFS: Implementing ListStatusRemoteIterator

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


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |   2m 28s |  |  Docker mode activated.  |
   |||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  |  No case conflicting files found.  |
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |   |   0m  0s | [test4tests](test4tests) |  The patch appears to include 1 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |  33m 37s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |   0m 37s |  |  trunk passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  compile  |   0m 33s |  |  trunk passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~20.04-b01  |
   | +1 :green_heart: |  checkstyle  |   0m 30s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   0m 38s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  14m  6s |  |  branch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 37s |  |  trunk passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javadoc  |   0m 32s |  |  trunk passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~20.04-b01  |
   | +0 :ok: |  spotbugs  |   0m 59s |  |  Used deprecated FindBugs config; considering switching to SpotBugs.  |
   | +1 :green_heart: |  findbugs  |   0m 57s |  |  trunk passed  |
   | -0 :warning: |  patch  |   1m 15s |  |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   |||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   0m 30s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 32s |  |  the patch passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javac  |   0m 32s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 27s |  |  the patch passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~20.04-b01  |
   | +1 :green_heart: |  javac  |   0m 27s |  |  the patch passed  |
   | +1 :green_heart: |  checkstyle  |   0m 17s |  |  the patch passed  |
   | +1 :green_heart: |  mvnsite  |   0m 30s |  |  the patch passed  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  xml  |   0m  1s |  |  The patch has no ill-formed XML file.  |
   | +1 :green_heart: |  shadedclient  |  12m 48s |  |  patch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 26s |  |  hadoop-tools_hadoop-azure-jdkUbuntu-11.0.9.1+1-Ubuntu-0ubuntu1.20.04 with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.20.04 generated 0 new + 15 unchanged - 2 fixed = 15 total (was 17)  |
   | +1 :green_heart: |  javadoc  |   0m 23s |  |  hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_275-8u275-b01-0ubuntu1~20.04-b01 with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~20.04-b01 generated 0 new + 15 unchanged - 2 fixed = 15 total (was 17)  |
   | -1 :x: |  findbugs  |   1m  4s | [/new-findbugs-hadoop-tools_hadoop-azure.html](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/19/artifact/out/new-findbugs-hadoop-tools_hadoop-azure.html) |  hadoop-tools/hadoop-azure generated 1 new + 0 unchanged - 0 fixed = 1 total (was 0)  |
   |||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   2m  0s |  |  hadoop-azure in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 33s |  |  The patch does not generate ASF License warnings.  |
   |  |   |  76m 16s |  |  |
   
   
   | Reason | Tests |
   |-------:|:------|
   | FindBugs | module:hadoop-tools/hadoop-azure |
   |  |  Inconsistent synchronization of org.apache.hadoop.fs.azurebfs.services.AbfsListStatusRemoteIterator.continuation; locked 50% of time  Unsynchronized access at AbfsListStatusRemoteIterator.java:50% of time  Unsynchronized access at AbfsListStatusRemoteIterator.java:[line 147] |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/19/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/2548 |
   | Optional Tests | dupname asflicense xml compile javac javadoc mvninstall mvnsite unit shadedclient findbugs checkstyle |
   | uname | Linux 58c7d8478c29 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 / 394b9f7a5c4 |
   | Default Java | Private Build-1.8.0_275-8u275-b01-0ubuntu1~20.04-b01 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.20.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_275-8u275-b01-0ubuntu1~20.04-b01 |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/19/testReport/ |
   | Max. process+thread count | 616 (vs. ulimit of 5500) |
   | modules | C: hadoop-tools/hadoop-azure U: hadoop-tools/hadoop-azure |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/19/console |
   | versions | git=2.25.1 maven=3.6.3 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 #2548: DRAFT PR: Implementing ListStatusRemoteIterator

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


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |   1m 12s |  |  Docker mode activated.  |
   |||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  |  No case conflicting files found.  |
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |   |   0m  0s | [test4tests](test4tests) |  The patch appears to include 1 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |  33m  6s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |   0m 37s |  |  trunk passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04  |
   | +1 :green_heart: |  compile  |   0m 34s |  |  trunk passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01  |
   | +1 :green_heart: |  checkstyle  |   0m 27s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   0m 39s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  16m 36s |  |  branch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 31s |  |  trunk passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04  |
   | +1 :green_heart: |  javadoc  |   0m 30s |  |  trunk passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01  |
   | +0 :ok: |  spotbugs  |   0m 58s |  |  Used deprecated FindBugs config; considering switching to SpotBugs.  |
   | +1 :green_heart: |  findbugs  |   0m 57s |  |  trunk passed  |
   | -0 :warning: |  patch  |   1m 18s |  |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   |||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   0m 31s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 32s |  |  the patch passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04  |
   | +1 :green_heart: |  javac  |   0m 32s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 27s |  |  the patch passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01  |
   | +1 :green_heart: |  javac  |   0m 27s |  |  the patch passed  |
   | -0 :warning: |  checkstyle  |   0m 19s | [/diff-checkstyle-hadoop-tools_hadoop-azure.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/10/artifact/out/diff-checkstyle-hadoop-tools_hadoop-azure.txt) |  hadoop-tools/hadoop-azure: The patch generated 7 new + 4 unchanged - 0 fixed = 11 total (was 4)  |
   | +1 :green_heart: |  mvnsite  |   0m 30s |  |  the patch passed  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  shadedclient  |  15m 52s |  |  patch has no errors when building and testing our client artifacts.  |
   | -1 :x: |  javadoc  |   0m 27s | [/diff-javadoc-javadoc-hadoop-tools_hadoop-azure-jdkUbuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/10/artifact/out/diff-javadoc-javadoc-hadoop-tools_hadoop-azure-jdkUbuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04.txt) |  hadoop-tools_hadoop-azure-jdkUbuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04 with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04 generated 3 new + 15 unchanged - 2 fixed = 18 total (was 17)  |
   | -1 :x: |  javadoc  |   0m 26s | [/diff-javadoc-javadoc-hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/10/artifact/out/diff-javadoc-javadoc-hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01.txt) |  hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01 with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01 generated 3 new + 15 unchanged - 2 fixed = 18 total (was 17)  |
   | -1 :x: |  findbugs  |   1m  2s | [/new-findbugs-hadoop-tools_hadoop-azure.html](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/10/artifact/out/new-findbugs-hadoop-tools_hadoop-azure.html) |  hadoop-tools/hadoop-azure generated 1 new + 0 unchanged - 0 fixed = 1 total (was 0)  |
   |||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   1m 28s |  |  hadoop-azure in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 30s |  |  The patch does not generate ASF License warnings.  |
   |  |   |  79m  5s |  |  |
   
   
   | Reason | Tests |
   |-------:|:------|
   | FindBugs | module:hadoop-tools/hadoop-azure |
   |  |  Inconsistent synchronization of org.apache.hadoop.fs.azurebfs.services.AbfsListStatusRemoteIterator.ioException; locked 66% of time  Unsynchronized access at AbfsListStatusRemoteIterator.java:66% of time  Unsynchronized access at AbfsListStatusRemoteIterator.java:[line 128] |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/10/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/2548 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient findbugs checkstyle |
   | uname | Linux ba35d2aab8dd 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 / 630f8ddd2c7 |
   | Default Java | Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01 |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/10/testReport/ |
   | Max. process+thread count | 683 (vs. ulimit of 5500) |
   | modules | C: hadoop-tools/hadoop-azure U: hadoop-tools/hadoop-azure |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/10/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] vinaysbadami commented on a change in pull request #2548: DRAFT PR: Implementing ListStatusRemoteIterator

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



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ListStatusRemoteIterator.java
##########
@@ -0,0 +1,129 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs.services;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.ListIterator;
+import java.util.NoSuchElementException;
+import java.util.Queue;
+import java.util.concurrent.CompletableFuture;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore;
+
+public class ListStatusRemoteIterator implements RemoteIterator<FileStatus> {
+
+  private static final Logger LOG = LoggerFactory
+      .getLogger(ListStatusRemoteIterator.class);
+
+  private static final boolean FETCH_ALL_FALSE = false;
+
+  private final Path path;
+  private final AzureBlobFileSystemStore abfsStore;
+  private final Queue<ListIterator<FileStatus>> iteratorsQueue =
+      new LinkedList<>();
+
+  private boolean firstRead = true;
+  private String continuation;
+  private ListIterator<FileStatus> currIterator;
+  private IOException ioException;
+
+  public ListStatusRemoteIterator(final Path path,
+      final AzureBlobFileSystemStore abfsStore) throws IOException {
+    this.path = path;
+    this.abfsStore = abfsStore;
+    fetchAllAsync();
+    updateCurrentIterator();
+  }
+
+  @Override
+  public boolean hasNext() throws IOException {
+    if (currIterator.hasNext()) {

Review comment:
       currentIterator && 




----------------------------------------------------------------
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] snvijaya commented on a change in pull request #2548: DRAFT PR: Implementing ListStatusRemoteIterator

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



##########
File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemListStatusIterator.java
##########
@@ -0,0 +1,190 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs;
+
+import java.io.IOException;
+import java.lang.reflect.Field;
+import java.lang.reflect.Modifier;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+
+import org.assertj.core.api.Assertions;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+import org.apache.hadoop.fs.azurebfs.services.ListStatusRemoteIterator;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+
+/**
+ * Test listStatus operation.
+ */
+public class ITestAzureBlobFileSystemListStatusIterator
+    extends AbstractAbfsIntegrationTest {
+
+  private static final int TEST_FILES_NUMBER = 1000;
+
+  public ITestAzureBlobFileSystemListStatusIterator() throws Exception {
+    super();
+  }
+
+  @Test
+  public void testListPath() throws Exception {
+    final AzureBlobFileSystem fs = getFileSystem();
+    String rootPath = "testRoot1";
+    final List<String> fileNames = createFiles(TEST_FILES_NUMBER, rootPath,
+        "testListPath");
+    AzureBlobFileSystemStore abfsStore = getAbfsStore(fs);
+    abfsStore.getAbfsConfiguration().setListMaxResults(10);
+    RemoteIterator<FileStatus> fsIt = fs.listStatusIterator(new Path(rootPath));
+    int itrCount = 0;
+    while (fsIt.hasNext()) {
+      FileStatus fileStatus = fsIt.next();
+      String pathStr = fileStatus.getPath().toString();
+      fileNames.remove(pathStr);
+      itrCount++;
+    }
+    assertEquals(TEST_FILES_NUMBER, itrCount);
+    assertEquals(0, fileNames.size());
+  }
+
+  @Test
+  public void testNextWhenNoMoreElementsPresent() throws Exception {
+    final AzureBlobFileSystem fs = getFileSystem();
+    String rootPathStr = "testRoot2";
+    Path rootPath = new Path(rootPathStr);
+    getFileSystem().mkdirs(rootPath);
+    RemoteIterator<FileStatus> fsItr = fs.listStatusIterator(rootPath);
+    fsItr = Mockito.spy(fsItr);
+    Mockito.doReturn(false).when(fsItr).hasNext();
+
+    RemoteIterator<FileStatus> finalFsItr = fsItr;
+    Assertions.assertThatThrownBy(() -> finalFsItr.next()).describedAs(
+        "next() should throw NoSuchElementException if hasNext() return "
+            + "false").isInstanceOf(NoSuchElementException.class);
+  }
+
+  @Test
+  public void testHasNextForEmptyDir() throws Exception {
+    final AzureBlobFileSystem fs = getFileSystem();
+    String rootPathStr = "testRoot3";
+    Path rootPath = new Path(rootPathStr);
+    getFileSystem().mkdirs(rootPath);
+    RemoteIterator<FileStatus> fsItr = fs.listStatusIterator(rootPath);
+    Assertions.assertThat(fsItr.hasNext())
+        .describedAs("hasNext returns false for empty directory").isFalse();
+  }
+
+  @Test
+  public void testHasNextForFile() throws Exception {
+    final AzureBlobFileSystem fs = getFileSystem();
+    String rootPathStr = "testRoot4";
+    Path rootPath = new Path(rootPathStr);
+    getFileSystem().create(rootPath);
+    RemoteIterator<FileStatus> fsItr = fs.listStatusIterator(rootPath);
+    Assertions.assertThat(fsItr.hasNext())
+        .describedAs("hasNext returns true for file").isTrue();
+  }
+
+  @Test
+  public void testHasNextForIOException() throws Exception {
+    final AzureBlobFileSystem fs = getFileSystem();
+    String rootPathStr = "testRoot5";
+    Path rootPath = new Path(rootPathStr);
+    getFileSystem().mkdirs(rootPath);
+    ListStatusRemoteIterator fsItr = (ListStatusRemoteIterator) fs
+        .listStatusIterator(rootPath);
+    Thread.sleep(1000);
+
+    String exceptionMessage = "test exception";
+    setPrivateField(fsItr, ListStatusRemoteIterator.class, "ioException",
+        new IOException(exceptionMessage));
+    setPrivateFinalField(fsItr, ListStatusRemoteIterator.class,

Review comment:
       Instead of testing failure case by setting ioException field, try to mock abfsStore or lower to give a failure response.




----------------------------------------------------------------
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] bilaharith commented on a change in pull request #2548: DRAFT PR: Implementing ListStatusRemoteIterator

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



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ListStatusRemoteIterator.java
##########
@@ -0,0 +1,129 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs.services;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.ListIterator;
+import java.util.NoSuchElementException;
+import java.util.Queue;
+import java.util.concurrent.CompletableFuture;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore;
+
+public class ListStatusRemoteIterator implements RemoteIterator<FileStatus> {
+
+  private static final Logger LOG = LoggerFactory
+      .getLogger(ListStatusRemoteIterator.class);
+
+  private static final boolean FETCH_ALL_FALSE = false;
+
+  private final Path path;
+  private final AzureBlobFileSystemStore abfsStore;
+  private final Queue<ListIterator<FileStatus>> iteratorsQueue =
+      new LinkedList<>();
+
+  private boolean firstRead = true;
+  private String continuation;
+  private ListIterator<FileStatus> currIterator;
+  private IOException ioException;
+
+  public ListStatusRemoteIterator(final Path path,
+      final AzureBlobFileSystemStore abfsStore) throws IOException {
+    this.path = path;
+    this.abfsStore = abfsStore;
+    fetchAllAsync();
+    updateCurrentIterator();
+  }
+
+  @Override
+  public boolean hasNext() throws IOException {
+    if (currIterator.hasNext()) {
+      return true;
+    }
+    updateCurrentIterator();
+    return currIterator.hasNext();
+  }
+
+  @Override
+  public FileStatus next() throws IOException {
+    if (!this.hasNext()) {
+      throw new NoSuchElementException();
+    }
+    return currIterator.next();
+  }
+
+  private void updateCurrentIterator() throws IOException {
+    synchronized (this) {
+      while (!isIterationComplete() && iteratorsQueue.isEmpty()) {
+        try {
+          this.wait();
+        } catch (InterruptedException e) {
+          Thread.currentThread().interrupt();
+          LOG.error("Thread got interrupted: {}", e);
+        }
+      }
+      if (!iteratorsQueue.isEmpty()) {
+        currIterator = iteratorsQueue.poll();
+      } else if (ioException != null) {
+        throw ioException;
+      }
+    }
+  }
+
+  private void fetchAllAsync() {
+    CompletableFuture.supplyAsync(() -> {
+      while (!isIterationComplete()) {
+        List<FileStatus> fileStatuses = new ArrayList<>();
+        try {
+          continuation = abfsStore
+              .listStatus(path, null, fileStatuses, FETCH_ALL_FALSE,
+                  continuation);
+        } catch (IOException e) {
+          ioException = e;
+          return null;
+        } finally {
+          if (firstRead) {
+            firstRead = false;
+          }
+        }
+        if (fileStatuses != null && !fileStatuses.isEmpty()) {
+          iteratorsQueue.add(fileStatuses.listIterator());
+          synchronized (this) {

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] hadoop-yetus commented on pull request #2548: DRAFT PR: Implementing ListStatusRemoteIterator

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


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |  30m 54s |  |  Docker mode activated.  |
   |||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  |  No case conflicting files found.  |
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |   |   0m  0s | [test4tests](test4tests) |  The patch appears to include 1 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |  33m 29s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |   0m 38s |  |  trunk passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04  |
   | +1 :green_heart: |  compile  |   0m 33s |  |  trunk passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01  |
   | +1 :green_heart: |  checkstyle  |   0m 27s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   0m 39s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  16m 28s |  |  branch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 32s |  |  trunk passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04  |
   | +1 :green_heart: |  javadoc  |   0m 29s |  |  trunk passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01  |
   | +0 :ok: |  spotbugs  |   0m 59s |  |  Used deprecated FindBugs config; considering switching to SpotBugs.  |
   | +1 :green_heart: |  findbugs  |   0m 57s |  |  trunk passed  |
   | -0 :warning: |  patch  |   1m 16s |  |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   |||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   0m 31s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 29s |  |  the patch passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04  |
   | +1 :green_heart: |  javac  |   0m 29s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 26s |  |  the patch passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01  |
   | +1 :green_heart: |  javac  |   0m 26s |  |  the patch passed  |
   | -0 :warning: |  checkstyle  |   0m 16s | [/diff-checkstyle-hadoop-tools_hadoop-azure.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/8/artifact/out/diff-checkstyle-hadoop-tools_hadoop-azure.txt) |  hadoop-tools/hadoop-azure: The patch generated 5 new + 4 unchanged - 0 fixed = 9 total (was 4)  |
   | +1 :green_heart: |  mvnsite  |   0m 29s |  |  the patch passed  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  shadedclient  |  14m 55s |  |  patch has no errors when building and testing our client artifacts.  |
   | -1 :x: |  javadoc  |   0m 27s | [/diff-javadoc-javadoc-hadoop-tools_hadoop-azure-jdkUbuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/8/artifact/out/diff-javadoc-javadoc-hadoop-tools_hadoop-azure-jdkUbuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04.txt) |  hadoop-tools_hadoop-azure-jdkUbuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04 with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04 generated 3 new + 15 unchanged - 2 fixed = 18 total (was 17)  |
   | -1 :x: |  javadoc  |   0m 25s | [/diff-javadoc-javadoc-hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/8/artifact/out/diff-javadoc-javadoc-hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01.txt) |  hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01 with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01 generated 3 new + 15 unchanged - 2 fixed = 18 total (was 17)  |
   | -1 :x: |  findbugs  |   1m  1s | [/new-findbugs-hadoop-tools_hadoop-azure.html](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/8/artifact/out/new-findbugs-hadoop-tools_hadoop-azure.html) |  hadoop-tools/hadoop-azure generated 1 new + 0 unchanged - 0 fixed = 1 total (was 0)  |
   |||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   1m 31s |  |  hadoop-azure in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 34s |  |  The patch does not generate ASF License warnings.  |
   |  |   | 108m 12s |  |  |
   
   
   | Reason | Tests |
   |-------:|:------|
   | FindBugs | module:hadoop-tools/hadoop-azure |
   |  |  Inconsistent synchronization of org.apache.hadoop.fs.azurebfs.services.AbfsListStatusRemoteIterator.ioException; locked 66% of time  Unsynchronized access at AbfsListStatusRemoteIterator.java:66% of time  Unsynchronized access at AbfsListStatusRemoteIterator.java:[line 128] |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/8/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/2548 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient findbugs checkstyle |
   | uname | Linux be81f3ee1b30 4.15.0-65-generic #74-Ubuntu SMP Tue Sep 17 17:06:04 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / 630f8ddd2c7 |
   | Default Java | Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01 |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/8/testReport/ |
   | Max. process+thread count | 546 (vs. ulimit of 5500) |
   | modules | C: hadoop-tools/hadoop-azure U: hadoop-tools/hadoop-azure |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/8/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] vinaysbadami commented on a change in pull request #2548: HADOOP-17475. ABFS: Implementing ListStatusRemoteIterator

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



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsListStatusRemoteIterator.java
##########
@@ -0,0 +1,159 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs.services;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.CompletableFuture;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.RemoteIterator;
+
+public class AbfsListStatusRemoteIterator implements RemoteIterator<FileStatus> {
+
+  private static final Logger LOG = LoggerFactory
+      .getLogger(AbfsListStatusRemoteIterator.class);
+
+  private static final boolean FETCH_ALL_FALSE = false;
+  private static final int MAX_QUEUE_SIZE = 10;
+
+  private final FileStatus fileStatus;
+  private final ListingSupport listingSupport;
+  private final ArrayBlockingQueue<Object> iteratorsQueue;
+  private final Object asyncOpLock = new Object();
+
+  private volatile boolean isAsyncInProgress = false;
+  private boolean isIterationComplete = false;
+  private String continuation;
+  private Iterator<FileStatus> currIterator;
+
+  public AbfsListStatusRemoteIterator(final FileStatus fileStatus,
+      final ListingSupport listingSupport) {
+    this.fileStatus = fileStatus;
+    this.listingSupport = listingSupport;
+    iteratorsQueue = new ArrayBlockingQueue<>(MAX_QUEUE_SIZE);
+    currIterator = Collections.emptyIterator();
+    fetchBatchesAsync();
+  }
+
+  @Override
+  public boolean hasNext() throws IOException {
+    if (currIterator.hasNext()) {
+      return true;
+    }
+    updateCurrentIterator();
+    if (currIterator == null) {
+      return false;
+    }
+    return currIterator.hasNext();
+  }
+
+  @Override
+  public FileStatus next() throws IOException {
+    if (!this.hasNext()) {
+      throw new NoSuchElementException();
+    }
+    return currIterator.next();
+  }
+
+  private void updateCurrentIterator() throws IOException {
+    do {
+      currIterator = getNextIterator();
+    } while (currIterator != null && !currIterator.hasNext()
+        && !isIterationComplete);
+  }
+
+  private Iterator<FileStatus> getNextIterator() throws IOException {
+    fetchBatchesAsync();
+    synchronized (this) {
+      if (iteratorsQueue.isEmpty() && isIterationComplete) {
+          return null;
+      }
+    }
+    try {
+      Object obj = iteratorsQueue.take();
+      if(obj instanceof Iterator){
+        return (Iterator<FileStatus>) obj;
+      }
+      throw (IOException) obj;
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      LOG.error("Thread got interrupted: {}", e);
+      return null;
+    }
+  }
+
+  private void fetchBatchesAsync() {
+    if (isAsyncInProgress) {
+      return;
+    }
+    synchronized (asyncOpLock) {
+      if (isAsyncInProgress) {
+        return;
+      }
+      isAsyncInProgress = true;
+    }
+    CompletableFuture.runAsync(() -> asyncOp());
+  }
+
+  private void asyncOp() {
+    try {
+      while (!isIterationComplete && iteratorsQueue.size() <= MAX_QUEUE_SIZE) {
+        addNextBatchIteratorToQueue();
+      }
+    } catch (IOException e) {
+      try {
+        iteratorsQueue.put(e);
+      } catch (InterruptedException interruptedException) {
+        Thread.currentThread().interrupt();
+        LOG.error("Thread got interrupted: {}", interruptedException);
+      }
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      LOG.error("Thread got interrupted: {}", e);
+    } finally {
+      synchronized (asyncOpLock) {
+        isAsyncInProgress = false;
+      }
+    }
+  }
+
+  private void addNextBatchIteratorToQueue()
+      throws IOException, InterruptedException {
+    List<FileStatus> fileStatuses = new ArrayList<>();
+    continuation = listingSupport
+        .listStatus(fileStatus.getPath(), null, fileStatuses, FETCH_ALL_FALSE,
+            continuation);
+    synchronized (this) {
+      if (continuation == null || continuation.isEmpty()) {
+        isIterationComplete = true;
+      }
+      iteratorsQueue.put(fileStatuses.iterator());

Review comment:
       can this be done outside the sync block? If put blocks u are holding the lock.

##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsListStatusRemoteIterator.java
##########
@@ -0,0 +1,159 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs.services;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.CompletableFuture;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.RemoteIterator;
+
+public class AbfsListStatusRemoteIterator implements RemoteIterator<FileStatus> {
+
+  private static final Logger LOG = LoggerFactory
+      .getLogger(AbfsListStatusRemoteIterator.class);
+
+  private static final boolean FETCH_ALL_FALSE = false;
+  private static final int MAX_QUEUE_SIZE = 10;
+
+  private final FileStatus fileStatus;
+  private final ListingSupport listingSupport;
+  private final ArrayBlockingQueue<Object> iteratorsQueue;
+  private final Object asyncOpLock = new Object();

Review comment:
       can we avoid this completely?

##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java
##########
@@ -982,6 +985,19 @@ public boolean exists(Path f) throws IOException {
     return super.exists(f);
   }
 
+  @Override
+  public RemoteIterator<FileStatus> listStatusIterator(Path path)
+      throws IOException {
+    LOG.debug("AzureBlobFileSystem.listStatusIterator path : {}", path);
+    if (abfsStore.getAbfsConfiguration().enableAbfsListIterator()) {

Review comment:
       after it is used for a while and we have confidence that it is baked

##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsListStatusRemoteIterator.java
##########
@@ -0,0 +1,159 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs.services;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.CompletableFuture;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.RemoteIterator;
+
+public class AbfsListStatusRemoteIterator implements RemoteIterator<FileStatus> {
+
+  private static final Logger LOG = LoggerFactory
+      .getLogger(AbfsListStatusRemoteIterator.class);
+
+  private static final boolean FETCH_ALL_FALSE = false;
+  private static final int MAX_QUEUE_SIZE = 10;
+
+  private final FileStatus fileStatus;
+  private final ListingSupport listingSupport;
+  private final ArrayBlockingQueue<Object> iteratorsQueue;
+  private final Object asyncOpLock = new Object();
+
+  private volatile boolean isAsyncInProgress = false;
+  private boolean isIterationComplete = false;
+  private String continuation;
+  private Iterator<FileStatus> currIterator;
+
+  public AbfsListStatusRemoteIterator(final FileStatus fileStatus,
+      final ListingSupport listingSupport) {
+    this.fileStatus = fileStatus;
+    this.listingSupport = listingSupport;
+    iteratorsQueue = new ArrayBlockingQueue<>(MAX_QUEUE_SIZE);
+    currIterator = Collections.emptyIterator();
+    fetchBatchesAsync();
+  }
+
+  @Override
+  public boolean hasNext() throws IOException {
+    if (currIterator.hasNext()) {
+      return true;
+    }
+    updateCurrentIterator();
+    if (currIterator == null) {
+      return false;
+    }
+    return currIterator.hasNext();
+  }
+
+  @Override
+  public FileStatus next() throws IOException {
+    if (!this.hasNext()) {
+      throw new NoSuchElementException();
+    }
+    return currIterator.next();
+  }
+
+  private void updateCurrentIterator() throws IOException {
+    do {
+      currIterator = getNextIterator();
+    } while (currIterator != null && !currIterator.hasNext()
+        && !isIterationComplete);
+  }
+
+  private Iterator<FileStatus> getNextIterator() throws IOException {
+    fetchBatchesAsync();
+    synchronized (this) {
+      if (iteratorsQueue.isEmpty() && isIterationComplete) {
+          return null;

Review comment:
       null or empty iterator? same question for line 106.
   Standrad with collections is to return an empty collection/iterator - not a null object.

##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsListStatusRemoteIterator.java
##########
@@ -0,0 +1,156 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs.services;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.CompletableFuture;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.RemoteIterator;
+
+public class AbfsListStatusRemoteIterator implements RemoteIterator<FileStatus> {
+
+  private static final Logger LOG = LoggerFactory
+      .getLogger(AbfsListStatusRemoteIterator.class);
+
+  private static final boolean FETCH_ALL_FALSE = false;
+  private static final int MAX_QUEUE_SIZE = 10;
+
+  private final FileStatus fileStatus;
+  private final ListingSupport listingSupport;
+  private final ArrayBlockingQueue<Object> iteratorsQueue;
+
+  private volatile boolean isAsyncInProgress = false;
+  private boolean isIterationComplete = false;
+  private String continuation;
+  private Iterator<FileStatus> currIterator;
+
+  public AbfsListStatusRemoteIterator(final FileStatus fileStatus,
+      final ListingSupport listingSupport) {
+    this.fileStatus = fileStatus;
+    this.listingSupport = listingSupport;
+    iteratorsQueue = new ArrayBlockingQueue<>(MAX_QUEUE_SIZE);
+    currIterator = Collections.emptyIterator();
+    fetchBatchesAsync();
+  }
+
+  @Override
+  public boolean hasNext() throws IOException {
+    if (currIterator.hasNext()) {
+      return true;
+    }
+    updateCurrentIterator();
+    return currIterator.hasNext();
+  }
+
+  @Override
+  public FileStatus next() throws IOException {
+    if (!this.hasNext()) {
+      throw new NoSuchElementException();
+    }
+    return currIterator.next();
+  }
+
+  private void updateCurrentIterator() throws IOException {
+    do {
+      currIterator = getNextIterator();
+    } while (currIterator != null && !currIterator.hasNext()
+        && !isIterationComplete);
+  }
+
+  private Iterator<FileStatus> getNextIterator() throws IOException {
+    fetchBatchesAsync();
+    synchronized (this) {
+      if (iteratorsQueue.isEmpty() && isIterationComplete) {
+          return Collections.emptyIterator();
+      }
+    }
+    try {
+      Object obj = iteratorsQueue.take();
+      if(obj instanceof Iterator){
+        return (Iterator<FileStatus>) obj;
+      }
+      throw (IOException) obj;
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      LOG.error("Thread got interrupted: {}", e);
+      return Collections.emptyIterator();
+    }
+  }
+
+  private void fetchBatchesAsync() {
+    if (isAsyncInProgress) {
+      return;
+    }
+    synchronized (this) {
+      if (isAsyncInProgress) {
+        return;
+      }
+      isAsyncInProgress = true;
+    }
+    CompletableFuture.runAsync(() -> asyncOp());
+  }
+
+  private void asyncOp() {
+    try {
+      while (!isIterationComplete && iteratorsQueue.size() <= MAX_QUEUE_SIZE) {
+        addNextBatchIteratorToQueue();
+      }
+    } catch (IOException e) {
+      try {
+        iteratorsQueue.put(e);
+      } catch (InterruptedException interruptedException) {
+        Thread.currentThread().interrupt();
+        LOG.error("Thread got interrupted: {}", interruptedException);
+      }
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      LOG.error("Thread got interrupted: {}", e);
+    } finally {
+      synchronized (this  ) {
+        isAsyncInProgress = false;
+      }
+    }
+  }
+
+  private void addNextBatchIteratorToQueue()
+      throws IOException, InterruptedException {
+    List<FileStatus> fileStatuses = new ArrayList<>();
+    continuation = listingSupport
+        .listStatus(fileStatus.getPath(), null, fileStatuses, FETCH_ALL_FALSE,
+            continuation);
+    iteratorsQueue.put(fileStatuses.iterator());
+    synchronized (this) {
+      if (continuation == null || continuation.isEmpty()) {
+        isIterationComplete = true;
+        iteratorsQueue.put(Collections.emptyIterator());

Review comment:
       not within sync block




----------------------------------------------------------------
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] bilaharith commented on pull request #2548: HADOOP-17475. ABFS: Implementing ListStatusRemoteIterator

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


   > merged to trunk; cp'd to branch-3.3 and will push up if a recompile works. I am not testing that ---can you do that and if there are problems we can do a followup?
   
   Sure


----------------------------------------------------------------
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] bilaharith commented on a change in pull request #2548: HADOOP-17475. ABFS: Implementing ListStatusRemoteIterator

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



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsListStatusRemoteIterator.java
##########
@@ -0,0 +1,151 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs.services;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.CompletableFuture;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.RemoteIterator;
+
+public class AbfsListStatusRemoteIterator implements RemoteIterator<FileStatus> {
+
+  private static final Logger LOG = LoggerFactory
+      .getLogger(AbfsListStatusRemoteIterator.class);
+
+  private static final boolean FETCH_ALL_FALSE = false;
+  private static final int MAX_QUEUE_SIZE = 10;
+
+  private final FileStatus fileStatus;
+  private final ListingSupport listingSupport;
+  private final ArrayBlockingQueue<Iterator<FileStatus>> iteratorsQueue;
+  private final Object asyncOpLock = new Object();
+
+  private volatile boolean isAsyncInProgress = false;
+  private boolean firstBatch = true;
+  private String continuation;
+  private Iterator<FileStatus> currIterator;
+  private IOException ioException;
+
+  public AbfsListStatusRemoteIterator(final FileStatus fileStatus,
+      final ListingSupport listingSupport) {
+    this.fileStatus = fileStatus;
+    this.listingSupport = listingSupport;
+    iteratorsQueue = new ArrayBlockingQueue<>(MAX_QUEUE_SIZE);
+    currIterator = Collections.emptyIterator();
+    fetchBatchesAsync();
+  }
+
+  @Override
+  public boolean hasNext() throws IOException {
+    if (currIterator.hasNext()) {
+      return true;
+    }
+    updateCurrentIterator();
+    return currIterator.hasNext();
+  }
+
+  @Override
+  public FileStatus next() throws IOException {
+    if (!this.hasNext()) {
+      throw new NoSuchElementException();
+    }
+    return currIterator.next();
+  }
+
+  private void updateCurrentIterator() throws IOException {
+    fetchBatchesAsync();
+    synchronized (this) {
+      if (iteratorsQueue.isEmpty()) {
+        if (ioException != null) {
+          throw ioException;
+        }
+        if (isListingComplete()) {
+          return;
+        }
+      }
+    }
+    try {
+      currIterator = iteratorsQueue.take();
+      if (!currIterator.hasNext() && !isListingComplete()) {
+        updateCurrentIterator();
+      }
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      LOG.error("Thread got interrupted: {}", e);
+    }
+  }
+
+  private synchronized boolean isListingComplete() {
+    return !firstBatch && (continuation == null || continuation.isEmpty());
+  }
+
+  private void fetchBatchesAsync() {
+    if (isAsyncInProgress) {
+      return;
+    }
+    synchronized (asyncOpLock) {
+      if (isAsyncInProgress) {
+        return;
+      }
+      isAsyncInProgress = true;
+    }
+    CompletableFuture.runAsync(() -> asyncOp());
+  }
+
+  private void asyncOp() {
+    try {
+      while (!isListingComplete() && iteratorsQueue.size() <= MAX_QUEUE_SIZE) {
+        addNextBatchIteratorToQueue();
+      }
+    } catch (IOException e) {
+      ioException = e;

Review comment:
       Exceptions are also put into the queue. uture is not used.

##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsListStatusRemoteIterator.java
##########
@@ -0,0 +1,151 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs.services;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.CompletableFuture;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.RemoteIterator;
+
+public class AbfsListStatusRemoteIterator implements RemoteIterator<FileStatus> {
+
+  private static final Logger LOG = LoggerFactory
+      .getLogger(AbfsListStatusRemoteIterator.class);
+
+  private static final boolean FETCH_ALL_FALSE = false;
+  private static final int MAX_QUEUE_SIZE = 10;
+
+  private final FileStatus fileStatus;
+  private final ListingSupport listingSupport;
+  private final ArrayBlockingQueue<Iterator<FileStatus>> iteratorsQueue;
+  private final Object asyncOpLock = new Object();
+
+  private volatile boolean isAsyncInProgress = false;
+  private boolean firstBatch = true;
+  private String continuation;
+  private Iterator<FileStatus> currIterator;
+  private IOException ioException;
+
+  public AbfsListStatusRemoteIterator(final FileStatus fileStatus,
+      final ListingSupport listingSupport) {
+    this.fileStatus = fileStatus;
+    this.listingSupport = listingSupport;
+    iteratorsQueue = new ArrayBlockingQueue<>(MAX_QUEUE_SIZE);
+    currIterator = Collections.emptyIterator();
+    fetchBatchesAsync();
+  }
+
+  @Override
+  public boolean hasNext() throws IOException {
+    if (currIterator.hasNext()) {
+      return true;
+    }
+    updateCurrentIterator();
+    return currIterator.hasNext();
+  }
+
+  @Override
+  public FileStatus next() throws IOException {
+    if (!this.hasNext()) {
+      throw new NoSuchElementException();
+    }
+    return currIterator.next();
+  }
+
+  private void updateCurrentIterator() throws IOException {
+    fetchBatchesAsync();
+    synchronized (this) {
+      if (iteratorsQueue.isEmpty()) {
+        if (ioException != null) {
+          throw ioException;
+        }
+        if (isListingComplete()) {
+          return;
+        }
+      }
+    }
+    try {
+      currIterator = iteratorsQueue.take();
+      if (!currIterator.hasNext() && !isListingComplete()) {
+        updateCurrentIterator();
+      }
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      LOG.error("Thread got interrupted: {}", e);
+    }
+  }
+
+  private synchronized boolean isListingComplete() {
+    return !firstBatch && (continuation == null || continuation.isEmpty());
+  }
+
+  private void fetchBatchesAsync() {
+    if (isAsyncInProgress) {
+      return;
+    }
+    synchronized (asyncOpLock) {
+      if (isAsyncInProgress) {
+        return;
+      }
+      isAsyncInProgress = true;
+    }
+    CompletableFuture.runAsync(() -> asyncOp());
+  }
+
+  private void asyncOp() {
+    try {
+      while (!isListingComplete() && iteratorsQueue.size() <= MAX_QUEUE_SIZE) {
+        addNextBatchIteratorToQueue();
+      }
+    } catch (IOException e) {
+      ioException = e;

Review comment:
       Exceptions are also put into the queue. Future is not used.




----------------------------------------------------------------
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 #2548: HADOOP-17475. ABFS: Implementing ListStatusRemoteIterator

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


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |   1m  2s |  |  Docker mode activated.  |
   |||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  |  No case conflicting files found.  |
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |   |   0m  0s | [test4tests](test4tests) |  The patch appears to include 1 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |  38m 10s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |   0m 41s |  |  trunk passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  compile  |   0m 30s |  |  trunk passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~20.04-b01  |
   | +1 :green_heart: |  checkstyle  |   0m 27s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   0m 37s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  14m 19s |  |  branch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 32s |  |  trunk passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javadoc  |   0m 29s |  |  trunk passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~20.04-b01  |
   | +0 :ok: |  spotbugs  |   0m 59s |  |  Used deprecated FindBugs config; considering switching to SpotBugs.  |
   | +1 :green_heart: |  findbugs  |   0m 58s |  |  trunk passed  |
   | -0 :warning: |  patch  |   1m 16s |  |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   |||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   0m 29s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 28s |  |  the patch passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javac  |   0m 28s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 25s |  |  the patch passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~20.04-b01  |
   | +1 :green_heart: |  javac  |   0m 25s |  |  the patch passed  |
   | +1 :green_heart: |  checkstyle  |   0m 17s |  |  the patch passed  |
   | +1 :green_heart: |  mvnsite  |   0m 28s |  |  the patch passed  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  shadedclient  |  12m 46s |  |  patch has no errors when building and testing our client artifacts.  |
   | -1 :x: |  javadoc  |   0m 26s | [/diff-javadoc-javadoc-hadoop-tools_hadoop-azure-jdkUbuntu-11.0.9.1+1-Ubuntu-0ubuntu1.20.04.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/17/artifact/out/diff-javadoc-javadoc-hadoop-tools_hadoop-azure-jdkUbuntu-11.0.9.1+1-Ubuntu-0ubuntu1.20.04.txt) |  hadoop-tools_hadoop-azure-jdkUbuntu-11.0.9.1+1-Ubuntu-0ubuntu1.20.04 with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.20.04 generated 3 new + 15 unchanged - 2 fixed = 18 total (was 17)  |
   | -1 :x: |  javadoc  |   0m 25s | [/diff-javadoc-javadoc-hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_275-8u275-b01-0ubuntu1~20.04-b01.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/17/artifact/out/diff-javadoc-javadoc-hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_275-8u275-b01-0ubuntu1~20.04-b01.txt) |  hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_275-8u275-b01-0ubuntu1~20.04-b01 with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~20.04-b01 generated 3 new + 15 unchanged - 2 fixed = 18 total (was 17)  |
   | -1 :x: |  findbugs  |   1m  0s | [/new-findbugs-hadoop-tools_hadoop-azure.html](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/17/artifact/out/new-findbugs-hadoop-tools_hadoop-azure.html) |  hadoop-tools/hadoop-azure generated 1 new + 0 unchanged - 0 fixed = 1 total (was 0)  |
   |||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   1m 55s |  |  hadoop-azure in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 32s |  |  The patch does not generate ASF License warnings.  |
   |  |   |  78m 53s |  |  |
   
   
   | Reason | Tests |
   |-------:|:------|
   | FindBugs | module:hadoop-tools/hadoop-azure |
   |  |  Inconsistent synchronization of org.apache.hadoop.fs.azurebfs.services.AbfsListStatusRemoteIterator.continuation; locked 50% of time  Unsynchronized access at AbfsListStatusRemoteIterator.java:50% of time  Unsynchronized access at AbfsListStatusRemoteIterator.java:[line 147] |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/17/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/2548 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient findbugs checkstyle |
   | uname | Linux 4d0b871b529a 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 / 18978f2e204 |
   | Default Java | Private Build-1.8.0_275-8u275-b01-0ubuntu1~20.04-b01 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.20.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_275-8u275-b01-0ubuntu1~20.04-b01 |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/17/testReport/ |
   | Max. process+thread count | 536 (vs. ulimit of 5500) |
   | modules | C: hadoop-tools/hadoop-azure U: hadoop-tools/hadoop-azure |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/17/console |
   | versions | git=2.25.1 maven=3.6.3 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 #2548: HADOOP-17475. ABFS: Implementing ListStatusRemoteIterator

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


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |  42m 17s |  |  Docker mode activated.  |
   |||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  |  No case conflicting files found.  |
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |   |   0m  0s | [test4tests](test4tests) |  The patch appears to include 1 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |  32m 24s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |   0m 38s |  |  trunk passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  compile  |   0m 33s |  |  trunk passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~20.04-b01  |
   | +1 :green_heart: |  checkstyle  |   0m 27s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   0m 39s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  14m 21s |  |  branch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 32s |  |  trunk passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javadoc  |   0m 30s |  |  trunk passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~20.04-b01  |
   | +0 :ok: |  spotbugs  |   1m  0s |  |  Used deprecated FindBugs config; considering switching to SpotBugs.  |
   | +1 :green_heart: |  findbugs  |   0m 58s |  |  trunk passed  |
   | -0 :warning: |  patch  |   1m 19s |  |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   |||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   0m 29s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 29s |  |  the patch passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javac  |   0m 29s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 26s |  |  the patch passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~20.04-b01  |
   | +1 :green_heart: |  javac  |   0m 26s |  |  the patch passed  |
   | -0 :warning: |  checkstyle  |   0m 18s | [/diff-checkstyle-hadoop-tools_hadoop-azure.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/16/artifact/out/diff-checkstyle-hadoop-tools_hadoop-azure.txt) |  hadoop-tools/hadoop-azure: The patch generated 1 new + 4 unchanged - 0 fixed = 5 total (was 4)  |
   | +1 :green_heart: |  mvnsite  |   0m 29s |  |  the patch passed  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  shadedclient  |  13m 10s |  |  patch has no errors when building and testing our client artifacts.  |
   | -1 :x: |  javadoc  |   0m 27s | [/diff-javadoc-javadoc-hadoop-tools_hadoop-azure-jdkUbuntu-11.0.9.1+1-Ubuntu-0ubuntu1.20.04.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/16/artifact/out/diff-javadoc-javadoc-hadoop-tools_hadoop-azure-jdkUbuntu-11.0.9.1+1-Ubuntu-0ubuntu1.20.04.txt) |  hadoop-tools_hadoop-azure-jdkUbuntu-11.0.9.1+1-Ubuntu-0ubuntu1.20.04 with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.20.04 generated 3 new + 15 unchanged - 2 fixed = 18 total (was 17)  |
   | -1 :x: |  javadoc  |   0m 25s | [/diff-javadoc-javadoc-hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_275-8u275-b01-0ubuntu1~20.04-b01.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/16/artifact/out/diff-javadoc-javadoc-hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_275-8u275-b01-0ubuntu1~20.04-b01.txt) |  hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_275-8u275-b01-0ubuntu1~20.04-b01 with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~20.04-b01 generated 3 new + 15 unchanged - 2 fixed = 18 total (was 17)  |
   | -1 :x: |  findbugs  |   0m 58s | [/new-findbugs-hadoop-tools_hadoop-azure.html](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/16/artifact/out/new-findbugs-hadoop-tools_hadoop-azure.html) |  hadoop-tools/hadoop-azure generated 1 new + 0 unchanged - 0 fixed = 1 total (was 0)  |
   |||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   1m 55s |  |  hadoop-azure in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 33s |  |  The patch does not generate ASF License warnings.  |
   |  |   | 115m 11s |  |  |
   
   
   | Reason | Tests |
   |-------:|:------|
   | FindBugs | module:hadoop-tools/hadoop-azure |
   |  |  Inconsistent synchronization of org.apache.hadoop.fs.azurebfs.services.AbfsListStatusRemoteIterator.continuation; locked 50% of time  Unsynchronized access at AbfsListStatusRemoteIterator.java:50% of time  Unsynchronized access at AbfsListStatusRemoteIterator.java:[line 147] |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/16/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/2548 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient findbugs checkstyle |
   | uname | Linux 91155a4390ce 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 / 80c7404b519 |
   | Default Java | Private Build-1.8.0_275-8u275-b01-0ubuntu1~20.04-b01 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.20.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_275-8u275-b01-0ubuntu1~20.04-b01 |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/16/testReport/ |
   | Max. process+thread count | 627 (vs. ulimit of 5500) |
   | modules | C: hadoop-tools/hadoop-azure U: hadoop-tools/hadoop-azure |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/16/console |
   | versions | git=2.25.1 maven=3.6.3 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 #2548: HADOOP-17475. ABFS: Implementing ListStatusRemoteIterator

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


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |   1m  5s |  |  Docker mode activated.  |
   |||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  |  No case conflicting files found.  |
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |   |   0m  0s | [test4tests](test4tests) |  The patch appears to include 1 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |  32m 51s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |   0m 39s |  |  trunk passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  compile  |   0m 35s |  |  trunk passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~20.04-b01  |
   | +1 :green_heart: |  checkstyle  |   0m 29s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   0m 40s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  14m  6s |  |  branch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 31s |  |  trunk passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javadoc  |   0m 30s |  |  trunk passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~20.04-b01  |
   | +0 :ok: |  spotbugs  |   0m 59s |  |  Used deprecated FindBugs config; considering switching to SpotBugs.  |
   | +1 :green_heart: |  findbugs  |   0m 56s |  |  trunk passed  |
   | -0 :warning: |  patch  |   1m 15s |  |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   |||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   0m 29s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 30s |  |  the patch passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javac  |   0m 30s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 26s |  |  the patch passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~20.04-b01  |
   | +1 :green_heart: |  javac  |   0m 26s |  |  the patch passed  |
   | +1 :green_heart: |  checkstyle  |   0m 18s |  |  the patch passed  |
   | +1 :green_heart: |  mvnsite  |   0m 29s |  |  the patch passed  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  xml  |   0m  1s |  |  The patch has no ill-formed XML file.  |
   | +1 :green_heart: |  shadedclient  |  12m 46s |  |  patch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 27s |  |  hadoop-tools_hadoop-azure-jdkUbuntu-11.0.9.1+1-Ubuntu-0ubuntu1.20.04 with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.20.04 generated 0 new + 15 unchanged - 2 fixed = 15 total (was 17)  |
   | +1 :green_heart: |  javadoc  |   0m 25s |  |  hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_275-8u275-b01-0ubuntu1~20.04-b01 with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~20.04-b01 generated 0 new + 15 unchanged - 2 fixed = 15 total (was 17)  |
   | +1 :green_heart: |  findbugs  |   0m 58s |  |  the patch passed  |
   |||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   1m 56s |  |  hadoop-azure in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 34s |  |  The patch does not generate ASF License warnings.  |
   |  |   |  73m 52s |  |  |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/20/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/2548 |
   | Optional Tests | dupname asflicense xml compile javac javadoc mvninstall mvnsite unit shadedclient findbugs checkstyle |
   | uname | Linux 2471c0bd7764 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 / 26b9d480e80 |
   | Default Java | Private Build-1.8.0_275-8u275-b01-0ubuntu1~20.04-b01 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.20.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_275-8u275-b01-0ubuntu1~20.04-b01 |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/20/testReport/ |
   | Max. process+thread count | 723 (vs. ulimit of 5500) |
   | modules | C: hadoop-tools/hadoop-azure U: hadoop-tools/hadoop-azure |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/20/console |
   | versions | git=2.25.1 maven=3.6.3 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 #2548: DRAFT PR: Implementing ListStatusRemoteIterator

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


   please, give it a JIRA ID, so when I look @ my notifications I know what it is about, same when I search in my inbox. I don't want to have put in that effort myself. 
   
   


----------------------------------------------------------------
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 #2548: HADOOP-17475. ABFS: Implementing ListStatusRemoteIterator

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






----------------------------------------------------------------
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] bilaharith commented on pull request #2548: HADOOP-17475. ABFS: Implementing ListStatusRemoteIterator

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


   > > There is one one findbugs warning remaining which is of medium priority. The same can be ignored since at line 147 the continuation token returned by the ListingOperation ouside the synchroniced lock since the same involve an http call.
   > 
   > well, it needs to be dealt with either by fixing or findbugs.xml. Define "medium priority" here
   
   Medium priority was a terminology used within findbugs report.
   The member variable continuation is returned from an external http call. Keeping this outside synchronized block since the same is costly.
   The same is excluded in the findbugs.xml file


----------------------------------------------------------------
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] bilaharith commented on a change in pull request #2548: DRAFT PR: Implementing ListStatusRemoteIterator

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



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ListStatusRemoteIterator.java
##########
@@ -0,0 +1,149 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs.services;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.CompletableFuture;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore;
+
+public class ListStatusRemoteIterator implements RemoteIterator<FileStatus> {
+
+  private static final Logger LOG = LoggerFactory
+      .getLogger(ListStatusRemoteIterator.class);
+
+  private static final boolean FETCH_ALL_FALSE = false;
+  private static final int MAX_QUEUE_SIZE = 10;
+
+  private final Path path;
+  private final AzureBlobFileSystemStore abfsStore;
+  private final ArrayBlockingQueue<Iterator<FileStatus>> iteratorsQueue;
+  private final Object asyncOpLock = new Object();
+
+  private boolean firstBatch = true;
+  private boolean isAsyncInProgress = false;
+  private String continuation;
+  private Iterator<FileStatus> currIterator;
+  private IOException ioException;
+
+  public ListStatusRemoteIterator(final Path path,
+      final AzureBlobFileSystemStore abfsStore) {
+    this.path = path;
+    this.abfsStore = abfsStore;
+    iteratorsQueue = new ArrayBlockingQueue<>(MAX_QUEUE_SIZE);
+    currIterator = Collections.emptyIterator();
+    fetchBatchesAsync();
+  }
+
+  @Override
+  public boolean hasNext() throws IOException {
+    if (currIterator.hasNext()) {
+      return true;
+    }
+    updateCurrentIterator();
+    return currIterator.hasNext();
+  }
+
+  @Override
+  public FileStatus next() throws IOException {
+    if (!this.hasNext()) {
+      throw new NoSuchElementException();
+    }
+    return currIterator.next();
+  }
+
+  private void updateCurrentIterator() throws IOException {
+    fetchBatchesAsync();
+    synchronized (this) {
+      if (iteratorsQueue.isEmpty()) {
+        if (ioException != null) {
+          throw ioException;
+        }
+        if (isListingComplete()) {
+          return;
+        }
+      }
+    }
+    try {
+      currIterator = iteratorsQueue.take();
+      if (!currIterator.hasNext() && !isListingComplete()) {
+        updateCurrentIterator();
+      }
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      LOG.error("Thread got interrupted: {}", e);
+    }
+  }
+
+  private boolean isListingComplete() {
+    return !firstBatch && (continuation == null || continuation.isEmpty());
+  }
+
+  private void fetchBatchesAsync() {
+    CompletableFuture.runAsync(() -> asyncOp());
+  }
+
+  private void asyncOp() {
+    if (isAsyncInProgress) {
+      return;
+    }
+    synchronized (asyncOpLock) {
+      if (isAsyncInProgress) {
+        return;
+      }
+      isAsyncInProgress = true;
+    }
+    try {
+      while (!isListingComplete() && iteratorsQueue.size() <= MAX_QUEUE_SIZE) {
+        addNextBatchIteratorToQueue();
+      }
+    } catch (IOException e) {
+      ioException = e;
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      LOG.error("Thread got interrupted: {}", e);
+    } finally {
+      isAsyncInProgress = false;

Review comment:
       Done

##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java
##########
@@ -982,6 +984,14 @@ public boolean exists(Path f) throws IOException {
     return super.exists(f);
   }
 
+  @Override
+  public RemoteIterator<FileStatus> listStatusIterator(Path path)
+      throws IOException {
+    LOG.debug("AzureBlobFileSystem.listStatusIterator path : {}", path);

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] bilaharith commented on a change in pull request #2548: DRAFT PR: Implementing ListStatusRemoteIterator

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



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ListStatusRemoteIterator.java
##########
@@ -0,0 +1,149 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs.services;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.CompletableFuture;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore;
+
+public class ListStatusRemoteIterator implements RemoteIterator<FileStatus> {
+
+  private static final Logger LOG = LoggerFactory
+      .getLogger(ListStatusRemoteIterator.class);
+
+  private static final boolean FETCH_ALL_FALSE = false;
+  private static final int MAX_QUEUE_SIZE = 10;
+
+  private final Path path;
+  private final AzureBlobFileSystemStore abfsStore;
+  private final ArrayBlockingQueue<Iterator<FileStatus>> iteratorsQueue;
+  private final Object asyncOpLock = new Object();
+
+  private boolean firstBatch = true;
+  private boolean isAsyncInProgress = false;
+  private String continuation;
+  private Iterator<FileStatus> currIterator;
+  private IOException ioException;
+
+  public ListStatusRemoteIterator(final Path path,
+      final AzureBlobFileSystemStore abfsStore) {
+    this.path = path;
+    this.abfsStore = abfsStore;
+    iteratorsQueue = new ArrayBlockingQueue<>(MAX_QUEUE_SIZE);
+    currIterator = Collections.emptyIterator();
+    fetchBatchesAsync();
+  }
+
+  @Override
+  public boolean hasNext() throws IOException {
+    if (currIterator.hasNext()) {
+      return true;
+    }
+    updateCurrentIterator();
+    return currIterator.hasNext();
+  }
+
+  @Override
+  public FileStatus next() throws IOException {
+    if (!this.hasNext()) {
+      throw new NoSuchElementException();
+    }
+    return currIterator.next();
+  }
+
+  private void updateCurrentIterator() throws IOException {
+    fetchBatchesAsync();
+    synchronized (this) {
+      if (iteratorsQueue.isEmpty()) {
+        if (ioException != null) {
+          throw ioException;
+        }
+        if (isListingComplete()) {
+          return;
+        }
+      }
+    }
+    try {
+      currIterator = iteratorsQueue.take();
+      if (!currIterator.hasNext() && !isListingComplete()) {
+        updateCurrentIterator();
+      }
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      LOG.error("Thread got interrupted: {}", e);
+    }
+  }
+
+  private boolean isListingComplete() {
+    return !firstBatch && (continuation == null || continuation.isEmpty());
+  }
+
+  private void fetchBatchesAsync() {
+    CompletableFuture.runAsync(() -> asyncOp());
+  }
+
+  private void asyncOp() {
+    if (isAsyncInProgress) {
+      return;
+    }
+    synchronized (asyncOpLock) {
+      if (isAsyncInProgress) {
+        return;
+      }
+      isAsyncInProgress = true;
+    }
+    try {
+      while (!isListingComplete() && iteratorsQueue.size() <= MAX_QUEUE_SIZE) {
+        addNextBatchIteratorToQueue();
+      }
+    } catch (IOException e) {
+      ioException = e;
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      LOG.error("Thread got interrupted: {}", e);

Review comment:
       Yes. Also we need to set the interrut flag again.

##########
File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemListStatusIterator.java
##########
@@ -0,0 +1,190 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs;
+
+import java.io.IOException;
+import java.lang.reflect.Field;
+import java.lang.reflect.Modifier;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+
+import org.assertj.core.api.Assertions;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+import org.apache.hadoop.fs.azurebfs.services.ListStatusRemoteIterator;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+
+/**
+ * Test listStatus operation.
+ */
+public class ITestAzureBlobFileSystemListStatusIterator
+    extends AbstractAbfsIntegrationTest {
+
+  private static final int TEST_FILES_NUMBER = 1000;
+
+  public ITestAzureBlobFileSystemListStatusIterator() throws Exception {
+    super();
+  }
+
+  @Test
+  public void testListPath() throws Exception {
+    final AzureBlobFileSystem fs = getFileSystem();
+    String rootPath = "testRoot1";
+    final List<String> fileNames = createFiles(TEST_FILES_NUMBER, rootPath,
+        "testListPath");
+    AzureBlobFileSystemStore abfsStore = getAbfsStore(fs);
+    abfsStore.getAbfsConfiguration().setListMaxResults(10);
+    RemoteIterator<FileStatus> fsIt = fs.listStatusIterator(new Path(rootPath));
+    int itrCount = 0;
+    while (fsIt.hasNext()) {
+      FileStatus fileStatus = fsIt.next();
+      String pathStr = fileStatus.getPath().toString();
+      fileNames.remove(pathStr);
+      itrCount++;
+    }
+    assertEquals(TEST_FILES_NUMBER, itrCount);
+    assertEquals(0, fileNames.size());
+  }
+
+  @Test
+  public void testNextWhenNoMoreElementsPresent() throws Exception {
+    final AzureBlobFileSystem fs = getFileSystem();
+    String rootPathStr = "testRoot2";
+    Path rootPath = new Path(rootPathStr);
+    getFileSystem().mkdirs(rootPath);
+    RemoteIterator<FileStatus> fsItr = fs.listStatusIterator(rootPath);
+    fsItr = Mockito.spy(fsItr);
+    Mockito.doReturn(false).when(fsItr).hasNext();
+
+    RemoteIterator<FileStatus> finalFsItr = fsItr;
+    Assertions.assertThatThrownBy(() -> finalFsItr.next()).describedAs(
+        "next() should throw NoSuchElementException if hasNext() return "
+            + "false").isInstanceOf(NoSuchElementException.class);
+  }
+
+  @Test
+  public void testHasNextForEmptyDir() throws Exception {
+    final AzureBlobFileSystem fs = getFileSystem();
+    String rootPathStr = "testRoot3";
+    Path rootPath = new Path(rootPathStr);
+    getFileSystem().mkdirs(rootPath);
+    RemoteIterator<FileStatus> fsItr = fs.listStatusIterator(rootPath);
+    Assertions.assertThat(fsItr.hasNext())

Review comment:
       Done

##########
File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemListStatusIterator.java
##########
@@ -0,0 +1,190 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs;
+
+import java.io.IOException;
+import java.lang.reflect.Field;
+import java.lang.reflect.Modifier;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+
+import org.assertj.core.api.Assertions;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+import org.apache.hadoop.fs.azurebfs.services.ListStatusRemoteIterator;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+
+/**
+ * Test listStatus operation.
+ */
+public class ITestAzureBlobFileSystemListStatusIterator
+    extends AbstractAbfsIntegrationTest {
+
+  private static final int TEST_FILES_NUMBER = 1000;
+
+  public ITestAzureBlobFileSystemListStatusIterator() throws Exception {
+    super();
+  }
+
+  @Test
+  public void testListPath() throws Exception {
+    final AzureBlobFileSystem fs = getFileSystem();
+    String rootPath = "testRoot1";
+    final List<String> fileNames = createFiles(TEST_FILES_NUMBER, rootPath,
+        "testListPath");
+    AzureBlobFileSystemStore abfsStore = getAbfsStore(fs);
+    abfsStore.getAbfsConfiguration().setListMaxResults(10);
+    RemoteIterator<FileStatus> fsIt = fs.listStatusIterator(new Path(rootPath));
+    int itrCount = 0;
+    while (fsIt.hasNext()) {
+      FileStatus fileStatus = fsIt.next();
+      String pathStr = fileStatus.getPath().toString();
+      fileNames.remove(pathStr);
+      itrCount++;
+    }
+    assertEquals(TEST_FILES_NUMBER, itrCount);
+    assertEquals(0, fileNames.size());
+  }
+
+  @Test
+  public void testNextWhenNoMoreElementsPresent() throws Exception {
+    final AzureBlobFileSystem fs = getFileSystem();
+    String rootPathStr = "testRoot2";
+    Path rootPath = new Path(rootPathStr);
+    getFileSystem().mkdirs(rootPath);
+    RemoteIterator<FileStatus> fsItr = fs.listStatusIterator(rootPath);
+    fsItr = Mockito.spy(fsItr);
+    Mockito.doReturn(false).when(fsItr).hasNext();
+
+    RemoteIterator<FileStatus> finalFsItr = fsItr;
+    Assertions.assertThatThrownBy(() -> finalFsItr.next()).describedAs(
+        "next() should throw NoSuchElementException if hasNext() return "
+            + "false").isInstanceOf(NoSuchElementException.class);
+  }
+
+  @Test
+  public void testHasNextForEmptyDir() throws Exception {
+    final AzureBlobFileSystem fs = getFileSystem();
+    String rootPathStr = "testRoot3";
+    Path rootPath = new Path(rootPathStr);
+    getFileSystem().mkdirs(rootPath);
+    RemoteIterator<FileStatus> fsItr = fs.listStatusIterator(rootPath);
+    Assertions.assertThat(fsItr.hasNext())
+        .describedAs("hasNext returns false for empty directory").isFalse();
+  }
+
+  @Test
+  public void testHasNextForFile() throws Exception {
+    final AzureBlobFileSystem fs = getFileSystem();
+    String rootPathStr = "testRoot4";

Review comment:
       Done

##########
File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemListStatusIterator.java
##########
@@ -0,0 +1,190 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs;
+
+import java.io.IOException;
+import java.lang.reflect.Field;
+import java.lang.reflect.Modifier;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+
+import org.assertj.core.api.Assertions;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+import org.apache.hadoop.fs.azurebfs.services.ListStatusRemoteIterator;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+
+/**
+ * Test listStatus operation.
+ */
+public class ITestAzureBlobFileSystemListStatusIterator
+    extends AbstractAbfsIntegrationTest {
+
+  private static final int TEST_FILES_NUMBER = 1000;
+
+  public ITestAzureBlobFileSystemListStatusIterator() throws Exception {
+    super();
+  }
+
+  @Test
+  public void testListPath() throws Exception {
+    final AzureBlobFileSystem fs = getFileSystem();
+    String rootPath = "testRoot1";
+    final List<String> fileNames = createFiles(TEST_FILES_NUMBER, rootPath,
+        "testListPath");
+    AzureBlobFileSystemStore abfsStore = getAbfsStore(fs);
+    abfsStore.getAbfsConfiguration().setListMaxResults(10);
+    RemoteIterator<FileStatus> fsIt = fs.listStatusIterator(new Path(rootPath));
+    int itrCount = 0;
+    while (fsIt.hasNext()) {
+      FileStatus fileStatus = fsIt.next();
+      String pathStr = fileStatus.getPath().toString();
+      fileNames.remove(pathStr);
+      itrCount++;
+    }
+    assertEquals(TEST_FILES_NUMBER, itrCount);
+    assertEquals(0, fileNames.size());
+  }
+
+  @Test
+  public void testNextWhenNoMoreElementsPresent() throws Exception {
+    final AzureBlobFileSystem fs = getFileSystem();
+    String rootPathStr = "testRoot2";
+    Path rootPath = new Path(rootPathStr);
+    getFileSystem().mkdirs(rootPath);
+    RemoteIterator<FileStatus> fsItr = fs.listStatusIterator(rootPath);
+    fsItr = Mockito.spy(fsItr);
+    Mockito.doReturn(false).when(fsItr).hasNext();
+
+    RemoteIterator<FileStatus> finalFsItr = fsItr;
+    Assertions.assertThatThrownBy(() -> finalFsItr.next()).describedAs(
+        "next() should throw NoSuchElementException if hasNext() return "
+            + "false").isInstanceOf(NoSuchElementException.class);
+  }
+
+  @Test
+  public void testHasNextForEmptyDir() throws Exception {
+    final AzureBlobFileSystem fs = getFileSystem();
+    String rootPathStr = "testRoot3";
+    Path rootPath = new Path(rootPathStr);
+    getFileSystem().mkdirs(rootPath);
+    RemoteIterator<FileStatus> fsItr = fs.listStatusIterator(rootPath);
+    Assertions.assertThat(fsItr.hasNext())
+        .describedAs("hasNext returns false for empty directory").isFalse();
+  }
+
+  @Test
+  public void testHasNextForFile() throws Exception {
+    final AzureBlobFileSystem fs = getFileSystem();
+    String rootPathStr = "testRoot4";
+    Path rootPath = new Path(rootPathStr);
+    getFileSystem().create(rootPath);
+    RemoteIterator<FileStatus> fsItr = fs.listStatusIterator(rootPath);
+    Assertions.assertThat(fsItr.hasNext())
+        .describedAs("hasNext returns true for file").isTrue();

Review comment:
       Done

##########
File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemListStatusIterator.java
##########
@@ -0,0 +1,190 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs;
+
+import java.io.IOException;
+import java.lang.reflect.Field;
+import java.lang.reflect.Modifier;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+
+import org.assertj.core.api.Assertions;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+import org.apache.hadoop.fs.azurebfs.services.ListStatusRemoteIterator;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+
+/**
+ * Test listStatus operation.
+ */
+public class ITestAzureBlobFileSystemListStatusIterator
+    extends AbstractAbfsIntegrationTest {
+
+  private static final int TEST_FILES_NUMBER = 1000;
+
+  public ITestAzureBlobFileSystemListStatusIterator() throws Exception {
+    super();
+  }
+
+  @Test
+  public void testListPath() throws Exception {
+    final AzureBlobFileSystem fs = getFileSystem();
+    String rootPath = "testRoot1";
+    final List<String> fileNames = createFiles(TEST_FILES_NUMBER, rootPath,
+        "testListPath");
+    AzureBlobFileSystemStore abfsStore = getAbfsStore(fs);
+    abfsStore.getAbfsConfiguration().setListMaxResults(10);
+    RemoteIterator<FileStatus> fsIt = fs.listStatusIterator(new Path(rootPath));
+    int itrCount = 0;
+    while (fsIt.hasNext()) {
+      FileStatus fileStatus = fsIt.next();
+      String pathStr = fileStatus.getPath().toString();
+      fileNames.remove(pathStr);
+      itrCount++;
+    }
+    assertEquals(TEST_FILES_NUMBER, itrCount);
+    assertEquals(0, fileNames.size());
+  }
+
+  @Test
+  public void testNextWhenNoMoreElementsPresent() throws Exception {
+    final AzureBlobFileSystem fs = getFileSystem();
+    String rootPathStr = "testRoot2";
+    Path rootPath = new Path(rootPathStr);
+    getFileSystem().mkdirs(rootPath);
+    RemoteIterator<FileStatus> fsItr = fs.listStatusIterator(rootPath);
+    fsItr = Mockito.spy(fsItr);
+    Mockito.doReturn(false).when(fsItr).hasNext();
+
+    RemoteIterator<FileStatus> finalFsItr = fsItr;
+    Assertions.assertThatThrownBy(() -> finalFsItr.next()).describedAs(
+        "next() should throw NoSuchElementException if hasNext() return "
+            + "false").isInstanceOf(NoSuchElementException.class);
+  }
+
+  @Test
+  public void testHasNextForEmptyDir() throws Exception {
+    final AzureBlobFileSystem fs = getFileSystem();
+    String rootPathStr = "testRoot3";
+    Path rootPath = new Path(rootPathStr);
+    getFileSystem().mkdirs(rootPath);
+    RemoteIterator<FileStatus> fsItr = fs.listStatusIterator(rootPath);
+    Assertions.assertThat(fsItr.hasNext())
+        .describedAs("hasNext returns false for empty directory").isFalse();
+  }
+
+  @Test
+  public void testHasNextForFile() throws Exception {
+    final AzureBlobFileSystem fs = getFileSystem();
+    String rootPathStr = "testRoot4";
+    Path rootPath = new Path(rootPathStr);
+    getFileSystem().create(rootPath);
+    RemoteIterator<FileStatus> fsItr = fs.listStatusIterator(rootPath);
+    Assertions.assertThat(fsItr.hasNext())
+        .describedAs("hasNext returns true for file").isTrue();
+  }
+
+  @Test
+  public void testHasNextForIOException() throws Exception {
+    final AzureBlobFileSystem fs = getFileSystem();
+    String rootPathStr = "testRoot5";
+    Path rootPath = new Path(rootPathStr);

Review comment:
       Done

##########
File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemListStatusIterator.java
##########
@@ -0,0 +1,190 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs;
+
+import java.io.IOException;
+import java.lang.reflect.Field;
+import java.lang.reflect.Modifier;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+
+import org.assertj.core.api.Assertions;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+import org.apache.hadoop.fs.azurebfs.services.ListStatusRemoteIterator;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+
+/**
+ * Test listStatus operation.
+ */
+public class ITestAzureBlobFileSystemListStatusIterator
+    extends AbstractAbfsIntegrationTest {
+
+  private static final int TEST_FILES_NUMBER = 1000;
+
+  public ITestAzureBlobFileSystemListStatusIterator() throws Exception {
+    super();
+  }
+
+  @Test
+  public void testListPath() throws Exception {
+    final AzureBlobFileSystem fs = getFileSystem();
+    String rootPath = "testRoot1";
+    final List<String> fileNames = createFiles(TEST_FILES_NUMBER, rootPath,
+        "testListPath");
+    AzureBlobFileSystemStore abfsStore = getAbfsStore(fs);
+    abfsStore.getAbfsConfiguration().setListMaxResults(10);
+    RemoteIterator<FileStatus> fsIt = fs.listStatusIterator(new Path(rootPath));
+    int itrCount = 0;
+    while (fsIt.hasNext()) {
+      FileStatus fileStatus = fsIt.next();
+      String pathStr = fileStatus.getPath().toString();
+      fileNames.remove(pathStr);
+      itrCount++;
+    }
+    assertEquals(TEST_FILES_NUMBER, itrCount);
+    assertEquals(0, fileNames.size());
+  }
+
+  @Test
+  public void testNextWhenNoMoreElementsPresent() throws Exception {
+    final AzureBlobFileSystem fs = getFileSystem();
+    String rootPathStr = "testRoot2";
+    Path rootPath = new Path(rootPathStr);
+    getFileSystem().mkdirs(rootPath);
+    RemoteIterator<FileStatus> fsItr = fs.listStatusIterator(rootPath);
+    fsItr = Mockito.spy(fsItr);
+    Mockito.doReturn(false).when(fsItr).hasNext();
+
+    RemoteIterator<FileStatus> finalFsItr = fsItr;
+    Assertions.assertThatThrownBy(() -> finalFsItr.next()).describedAs(
+        "next() should throw NoSuchElementException if hasNext() return "
+            + "false").isInstanceOf(NoSuchElementException.class);
+  }
+
+  @Test
+  public void testHasNextForEmptyDir() throws Exception {
+    final AzureBlobFileSystem fs = getFileSystem();
+    String rootPathStr = "testRoot3";
+    Path rootPath = new Path(rootPathStr);
+    getFileSystem().mkdirs(rootPath);
+    RemoteIterator<FileStatus> fsItr = fs.listStatusIterator(rootPath);
+    Assertions.assertThat(fsItr.hasNext())
+        .describedAs("hasNext returns false for empty directory").isFalse();
+  }
+
+  @Test
+  public void testHasNextForFile() throws Exception {
+    final AzureBlobFileSystem fs = getFileSystem();
+    String rootPathStr = "testRoot4";
+    Path rootPath = new Path(rootPathStr);
+    getFileSystem().create(rootPath);
+    RemoteIterator<FileStatus> fsItr = fs.listStatusIterator(rootPath);
+    Assertions.assertThat(fsItr.hasNext())
+        .describedAs("hasNext returns true for file").isTrue();
+  }
+
+  @Test
+  public void testHasNextForIOException() throws Exception {
+    final AzureBlobFileSystem fs = getFileSystem();
+    String rootPathStr = "testRoot5";
+    Path rootPath = new Path(rootPathStr);
+    getFileSystem().mkdirs(rootPath);
+    ListStatusRemoteIterator fsItr = (ListStatusRemoteIterator) fs
+        .listStatusIterator(rootPath);
+    Thread.sleep(1000);
+
+    String exceptionMessage = "test exception";

Review comment:
       Done

##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java
##########
@@ -865,16 +873,16 @@ public FileStatus getFileStatus(final Path path) throws IOException {
             startFrom);
 
     final String relativePath = getRelativePath(path);
-    String continuation = null;
 
-    // generate continuation token if a valid startFrom is provided.
-    if (startFrom != null && !startFrom.isEmpty()) {
-      continuation = getIsNamespaceEnabled()
-              ? generateContinuationTokenForXns(startFrom)
-              : generateContinuationTokenForNonXns(relativePath, startFrom);
+    if (continuation == null || continuation.length() < 1) {

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] steveloughran commented on pull request #2548: HADOOP-17475. ABFS: Implementing ListStatusRemoteIterator

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


   > There is one one findbugs warning remaining which is of medium priority. The same can be ignored since at line 147 the continuation token returned by the ListingOperation ouside the synchroniced lock since the same involve an http call.
   
   well, it needs to be dealt with either by fixing or findbugs.xml. Define "medium priority" here


----------------------------------------------------------------
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] bilaharith commented on a change in pull request #2548: HADOOP-17475. ABFS: Implementing ListStatusRemoteIterator

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



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java
##########
@@ -982,6 +985,19 @@ public boolean exists(Path f) throws IOException {
     return super.exists(f);
   }
 
+  @Override
+  public RemoteIterator<FileStatus> listStatusIterator(Path path)
+      throws IOException {
+    LOG.debug("AzureBlobFileSystem.listStatusIterator path : {}", path);
+    if (abfsStore.getAbfsConfiguration().enableAbfsListIterator()) {
+      AbfsListStatusRemoteIterator abfsLsItr =
+          new AbfsListStatusRemoteIterator(getFileStatus(path), abfsStore);
+      return RemoteIterators.typeCastingRemoteIterator(abfsLsItr);

Review comment:
       > very close to getting in. other than some minor checkstyle/javadoc complaints, that findbugs needs to be made to stop complaining. Usually it is indicating a real risk of a problem, so I'd like to see what can be done about it -rather than just edit the XML file to turn off the check
   
   There is one one findbugs warning remaining which is of medium priority. The same can be ignored since at line 147 the continuation token returned by the ListingOperation ouside the synchroniced lock since the same involve an http call.




----------------------------------------------------------------
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 #2548: HADOOP-17475. ABFS: Implementing ListStatusRemoteIterator

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


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |   1m  4s |  |  Docker mode activated.  |
   |||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  |  No case conflicting files found.  |
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |   |   0m  0s | [test4tests](test4tests) |  The patch appears to include 1 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |  33m 11s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |   0m 37s |  |  trunk passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04  |
   | +1 :green_heart: |  compile  |   0m 34s |  |  trunk passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01  |
   | +1 :green_heart: |  checkstyle  |   0m 28s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   0m 39s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  16m 48s |  |  branch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 32s |  |  trunk passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04  |
   | +1 :green_heart: |  javadoc  |   0m 30s |  |  trunk passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01  |
   | +0 :ok: |  spotbugs  |   0m 58s |  |  Used deprecated FindBugs config; considering switching to SpotBugs.  |
   | +1 :green_heart: |  findbugs  |   0m 56s |  |  trunk passed  |
   | -0 :warning: |  patch  |   1m 16s |  |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   |||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   0m 30s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 29s |  |  the patch passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04  |
   | +1 :green_heart: |  javac  |   0m 29s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 25s |  |  the patch passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01  |
   | +1 :green_heart: |  javac  |   0m 25s |  |  the patch passed  |
   | -0 :warning: |  checkstyle  |   0m 18s | [/diff-checkstyle-hadoop-tools_hadoop-azure.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/13/artifact/out/diff-checkstyle-hadoop-tools_hadoop-azure.txt) |  hadoop-tools/hadoop-azure: The patch generated 6 new + 4 unchanged - 0 fixed = 10 total (was 4)  |
   | +1 :green_heart: |  mvnsite  |   0m 29s |  |  the patch passed  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  shadedclient  |  15m  8s |  |  patch has no errors when building and testing our client artifacts.  |
   | -1 :x: |  javadoc  |   0m 26s | [/diff-javadoc-javadoc-hadoop-tools_hadoop-azure-jdkUbuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/13/artifact/out/diff-javadoc-javadoc-hadoop-tools_hadoop-azure-jdkUbuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04.txt) |  hadoop-tools_hadoop-azure-jdkUbuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04 with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04 generated 3 new + 15 unchanged - 2 fixed = 18 total (was 17)  |
   | -1 :x: |  javadoc  |   0m 25s | [/diff-javadoc-javadoc-hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/13/artifact/out/diff-javadoc-javadoc-hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01.txt) |  hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01 with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01 generated 3 new + 15 unchanged - 2 fixed = 18 total (was 17)  |
   | -1 :x: |  findbugs  |   1m  2s | [/new-findbugs-hadoop-tools_hadoop-azure.html](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/13/artifact/out/new-findbugs-hadoop-tools_hadoop-azure.html) |  hadoop-tools/hadoop-azure generated 2 new + 0 unchanged - 0 fixed = 2 total (was 0)  |
   |||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   1m 56s |  |  hadoop-azure in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 33s |  |  The patch does not generate ASF License warnings.  |
   |  |   |  79m 17s |  |  |
   
   
   | Reason | Tests |
   |-------:|:------|
   | FindBugs | module:hadoop-tools/hadoop-azure |
   |  |  Inconsistent synchronization of org.apache.hadoop.fs.azurebfs.services.AbfsListStatusRemoteIterator.continuation; locked 50% of time  Unsynchronized access at AbfsListStatusRemoteIterator.java:50% of time  Unsynchronized access at AbfsListStatusRemoteIterator.java:[line 149] |
   |  |  Inconsistent synchronization of org.apache.hadoop.fs.azurebfs.services.AbfsListStatusRemoteIterator.isIterationComplete; locked 50% of time  Unsynchronized access at AbfsListStatusRemoteIterator.java:50% of time  Unsynchronized access at AbfsListStatusRemoteIterator.java:[line 86] |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/13/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/2548 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient findbugs checkstyle |
   | uname | Linux cf1f444f6be9 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 / d82009599a2 |
   | Default Java | Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01 |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/13/testReport/ |
   | Max. process+thread count | 536 (vs. ulimit of 5500) |
   | modules | C: hadoop-tools/hadoop-azure U: hadoop-tools/hadoop-azure |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/13/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 merged pull request #2548: HADOOP-17475. ABFS: Implementing ListStatusRemoteIterator

Posted by GitBox <gi...@apache.org>.
steveloughran merged pull request #2548:
URL: https://github.com/apache/hadoop/pull/2548


   


----------------------------------------------------------------
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] bilaharith removed a comment on pull request #2548: HADOOP-17475. ABFS: Implementing ListStatusRemoteIterator

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


   > rs propagate the IOStatisticsSource interface, so when the innermost iterator collects cost/count of list calls, the stats will be visible to and collectable
   
   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] vinaysbadami commented on a change in pull request #2548: DRAFT PR: Implementing ListStatusRemoteIterator

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



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsListStatusRemoteIterator.java
##########
@@ -0,0 +1,157 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs.services;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.CompletableFuture;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+
+public class AbfsListStatusRemoteIterator implements RemoteIterator<FileStatus> {
+
+  private static final Logger LOG = LoggerFactory
+      .getLogger(AbfsListStatusRemoteIterator.class);
+
+  private static final boolean FETCH_ALL_FALSE = false;
+  private static final int MAX_QUEUE_SIZE = 10;
+
+  private final FileStatus fileStatus;
+  private final ListingSupport listingSupport;
+  private final ArrayBlockingQueue<Iterator<FileStatus>> iteratorsQueue;
+  private final Object asyncOpLock = new Object();
+
+  private volatile boolean isAsyncInProgress = false;
+  private boolean firstBatch = true;
+  private String continuation;
+  private Iterator<FileStatus> currIterator;
+  private IOException ioException;
+
+  public AbfsListStatusRemoteIterator(final FileStatus fileStatus,
+      final ListingSupport listingSupport) {
+    this.fileStatus = fileStatus;
+    this.listingSupport = listingSupport;
+    iteratorsQueue = new ArrayBlockingQueue<>(MAX_QUEUE_SIZE);
+    currIterator = Collections.emptyIterator();
+    fetchBatchesAsync();
+  }
+
+  @Override
+  public boolean hasNext() throws IOException {
+    if (currIterator.hasNext()) {
+      return true;
+    }
+    updateCurrentIterator();
+    return currIterator.hasNext();
+  }
+
+  @Override
+  public FileStatus next() throws IOException {
+    if (!this.hasNext()) {
+      throw new NoSuchElementException();
+    }
+    return currIterator.next();
+  }
+
+  private void updateCurrentIterator() throws IOException {
+    fetchBatchesAsync();
+    synchronized (this) {
+      if (iteratorsQueue.isEmpty()) {
+        if (ioException != null) {
+          throw ioException;
+        }
+        if (isListingComplete()) {
+          return;
+        }
+      }
+    }
+    try {
+      currIterator = iteratorsQueue.take();
+      if (!currIterator.hasNext() && !isListingComplete()) {
+        updateCurrentIterator();
+      }
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      LOG.error("Thread got interrupted: {}", e);
+    }
+  }
+
+  private synchronized boolean isListingComplete() {
+    return !firstBatch && (continuation == null || continuation.isEmpty());
+  }
+
+  private void fetchBatchesAsync() {
+    if (isAsyncInProgress) {
+      return;
+    }
+    synchronized (asyncOpLock) {
+      if (isAsyncInProgress) {
+        return;
+      }
+      isAsyncInProgress = true;
+    }
+    CompletableFuture.runAsync(() -> asyncOp());
+  }
+
+  private void asyncOp() {
+    try {
+      while (!isListingComplete() && iteratorsQueue.size() <= MAX_QUEUE_SIZE) {
+        addNextBatchIteratorToQueue();
+      }
+    } catch (IOException e) {
+      ioException = e;
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      LOG.error("Thread got interrupted: {}", e);
+    } finally {
+      synchronized (asyncOpLock) {
+        try {
+          iteratorsQueue.put(Collections.emptyIterator());

Review comment:
       did u want to do this even when u did not have an exception? 




----------------------------------------------------------------
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] bilaharith commented on pull request #2548: DRAFT PR: Implementing ListStatusRemoteIterator

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


   **Driver test results using accounts in Central India**
   mvn -T 1C -Dparallel-tests=abfs -Dscale -DtestsThreadCount=8 clean verify
   
   HNS-OAuth
   ========================
   [INFO] Results:
   [INFO] 
   [INFO] Tests run: 90, Failures: 0, Errors: 0, Skipped: 0
   [INFO] Results:
   [INFO] 
   [WARNING] Tests run: 507, Failures: 0, Errors: 0, Skipped: 68
   [INFO] Results:
   [INFO] 
   [WARNING] Tests run: 256, Failures: 0, Errors: 0, Skipped: 48
   
   HNS-SharedKey
   ========================
   [INFO] Results:
   [INFO] 
   [INFO] Tests run: 90, Failures: 0, Errors: 0, Skipped: 0
   [INFO] Results:
   [INFO] 
   [WARNING] Tests run: 507, Failures: 0, Errors: 0, Skipped: 24
   [INFO] Results:
   [INFO] 
   [WARNING] Tests run: 256, Failures: 0, Errors: 0, Skipped: 40
   
   NonHNS-SharedKey
   ========================
   [INFO] Results:
   [INFO] 
   [INFO] Tests run: 90, Failures: 0, Errors: 0, Skipped: 0
   [INFO] Results:
   [INFO] 
   [WARNING] Tests run: 507, Failures: 0, Errors: 0, Skipped: 249
   [INFO] Results:
   [INFO] 
   [WARNING] Tests run: 256, Failures: 0, Errors: 0, Skipped: 40


----------------------------------------------------------------
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 #2548: DRAFT PR: Implementing ListStatusRemoteIterator

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


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |   1m 16s |  |  Docker mode activated.  |
   |||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  |  No case conflicting files found.  |
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |   |   0m  0s | [test4tests](test4tests) |  The patch appears to include 1 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |  35m 26s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |   0m 36s |  |  trunk passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04  |
   | +1 :green_heart: |  compile  |   0m 31s |  |  trunk passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01  |
   | +1 :green_heart: |  checkstyle  |   0m 23s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   0m 37s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  17m 57s |  |  branch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 27s |  |  trunk passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04  |
   | +1 :green_heart: |  javadoc  |   0m 25s |  |  trunk passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01  |
   | +0 :ok: |  spotbugs  |   0m 57s |  |  Used deprecated FindBugs config; considering switching to SpotBugs.  |
   | +1 :green_heart: |  findbugs  |   0m 55s |  |  trunk passed  |
   |||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   0m 28s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 29s |  |  the patch passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04  |
   | -1 :x: |  javac  |   0m 29s | [/diff-compile-javac-hadoop-tools_hadoop-azure-jdkUbuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/1/artifact/out/diff-compile-javac-hadoop-tools_hadoop-azure-jdkUbuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04.txt) |  hadoop-tools_hadoop-azure-jdkUbuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04 with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04 generated 2 new + 13 unchanged - 2 fixed = 15 total (was 15)  |
   | +1 :green_heart: |  compile  |   0m 25s |  |  the patch passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01  |
   | -1 :x: |  javac  |   0m 25s | [/diff-compile-javac-hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/1/artifact/out/diff-compile-javac-hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01.txt) |  hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01 with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01 generated 2 new + 7 unchanged - 2 fixed = 9 total (was 9)  |
   | -0 :warning: |  checkstyle  |   0m 15s | [/diff-checkstyle-hadoop-tools_hadoop-azure.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/1/artifact/out/diff-checkstyle-hadoop-tools_hadoop-azure.txt) |  hadoop-tools/hadoop-azure: The patch generated 16 new + 2 unchanged - 0 fixed = 18 total (was 2)  |
   | +1 :green_heart: |  mvnsite  |   0m 29s |  |  the patch passed  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  shadedclient  |  16m 51s |  |  patch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 23s |  |  the patch passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04  |
   | +1 :green_heart: |  javadoc  |   0m 21s |  |  the patch passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01  |
   | +1 :green_heart: |  findbugs  |   1m  0s |  |  the patch passed  |
   |||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   1m 24s |  |  hadoop-azure in the patch passed.  |
   | -1 :x: |  asflicense  |   0m 30s | [/patch-asflicense-problems.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/1/artifact/out/patch-asflicense-problems.txt) |  The patch generated 1 ASF License warnings.  |
   |  |   |  83m  9s |  |  |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/1/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/2548 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient findbugs checkstyle |
   | uname | Linux 4b671b4f7ef3 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 / 5bf977e6b16 |
   | Default Java | Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01 |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/1/testReport/ |
   | Max. process+thread count | 511 (vs. ulimit of 5500) |
   | modules | C: hadoop-tools/hadoop-azure U: hadoop-tools/hadoop-azure |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/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 #2548: DRAFT PR: Implementing ListStatusRemoteIterator

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


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |  32m  4s |  |  Docker mode activated.  |
   |||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  |  No case conflicting files found.  |
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |   |   0m  0s | [test4tests](test4tests) |  The patch appears to include 1 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |  35m 14s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |   0m 35s |  |  trunk passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04  |
   | +1 :green_heart: |  compile  |   0m 30s |  |  trunk passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01  |
   | +1 :green_heart: |  checkstyle  |   0m 23s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   0m 36s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  18m 19s |  |  branch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 29s |  |  trunk passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04  |
   | +1 :green_heart: |  javadoc  |   0m 24s |  |  trunk passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01  |
   | +0 :ok: |  spotbugs  |   0m 58s |  |  Used deprecated FindBugs config; considering switching to SpotBugs.  |
   | +1 :green_heart: |  findbugs  |   0m 56s |  |  trunk passed  |
   |||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   0m 29s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 29s |  |  the patch passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04  |
   | -1 :x: |  javac  |   0m 29s | [/diff-compile-javac-hadoop-tools_hadoop-azure-jdkUbuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/5/artifact/out/diff-compile-javac-hadoop-tools_hadoop-azure-jdkUbuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04.txt) |  hadoop-tools_hadoop-azure-jdkUbuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04 with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04 generated 2 new + 13 unchanged - 2 fixed = 15 total (was 15)  |
   | +1 :green_heart: |  compile  |   0m 24s |  |  the patch passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01  |
   | -1 :x: |  javac  |   0m 24s | [/diff-compile-javac-hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/5/artifact/out/diff-compile-javac-hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01.txt) |  hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01 with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01 generated 2 new + 7 unchanged - 2 fixed = 9 total (was 9)  |
   | -0 :warning: |  checkstyle  |   0m 15s | [/diff-checkstyle-hadoop-tools_hadoop-azure.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/5/artifact/out/diff-checkstyle-hadoop-tools_hadoop-azure.txt) |  hadoop-tools/hadoop-azure: The patch generated 6 new + 2 unchanged - 0 fixed = 8 total (was 2)  |
   | +1 :green_heart: |  mvnsite  |   0m 26s |  |  the patch passed  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  shadedclient  |  16m 36s |  |  patch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 23s |  |  the patch passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04  |
   | +1 :green_heart: |  javadoc  |   0m 21s |  |  the patch passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01  |
   | -1 :x: |  findbugs  |   1m  2s | [/new-findbugs-hadoop-tools_hadoop-azure.html](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/5/artifact/out/new-findbugs-hadoop-tools_hadoop-azure.html) |  hadoop-tools/hadoop-azure generated 2 new + 0 unchanged - 0 fixed = 2 total (was 0)  |
   |||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   1m 24s |  |  hadoop-azure in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 29s |  |  The patch does not generate ASF License warnings.  |
   |  |   | 113m 41s |  |  |
   
   
   | Reason | Tests |
   |-------:|:------|
   | FindBugs | module:hadoop-tools/hadoop-azure |
   |  |  Inconsistent synchronization of org.apache.hadoop.fs.azurebfs.services.ListStatusRemoteIterator.ioException; locked 66% of time  Unsynchronized access at ListStatusRemoteIterator.java:66% of time  Unsynchronized access at ListStatusRemoteIterator.java:[line 107] |
   |  |  Naked notify in org.apache.hadoop.fs.azurebfs.services.ListStatusRemoteIterator.lambda$fetchAllAsync$0()  At ListStatusRemoteIterator.java:At ListStatusRemoteIterator.java:[line 117] |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/5/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/2548 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient findbugs checkstyle |
   | uname | Linux 6c527b9e492b 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 / 513f1995adc |
   | Default Java | Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01 |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/5/testReport/ |
   | Max. process+thread count | 540 (vs. ulimit of 5500) |
   | modules | C: hadoop-tools/hadoop-azure U: hadoop-tools/hadoop-azure |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/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 commented on pull request #2548: DRAFT PR: Implementing ListStatusRemoteIterator

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


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |   1m  7s |  |  Docker mode activated.  |
   |||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  |  No case conflicting files found.  |
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |   |   0m  0s | [test4tests](test4tests) |  The patch appears to include 1 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |  33m 34s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |   0m 40s |  |  trunk passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04  |
   | +1 :green_heart: |  compile  |   0m 35s |  |  trunk passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01  |
   | +1 :green_heart: |  checkstyle  |   0m 27s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   0m 40s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  16m 25s |  |  branch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 26s |  |  trunk passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04  |
   | +1 :green_heart: |  javadoc  |   0m 29s |  |  trunk passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01  |
   | +0 :ok: |  spotbugs  |   1m  2s |  |  Used deprecated FindBugs config; considering switching to SpotBugs.  |
   | +1 :green_heart: |  findbugs  |   1m  0s |  |  trunk passed  |
   |||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   0m 30s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 31s |  |  the patch passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04  |
   | -1 :x: |  javac  |   0m 31s | [/diff-compile-javac-hadoop-tools_hadoop-azure-jdkUbuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/2/artifact/out/diff-compile-javac-hadoop-tools_hadoop-azure-jdkUbuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04.txt) |  hadoop-tools_hadoop-azure-jdkUbuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04 with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04 generated 2 new + 13 unchanged - 2 fixed = 15 total (was 15)  |
   | +1 :green_heart: |  compile  |   0m 26s |  |  the patch passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01  |
   | -1 :x: |  javac  |   0m 26s | [/diff-compile-javac-hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/2/artifact/out/diff-compile-javac-hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01.txt) |  hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01 with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01 generated 2 new + 7 unchanged - 2 fixed = 9 total (was 9)  |
   | -0 :warning: |  checkstyle  |   0m 17s | [/diff-checkstyle-hadoop-tools_hadoop-azure.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/2/artifact/out/diff-checkstyle-hadoop-tools_hadoop-azure.txt) |  hadoop-tools/hadoop-azure: The patch generated 16 new + 2 unchanged - 0 fixed = 18 total (was 2)  |
   | +1 :green_heart: |  mvnsite  |   0m 29s |  |  the patch passed  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  shadedclient  |  14m 52s |  |  patch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 27s |  |  the patch passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04  |
   | +1 :green_heart: |  javadoc  |   0m 25s |  |  the patch passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01  |
   | +1 :green_heart: |  findbugs  |   1m  1s |  |  the patch passed  |
   |||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   1m 28s |  |  hadoop-azure in the patch passed.  |
   | -1 :x: |  asflicense  |   0m 30s | [/patch-asflicense-problems.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/2/artifact/out/patch-asflicense-problems.txt) |  The patch generated 1 ASF License warnings.  |
   |  |   |  78m 27s |  |  |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/2/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/2548 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient findbugs checkstyle |
   | uname | Linux db9c212bf37f 4.15.0-65-generic #74-Ubuntu SMP Tue Sep 17 17:06:04 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / 5bf977e6b16 |
   | Default Java | Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01 |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/2/testReport/ |
   | Max. process+thread count | 608 (vs. ulimit of 5500) |
   | modules | C: hadoop-tools/hadoop-azure U: hadoop-tools/hadoop-azure |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/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] steveloughran commented on pull request #2548: HADOOP-17475. ABFS: Implementing ListStatusRemoteIterator

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






----------------------------------------------------------------
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 #2548: DRAFT PR: Implementing ListStatusRemoteIterator

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


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |   2m 12s |  |  Docker mode activated.  |
   |||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  |  No case conflicting files found.  |
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |   |   0m  0s | [test4tests](test4tests) |  The patch appears to include 1 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |  36m 53s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |   0m 36s |  |  trunk passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04  |
   | +1 :green_heart: |  compile  |   0m 33s |  |  trunk passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01  |
   | +1 :green_heart: |  checkstyle  |   0m 29s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   0m 38s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  20m 33s |  |  branch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 34s |  |  trunk passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04  |
   | +1 :green_heart: |  javadoc  |   0m 29s |  |  trunk passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01  |
   | +0 :ok: |  spotbugs  |   1m 10s |  |  Used deprecated FindBugs config; considering switching to SpotBugs.  |
   | +1 :green_heart: |  findbugs  |   1m  7s |  |  trunk passed  |
   | -0 :warning: |  patch  |   1m 30s |  |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   |||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   0m 33s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 35s |  |  the patch passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04  |
   | +1 :green_heart: |  javac  |   0m 35s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 29s |  |  the patch passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01  |
   | +1 :green_heart: |  javac  |   0m 29s |  |  the patch passed  |
   | -0 :warning: |  checkstyle  |   0m 20s | [/diff-checkstyle-hadoop-tools_hadoop-azure.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/9/artifact/out/diff-checkstyle-hadoop-tools_hadoop-azure.txt) |  hadoop-tools/hadoop-azure: The patch generated 5 new + 4 unchanged - 0 fixed = 9 total (was 4)  |
   | +1 :green_heart: |  mvnsite  |   0m 33s |  |  the patch passed  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  shadedclient  |  19m  7s |  |  patch has no errors when building and testing our client artifacts.  |
   | -1 :x: |  javadoc  |   0m 27s | [/diff-javadoc-javadoc-hadoop-tools_hadoop-azure-jdkUbuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/9/artifact/out/diff-javadoc-javadoc-hadoop-tools_hadoop-azure-jdkUbuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04.txt) |  hadoop-tools_hadoop-azure-jdkUbuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04 with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04 generated 3 new + 15 unchanged - 2 fixed = 18 total (was 17)  |
   | -1 :x: |  javadoc  |   0m 25s | [/diff-javadoc-javadoc-hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/9/artifact/out/diff-javadoc-javadoc-hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01.txt) |  hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01 with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01 generated 3 new + 15 unchanged - 2 fixed = 18 total (was 17)  |
   | -1 :x: |  findbugs  |   1m 14s | [/new-findbugs-hadoop-tools_hadoop-azure.html](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/9/artifact/out/new-findbugs-hadoop-tools_hadoop-azure.html) |  hadoop-tools/hadoop-azure generated 1 new + 0 unchanged - 0 fixed = 1 total (was 0)  |
   |||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   1m 32s |  |  hadoop-azure in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 38s |  |  The patch does not generate ASF License warnings.  |
   |  |   |  92m 18s |  |  |
   
   
   | Reason | Tests |
   |-------:|:------|
   | FindBugs | module:hadoop-tools/hadoop-azure |
   |  |  Inconsistent synchronization of org.apache.hadoop.fs.azurebfs.services.AbfsListStatusRemoteIterator.ioException; locked 66% of time  Unsynchronized access at AbfsListStatusRemoteIterator.java:66% of time  Unsynchronized access at AbfsListStatusRemoteIterator.java:[line 128] |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/9/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/2548 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient findbugs checkstyle |
   | uname | Linux 41dd7a1a9ef1 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 / 630f8ddd2c7 |
   | Default Java | Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01 |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/9/testReport/ |
   | Max. process+thread count | 709 (vs. ulimit of 5500) |
   | modules | C: hadoop-tools/hadoop-azure U: hadoop-tools/hadoop-azure |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/9/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 #2548: HADOOP-17475. ABFS: Implementing ListStatusRemoteIterator

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


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |   1m 51s |  |  Docker mode activated.  |
   |||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  |  No case conflicting files found.  |
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |   |   0m  0s | [test4tests](test4tests) |  The patch appears to include 1 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |  38m  3s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |   0m 42s |  |  trunk passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04  |
   | +1 :green_heart: |  compile  |   0m 34s |  |  trunk passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01  |
   | +1 :green_heart: |  checkstyle  |   0m 26s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   0m 38s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  19m  7s |  |  branch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 32s |  |  trunk passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04  |
   | +1 :green_heart: |  javadoc  |   0m 26s |  |  trunk passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01  |
   | +0 :ok: |  spotbugs  |   1m  4s |  |  Used deprecated FindBugs config; considering switching to SpotBugs.  |
   | +1 :green_heart: |  findbugs  |   1m  0s |  |  trunk passed  |
   | -0 :warning: |  patch  |   1m 20s |  |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   |||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   0m 31s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 32s |  |  the patch passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04  |
   | +1 :green_heart: |  javac  |   0m 32s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 25s |  |  the patch passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01  |
   | +1 :green_heart: |  javac  |   0m 25s |  |  the patch passed  |
   | -0 :warning: |  checkstyle  |   0m 16s | [/diff-checkstyle-hadoop-tools_hadoop-azure.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/14/artifact/out/diff-checkstyle-hadoop-tools_hadoop-azure.txt) |  hadoop-tools/hadoop-azure: The patch generated 7 new + 4 unchanged - 0 fixed = 11 total (was 4)  |
   | +1 :green_heart: |  mvnsite  |   0m 28s |  |  the patch passed  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  shadedclient  |  16m 50s |  |  patch has no errors when building and testing our client artifacts.  |
   | -1 :x: |  javadoc  |   0m 24s | [/diff-javadoc-javadoc-hadoop-tools_hadoop-azure-jdkUbuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/14/artifact/out/diff-javadoc-javadoc-hadoop-tools_hadoop-azure-jdkUbuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04.txt) |  hadoop-tools_hadoop-azure-jdkUbuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04 with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04 generated 3 new + 15 unchanged - 2 fixed = 18 total (was 17)  |
   | -1 :x: |  javadoc  |   0m 21s | [/diff-javadoc-javadoc-hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/14/artifact/out/diff-javadoc-javadoc-hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01.txt) |  hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01 with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01 generated 3 new + 15 unchanged - 2 fixed = 18 total (was 17)  |
   | -1 :x: |  findbugs  |   1m  4s | [/new-findbugs-hadoop-tools_hadoop-azure.html](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/14/artifact/out/new-findbugs-hadoop-tools_hadoop-azure.html) |  hadoop-tools/hadoop-azure generated 2 new + 0 unchanged - 0 fixed = 2 total (was 0)  |
   |||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   2m  4s |  |  hadoop-azure in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 32s |  |  The patch does not generate ASF License warnings.  |
   |  |   |  89m  2s |  |  |
   
   
   | Reason | Tests |
   |-------:|:------|
   | FindBugs | module:hadoop-tools/hadoop-azure |
   |  |  Inconsistent synchronization of org.apache.hadoop.fs.azurebfs.services.AbfsListStatusRemoteIterator.continuation; locked 50% of time  Unsynchronized access at AbfsListStatusRemoteIterator.java:50% of time  Unsynchronized access at AbfsListStatusRemoteIterator.java:[line 145] |
   |  |  Inconsistent synchronization of org.apache.hadoop.fs.azurebfs.services.AbfsListStatusRemoteIterator.isIterationComplete; locked 50% of time  Unsynchronized access at AbfsListStatusRemoteIterator.java:50% of time  Unsynchronized access at AbfsListStatusRemoteIterator.java:[line 82] |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/14/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/2548 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient findbugs checkstyle |
   | uname | Linux 56449b6de7f3 4.15.0-101-generic #102-Ubuntu SMP Mon May 11 10:07:26 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / 06a5d3437f6 |
   | Default Java | Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01 |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/14/testReport/ |
   | Max. process+thread count | 509 (vs. ulimit of 5500) |
   | modules | C: hadoop-tools/hadoop-azure U: hadoop-tools/hadoop-azure |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/14/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] bilaharith commented on a change in pull request #2548: HADOOP-17475. ABFS: Implementing ListStatusRemoteIterator

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



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsListStatusRemoteIterator.java
##########
@@ -0,0 +1,156 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs.services;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.CompletableFuture;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.RemoteIterator;
+
+public class AbfsListStatusRemoteIterator implements RemoteIterator<FileStatus> {
+
+  private static final Logger LOG = LoggerFactory
+      .getLogger(AbfsListStatusRemoteIterator.class);
+
+  private static final boolean FETCH_ALL_FALSE = false;
+  private static final int MAX_QUEUE_SIZE = 10;
+
+  private final FileStatus fileStatus;
+  private final ListingSupport listingSupport;
+  private final ArrayBlockingQueue<Object> iteratorsQueue;
+
+  private volatile boolean isAsyncInProgress = false;
+  private boolean isIterationComplete = false;
+  private String continuation;
+  private Iterator<FileStatus> currIterator;
+
+  public AbfsListStatusRemoteIterator(final FileStatus fileStatus,
+      final ListingSupport listingSupport) {
+    this.fileStatus = fileStatus;
+    this.listingSupport = listingSupport;
+    iteratorsQueue = new ArrayBlockingQueue<>(MAX_QUEUE_SIZE);
+    currIterator = Collections.emptyIterator();
+    fetchBatchesAsync();
+  }
+
+  @Override
+  public boolean hasNext() throws IOException {
+    if (currIterator.hasNext()) {
+      return true;
+    }
+    updateCurrentIterator();
+    return currIterator.hasNext();
+  }
+
+  @Override
+  public FileStatus next() throws IOException {
+    if (!this.hasNext()) {
+      throw new NoSuchElementException();
+    }
+    return currIterator.next();
+  }
+
+  private void updateCurrentIterator() throws IOException {
+    do {
+      currIterator = getNextIterator();
+    } while (currIterator != null && !currIterator.hasNext()
+        && !isIterationComplete);
+  }
+
+  private Iterator<FileStatus> getNextIterator() throws IOException {
+    fetchBatchesAsync();
+    synchronized (this) {
+      if (iteratorsQueue.isEmpty() && isIterationComplete) {
+          return Collections.emptyIterator();
+      }
+    }
+    try {
+      Object obj = iteratorsQueue.take();
+      if(obj instanceof Iterator){
+        return (Iterator<FileStatus>) obj;
+      }
+      throw (IOException) obj;
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      LOG.error("Thread got interrupted: {}", e);
+      return Collections.emptyIterator();
+    }
+  }
+
+  private void fetchBatchesAsync() {
+    if (isAsyncInProgress) {
+      return;
+    }
+    synchronized (this) {
+      if (isAsyncInProgress) {
+        return;
+      }
+      isAsyncInProgress = true;
+    }
+    CompletableFuture.runAsync(() -> asyncOp());
+  }
+
+  private void asyncOp() {
+    try {
+      while (!isIterationComplete && iteratorsQueue.size() <= MAX_QUEUE_SIZE) {
+        addNextBatchIteratorToQueue();
+      }
+    } catch (IOException e) {
+      try {
+        iteratorsQueue.put(e);
+      } catch (InterruptedException interruptedException) {
+        Thread.currentThread().interrupt();
+        LOG.error("Thread got interrupted: {}", interruptedException);
+      }
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      LOG.error("Thread got interrupted: {}", e);
+    } finally {
+      synchronized (this  ) {
+        isAsyncInProgress = false;
+      }
+    }
+  }
+
+  private void addNextBatchIteratorToQueue()
+      throws IOException, InterruptedException {
+    List<FileStatus> fileStatuses = new ArrayList<>();
+    continuation = listingSupport
+        .listStatus(fileStatus.getPath(), null, fileStatuses, FETCH_ALL_FALSE,
+            continuation);
+    iteratorsQueue.put(fileStatuses.iterator());
+    synchronized (this) {
+      if (continuation == null || continuation.isEmpty()) {
+        isIterationComplete = true;
+        iteratorsQueue.put(Collections.emptyIterator());

Review comment:
       Done

##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsListStatusRemoteIterator.java
##########
@@ -0,0 +1,159 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs.services;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.CompletableFuture;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.RemoteIterator;
+
+public class AbfsListStatusRemoteIterator implements RemoteIterator<FileStatus> {
+
+  private static final Logger LOG = LoggerFactory
+      .getLogger(AbfsListStatusRemoteIterator.class);
+
+  private static final boolean FETCH_ALL_FALSE = false;
+  private static final int MAX_QUEUE_SIZE = 10;
+
+  private final FileStatus fileStatus;
+  private final ListingSupport listingSupport;
+  private final ArrayBlockingQueue<Object> iteratorsQueue;
+  private final Object asyncOpLock = new Object();
+
+  private volatile boolean isAsyncInProgress = false;
+  private boolean isIterationComplete = false;
+  private String continuation;
+  private Iterator<FileStatus> currIterator;
+
+  public AbfsListStatusRemoteIterator(final FileStatus fileStatus,
+      final ListingSupport listingSupport) {
+    this.fileStatus = fileStatus;
+    this.listingSupport = listingSupport;
+    iteratorsQueue = new ArrayBlockingQueue<>(MAX_QUEUE_SIZE);
+    currIterator = Collections.emptyIterator();
+    fetchBatchesAsync();
+  }
+
+  @Override
+  public boolean hasNext() throws IOException {
+    if (currIterator.hasNext()) {
+      return true;
+    }
+    updateCurrentIterator();
+    if (currIterator == null) {
+      return false;
+    }
+    return currIterator.hasNext();
+  }
+
+  @Override
+  public FileStatus next() throws IOException {
+    if (!this.hasNext()) {
+      throw new NoSuchElementException();
+    }
+    return currIterator.next();
+  }
+
+  private void updateCurrentIterator() throws IOException {
+    do {
+      currIterator = getNextIterator();
+    } while (currIterator != null && !currIterator.hasNext()
+        && !isIterationComplete);
+  }
+
+  private Iterator<FileStatus> getNextIterator() throws IOException {
+    fetchBatchesAsync();
+    synchronized (this) {
+      if (iteratorsQueue.isEmpty() && isIterationComplete) {
+          return null;

Review comment:
       Done

##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsListStatusRemoteIterator.java
##########
@@ -0,0 +1,159 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs.services;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.CompletableFuture;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.RemoteIterator;
+
+public class AbfsListStatusRemoteIterator implements RemoteIterator<FileStatus> {
+
+  private static final Logger LOG = LoggerFactory
+      .getLogger(AbfsListStatusRemoteIterator.class);
+
+  private static final boolean FETCH_ALL_FALSE = false;
+  private static final int MAX_QUEUE_SIZE = 10;
+
+  private final FileStatus fileStatus;
+  private final ListingSupport listingSupport;
+  private final ArrayBlockingQueue<Object> iteratorsQueue;
+  private final Object asyncOpLock = new Object();

Review comment:
       Done

##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsListStatusRemoteIterator.java
##########
@@ -0,0 +1,159 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs.services;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.CompletableFuture;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.RemoteIterator;
+
+public class AbfsListStatusRemoteIterator implements RemoteIterator<FileStatus> {
+
+  private static final Logger LOG = LoggerFactory
+      .getLogger(AbfsListStatusRemoteIterator.class);
+
+  private static final boolean FETCH_ALL_FALSE = false;
+  private static final int MAX_QUEUE_SIZE = 10;
+
+  private final FileStatus fileStatus;
+  private final ListingSupport listingSupport;
+  private final ArrayBlockingQueue<Object> iteratorsQueue;
+  private final Object asyncOpLock = new Object();
+
+  private volatile boolean isAsyncInProgress = false;
+  private boolean isIterationComplete = false;
+  private String continuation;
+  private Iterator<FileStatus> currIterator;
+
+  public AbfsListStatusRemoteIterator(final FileStatus fileStatus,
+      final ListingSupport listingSupport) {
+    this.fileStatus = fileStatus;
+    this.listingSupport = listingSupport;
+    iteratorsQueue = new ArrayBlockingQueue<>(MAX_QUEUE_SIZE);
+    currIterator = Collections.emptyIterator();
+    fetchBatchesAsync();
+  }
+
+  @Override
+  public boolean hasNext() throws IOException {
+    if (currIterator.hasNext()) {
+      return true;
+    }
+    updateCurrentIterator();
+    if (currIterator == null) {
+      return false;
+    }
+    return currIterator.hasNext();
+  }
+
+  @Override
+  public FileStatus next() throws IOException {
+    if (!this.hasNext()) {
+      throw new NoSuchElementException();
+    }
+    return currIterator.next();
+  }
+
+  private void updateCurrentIterator() throws IOException {
+    do {
+      currIterator = getNextIterator();
+    } while (currIterator != null && !currIterator.hasNext()
+        && !isIterationComplete);
+  }
+
+  private Iterator<FileStatus> getNextIterator() throws IOException {
+    fetchBatchesAsync();
+    synchronized (this) {
+      if (iteratorsQueue.isEmpty() && isIterationComplete) {
+          return null;
+      }
+    }
+    try {
+      Object obj = iteratorsQueue.take();
+      if(obj instanceof Iterator){
+        return (Iterator<FileStatus>) obj;
+      }
+      throw (IOException) obj;
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      LOG.error("Thread got interrupted: {}", e);
+      return null;
+    }
+  }
+
+  private void fetchBatchesAsync() {
+    if (isAsyncInProgress) {
+      return;
+    }
+    synchronized (asyncOpLock) {
+      if (isAsyncInProgress) {
+        return;
+      }
+      isAsyncInProgress = true;
+    }
+    CompletableFuture.runAsync(() -> asyncOp());
+  }
+
+  private void asyncOp() {
+    try {
+      while (!isIterationComplete && iteratorsQueue.size() <= MAX_QUEUE_SIZE) {
+        addNextBatchIteratorToQueue();
+      }
+    } catch (IOException e) {
+      try {
+        iteratorsQueue.put(e);
+      } catch (InterruptedException interruptedException) {
+        Thread.currentThread().interrupt();
+        LOG.error("Thread got interrupted: {}", interruptedException);
+      }
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      LOG.error("Thread got interrupted: {}", e);
+    } finally {
+      synchronized (asyncOpLock) {
+        isAsyncInProgress = false;
+      }
+    }
+  }
+
+  private void addNextBatchIteratorToQueue()
+      throws IOException, InterruptedException {
+    List<FileStatus> fileStatuses = new ArrayList<>();
+    continuation = listingSupport
+        .listStatus(fileStatus.getPath(), null, fileStatuses, FETCH_ALL_FALSE,
+            continuation);
+    synchronized (this) {
+      if (continuation == null || continuation.isEmpty()) {
+        isIterationComplete = true;
+      }
+      iteratorsQueue.put(fileStatuses.iterator());

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] bilaharith commented on a change in pull request #2548: HADOOP-17475. ABFS: Implementing ListStatusRemoteIterator

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



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsListStatusRemoteIterator.java
##########
@@ -0,0 +1,159 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs.services;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeUnit;
+import javax.activation.UnsupportedDataTypeException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.RemoteIterator;
+
+public class AbfsListStatusRemoteIterator
+    implements RemoteIterator<FileStatus> {
+
+  private static final Logger LOG = LoggerFactory
+      .getLogger(AbfsListStatusRemoteIterator.class);
+
+  private static final boolean FETCH_ALL_FALSE = false;
+  private static final int MAX_QUEUE_SIZE = 10;
+  private static final long POLL_WAIT_TIME_IN_MS = 250;
+
+  private final FileStatus fileStatus;
+  private final ListingSupport listingSupport;
+  private final ArrayBlockingQueue<Object> iteratorsQueue;
+
+  private volatile boolean isAsyncInProgress = false;
+  private boolean isIterationComplete = false;
+  private String continuation;
+  private Iterator<FileStatus> currIterator;
+
+  public AbfsListStatusRemoteIterator(final FileStatus fileStatus,
+      final ListingSupport listingSupport) {
+    this.fileStatus = fileStatus;
+    this.listingSupport = listingSupport;
+    iteratorsQueue = new ArrayBlockingQueue<>(MAX_QUEUE_SIZE);
+    currIterator = Collections.emptyIterator();
+    fetchBatchesAsync();
+  }
+
+  @Override
+  public boolean hasNext() throws IOException {
+    if (currIterator.hasNext()) {
+      return true;
+    }
+    currIterator = getNextIterator();
+    return currIterator.hasNext();
+  }
+
+  @Override
+  public FileStatus next() throws IOException {
+    if (!this.hasNext()) {
+      throw new NoSuchElementException();
+    }
+    return currIterator.next();
+  }
+
+  private Iterator<FileStatus> getNextIterator() throws IOException {
+    fetchBatchesAsync();
+    try {
+      Object obj = null;
+      while (obj == null
+          && (!isIterationComplete || !iteratorsQueue.isEmpty())) {
+        obj = iteratorsQueue.poll(POLL_WAIT_TIME_IN_MS, TimeUnit.MILLISECONDS);
+      }
+      if (obj == null) {
+        return Collections.emptyIterator();
+      } else if (obj instanceof Iterator) {

Review comment:
       It is added

##########
File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsListStatusRemoteIterator.java
##########
@@ -0,0 +1,338 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+
+import org.assertj.core.api.Assertions;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.fs.azurebfs.services.AbfsListStatusRemoteIterator;
+import org.apache.hadoop.fs.azurebfs.services.ListingSupport;
+
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.anyBoolean;
+import static org.mockito.ArgumentMatchers.anyList;
+import static org.mockito.ArgumentMatchers.nullable;
+import static org.mockito.Mockito.verify;
+
+/**
+ * Test ListStatusRemoteIterator operation.
+ */
+public class ITestAbfsListStatusRemoteIterator extends AbstractAbfsIntegrationTest {
+
+  private static final int TEST_FILES_NUMBER = 1000;
+
+  public ITestAbfsListStatusRemoteIterator() throws Exception {
+    super();
+  }
+
+  @Test
+  public void testAbfsIteratorWithHasNext() throws Exception {
+    Path testDir = createTestDirectory();
+    setPageSize(10);
+    final List<String> fileNames = createFilesUnderDirectory(TEST_FILES_NUMBER,
+        testDir, "testListPath");
+
+    ListingSupport listngSupport = Mockito.spy(getFileSystem().getAbfsStore());
+    RemoteIterator<FileStatus> fsItr = new AbfsListStatusRemoteIterator(
+        getFileSystem().getFileStatus(testDir), listngSupport);
+    Assertions.assertThat(fsItr)
+        .describedAs("RemoteIterator should be instance of "
+            + "AbfsListStatusRemoteIterator by default")
+        .isInstanceOf(AbfsListStatusRemoteIterator.class);
+    int itrCount = 0;
+    while (fsItr.hasNext()) {
+      FileStatus fileStatus = fsItr.next();
+      String pathStr = fileStatus.getPath().toString();
+      fileNames.remove(pathStr);
+      itrCount++;
+    }
+    Assertions.assertThat(itrCount)
+        .describedAs("Number of iterations should be equal to the files "
+            + "created")
+        .isEqualTo(TEST_FILES_NUMBER);
+    Assertions.assertThat(fileNames.size())
+        .describedAs("After removing every iterm found from the iterator, "
+            + "there should be no more elements in the fileNames")
+        .isEqualTo(0);
+    int minNumberOfInvokations = TEST_FILES_NUMBER / 10;
+    verify(listngSupport, Mockito.atLeast(minNumberOfInvokations))
+        .listStatus(any(Path.class), nullable(String.class),
+            anyList(), anyBoolean(),
+            nullable(String.class));
+  }
+
+  @Test
+  public void testAbfsIteratorWithoutHasNext() throws Exception {
+    Path testDir = createTestDirectory();
+    setPageSize(10);
+    final List<String> fileNames = createFilesUnderDirectory(TEST_FILES_NUMBER,
+        testDir, "testListPath");
+
+    ListingSupport listngSupport = Mockito.spy(getFileSystem().getAbfsStore());
+    RemoteIterator<FileStatus> fsItr = new AbfsListStatusRemoteIterator(
+        getFileSystem().getFileStatus(testDir), listngSupport);
+    Assertions.assertThat(fsItr)
+        .describedAs("RemoteIterator should be instance of "
+            + "AbfsListStatusRemoteIterator by default")
+        .isInstanceOf(AbfsListStatusRemoteIterator.class);
+    int itrCount = 0;
+    for (int i = 0; i < TEST_FILES_NUMBER; i++) {
+      FileStatus fileStatus = fsItr.next();
+      String pathStr = fileStatus.getPath().toString();
+      fileNames.remove(pathStr);
+      itrCount++;
+    }
+    Assertions.assertThatThrownBy(() -> fsItr.next())
+        .describedAs(
+            "next() should throw NoSuchElementException since next has been "
+                + "called " + TEST_FILES_NUMBER + " times")
+        .isInstanceOf(NoSuchElementException.class);
+    Assertions.assertThat(itrCount)
+        .describedAs("Number of iterations should be equal to the files "
+            + "created")
+        .isEqualTo(TEST_FILES_NUMBER);
+    Assertions.assertThat(fileNames.size())
+        .describedAs("After removing every iterm found from the iterator, "
+            + "there should be no more elements in the fileNames")
+        .isEqualTo(0);
+    int minNumberOfInvokations = TEST_FILES_NUMBER / 10;
+    verify(listngSupport, Mockito.atLeast(minNumberOfInvokations))
+        .listStatus(any(Path.class), nullable(String.class),
+            anyList(), anyBoolean(),
+            nullable(String.class));
+  }
+
+  @Test
+  public void testWithAbfsIteratorDisabled() throws Exception {
+    Path testDir = createTestDirectory();
+    setPageSize(10);
+    setEnableAbfsIterator(false);
+    final List<String> fileNames = createFilesUnderDirectory(TEST_FILES_NUMBER,
+        testDir, "testListPath");
+
+    RemoteIterator<FileStatus> fsItr =
+        getFileSystem().listStatusIterator(testDir);
+    Assertions.assertThat(fsItr)
+        .describedAs("RemoteIterator should not be instance of "
+            + "AbfsListStatusRemoteIterator when it is disabled")
+        .isNotInstanceOf(AbfsListStatusRemoteIterator.class);
+    int itrCount = 0;
+    while (fsItr.hasNext()) {
+      FileStatus fileStatus = fsItr.next();
+      String pathStr = fileStatus.getPath().toString();
+      fileNames.remove(pathStr);
+      itrCount++;
+    }
+    Assertions.assertThat(itrCount)
+        .describedAs("Number of iterations should be equal to the files "
+            + "created")
+        .isEqualTo(TEST_FILES_NUMBER);
+    Assertions.assertThat(fileNames.size())
+        .describedAs("After removing every iterm found from the iterator, "
+            + "there should be no more elements in the fileNames")
+        .isEqualTo(0);
+  }
+
+  @Test
+  public void testWithAbfsIteratorDisabledWithoutHasNext() throws Exception {
+    Path testDir = createTestDirectory();
+    setPageSize(10);
+    setEnableAbfsIterator(false);
+    final List<String> fileNames = createFilesUnderDirectory(TEST_FILES_NUMBER,
+        testDir, "testListPath");
+
+    RemoteIterator<FileStatus> fsItr =
+        getFileSystem().listStatusIterator(testDir);
+    Assertions.assertThat(fsItr)
+        .describedAs("RemoteIterator should not be instance of "
+            + "AbfsListStatusRemoteIterator when it is disabled")
+        .isNotInstanceOf(AbfsListStatusRemoteIterator.class);
+    int itrCount = 0;
+    for (int i = 0; i < TEST_FILES_NUMBER; i++) {
+      FileStatus fileStatus = fsItr.next();
+      String pathStr = fileStatus.getPath().toString();
+      fileNames.remove(pathStr);
+      itrCount++;
+    }
+    Assertions.assertThatThrownBy(() -> fsItr.next())
+        .describedAs(
+            "next() should throw NoSuchElementException since next has been "
+                + "called " + TEST_FILES_NUMBER + " times")
+        .isInstanceOf(NoSuchElementException.class);
+    Assertions.assertThat(itrCount)
+        .describedAs("Number of iterations should be equal to the files "
+            + "created")
+        .isEqualTo(TEST_FILES_NUMBER);
+    Assertions.assertThat(fileNames.size())
+        .describedAs("After removing every iterm found from the iterator, "
+            + "there should be no more elements in the fileNames")
+        .isEqualTo(0);
+  }
+
+  @Test
+  public void testNextWhenNoMoreElementsPresent() throws Exception {
+    Path testDir = createTestDirectory();
+    setPageSize(10);
+    RemoteIterator fsItr =
+        new AbfsListStatusRemoteIterator(getFileSystem().getFileStatus(testDir),
+            getFileSystem().getAbfsStore());
+    fsItr = Mockito.spy(fsItr);
+    Mockito.doReturn(false).when(fsItr).hasNext();
+
+    RemoteIterator<FileStatus> finalFsItr = fsItr;
+    Assertions.assertThatThrownBy(() -> finalFsItr.next())
+        .describedAs(
+        "next() should throw NoSuchElementException if hasNext() return "
+            + "false")
+        .isInstanceOf(NoSuchElementException.class);
+  }
+
+  @Test
+  public void testHasNextForEmptyDir() throws Exception {
+    Path testDir = createTestDirectory();
+    setPageSize(10);
+    RemoteIterator<FileStatus> fsItr = getFileSystem()
+        .listStatusIterator(testDir);
+    Assertions.assertThat(fsItr.hasNext())
+        .describedAs("hasNext returns false for empty directory")
+        .isFalse();
+  }
+
+  @Test
+  public void testHasNextForFile() throws Exception {
+    final AzureBlobFileSystem fs = getFileSystem();
+    String testFileName = "testFile";
+    Path testFile = new Path(testFileName);
+    getFileSystem().create(testFile);
+    setPageSize(10);
+    RemoteIterator<FileStatus> fsItr = fs.listStatusIterator(testFile);
+    Assertions.assertThat(fsItr.hasNext())
+        .describedAs("hasNext returns true for file").isTrue();
+    Assertions.assertThat(fsItr.next().getPath().toString())
+        .describedAs("next returns the file itself")
+        .endsWith(testFileName);
+  }
+
+  @Test
+  public void testIOException() throws Exception {
+    Path testDir = createTestDirectory();
+    setPageSize(10);
+    getFileSystem().mkdirs(testDir);
+
+    String exceptionMessage = "test exception";
+    ListingSupport lsSupport =getMockListingSupport(exceptionMessage);
+    RemoteIterator fsItr =
+        new AbfsListStatusRemoteIterator(getFileSystem().getFileStatus(testDir),
+        lsSupport);
+
+    Assertions.assertThatThrownBy(() -> fsItr.next())
+        .describedAs(
+        "When ioException is not null and queue is empty exception should be "
+            + "thrown")
+        .isInstanceOf(IOException.class)
+        .hasMessage(exceptionMessage);
+  }
+
+  @Test
+  public void testNonExistingPath() throws Throwable {
+    Path nonExistingDir = new Path("nonExistingPath");
+    Assertions.assertThatThrownBy(
+        () -> getFileSystem().listStatusIterator(nonExistingDir)).describedAs(
+        "test the listStatusIterator call on a path which is not "
+            + "present should result in FileNotFoundException")
+        .isInstanceOf(FileNotFoundException.class);
+  }
+
+  private ListingSupport getMockListingSupport(String exceptionMessage) {
+    return new ListingSupport() {
+      @Override
+      public FileStatus[] listStatus(Path path) throws IOException {
+        return null;
+      }
+
+      @Override
+      public FileStatus[] listStatus(Path path, String startFrom)
+          throws IOException {
+        return null;
+      }
+
+      @Override
+      public String listStatus(Path path, String startFrom,
+          List<FileStatus> fileStatuses, boolean fetchAll, String continuation)
+          throws IOException {
+        throw new IOException(exceptionMessage);
+      }
+    };
+  }
+
+  private Path createTestDirectory() throws IOException {
+    String testDirectoryName = "testDirectory" + System.currentTimeMillis();
+    Path testDirectory = new Path(testDirectoryName);
+    getFileSystem().mkdirs(testDirectory);
+    return testDirectory;
+  }
+
+  private void setEnableAbfsIterator(boolean shouldEnable) throws IOException {
+    AzureBlobFileSystemStore abfsStore = getAbfsStore(getFileSystem());
+    abfsStore.getAbfsConfiguration().setEnableAbfsListIterator(shouldEnable);
+  }
+
+  private void setPageSize(int pageSize) throws IOException {
+    AzureBlobFileSystemStore abfsStore = getAbfsStore(getFileSystem());
+    abfsStore.getAbfsConfiguration().setListMaxResults(pageSize);
+  }
+
+  private List<String> createFilesUnderDirectory(int numFiles, Path rootPath,
+      String filenamePrefix)
+      throws ExecutionException, InterruptedException, IOException {
+    final List<Future<Void>> tasks = new ArrayList<>();
+    final List<String> fileNames = new ArrayList<>();
+    ExecutorService es = Executors.newFixedThreadPool(10);
+    for (int i = 0; i < numFiles; i++) {
+      final Path filePath = new Path(rootPath, filenamePrefix + i);
+      Callable<Void> callable = () -> {
+        getFileSystem().create(filePath);
+        fileNames.add(makeQualified(filePath).toString());
+        return null;
+      };
+      tasks.add(es.submit(callable));
+    }
+    for (Future<Void> task : tasks) {
+      task.get();
+    }
+    es.shutdownNow();

Review comment:
       Done

##########
File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsListStatusRemoteIterator.java
##########
@@ -0,0 +1,338 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+
+import org.assertj.core.api.Assertions;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.fs.azurebfs.services.AbfsListStatusRemoteIterator;
+import org.apache.hadoop.fs.azurebfs.services.ListingSupport;
+
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.anyBoolean;
+import static org.mockito.ArgumentMatchers.anyList;
+import static org.mockito.ArgumentMatchers.nullable;
+import static org.mockito.Mockito.verify;
+
+/**
+ * Test ListStatusRemoteIterator operation.
+ */
+public class ITestAbfsListStatusRemoteIterator extends AbstractAbfsIntegrationTest {
+
+  private static final int TEST_FILES_NUMBER = 1000;
+
+  public ITestAbfsListStatusRemoteIterator() throws Exception {
+    super();

Review comment:
       Done

##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ListingSupport.java
##########
@@ -0,0 +1,75 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs.services;
+
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+
+public interface ListingSupport {
+
+  /**
+   * @param path The list path.
+   * @return the entries in the path.
+   */
+  FileStatus[] listStatus(Path path) throws IOException;
+
+  /**
+   * @param path      Path the list path.
+   * @param startFrom The entry name that list results should start with.
+   *                  For example, if folder "/folder" contains four
+   *                  files: "afile", "bfile", "hfile", "ifile". Then
+   *                  listStatus(Path("/folder"), "hfile") will return
+   *                  "/folder/hfile" and "folder/ifile" Notice that if
+   *                  startFrom is a non-existent entry name, then the
+   *                  list response contains all entries after this
+   *                  non-existent entry in lexical order: listStatus
+   *                  (Path("/folder"), "cfile") will return
+   *                  "/folder/hfile" and "/folder/ifile".
+   * @return the entries in the path start from  "startFrom" in lexical order.
+   */
+  @InterfaceStability.Unstable

Review comment:
       Done

##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsListStatusRemoteIterator.java
##########
@@ -0,0 +1,159 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs.services;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeUnit;
+import javax.activation.UnsupportedDataTypeException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.RemoteIterator;
+
+public class AbfsListStatusRemoteIterator
+    implements RemoteIterator<FileStatus> {
+
+  private static final Logger LOG = LoggerFactory
+      .getLogger(AbfsListStatusRemoteIterator.class);
+
+  private static final boolean FETCH_ALL_FALSE = false;
+  private static final int MAX_QUEUE_SIZE = 10;
+  private static final long POLL_WAIT_TIME_IN_MS = 250;
+
+  private final FileStatus fileStatus;
+  private final ListingSupport listingSupport;
+  private final ArrayBlockingQueue<Object> iteratorsQueue;
+
+  private volatile boolean isAsyncInProgress = false;
+  private boolean isIterationComplete = false;
+  private String continuation;
+  private Iterator<FileStatus> currIterator;
+
+  public AbfsListStatusRemoteIterator(final FileStatus fileStatus,
+      final ListingSupport listingSupport) {
+    this.fileStatus = fileStatus;
+    this.listingSupport = listingSupport;
+    iteratorsQueue = new ArrayBlockingQueue<>(MAX_QUEUE_SIZE);
+    currIterator = Collections.emptyIterator();
+    fetchBatchesAsync();
+  }
+
+  @Override
+  public boolean hasNext() throws IOException {
+    if (currIterator.hasNext()) {
+      return true;
+    }
+    currIterator = getNextIterator();
+    return currIterator.hasNext();
+  }
+
+  @Override
+  public FileStatus next() throws IOException {
+    if (!this.hasNext()) {
+      throw new NoSuchElementException();
+    }
+    return currIterator.next();
+  }
+
+  private Iterator<FileStatus> getNextIterator() throws IOException {
+    fetchBatchesAsync();
+    try {
+      Object obj = null;
+      while (obj == null
+          && (!isIterationComplete || !iteratorsQueue.isEmpty())) {
+        obj = iteratorsQueue.poll(POLL_WAIT_TIME_IN_MS, TimeUnit.MILLISECONDS);
+      }
+      if (obj == null) {
+        return Collections.emptyIterator();
+      } else if (obj instanceof Iterator) {
+        return (Iterator<FileStatus>) obj;
+      } else if (obj instanceof IOException) {
+        throw (IOException) obj;
+      } else {
+        throw new UnsupportedDataTypeException();
+      }
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      LOG.error("Thread got interrupted: {}", e);
+      throw new IOException(e);
+    }
+  }
+
+  private void fetchBatchesAsync() {
+    if (isAsyncInProgress || isIterationComplete) {
+      return;
+    }
+    synchronized (this) {
+      if (isAsyncInProgress || isIterationComplete) {
+        return;
+      }
+      isAsyncInProgress = true;
+    }
+    CompletableFuture.runAsync(() -> asyncOp());
+  }
+
+  private void asyncOp() {
+    try {
+      while (!isIterationComplete && iteratorsQueue.size() <= MAX_QUEUE_SIZE) {
+        addNextBatchIteratorToQueue();
+      }
+    } catch (IOException ioe) {
+      LOG.error("Fetching filestatuses failed", ioe);
+      try {
+        iteratorsQueue.put(ioe);
+      } catch (InterruptedException interruptedException) {
+        Thread.currentThread().interrupt();
+        LOG.error("Thread got interrupted: {}", interruptedException);
+      }
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      LOG.error("Thread got interrupted: {}", e);
+    } finally {
+      synchronized (this) {
+        isAsyncInProgress = false;
+      }
+    }
+  }
+
+  private void addNextBatchIteratorToQueue()
+      throws IOException, InterruptedException {
+    List<FileStatus> fileStatuses = new ArrayList<>();
+    continuation = listingSupport
+        .listStatus(fileStatus.getPath(), null, fileStatuses, FETCH_ALL_FALSE,
+            continuation);
+    if(!fileStatuses.isEmpty()) {

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] bilaharith commented on a change in pull request #2548: DRAFT PR: Implementing ListStatusRemoteIterator

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



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ListStatusRemoteIterator.java
##########
@@ -0,0 +1,129 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs.services;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.ListIterator;
+import java.util.NoSuchElementException;
+import java.util.Queue;
+import java.util.concurrent.CompletableFuture;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore;
+
+public class ListStatusRemoteIterator implements RemoteIterator<FileStatus> {
+
+  private static final Logger LOG = LoggerFactory
+      .getLogger(ListStatusRemoteIterator.class);
+
+  private static final boolean FETCH_ALL_FALSE = false;
+
+  private final Path path;
+  private final AzureBlobFileSystemStore abfsStore;
+  private final Queue<ListIterator<FileStatus>> iteratorsQueue =
+      new LinkedList<>();
+
+  private boolean firstRead = true;
+  private String continuation;
+  private ListIterator<FileStatus> currIterator;
+  private IOException ioException;
+
+  public ListStatusRemoteIterator(final Path path,
+      final AzureBlobFileSystemStore abfsStore) throws IOException {
+    this.path = path;
+    this.abfsStore = abfsStore;
+    fetchAllAsync();
+    updateCurrentIterator();

Review comment:
       Done

##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java
##########
@@ -862,16 +871,16 @@ public FileStatus getFileStatus(final Path path) throws IOException {
             startFrom);
 
     final String relativePath = getRelativePath(path);
-    String continuation = null;
 
-    // generate continuation token if a valid startFrom is provided.
-    if (startFrom != null && !startFrom.isEmpty()) {
-      continuation = getIsNamespaceEnabled()
-              ? generateContinuationTokenForXns(startFrom)
-              : generateContinuationTokenForNonXns(relativePath, startFrom);
+    if(continuation==null ||continuation.length()<1) {

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] bilaharith commented on pull request #2548: DRAFT PR: Implementing ListStatusRemoteIterator

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


   > rs propagate the IOStatisticsSource interface, so when the innermost iterator collects cost/count of list calls, the stats will be visible to and collectable
   
   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] steveloughran commented on pull request #2548: HADOOP-17475. ABFS: Implementing ListStatusRemoteIterator

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


   merged to trunk; cp'd to branch-3.3 and will push up if a recompile works. I am not testing that ---can you do that and if there are problems we can do a followup?


----------------------------------------------------------------
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 merged pull request #2548: HADOOP-17475. ABFS: Implementing ListStatusRemoteIterator

Posted by GitBox <gi...@apache.org>.
steveloughran merged pull request #2548:
URL: https://github.com/apache/hadoop/pull/2548


   


----------------------------------------------------------------
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] bilaharith commented on a change in pull request #2548: HADOOP-17475. ABFS: Implementing ListStatusRemoteIterator

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



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsListStatusRemoteIterator.java
##########
@@ -0,0 +1,151 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs.services;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.CompletableFuture;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.RemoteIterator;
+
+public class AbfsListStatusRemoteIterator implements RemoteIterator<FileStatus> {
+
+  private static final Logger LOG = LoggerFactory
+      .getLogger(AbfsListStatusRemoteIterator.class);
+
+  private static final boolean FETCH_ALL_FALSE = false;
+  private static final int MAX_QUEUE_SIZE = 10;
+
+  private final FileStatus fileStatus;
+  private final ListingSupport listingSupport;
+  private final ArrayBlockingQueue<Iterator<FileStatus>> iteratorsQueue;
+  private final Object asyncOpLock = new Object();
+
+  private volatile boolean isAsyncInProgress = false;
+  private boolean firstBatch = true;
+  private String continuation;
+  private Iterator<FileStatus> currIterator;
+  private IOException ioException;
+
+  public AbfsListStatusRemoteIterator(final FileStatus fileStatus,
+      final ListingSupport listingSupport) {
+    this.fileStatus = fileStatus;
+    this.listingSupport = listingSupport;
+    iteratorsQueue = new ArrayBlockingQueue<>(MAX_QUEUE_SIZE);
+    currIterator = Collections.emptyIterator();
+    fetchBatchesAsync();
+  }
+
+  @Override
+  public boolean hasNext() throws IOException {
+    if (currIterator.hasNext()) {
+      return true;
+    }
+    updateCurrentIterator();
+    return currIterator.hasNext();
+  }
+
+  @Override
+  public FileStatus next() throws IOException {
+    if (!this.hasNext()) {
+      throw new NoSuchElementException();
+    }
+    return currIterator.next();
+  }
+
+  private void updateCurrentIterator() throws IOException {
+    fetchBatchesAsync();
+    synchronized (this) {
+      if (iteratorsQueue.isEmpty()) {
+        if (ioException != null) {
+          throw ioException;
+        }
+        if (isListingComplete()) {
+          return;
+        }
+      }
+    }
+    try {
+      currIterator = iteratorsQueue.take();
+      if (!currIterator.hasNext() && !isListingComplete()) {
+        updateCurrentIterator();

Review comment:
       Moved away fro recursion




----------------------------------------------------------------
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] bilaharith commented on a change in pull request #2548: HADOOP-17475. ABFS: Implementing ListStatusRemoteIterator

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



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsListStatusRemoteIterator.java
##########
@@ -0,0 +1,159 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs.services;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeUnit;
+import javax.activation.UnsupportedDataTypeException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.RemoteIterator;
+
+public class AbfsListStatusRemoteIterator
+    implements RemoteIterator<FileStatus> {
+
+  private static final Logger LOG = LoggerFactory
+      .getLogger(AbfsListStatusRemoteIterator.class);
+
+  private static final boolean FETCH_ALL_FALSE = false;
+  private static final int MAX_QUEUE_SIZE = 10;
+  private static final long POLL_WAIT_TIME_IN_MS = 250;
+
+  private final FileStatus fileStatus;
+  private final ListingSupport listingSupport;
+  private final ArrayBlockingQueue<Object> iteratorsQueue;
+
+  private volatile boolean isAsyncInProgress = false;
+  private boolean isIterationComplete = false;
+  private String continuation;
+  private Iterator<FileStatus> currIterator;
+
+  public AbfsListStatusRemoteIterator(final FileStatus fileStatus,
+      final ListingSupport listingSupport) {
+    this.fileStatus = fileStatus;
+    this.listingSupport = listingSupport;
+    iteratorsQueue = new ArrayBlockingQueue<>(MAX_QUEUE_SIZE);
+    currIterator = Collections.emptyIterator();
+    fetchBatchesAsync();
+  }
+
+  @Override
+  public boolean hasNext() throws IOException {
+    if (currIterator.hasNext()) {
+      return true;
+    }
+    currIterator = getNextIterator();
+    return currIterator.hasNext();
+  }
+
+  @Override
+  public FileStatus next() throws IOException {
+    if (!this.hasNext()) {
+      throw new NoSuchElementException();
+    }
+    return currIterator.next();
+  }
+
+  private Iterator<FileStatus> getNextIterator() throws IOException {
+    fetchBatchesAsync();
+    try {
+      Object obj = null;
+      while (obj == null
+          && (!isIterationComplete || !iteratorsQueue.isEmpty())) {
+        obj = iteratorsQueue.poll(POLL_WAIT_TIME_IN_MS, TimeUnit.MILLISECONDS);
+      }
+      if (obj == null) {
+        return Collections.emptyIterator();
+      } else if (obj instanceof Iterator) {

Review comment:
       Done

##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsListStatusRemoteIterator.java
##########
@@ -0,0 +1,159 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs.services;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeUnit;
+import javax.activation.UnsupportedDataTypeException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.RemoteIterator;
+
+public class AbfsListStatusRemoteIterator
+    implements RemoteIterator<FileStatus> {
+
+  private static final Logger LOG = LoggerFactory
+      .getLogger(AbfsListStatusRemoteIterator.class);
+
+  private static final boolean FETCH_ALL_FALSE = false;
+  private static final int MAX_QUEUE_SIZE = 10;
+  private static final long POLL_WAIT_TIME_IN_MS = 250;
+
+  private final FileStatus fileStatus;
+  private final ListingSupport listingSupport;
+  private final ArrayBlockingQueue<Object> iteratorsQueue;
+
+  private volatile boolean isAsyncInProgress = false;
+  private boolean isIterationComplete = false;
+  private String continuation;
+  private Iterator<FileStatus> currIterator;
+
+  public AbfsListStatusRemoteIterator(final FileStatus fileStatus,
+      final ListingSupport listingSupport) {
+    this.fileStatus = fileStatus;
+    this.listingSupport = listingSupport;
+    iteratorsQueue = new ArrayBlockingQueue<>(MAX_QUEUE_SIZE);
+    currIterator = Collections.emptyIterator();
+    fetchBatchesAsync();
+  }
+
+  @Override
+  public boolean hasNext() throws IOException {
+    if (currIterator.hasNext()) {
+      return true;
+    }
+    currIterator = getNextIterator();
+    return currIterator.hasNext();
+  }
+
+  @Override
+  public FileStatus next() throws IOException {
+    if (!this.hasNext()) {
+      throw new NoSuchElementException();
+    }
+    return currIterator.next();
+  }
+
+  private Iterator<FileStatus> getNextIterator() throws IOException {
+    fetchBatchesAsync();
+    try {
+      Object obj = null;
+      while (obj == null
+          && (!isIterationComplete || !iteratorsQueue.isEmpty())) {
+        obj = iteratorsQueue.poll(POLL_WAIT_TIME_IN_MS, TimeUnit.MILLISECONDS);
+      }
+      if (obj == null) {
+        return Collections.emptyIterator();
+      } else if (obj instanceof Iterator) {

Review comment:
       It is added

##########
File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsListStatusRemoteIterator.java
##########
@@ -0,0 +1,338 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+
+import org.assertj.core.api.Assertions;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.fs.azurebfs.services.AbfsListStatusRemoteIterator;
+import org.apache.hadoop.fs.azurebfs.services.ListingSupport;
+
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.anyBoolean;
+import static org.mockito.ArgumentMatchers.anyList;
+import static org.mockito.ArgumentMatchers.nullable;
+import static org.mockito.Mockito.verify;
+
+/**
+ * Test ListStatusRemoteIterator operation.
+ */
+public class ITestAbfsListStatusRemoteIterator extends AbstractAbfsIntegrationTest {
+
+  private static final int TEST_FILES_NUMBER = 1000;
+
+  public ITestAbfsListStatusRemoteIterator() throws Exception {
+    super();
+  }
+
+  @Test
+  public void testAbfsIteratorWithHasNext() throws Exception {
+    Path testDir = createTestDirectory();
+    setPageSize(10);
+    final List<String> fileNames = createFilesUnderDirectory(TEST_FILES_NUMBER,
+        testDir, "testListPath");
+
+    ListingSupport listngSupport = Mockito.spy(getFileSystem().getAbfsStore());
+    RemoteIterator<FileStatus> fsItr = new AbfsListStatusRemoteIterator(
+        getFileSystem().getFileStatus(testDir), listngSupport);
+    Assertions.assertThat(fsItr)
+        .describedAs("RemoteIterator should be instance of "
+            + "AbfsListStatusRemoteIterator by default")
+        .isInstanceOf(AbfsListStatusRemoteIterator.class);
+    int itrCount = 0;
+    while (fsItr.hasNext()) {
+      FileStatus fileStatus = fsItr.next();
+      String pathStr = fileStatus.getPath().toString();
+      fileNames.remove(pathStr);
+      itrCount++;
+    }
+    Assertions.assertThat(itrCount)
+        .describedAs("Number of iterations should be equal to the files "
+            + "created")
+        .isEqualTo(TEST_FILES_NUMBER);
+    Assertions.assertThat(fileNames.size())
+        .describedAs("After removing every iterm found from the iterator, "
+            + "there should be no more elements in the fileNames")
+        .isEqualTo(0);
+    int minNumberOfInvokations = TEST_FILES_NUMBER / 10;
+    verify(listngSupport, Mockito.atLeast(minNumberOfInvokations))
+        .listStatus(any(Path.class), nullable(String.class),
+            anyList(), anyBoolean(),
+            nullable(String.class));
+  }
+
+  @Test
+  public void testAbfsIteratorWithoutHasNext() throws Exception {
+    Path testDir = createTestDirectory();
+    setPageSize(10);
+    final List<String> fileNames = createFilesUnderDirectory(TEST_FILES_NUMBER,
+        testDir, "testListPath");
+
+    ListingSupport listngSupport = Mockito.spy(getFileSystem().getAbfsStore());
+    RemoteIterator<FileStatus> fsItr = new AbfsListStatusRemoteIterator(
+        getFileSystem().getFileStatus(testDir), listngSupport);
+    Assertions.assertThat(fsItr)
+        .describedAs("RemoteIterator should be instance of "
+            + "AbfsListStatusRemoteIterator by default")
+        .isInstanceOf(AbfsListStatusRemoteIterator.class);
+    int itrCount = 0;
+    for (int i = 0; i < TEST_FILES_NUMBER; i++) {
+      FileStatus fileStatus = fsItr.next();
+      String pathStr = fileStatus.getPath().toString();
+      fileNames.remove(pathStr);
+      itrCount++;
+    }
+    Assertions.assertThatThrownBy(() -> fsItr.next())
+        .describedAs(
+            "next() should throw NoSuchElementException since next has been "
+                + "called " + TEST_FILES_NUMBER + " times")
+        .isInstanceOf(NoSuchElementException.class);
+    Assertions.assertThat(itrCount)
+        .describedAs("Number of iterations should be equal to the files "
+            + "created")
+        .isEqualTo(TEST_FILES_NUMBER);
+    Assertions.assertThat(fileNames.size())
+        .describedAs("After removing every iterm found from the iterator, "
+            + "there should be no more elements in the fileNames")
+        .isEqualTo(0);
+    int minNumberOfInvokations = TEST_FILES_NUMBER / 10;
+    verify(listngSupport, Mockito.atLeast(minNumberOfInvokations))
+        .listStatus(any(Path.class), nullable(String.class),
+            anyList(), anyBoolean(),
+            nullable(String.class));
+  }
+
+  @Test
+  public void testWithAbfsIteratorDisabled() throws Exception {
+    Path testDir = createTestDirectory();
+    setPageSize(10);
+    setEnableAbfsIterator(false);
+    final List<String> fileNames = createFilesUnderDirectory(TEST_FILES_NUMBER,
+        testDir, "testListPath");
+
+    RemoteIterator<FileStatus> fsItr =
+        getFileSystem().listStatusIterator(testDir);
+    Assertions.assertThat(fsItr)
+        .describedAs("RemoteIterator should not be instance of "
+            + "AbfsListStatusRemoteIterator when it is disabled")
+        .isNotInstanceOf(AbfsListStatusRemoteIterator.class);
+    int itrCount = 0;
+    while (fsItr.hasNext()) {
+      FileStatus fileStatus = fsItr.next();
+      String pathStr = fileStatus.getPath().toString();
+      fileNames.remove(pathStr);
+      itrCount++;
+    }
+    Assertions.assertThat(itrCount)
+        .describedAs("Number of iterations should be equal to the files "
+            + "created")
+        .isEqualTo(TEST_FILES_NUMBER);
+    Assertions.assertThat(fileNames.size())
+        .describedAs("After removing every iterm found from the iterator, "
+            + "there should be no more elements in the fileNames")
+        .isEqualTo(0);
+  }
+
+  @Test
+  public void testWithAbfsIteratorDisabledWithoutHasNext() throws Exception {
+    Path testDir = createTestDirectory();
+    setPageSize(10);
+    setEnableAbfsIterator(false);
+    final List<String> fileNames = createFilesUnderDirectory(TEST_FILES_NUMBER,
+        testDir, "testListPath");
+
+    RemoteIterator<FileStatus> fsItr =
+        getFileSystem().listStatusIterator(testDir);
+    Assertions.assertThat(fsItr)
+        .describedAs("RemoteIterator should not be instance of "
+            + "AbfsListStatusRemoteIterator when it is disabled")
+        .isNotInstanceOf(AbfsListStatusRemoteIterator.class);
+    int itrCount = 0;
+    for (int i = 0; i < TEST_FILES_NUMBER; i++) {
+      FileStatus fileStatus = fsItr.next();
+      String pathStr = fileStatus.getPath().toString();
+      fileNames.remove(pathStr);
+      itrCount++;
+    }
+    Assertions.assertThatThrownBy(() -> fsItr.next())
+        .describedAs(
+            "next() should throw NoSuchElementException since next has been "
+                + "called " + TEST_FILES_NUMBER + " times")
+        .isInstanceOf(NoSuchElementException.class);
+    Assertions.assertThat(itrCount)
+        .describedAs("Number of iterations should be equal to the files "
+            + "created")
+        .isEqualTo(TEST_FILES_NUMBER);
+    Assertions.assertThat(fileNames.size())
+        .describedAs("After removing every iterm found from the iterator, "
+            + "there should be no more elements in the fileNames")
+        .isEqualTo(0);
+  }
+
+  @Test
+  public void testNextWhenNoMoreElementsPresent() throws Exception {
+    Path testDir = createTestDirectory();
+    setPageSize(10);
+    RemoteIterator fsItr =
+        new AbfsListStatusRemoteIterator(getFileSystem().getFileStatus(testDir),
+            getFileSystem().getAbfsStore());
+    fsItr = Mockito.spy(fsItr);
+    Mockito.doReturn(false).when(fsItr).hasNext();
+
+    RemoteIterator<FileStatus> finalFsItr = fsItr;
+    Assertions.assertThatThrownBy(() -> finalFsItr.next())
+        .describedAs(
+        "next() should throw NoSuchElementException if hasNext() return "
+            + "false")
+        .isInstanceOf(NoSuchElementException.class);
+  }
+
+  @Test
+  public void testHasNextForEmptyDir() throws Exception {
+    Path testDir = createTestDirectory();
+    setPageSize(10);
+    RemoteIterator<FileStatus> fsItr = getFileSystem()
+        .listStatusIterator(testDir);
+    Assertions.assertThat(fsItr.hasNext())
+        .describedAs("hasNext returns false for empty directory")
+        .isFalse();
+  }
+
+  @Test
+  public void testHasNextForFile() throws Exception {
+    final AzureBlobFileSystem fs = getFileSystem();
+    String testFileName = "testFile";
+    Path testFile = new Path(testFileName);
+    getFileSystem().create(testFile);
+    setPageSize(10);
+    RemoteIterator<FileStatus> fsItr = fs.listStatusIterator(testFile);
+    Assertions.assertThat(fsItr.hasNext())
+        .describedAs("hasNext returns true for file").isTrue();
+    Assertions.assertThat(fsItr.next().getPath().toString())
+        .describedAs("next returns the file itself")
+        .endsWith(testFileName);
+  }
+
+  @Test
+  public void testIOException() throws Exception {
+    Path testDir = createTestDirectory();
+    setPageSize(10);
+    getFileSystem().mkdirs(testDir);
+
+    String exceptionMessage = "test exception";
+    ListingSupport lsSupport =getMockListingSupport(exceptionMessage);
+    RemoteIterator fsItr =
+        new AbfsListStatusRemoteIterator(getFileSystem().getFileStatus(testDir),
+        lsSupport);
+
+    Assertions.assertThatThrownBy(() -> fsItr.next())
+        .describedAs(
+        "When ioException is not null and queue is empty exception should be "
+            + "thrown")
+        .isInstanceOf(IOException.class)
+        .hasMessage(exceptionMessage);
+  }
+
+  @Test
+  public void testNonExistingPath() throws Throwable {
+    Path nonExistingDir = new Path("nonExistingPath");
+    Assertions.assertThatThrownBy(
+        () -> getFileSystem().listStatusIterator(nonExistingDir)).describedAs(
+        "test the listStatusIterator call on a path which is not "
+            + "present should result in FileNotFoundException")
+        .isInstanceOf(FileNotFoundException.class);
+  }
+
+  private ListingSupport getMockListingSupport(String exceptionMessage) {
+    return new ListingSupport() {
+      @Override
+      public FileStatus[] listStatus(Path path) throws IOException {
+        return null;
+      }
+
+      @Override
+      public FileStatus[] listStatus(Path path, String startFrom)
+          throws IOException {
+        return null;
+      }
+
+      @Override
+      public String listStatus(Path path, String startFrom,
+          List<FileStatus> fileStatuses, boolean fetchAll, String continuation)
+          throws IOException {
+        throw new IOException(exceptionMessage);
+      }
+    };
+  }
+
+  private Path createTestDirectory() throws IOException {
+    String testDirectoryName = "testDirectory" + System.currentTimeMillis();
+    Path testDirectory = new Path(testDirectoryName);
+    getFileSystem().mkdirs(testDirectory);
+    return testDirectory;
+  }
+
+  private void setEnableAbfsIterator(boolean shouldEnable) throws IOException {
+    AzureBlobFileSystemStore abfsStore = getAbfsStore(getFileSystem());
+    abfsStore.getAbfsConfiguration().setEnableAbfsListIterator(shouldEnable);
+  }
+
+  private void setPageSize(int pageSize) throws IOException {
+    AzureBlobFileSystemStore abfsStore = getAbfsStore(getFileSystem());
+    abfsStore.getAbfsConfiguration().setListMaxResults(pageSize);
+  }
+
+  private List<String> createFilesUnderDirectory(int numFiles, Path rootPath,
+      String filenamePrefix)
+      throws ExecutionException, InterruptedException, IOException {
+    final List<Future<Void>> tasks = new ArrayList<>();
+    final List<String> fileNames = new ArrayList<>();
+    ExecutorService es = Executors.newFixedThreadPool(10);
+    for (int i = 0; i < numFiles; i++) {
+      final Path filePath = new Path(rootPath, filenamePrefix + i);
+      Callable<Void> callable = () -> {
+        getFileSystem().create(filePath);
+        fileNames.add(makeQualified(filePath).toString());
+        return null;
+      };
+      tasks.add(es.submit(callable));
+    }
+    for (Future<Void> task : tasks) {
+      task.get();
+    }
+    es.shutdownNow();

Review comment:
       Done

##########
File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsListStatusRemoteIterator.java
##########
@@ -0,0 +1,338 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+
+import org.assertj.core.api.Assertions;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.fs.azurebfs.services.AbfsListStatusRemoteIterator;
+import org.apache.hadoop.fs.azurebfs.services.ListingSupport;
+
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.anyBoolean;
+import static org.mockito.ArgumentMatchers.anyList;
+import static org.mockito.ArgumentMatchers.nullable;
+import static org.mockito.Mockito.verify;
+
+/**
+ * Test ListStatusRemoteIterator operation.
+ */
+public class ITestAbfsListStatusRemoteIterator extends AbstractAbfsIntegrationTest {
+
+  private static final int TEST_FILES_NUMBER = 1000;
+
+  public ITestAbfsListStatusRemoteIterator() throws Exception {
+    super();

Review comment:
       Done

##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ListingSupport.java
##########
@@ -0,0 +1,75 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs.services;
+
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+
+public interface ListingSupport {
+
+  /**
+   * @param path The list path.
+   * @return the entries in the path.
+   */
+  FileStatus[] listStatus(Path path) throws IOException;
+
+  /**
+   * @param path      Path the list path.
+   * @param startFrom The entry name that list results should start with.
+   *                  For example, if folder "/folder" contains four
+   *                  files: "afile", "bfile", "hfile", "ifile". Then
+   *                  listStatus(Path("/folder"), "hfile") will return
+   *                  "/folder/hfile" and "folder/ifile" Notice that if
+   *                  startFrom is a non-existent entry name, then the
+   *                  list response contains all entries after this
+   *                  non-existent entry in lexical order: listStatus
+   *                  (Path("/folder"), "cfile") will return
+   *                  "/folder/hfile" and "/folder/ifile".
+   * @return the entries in the path start from  "startFrom" in lexical order.
+   */
+  @InterfaceStability.Unstable

Review comment:
       Done

##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsListStatusRemoteIterator.java
##########
@@ -0,0 +1,159 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs.services;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeUnit;
+import javax.activation.UnsupportedDataTypeException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.RemoteIterator;
+
+public class AbfsListStatusRemoteIterator
+    implements RemoteIterator<FileStatus> {
+
+  private static final Logger LOG = LoggerFactory
+      .getLogger(AbfsListStatusRemoteIterator.class);
+
+  private static final boolean FETCH_ALL_FALSE = false;
+  private static final int MAX_QUEUE_SIZE = 10;
+  private static final long POLL_WAIT_TIME_IN_MS = 250;
+
+  private final FileStatus fileStatus;
+  private final ListingSupport listingSupport;
+  private final ArrayBlockingQueue<Object> iteratorsQueue;
+
+  private volatile boolean isAsyncInProgress = false;
+  private boolean isIterationComplete = false;
+  private String continuation;
+  private Iterator<FileStatus> currIterator;
+
+  public AbfsListStatusRemoteIterator(final FileStatus fileStatus,
+      final ListingSupport listingSupport) {
+    this.fileStatus = fileStatus;
+    this.listingSupport = listingSupport;
+    iteratorsQueue = new ArrayBlockingQueue<>(MAX_QUEUE_SIZE);
+    currIterator = Collections.emptyIterator();
+    fetchBatchesAsync();
+  }
+
+  @Override
+  public boolean hasNext() throws IOException {
+    if (currIterator.hasNext()) {
+      return true;
+    }
+    currIterator = getNextIterator();
+    return currIterator.hasNext();
+  }
+
+  @Override
+  public FileStatus next() throws IOException {
+    if (!this.hasNext()) {
+      throw new NoSuchElementException();
+    }
+    return currIterator.next();
+  }
+
+  private Iterator<FileStatus> getNextIterator() throws IOException {
+    fetchBatchesAsync();
+    try {
+      Object obj = null;
+      while (obj == null
+          && (!isIterationComplete || !iteratorsQueue.isEmpty())) {
+        obj = iteratorsQueue.poll(POLL_WAIT_TIME_IN_MS, TimeUnit.MILLISECONDS);
+      }
+      if (obj == null) {
+        return Collections.emptyIterator();
+      } else if (obj instanceof Iterator) {
+        return (Iterator<FileStatus>) obj;
+      } else if (obj instanceof IOException) {
+        throw (IOException) obj;
+      } else {
+        throw new UnsupportedDataTypeException();
+      }
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      LOG.error("Thread got interrupted: {}", e);
+      throw new IOException(e);
+    }
+  }
+
+  private void fetchBatchesAsync() {
+    if (isAsyncInProgress || isIterationComplete) {
+      return;
+    }
+    synchronized (this) {
+      if (isAsyncInProgress || isIterationComplete) {
+        return;
+      }
+      isAsyncInProgress = true;
+    }
+    CompletableFuture.runAsync(() -> asyncOp());
+  }
+
+  private void asyncOp() {
+    try {
+      while (!isIterationComplete && iteratorsQueue.size() <= MAX_QUEUE_SIZE) {
+        addNextBatchIteratorToQueue();
+      }
+    } catch (IOException ioe) {
+      LOG.error("Fetching filestatuses failed", ioe);
+      try {
+        iteratorsQueue.put(ioe);
+      } catch (InterruptedException interruptedException) {
+        Thread.currentThread().interrupt();
+        LOG.error("Thread got interrupted: {}", interruptedException);
+      }
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      LOG.error("Thread got interrupted: {}", e);
+    } finally {
+      synchronized (this) {
+        isAsyncInProgress = false;
+      }
+    }
+  }
+
+  private void addNextBatchIteratorToQueue()
+      throws IOException, InterruptedException {
+    List<FileStatus> fileStatuses = new ArrayList<>();
+    continuation = listingSupport
+        .listStatus(fileStatus.getPath(), null, fileStatuses, FETCH_ALL_FALSE,
+            continuation);
+    if(!fileStatuses.isEmpty()) {

Review comment:
       Done

##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java
##########
@@ -982,6 +985,19 @@ public boolean exists(Path f) throws IOException {
     return super.exists(f);
   }
 
+  @Override
+  public RemoteIterator<FileStatus> listStatusIterator(Path path)
+      throws IOException {
+    LOG.debug("AzureBlobFileSystem.listStatusIterator path : {}", path);
+    if (abfsStore.getAbfsConfiguration().enableAbfsListIterator()) {
+      AbfsListStatusRemoteIterator abfsLsItr =
+          new AbfsListStatusRemoteIterator(getFileStatus(path), abfsStore);
+      return RemoteIterators.typeCastingRemoteIterator(abfsLsItr);

Review comment:
       > very close to getting in. other than some minor checkstyle/javadoc complaints, that findbugs needs to be made to stop complaining. Usually it is indicating a real risk of a problem, so I'd like to see what can be done about it -rather than just edit the XML file to turn off the check
   
   There is one one findbugs warning remaining which is of medium priority. The same can be ignored since at line 147 the continuation token returned by the ListingOperation ouside the synchroniced lock since the same involve an http call.

##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java
##########
@@ -982,6 +985,19 @@ public boolean exists(Path f) throws IOException {
     return super.exists(f);
   }
 
+  @Override
+  public RemoteIterator<FileStatus> listStatusIterator(Path path)
+      throws IOException {
+    LOG.debug("AzureBlobFileSystem.listStatusIterator path : {}", path);
+    if (abfsStore.getAbfsConfiguration().enableAbfsListIterator()) {

Review comment:
       Done
   https://issues.apache.org/jira/browse/HADOOP-17512




----------------------------------------------------------------
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 #2548: DRAFT PR: Implementing ListStatusRemoteIterator

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


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |  30m 59s |  |  Docker mode activated.  |
   |||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  |  No case conflicting files found.  |
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |   |   0m  0s | [test4tests](test4tests) |  The patch appears to include 1 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |  33m  7s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |   0m 38s |  |  trunk passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04  |
   | +1 :green_heart: |  compile  |   0m 33s |  |  trunk passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01  |
   | +1 :green_heart: |  checkstyle  |   0m 27s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   0m 40s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  16m 25s |  |  branch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 33s |  |  trunk passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04  |
   | +1 :green_heart: |  javadoc  |   0m 29s |  |  trunk passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01  |
   | +0 :ok: |  spotbugs  |   0m 59s |  |  Used deprecated FindBugs config; considering switching to SpotBugs.  |
   | +1 :green_heart: |  findbugs  |   0m 58s |  |  trunk passed  |
   |||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   0m 31s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 29s |  |  the patch passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04  |
   | -1 :x: |  javac  |   0m 29s | [/diff-compile-javac-hadoop-tools_hadoop-azure-jdkUbuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/3/artifact/out/diff-compile-javac-hadoop-tools_hadoop-azure-jdkUbuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04.txt) |  hadoop-tools_hadoop-azure-jdkUbuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04 with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04 generated 2 new + 13 unchanged - 2 fixed = 15 total (was 15)  |
   | +1 :green_heart: |  compile  |   0m 26s |  |  the patch passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01  |
   | -1 :x: |  javac  |   0m 26s | [/diff-compile-javac-hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/3/artifact/out/diff-compile-javac-hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01.txt) |  hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01 with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01 generated 2 new + 7 unchanged - 2 fixed = 9 total (was 9)  |
   | -0 :warning: |  checkstyle  |   0m 17s | [/diff-checkstyle-hadoop-tools_hadoop-azure.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/3/artifact/out/diff-checkstyle-hadoop-tools_hadoop-azure.txt) |  hadoop-tools/hadoop-azure: The patch generated 6 new + 2 unchanged - 0 fixed = 8 total (was 2)  |
   | +1 :green_heart: |  mvnsite  |   0m 28s |  |  the patch passed  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  shadedclient  |  14m 54s |  |  patch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 27s |  |  the patch passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04  |
   | +1 :green_heart: |  javadoc  |   0m 25s |  |  the patch passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01  |
   | +1 :green_heart: |  findbugs  |   0m 59s |  |  the patch passed  |
   |||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   1m 28s |  |  hadoop-azure in the patch passed.  |
   | -1 :x: |  asflicense  |   0m 33s | [/patch-asflicense-problems.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/3/artifact/out/patch-asflicense-problems.txt) |  The patch generated 1 ASF License warnings.  |
   |  |   | 107m 45s |  |  |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/3/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/2548 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient findbugs checkstyle |
   | uname | Linux 7af052b03163 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 / 513f1995adc |
   | Default Java | Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01 |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/3/testReport/ |
   | Max. process+thread count | 542 (vs. ulimit of 5500) |
   | modules | C: hadoop-tools/hadoop-azure U: hadoop-tools/hadoop-azure |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/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] hadoop-yetus commented on pull request #2548: HADOOP-17475. ABFS: Implementing ListStatusRemoteIterator

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


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |   6m 10s |  |  Docker mode activated.  |
   |||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  |  No case conflicting files found.  |
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |   |   0m  0s | [test4tests](test4tests) |  The patch appears to include 1 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |  33m 28s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |   0m 39s |  |  trunk passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  compile  |   0m 29s |  |  trunk passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~20.04-b01  |
   | +1 :green_heart: |  checkstyle  |   0m 26s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   0m 40s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  14m  6s |  |  branch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 30s |  |  trunk passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javadoc  |   0m 34s |  |  trunk passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~20.04-b01  |
   | +0 :ok: |  spotbugs  |   1m  8s |  |  Used deprecated FindBugs config; considering switching to SpotBugs.  |
   | +1 :green_heart: |  findbugs  |   1m  6s |  |  trunk passed  |
   | -0 :warning: |  patch  |   1m 25s |  |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   |||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   0m 29s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 30s |  |  the patch passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javac  |   0m 30s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 26s |  |  the patch passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~20.04-b01  |
   | +1 :green_heart: |  javac  |   0m 26s |  |  the patch passed  |
   | +1 :green_heart: |  checkstyle  |   0m 19s |  |  the patch passed  |
   | +1 :green_heart: |  mvnsite  |   0m 29s |  |  the patch passed  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  xml  |   0m  1s |  |  The patch has no ill-formed XML file.  |
   | +1 :green_heart: |  shadedclient  |  12m 36s |  |  patch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 28s |  |  hadoop-tools_hadoop-azure-jdkUbuntu-11.0.9.1+1-Ubuntu-0ubuntu1.20.04 with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.20.04 generated 0 new + 15 unchanged - 2 fixed = 15 total (was 17)  |
   | +1 :green_heart: |  javadoc  |   0m 24s |  |  hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_275-8u275-b01-0ubuntu1~20.04-b01 with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~20.04-b01 generated 0 new + 15 unchanged - 2 fixed = 15 total (was 17)  |
   | -1 :x: |  findbugs  |   1m  7s | [/new-findbugs-hadoop-tools_hadoop-azure.html](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/18/artifact/out/new-findbugs-hadoop-tools_hadoop-azure.html) |  hadoop-tools/hadoop-azure generated 1 new + 0 unchanged - 0 fixed = 1 total (was 0)  |
   |||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   1m 56s |  |  hadoop-azure in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 32s |  |  The patch does not generate ASF License warnings.  |
   |  |   |  79m 34s |  |  |
   
   
   | Reason | Tests |
   |-------:|:------|
   | FindBugs | module:hadoop-tools/hadoop-azure |
   |  |  Inconsistent synchronization of org.apache.hadoop.fs.azurebfs.services.AbfsListStatusRemoteIterator.continuation; locked 50% of time  Unsynchronized access at AbfsListStatusRemoteIterator.java:50% of time  Unsynchronized access at AbfsListStatusRemoteIterator.java:[line 147] |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/18/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/2548 |
   | Optional Tests | dupname asflicense xml compile javac javadoc mvninstall mvnsite unit shadedclient findbugs checkstyle |
   | uname | Linux c0cbb8a0f34f 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 / 394b9f7a5c4 |
   | Default Java | Private Build-1.8.0_275-8u275-b01-0ubuntu1~20.04-b01 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.20.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_275-8u275-b01-0ubuntu1~20.04-b01 |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/18/testReport/ |
   | Max. process+thread count | 664 (vs. ulimit of 5500) |
   | modules | C: hadoop-tools/hadoop-azure U: hadoop-tools/hadoop-azure |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/18/console |
   | versions | git=2.25.1 maven=3.6.3 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] snvijaya commented on a change in pull request #2548: DRAFT PR: Implementing ListStatusRemoteIterator

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



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java
##########
@@ -865,16 +873,16 @@ public FileStatus getFileStatus(final Path path) throws IOException {
             startFrom);
 
     final String relativePath = getRelativePath(path);
-    String continuation = null;
 
-    // generate continuation token if a valid startFrom is provided.
-    if (startFrom != null && !startFrom.isEmpty()) {
-      continuation = getIsNamespaceEnabled()
-              ? generateContinuationTokenForXns(startFrom)
-              : generateContinuationTokenForNonXns(relativePath, startFrom);
+    if (continuation == null || continuation.length() < 1) {

Review comment:
       continuation.isEmpty()




----------------------------------------------------------------
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] bilaharith commented on a change in pull request #2548: DRAFT PR: Implementing ListStatusRemoteIterator

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



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ListStatusRemoteIterator.java
##########
@@ -0,0 +1,149 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs.services;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.CompletableFuture;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore;
+
+public class ListStatusRemoteIterator implements RemoteIterator<FileStatus> {
+
+  private static final Logger LOG = LoggerFactory
+      .getLogger(ListStatusRemoteIterator.class);
+
+  private static final boolean FETCH_ALL_FALSE = false;
+  private static final int MAX_QUEUE_SIZE = 10;
+
+  private final Path path;
+  private final AzureBlobFileSystemStore abfsStore;
+  private final ArrayBlockingQueue<Iterator<FileStatus>> iteratorsQueue;
+  private final Object asyncOpLock = new Object();
+
+  private boolean firstBatch = true;
+  private boolean isAsyncInProgress = false;
+  private String continuation;
+  private Iterator<FileStatus> currIterator;
+  private IOException ioException;
+
+  public ListStatusRemoteIterator(final Path path,

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] vinaysbadami commented on a change in pull request #2548: HADOOP-17475. ABFS: Implementing ListStatusRemoteIterator

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



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java
##########
@@ -982,6 +985,19 @@ public boolean exists(Path f) throws IOException {
     return super.exists(f);
   }
 
+  @Override
+  public RemoteIterator<FileStatus> listStatusIterator(Path path)
+      throws IOException {
+    LOG.debug("AzureBlobFileSystem.listStatusIterator path : {}", path);
+    if (abfsStore.getAbfsConfiguration().enableAbfsListIterator()) {

Review comment:
       after it is used for a while and we have confidence that it is baked




----------------------------------------------------------------
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] bilaharith commented on a change in pull request #2548: HADOOP-17475. ABFS: Implementing ListStatusRemoteIterator

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



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsListStatusRemoteIterator.java
##########
@@ -0,0 +1,151 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs.services;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.CompletableFuture;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.RemoteIterator;
+
+public class AbfsListStatusRemoteIterator implements RemoteIterator<FileStatus> {
+
+  private static final Logger LOG = LoggerFactory
+      .getLogger(AbfsListStatusRemoteIterator.class);
+
+  private static final boolean FETCH_ALL_FALSE = false;
+  private static final int MAX_QUEUE_SIZE = 10;
+
+  private final FileStatus fileStatus;
+  private final ListingSupport listingSupport;
+  private final ArrayBlockingQueue<Iterator<FileStatus>> iteratorsQueue;
+  private final Object asyncOpLock = new Object();
+
+  private volatile boolean isAsyncInProgress = false;
+  private boolean firstBatch = true;

Review comment:
       Done, introduced a new variable isIterationComplete.




----------------------------------------------------------------
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] bilaharith commented on a change in pull request #2548: DRAFT PR: Implementing ListStatusRemoteIterator

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



##########
File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemListStatusIterator.java
##########
@@ -0,0 +1,190 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs;
+
+import java.io.IOException;
+import java.lang.reflect.Field;
+import java.lang.reflect.Modifier;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+
+import org.assertj.core.api.Assertions;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+import org.apache.hadoop.fs.azurebfs.services.ListStatusRemoteIterator;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+
+/**
+ * Test listStatus operation.
+ */
+public class ITestAzureBlobFileSystemListStatusIterator
+    extends AbstractAbfsIntegrationTest {
+
+  private static final int TEST_FILES_NUMBER = 1000;
+
+  public ITestAzureBlobFileSystemListStatusIterator() throws Exception {
+    super();
+  }
+
+  @Test
+  public void testListPath() throws Exception {
+    final AzureBlobFileSystem fs = getFileSystem();
+    String rootPath = "testRoot1";
+    final List<String> fileNames = createFiles(TEST_FILES_NUMBER, rootPath,
+        "testListPath");
+    AzureBlobFileSystemStore abfsStore = getAbfsStore(fs);
+    abfsStore.getAbfsConfiguration().setListMaxResults(10);
+    RemoteIterator<FileStatus> fsIt = fs.listStatusIterator(new Path(rootPath));
+    int itrCount = 0;
+    while (fsIt.hasNext()) {
+      FileStatus fileStatus = fsIt.next();
+      String pathStr = fileStatus.getPath().toString();
+      fileNames.remove(pathStr);
+      itrCount++;
+    }
+    assertEquals(TEST_FILES_NUMBER, itrCount);
+    assertEquals(0, fileNames.size());
+  }
+
+  @Test
+  public void testNextWhenNoMoreElementsPresent() throws Exception {
+    final AzureBlobFileSystem fs = getFileSystem();
+    String rootPathStr = "testRoot2";
+    Path rootPath = new Path(rootPathStr);
+    getFileSystem().mkdirs(rootPath);
+    RemoteIterator<FileStatus> fsItr = fs.listStatusIterator(rootPath);
+    fsItr = Mockito.spy(fsItr);
+    Mockito.doReturn(false).when(fsItr).hasNext();
+
+    RemoteIterator<FileStatus> finalFsItr = fsItr;
+    Assertions.assertThatThrownBy(() -> finalFsItr.next()).describedAs(
+        "next() should throw NoSuchElementException if hasNext() return "
+            + "false").isInstanceOf(NoSuchElementException.class);
+  }
+
+  @Test
+  public void testHasNextForEmptyDir() throws Exception {
+    final AzureBlobFileSystem fs = getFileSystem();
+    String rootPathStr = "testRoot3";
+    Path rootPath = new Path(rootPathStr);
+    getFileSystem().mkdirs(rootPath);
+    RemoteIterator<FileStatus> fsItr = fs.listStatusIterator(rootPath);
+    Assertions.assertThat(fsItr.hasNext())
+        .describedAs("hasNext returns false for empty directory").isFalse();
+  }
+
+  @Test
+  public void testHasNextForFile() throws Exception {
+    final AzureBlobFileSystem fs = getFileSystem();
+    String rootPathStr = "testRoot4";
+    Path rootPath = new Path(rootPathStr);
+    getFileSystem().create(rootPath);
+    RemoteIterator<FileStatus> fsItr = fs.listStatusIterator(rootPath);
+    Assertions.assertThat(fsItr.hasNext())
+        .describedAs("hasNext returns true for file").isTrue();
+  }
+
+  @Test
+  public void testHasNextForIOException() throws Exception {
+    final AzureBlobFileSystem fs = getFileSystem();
+    String rootPathStr = "testRoot5";
+    Path rootPath = new Path(rootPathStr);
+    getFileSystem().mkdirs(rootPath);
+    ListStatusRemoteIterator fsItr = (ListStatusRemoteIterator) fs
+        .listStatusIterator(rootPath);
+    Thread.sleep(1000);
+
+    String exceptionMessage = "test exception";
+    setPrivateField(fsItr, ListStatusRemoteIterator.class, "ioException",
+        new IOException(exceptionMessage));
+    setPrivateFinalField(fsItr, ListStatusRemoteIterator.class,
+        "iteratorsQueue", new ArrayBlockingQueue<Iterator>(1));
+
+    Assertions.assertThatThrownBy(() -> fsItr.hasNext()).describedAs(
+        "When ioException is not null and queue is empty exception should be "
+            + "thrown").isInstanceOf(IOException.class)
+        .hasMessage(exceptionMessage);
+  }
+
+  private void setPrivateField(Object obj, Class classObj, String fieldName,
+      Object value) throws NoSuchFieldException, IllegalAccessException {
+    Field field = classObj.getDeclaredField(fieldName);
+    field.setAccessible(true);
+    field.set(obj, value);
+  }
+
+  private void setPrivateFinalField(Object obj, Class classObj,
+      String fieldName, Object value)
+      throws NoSuchFieldException, IllegalAccessException {
+    Field field = classObj.getDeclaredField(fieldName);
+    field.setAccessible(true);
+    Field modifiersField = Field.class.getDeclaredField("modifiers");
+    modifiersField.setAccessible(true);
+    modifiersField.setInt(field, field.getModifiers() & ~Modifier.FINAL);
+    field.set(obj, value);
+  }
+
+  private List<String> createFiles(int numFiles, String rootPathStr,
+      String filenamePrefix)
+      throws ExecutionException, InterruptedException, IOException {
+    final List<Future<Void>> tasks = new ArrayList<>();
+    final List<String> fileNames = new ArrayList<>();
+    ExecutorService es = Executors.newFixedThreadPool(10);
+    final Path rootPath = new Path(rootPathStr);
+    for (int i = 0; i < numFiles; i++) {
+      final Path filePath = new Path(rootPath, filenamePrefix + i);
+      Callable<Void> callable = new Callable<Void>() {
+        @Override
+        public Void call() throws Exception {
+          getFileSystem().create(filePath);
+          fileNames.add(makeQualified(filePath).toString());
+          return null;
+        }
+      };
+      tasks.add(es.submit(callable));
+    }
+    for (Future<Void> task : tasks) {
+      task.get();
+    }
+    es.shutdownNow();
+    return fileNames;
+  }
+
+  private AzureBlobFileSystemStore getAbfsStore(FileSystem fs)

Review comment:
       Done

##########
File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemListStatusIterator.java
##########
@@ -0,0 +1,190 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs;
+
+import java.io.IOException;
+import java.lang.reflect.Field;
+import java.lang.reflect.Modifier;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+
+import org.assertj.core.api.Assertions;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+import org.apache.hadoop.fs.azurebfs.services.ListStatusRemoteIterator;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+
+/**
+ * Test listStatus operation.
+ */
+public class ITestAzureBlobFileSystemListStatusIterator
+    extends AbstractAbfsIntegrationTest {
+
+  private static final int TEST_FILES_NUMBER = 1000;
+
+  public ITestAzureBlobFileSystemListStatusIterator() throws Exception {
+    super();
+  }
+
+  @Test
+  public void testListPath() throws Exception {
+    final AzureBlobFileSystem fs = getFileSystem();
+    String rootPath = "testRoot1";
+    final List<String> fileNames = createFiles(TEST_FILES_NUMBER, rootPath,
+        "testListPath");
+    AzureBlobFileSystemStore abfsStore = getAbfsStore(fs);
+    abfsStore.getAbfsConfiguration().setListMaxResults(10);
+    RemoteIterator<FileStatus> fsIt = fs.listStatusIterator(new Path(rootPath));
+    int itrCount = 0;
+    while (fsIt.hasNext()) {

Review comment:
       Done

##########
File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemListStatusIterator.java
##########
@@ -0,0 +1,190 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs;
+
+import java.io.IOException;
+import java.lang.reflect.Field;
+import java.lang.reflect.Modifier;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+
+import org.assertj.core.api.Assertions;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+import org.apache.hadoop.fs.azurebfs.services.ListStatusRemoteIterator;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+
+/**
+ * Test listStatus operation.
+ */
+public class ITestAzureBlobFileSystemListStatusIterator
+    extends AbstractAbfsIntegrationTest {
+
+  private static final int TEST_FILES_NUMBER = 1000;
+
+  public ITestAzureBlobFileSystemListStatusIterator() throws Exception {
+    super();
+  }
+
+  @Test
+  public void testListPath() throws Exception {
+    final AzureBlobFileSystem fs = getFileSystem();
+    String rootPath = "testRoot1";
+    final List<String> fileNames = createFiles(TEST_FILES_NUMBER, rootPath,
+        "testListPath");
+    AzureBlobFileSystemStore abfsStore = getAbfsStore(fs);
+    abfsStore.getAbfsConfiguration().setListMaxResults(10);
+    RemoteIterator<FileStatus> fsIt = fs.listStatusIterator(new Path(rootPath));
+    int itrCount = 0;
+    while (fsIt.hasNext()) {
+      FileStatus fileStatus = fsIt.next();
+      String pathStr = fileStatus.getPath().toString();
+      fileNames.remove(pathStr);
+      itrCount++;
+    }
+    assertEquals(TEST_FILES_NUMBER, itrCount);
+    assertEquals(0, fileNames.size());
+  }
+
+  @Test
+  public void testNextWhenNoMoreElementsPresent() throws Exception {
+    final AzureBlobFileSystem fs = getFileSystem();
+    String rootPathStr = "testRoot2";
+    Path rootPath = new Path(rootPathStr);
+    getFileSystem().mkdirs(rootPath);
+    RemoteIterator<FileStatus> fsItr = fs.listStatusIterator(rootPath);
+    fsItr = Mockito.spy(fsItr);
+    Mockito.doReturn(false).when(fsItr).hasNext();
+
+    RemoteIterator<FileStatus> finalFsItr = fsItr;
+    Assertions.assertThatThrownBy(() -> finalFsItr.next()).describedAs(
+        "next() should throw NoSuchElementException if hasNext() return "
+            + "false").isInstanceOf(NoSuchElementException.class);
+  }
+
+  @Test
+  public void testHasNextForEmptyDir() throws Exception {
+    final AzureBlobFileSystem fs = getFileSystem();
+    String rootPathStr = "testRoot3";
+    Path rootPath = new Path(rootPathStr);
+    getFileSystem().mkdirs(rootPath);
+    RemoteIterator<FileStatus> fsItr = fs.listStatusIterator(rootPath);
+    Assertions.assertThat(fsItr.hasNext())
+        .describedAs("hasNext returns false for empty directory").isFalse();
+  }
+
+  @Test
+  public void testHasNextForFile() throws Exception {
+    final AzureBlobFileSystem fs = getFileSystem();
+    String rootPathStr = "testRoot4";
+    Path rootPath = new Path(rootPathStr);
+    getFileSystem().create(rootPath);
+    RemoteIterator<FileStatus> fsItr = fs.listStatusIterator(rootPath);
+    Assertions.assertThat(fsItr.hasNext())
+        .describedAs("hasNext returns true for file").isTrue();
+  }
+
+  @Test
+  public void testHasNextForIOException() throws Exception {
+    final AzureBlobFileSystem fs = getFileSystem();
+    String rootPathStr = "testRoot5";
+    Path rootPath = new Path(rootPathStr);
+    getFileSystem().mkdirs(rootPath);
+    ListStatusRemoteIterator fsItr = (ListStatusRemoteIterator) fs
+        .listStatusIterator(rootPath);
+    Thread.sleep(1000);
+
+    String exceptionMessage = "test exception";
+    setPrivateField(fsItr, ListStatusRemoteIterator.class, "ioException",

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] hadoop-yetus removed a comment on pull request #2548: DRAFT PR: Implementing ListStatusRemoteIterator

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


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |   1m  5s |  |  Docker mode activated.  |
   |||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  |  No case conflicting files found.  |
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |   |   0m  0s | [test4tests](test4tests) |  The patch appears to include 1 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |  34m 29s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |   0m 36s |  |  trunk passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04  |
   | +1 :green_heart: |  compile  |   0m 33s |  |  trunk passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01  |
   | +1 :green_heart: |  checkstyle  |   0m 26s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   0m 38s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  16m 28s |  |  branch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 31s |  |  trunk passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04  |
   | +1 :green_heart: |  javadoc  |   0m 28s |  |  trunk passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01  |
   | +0 :ok: |  spotbugs  |   0m 58s |  |  Used deprecated FindBugs config; considering switching to SpotBugs.  |
   | +1 :green_heart: |  findbugs  |   0m 57s |  |  trunk passed  |
   |||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   0m 30s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 30s |  |  the patch passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04  |
   | -1 :x: |  javac  |   0m 30s | [/diff-compile-javac-hadoop-tools_hadoop-azure-jdkUbuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/6/artifact/out/diff-compile-javac-hadoop-tools_hadoop-azure-jdkUbuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04.txt) |  hadoop-tools_hadoop-azure-jdkUbuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04 with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04 generated 1 new + 14 unchanged - 1 fixed = 15 total (was 15)  |
   | +1 :green_heart: |  compile  |   0m 26s |  |  the patch passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01  |
   | -1 :x: |  javac  |   0m 26s | [/diff-compile-javac-hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/6/artifact/out/diff-compile-javac-hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01.txt) |  hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01 with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01 generated 1 new + 8 unchanged - 1 fixed = 9 total (was 9)  |
   | +1 :green_heart: |  checkstyle  |   0m 17s |  |  the patch passed  |
   | +1 :green_heart: |  mvnsite  |   0m 29s |  |  the patch passed  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  shadedclient  |  15m 37s |  |  patch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 24s |  |  the patch passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04  |
   | +1 :green_heart: |  javadoc  |   0m 23s |  |  the patch passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01  |
   | -1 :x: |  findbugs  |   1m  1s | [/new-findbugs-hadoop-tools_hadoop-azure.html](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/6/artifact/out/new-findbugs-hadoop-tools_hadoop-azure.html) |  hadoop-tools/hadoop-azure generated 4 new + 0 unchanged - 0 fixed = 4 total (was 0)  |
   |||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   1m 25s |  |  hadoop-azure in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 33s |  |  The patch does not generate ASF License warnings.  |
   |  |   |  79m 38s |  |  |
   
   
   | Reason | Tests |
   |-------:|:------|
   | FindBugs | module:hadoop-tools/hadoop-azure |
   |  |  Possible doublecheck on org.apache.hadoop.fs.azurebfs.services.ListStatusRemoteIterator.isAsyncInProgress in org.apache.hadoop.fs.azurebfs.services.ListStatusRemoteIterator.asyncOp()  At ListStatusRemoteIterator.java:org.apache.hadoop.fs.azurebfs.services.ListStatusRemoteIterator.asyncOp()  At ListStatusRemoteIterator.java:[lines 115-119] |
   |  |  Inconsistent synchronization of org.apache.hadoop.fs.azurebfs.services.ListStatusRemoteIterator.continuation; locked 40% of time  Unsynchronized access at ListStatusRemoteIterator.java:40% of time  Unsynchronized access at ListStatusRemoteIterator.java:[line 107] |
   |  |  Inconsistent synchronization of org.apache.hadoop.fs.azurebfs.services.ListStatusRemoteIterator.firstBatch; locked 66% of time  Unsynchronized access at ListStatusRemoteIterator.java:66% of time  Unsynchronized access at ListStatusRemoteIterator.java:[line 107] |
   |  |  Inconsistent synchronization of org.apache.hadoop.fs.azurebfs.services.ListStatusRemoteIterator.ioException; locked 66% of time  Unsynchronized access at ListStatusRemoteIterator.java:66% of time  Unsynchronized access at ListStatusRemoteIterator.java:[line 129] |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/6/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/2548 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient findbugs checkstyle |
   | uname | Linux 7a17bcb50465 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 / ec22850dbef |
   | Default Java | Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01 |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/6/testReport/ |
   | Max. process+thread count | 544 (vs. ulimit of 5500) |
   | modules | C: hadoop-tools/hadoop-azure U: hadoop-tools/hadoop-azure |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/6/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] bilaharith commented on a change in pull request #2548: DRAFT PR: Implementing ListStatusRemoteIterator

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



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ListStatusRemoteIterator.java
##########
@@ -0,0 +1,129 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs.services;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.ListIterator;
+import java.util.NoSuchElementException;
+import java.util.Queue;
+import java.util.concurrent.CompletableFuture;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore;
+
+public class ListStatusRemoteIterator implements RemoteIterator<FileStatus> {
+
+  private static final Logger LOG = LoggerFactory
+      .getLogger(ListStatusRemoteIterator.class);
+
+  private static final boolean FETCH_ALL_FALSE = false;
+
+  private final Path path;
+  private final AzureBlobFileSystemStore abfsStore;
+  private final Queue<ListIterator<FileStatus>> iteratorsQueue =
+      new LinkedList<>();
+
+  private boolean firstRead = true;
+  private String continuation;
+  private ListIterator<FileStatus> currIterator;
+  private IOException ioException;
+
+  public ListStatusRemoteIterator(final Path path,
+      final AzureBlobFileSystemStore abfsStore) throws IOException {
+    this.path = path;
+    this.abfsStore = abfsStore;

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] steveloughran commented on a change in pull request #2548: HADOOP-17475. ABFS: Implementing ListStatusRemoteIterator

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



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java
##########
@@ -45,6 +45,8 @@
 import org.apache.commons.lang3.ArrayUtils;
 import org.apache.hadoop.fs.azurebfs.services.AbfsClient;
 import org.apache.hadoop.fs.azurebfs.services.AbfsClientThrottlingIntercept;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.fs.azurebfs.services.AbfsListStatusRemoteIterator;

Review comment:
       should go above so imports are in order; goal is to reduce conflict between patches, branches and versions

##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java
##########
@@ -982,6 +985,19 @@ public boolean exists(Path f) throws IOException {
     return super.exists(f);
   }
 
+  @Override
+  public RemoteIterator<FileStatus> listStatusIterator(Path path)
+      throws IOException {
+    LOG.debug("AzureBlobFileSystem.listStatusIterator path : {}", path);
+    if (abfsStore.getAbfsConfiguration().enableAbfsListIterator()) {

Review comment:
       would you ever want to make this not optional? one code path == better testing

##########
File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsListStatusIterator.java
##########
@@ -0,0 +1,339 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+
+import org.assertj.core.api.Assertions;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.fs.azurebfs.services.AbfsListStatusRemoteIterator;
+import org.apache.hadoop.fs.azurebfs.services.ListingSupport;
+
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.anyBoolean;
+import static org.mockito.ArgumentMatchers.anyList;
+import static org.mockito.ArgumentMatchers.nullable;
+import static org.mockito.Mockito.verify;
+
+/**
+ * Test ListStatusRemoteIterator operation.
+ */
+public class ITestAbfsListStatusIterator extends AbstractAbfsIntegrationTest {
+
+  private static final int TEST_FILES_NUMBER = 1000;
+
+  public ITestAbfsListStatusIterator() throws Exception {
+    super();
+  }
+
+  @Test
+  public void testListStatusRemoteIterator() throws Exception {
+    Path testDir = createTestDirectory();
+    setPageSize(10);
+    final List<String> fileNames = createFilesUnderDirectory(TEST_FILES_NUMBER,
+        testDir, "testListPath");
+
+    ListingSupport listngSupport = Mockito.spy(getFileSystem().getAbfsStore());

Review comment:
       With an interface, you don't need to play mockito games any more. Instead just provide a dummy impl to simulate deep/wide directories, controllable page size etc

##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsListStatusRemoteIterator.java
##########
@@ -0,0 +1,151 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs.services;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.CompletableFuture;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.RemoteIterator;
+
+public class AbfsListStatusRemoteIterator implements RemoteIterator<FileStatus> {
+
+  private static final Logger LOG = LoggerFactory
+      .getLogger(AbfsListStatusRemoteIterator.class);
+
+  private static final boolean FETCH_ALL_FALSE = false;
+  private static final int MAX_QUEUE_SIZE = 10;
+
+  private final FileStatus fileStatus;
+  private final ListingSupport listingSupport;
+  private final ArrayBlockingQueue<Iterator<FileStatus>> iteratorsQueue;
+  private final Object asyncOpLock = new Object();
+
+  private volatile boolean isAsyncInProgress = false;
+  private boolean firstBatch = true;
+  private String continuation;
+  private Iterator<FileStatus> currIterator;
+  private IOException ioException;
+
+  public AbfsListStatusRemoteIterator(final FileStatus fileStatus,
+      final ListingSupport listingSupport) {
+    this.fileStatus = fileStatus;
+    this.listingSupport = listingSupport;
+    iteratorsQueue = new ArrayBlockingQueue<>(MAX_QUEUE_SIZE);
+    currIterator = Collections.emptyIterator();
+    fetchBatchesAsync();
+  }
+
+  @Override
+  public boolean hasNext() throws IOException {
+    if (currIterator.hasNext()) {
+      return true;
+    }
+    updateCurrentIterator();
+    return currIterator.hasNext();
+  }
+
+  @Override
+  public FileStatus next() throws IOException {
+    if (!this.hasNext()) {
+      throw new NoSuchElementException();
+    }
+    return currIterator.next();
+  }
+
+  private void updateCurrentIterator() throws IOException {
+    fetchBatchesAsync();
+    synchronized (this) {
+      if (iteratorsQueue.isEmpty()) {
+        if (ioException != null) {
+          throw ioException;
+        }
+        if (isListingComplete()) {
+          return;
+        }
+      }
+    }
+    try {
+      currIterator = iteratorsQueue.take();
+      if (!currIterator.hasNext() && !isListingComplete()) {
+        updateCurrentIterator();
+      }
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      LOG.error("Thread got interrupted: {}", e);
+    }
+  }
+
+  private synchronized boolean isListingComplete() {
+    return !firstBatch && (continuation == null || continuation.isEmpty());
+  }
+
+  private void fetchBatchesAsync() {
+    if (isAsyncInProgress) {
+      return;
+    }
+    synchronized (asyncOpLock) {
+      if (isAsyncInProgress) {
+        return;
+      }
+      isAsyncInProgress = true;
+    }
+    CompletableFuture.runAsync(() -> asyncOp());
+  }
+
+  private void asyncOp() {
+    try {
+      while (!isListingComplete() && iteratorsQueue.size() <= MAX_QUEUE_SIZE) {
+        addNextBatchIteratorToQueue();
+      }
+    } catch (IOException e) {
+      ioException = e;

Review comment:
       * should only be set with first IOE, as that's usually first sign of failure. Rest just log message @ warn and stack @ debug.
   * what about other exceptions? are they handled by the normal Futures code? In which case ExecutionException needs to be picked up and unwrapped. The code in org.apache.hadoop.util.functional.FutureIO can help there

##########
File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsListStatusIterator.java
##########
@@ -0,0 +1,339 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+
+import org.assertj.core.api.Assertions;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.fs.azurebfs.services.AbfsListStatusRemoteIterator;
+import org.apache.hadoop.fs.azurebfs.services.ListingSupport;
+
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.anyBoolean;
+import static org.mockito.ArgumentMatchers.anyList;
+import static org.mockito.ArgumentMatchers.nullable;
+import static org.mockito.Mockito.verify;
+
+/**
+ * Test ListStatusRemoteIterator operation.
+ */
+public class ITestAbfsListStatusIterator extends AbstractAbfsIntegrationTest {
+
+  private static final int TEST_FILES_NUMBER = 1000;
+
+  public ITestAbfsListStatusIterator() throws Exception {
+    super();
+  }
+
+  @Test
+  public void testListStatusRemoteIterator() throws Exception {
+    Path testDir = createTestDirectory();
+    setPageSize(10);
+    final List<String> fileNames = createFilesUnderDirectory(TEST_FILES_NUMBER,
+        testDir, "testListPath");
+
+    ListingSupport listngSupport = Mockito.spy(getFileSystem().getAbfsStore());
+    RemoteIterator<FileStatus> fsItr = new AbfsListStatusRemoteIterator(
+        getFileSystem().getFileStatus(testDir), listngSupport);
+    Assertions.assertThat(fsItr)
+        .describedAs("RemoteIterator should be instance of "
+            + "AbfsListStatusRemoteIterator by default")
+        .isInstanceOf(AbfsListStatusRemoteIterator.class);
+    int itrCount = 0;
+    while (fsItr.hasNext()) {
+      FileStatus fileStatus = fsItr.next();
+      String pathStr = fileStatus.getPath().toString();
+      fileNames.remove(pathStr);
+      itrCount++;
+    }
+    Assertions.assertThat(itrCount)
+        .describedAs("Number of iterations should be equal to the files "
+            + "created")
+        .isEqualTo(TEST_FILES_NUMBER);
+    Assertions.assertThat(fileNames.size())
+        .describedAs("After removing every iterm found from the iterator, "
+            + "there should be no more elements in the fileNames")
+        .isEqualTo(0);
+    verify(listngSupport, Mockito.atLeast(100))
+        .listStatus(any(Path.class), nullable(String.class),
+            anyList(), anyBoolean(),
+            nullable(String.class));
+  }
+
+  @Test
+  public void testListStatusRemoteIteratorWithoutHasNext() throws Exception {
+    Path testDir = createTestDirectory();
+    setPageSize(10);
+    final List<String> fileNames = createFilesUnderDirectory(TEST_FILES_NUMBER,
+        testDir, "testListPath");
+
+    ListingSupport listngSupport = Mockito.spy(getFileSystem().getAbfsStore());
+    RemoteIterator<FileStatus> fsItr = new AbfsListStatusRemoteIterator(
+        getFileSystem().getFileStatus(testDir), listngSupport);
+    Assertions.assertThat(fsItr)
+        .describedAs("RemoteIterator should be instance of "
+            + "AbfsListStatusRemoteIterator by default")
+        .isInstanceOf(AbfsListStatusRemoteIterator.class);
+    int itrCount = 0;
+    for (int i = 0; i < TEST_FILES_NUMBER; i++) {
+      FileStatus fileStatus = fsItr.next();
+      String pathStr = fileStatus.getPath().toString();
+      fileNames.remove(pathStr);
+      itrCount++;
+    }
+    Assertions.assertThatThrownBy(() -> fsItr.next())
+        .describedAs(
+            "next() should throw NoSuchElementException since next has been "
+                + "called " + TEST_FILES_NUMBER + " times")
+        .isInstanceOf(NoSuchElementException.class);
+    Assertions.assertThat(itrCount)
+        .describedAs("Number of iterations should be equal to the files "
+            + "created")
+        .isEqualTo(TEST_FILES_NUMBER);
+    Assertions.assertThat(fileNames.size())
+        .describedAs("After removing every iterm found from the iterator, "
+            + "there should be no more elements in the fileNames")
+        .isEqualTo(0);
+    verify(listngSupport, Mockito.atLeast(100))
+        .listStatus(any(Path.class), nullable(String.class),
+            anyList(), anyBoolean(),
+            nullable(String.class));
+  }
+
+  @Test
+  public void testWithAbfsIteratorDisabled() throws Exception {
+    Path testDir = createTestDirectory();
+    setPageSize(10);
+    setEnableAbfsIterator(false);
+    final List<String> fileNames = createFilesUnderDirectory(TEST_FILES_NUMBER,
+        testDir, "testListPath");
+
+    RemoteIterator<FileStatus> fsItr =
+        getFileSystem().listStatusIterator(testDir);
+    Assertions.assertThat(fsItr)
+        .describedAs("RemoteIterator should not be instance of "
+            + "AbfsListStatusRemoteIterator when it is disabled")
+        .isNotInstanceOf(AbfsListStatusRemoteIterator.class);
+    int itrCount = 0;
+    while (fsItr.hasNext()) {
+      FileStatus fileStatus = fsItr.next();
+      String pathStr = fileStatus.getPath().toString();
+      fileNames.remove(pathStr);
+      itrCount++;
+    }
+    Assertions.assertThat(itrCount)
+        .describedAs("Number of iterations should be equal to the files "
+            + "created")
+        .isEqualTo(TEST_FILES_NUMBER);
+    Assertions.assertThat(fileNames.size())
+        .describedAs("After removing every iterm found from the iterator, "
+            + "there should be no more elements in the fileNames")
+        .isEqualTo(0);
+  }
+
+  @Test
+  public void testWithAbfsIteratorDisabledWithutHasNext() throws Exception {
+    Path testDir = createTestDirectory();
+    setPageSize(10);
+    setEnableAbfsIterator(false);
+    final List<String> fileNames = createFilesUnderDirectory(TEST_FILES_NUMBER,
+        testDir, "testListPath");
+
+    RemoteIterator<FileStatus> fsItr =
+        getFileSystem().listStatusIterator(testDir);
+    Assertions.assertThat(fsItr)
+        .describedAs("RemoteIterator should not be instance of "
+            + "AbfsListStatusRemoteIterator when it is disabled")
+        .isNotInstanceOf(AbfsListStatusRemoteIterator.class);
+    int itrCount = 0;
+    for (int i = 0; i < TEST_FILES_NUMBER; i++) {
+      FileStatus fileStatus = fsItr.next();
+      String pathStr = fileStatus.getPath().toString();
+      fileNames.remove(pathStr);
+      itrCount++;
+    }
+    Assertions.assertThatThrownBy(() -> fsItr.next())
+        .describedAs(
+            "next() should throw NoSuchElementException since next has been "
+                + "called " + TEST_FILES_NUMBER + " times")
+        .isInstanceOf(NoSuchElementException.class);
+    Assertions.assertThat(itrCount)
+        .describedAs("Number of iterations should be equal to the files "
+            + "created")
+        .isEqualTo(TEST_FILES_NUMBER);
+    Assertions.assertThat(fileNames.size())
+        .describedAs("After removing every iterm found from the iterator, "
+            + "there should be no more elements in the fileNames")
+        .isEqualTo(0);
+  }
+
+  @Test
+  public void testNextWhenNoMoreElementsPresent() throws Exception {
+    Path testDir = createTestDirectory();
+    setPageSize(10);
+    RemoteIterator fsItr =
+        new AbfsListStatusRemoteIterator(getFileSystem().getFileStatus(testDir),
+            getFileSystem().getAbfsStore());
+    fsItr = Mockito.spy(fsItr);
+    Mockito.doReturn(false).when(fsItr).hasNext();
+
+    RemoteIterator<FileStatus> finalFsItr = fsItr;
+    Assertions.assertThatThrownBy(() -> finalFsItr.next())
+        .describedAs(
+        "next() should throw NoSuchElementException if hasNext() return "
+            + "false")
+        .isInstanceOf(NoSuchElementException.class);
+  }
+
+  @Test
+  public void testHasNextForEmptyDir() throws Exception {
+    Path testDir = createTestDirectory();
+    setPageSize(10);
+    RemoteIterator<FileStatus> fsItr = getFileSystem()
+        .listStatusIterator(testDir);
+    Assertions.assertThat(fsItr.hasNext())
+        .describedAs("hasNext returns false for empty directory")
+        .isFalse();
+  }
+
+  @Test
+  public void testHasNextForFile() throws Exception {
+    final AzureBlobFileSystem fs = getFileSystem();
+    String testFileName = "testFile";
+    Path testFile = new Path(testFileName);
+    getFileSystem().create(testFile);
+    setPageSize(10);
+    RemoteIterator<FileStatus> fsItr = fs.listStatusIterator(testFile);
+    Assertions.assertThat(fsItr.hasNext())
+        .describedAs("hasNext returns true for file").isTrue();
+    Assertions.assertThat(fsItr.next().getPath().toString())
+        .describedAs("next returns the file itself")
+        .endsWith(testFileName);
+  }
+
+  @Test
+  public void testIOException() throws Exception {
+    Path testDir = createTestDirectory();
+    setPageSize(10);
+    getFileSystem().mkdirs(testDir);
+
+    String exceptionMessage = "test exception";
+    ListingSupport lsSupport =getMockListingSupport(exceptionMessage);
+    RemoteIterator fsItr =
+        new AbfsListStatusRemoteIterator(getFileSystem().getFileStatus(testDir),
+        lsSupport);
+
+    Assertions.assertThatThrownBy(() -> fsItr.next())
+        .describedAs(
+        "When ioException is not null and queue is empty exception should be "
+            + "thrown")
+        .isInstanceOf(IOException.class)
+        .hasMessage(exceptionMessage);
+  }
+
+  @Test
+  public void testNonExistingPath() throws Throwable {
+    Path nonExistingDir = new Path("nonExistingPath");
+    Assertions.assertThatThrownBy(
+        () -> getFileSystem().listStatusIterator(nonExistingDir)).describedAs(
+        "test the listStatusIterator call on a path which is not "
+            + "present should result in FileNotFoundException")
+        .isInstanceOf(FileNotFoundException.class);
+  }
+
+  private ListingSupport getMockListingSupport(String exceptionMessage) {
+    return new ListingSupport() {
+      @Override
+      public FileStatus[] listStatus(Path path) throws IOException {
+        return null;
+      }
+
+      @Override
+      public FileStatus[] listStatus(Path path, String startFrom)
+          throws IOException {
+        return null;
+      }
+
+      @Override
+      public String listStatus(Path path, String startFrom,
+          List<FileStatus> fileStatuses, boolean fetchAll, String continuation)
+          throws IOException {
+        throw new IOException(exceptionMessage);
+      }
+    };
+  }
+
+  private Path createTestDirectory() throws IOException {
+    String testDirectoryName = "testDirectory" + System.currentTimeMillis();
+    Path testDirectory = new Path(testDirectoryName);
+    getFileSystem().mkdirs(testDirectory);
+    return testDirectory;
+  }
+
+  private void setEnableAbfsIterator(boolean shouldEnable) throws IOException {
+    AzureBlobFileSystemStore abfsStore = getAbfsStore(getFileSystem());
+    abfsStore.getAbfsConfiguration().setEnableAbfsListIterator(shouldEnable);
+  }
+
+  private void setPageSize(int pageSize) throws IOException {
+    AzureBlobFileSystemStore abfsStore = getAbfsStore(getFileSystem());
+    abfsStore.getAbfsConfiguration().setListMaxResults(pageSize);
+  }
+
+  private List<String> createFilesUnderDirectory(int numFiles, Path rootPath,
+      String filenamePrefix)
+      throws ExecutionException, InterruptedException, IOException {
+    final List<Future<Void>> tasks = new ArrayList<>();
+    final List<String> fileNames = new ArrayList<>();
+    ExecutorService es = Executors.newFixedThreadPool(10);
+    for (int i = 0; i < numFiles; i++) {
+      final Path filePath = new Path(rootPath, filenamePrefix + i);
+      Callable<Void> callable = new Callable<Void>() {

Review comment:
       you can just use a () -> { } closure here




----------------------------------------------------------------
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 #2548: HADOOP-17475. ABFS: Implementing ListStatusRemoteIterator

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


   that's fine: key is "findbugs must be happy, sometimes it overreacts -but you need to justify its complaints. Sometimes it's a losing battler with the sync stuff; I had that with the MeanStatistic stuff in IOStatistics and just made everything sync even when it was overkill


----------------------------------------------------------------
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 #2548: HADOOP-17475. ABFS: Implementing ListStatusRemoteIterator

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


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |   1m  2s |  |  Docker mode activated.  |
   |||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  |  No case conflicting files found.  |
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |   |   0m  0s | [test4tests](test4tests) |  The patch appears to include 1 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |  38m 10s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |   0m 41s |  |  trunk passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  compile  |   0m 30s |  |  trunk passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~20.04-b01  |
   | +1 :green_heart: |  checkstyle  |   0m 27s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   0m 37s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  14m 19s |  |  branch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 32s |  |  trunk passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javadoc  |   0m 29s |  |  trunk passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~20.04-b01  |
   | +0 :ok: |  spotbugs  |   0m 59s |  |  Used deprecated FindBugs config; considering switching to SpotBugs.  |
   | +1 :green_heart: |  findbugs  |   0m 58s |  |  trunk passed  |
   | -0 :warning: |  patch  |   1m 16s |  |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   |||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   0m 29s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 28s |  |  the patch passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javac  |   0m 28s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 25s |  |  the patch passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~20.04-b01  |
   | +1 :green_heart: |  javac  |   0m 25s |  |  the patch passed  |
   | +1 :green_heart: |  checkstyle  |   0m 17s |  |  the patch passed  |
   | +1 :green_heart: |  mvnsite  |   0m 28s |  |  the patch passed  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  shadedclient  |  12m 46s |  |  patch has no errors when building and testing our client artifacts.  |
   | -1 :x: |  javadoc  |   0m 26s | [/diff-javadoc-javadoc-hadoop-tools_hadoop-azure-jdkUbuntu-11.0.9.1+1-Ubuntu-0ubuntu1.20.04.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/17/artifact/out/diff-javadoc-javadoc-hadoop-tools_hadoop-azure-jdkUbuntu-11.0.9.1+1-Ubuntu-0ubuntu1.20.04.txt) |  hadoop-tools_hadoop-azure-jdkUbuntu-11.0.9.1+1-Ubuntu-0ubuntu1.20.04 with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.20.04 generated 3 new + 15 unchanged - 2 fixed = 18 total (was 17)  |
   | -1 :x: |  javadoc  |   0m 25s | [/diff-javadoc-javadoc-hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_275-8u275-b01-0ubuntu1~20.04-b01.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/17/artifact/out/diff-javadoc-javadoc-hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_275-8u275-b01-0ubuntu1~20.04-b01.txt) |  hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_275-8u275-b01-0ubuntu1~20.04-b01 with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~20.04-b01 generated 3 new + 15 unchanged - 2 fixed = 18 total (was 17)  |
   | -1 :x: |  findbugs  |   1m  0s | [/new-findbugs-hadoop-tools_hadoop-azure.html](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/17/artifact/out/new-findbugs-hadoop-tools_hadoop-azure.html) |  hadoop-tools/hadoop-azure generated 1 new + 0 unchanged - 0 fixed = 1 total (was 0)  |
   |||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   1m 55s |  |  hadoop-azure in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 32s |  |  The patch does not generate ASF License warnings.  |
   |  |   |  78m 53s |  |  |
   
   
   | Reason | Tests |
   |-------:|:------|
   | FindBugs | module:hadoop-tools/hadoop-azure |
   |  |  Inconsistent synchronization of org.apache.hadoop.fs.azurebfs.services.AbfsListStatusRemoteIterator.continuation; locked 50% of time  Unsynchronized access at AbfsListStatusRemoteIterator.java:50% of time  Unsynchronized access at AbfsListStatusRemoteIterator.java:[line 147] |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/17/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/2548 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient findbugs checkstyle |
   | uname | Linux 4d0b871b529a 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 / 18978f2e204 |
   | Default Java | Private Build-1.8.0_275-8u275-b01-0ubuntu1~20.04-b01 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.20.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_275-8u275-b01-0ubuntu1~20.04-b01 |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/17/testReport/ |
   | Max. process+thread count | 536 (vs. ulimit of 5500) |
   | modules | C: hadoop-tools/hadoop-azure U: hadoop-tools/hadoop-azure |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/17/console |
   | versions | git=2.25.1 maven=3.6.3 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 #2548: HADOOP-17475. ABFS: Implementing ListStatusRemoteIterator

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


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |   1m  3s |  |  Docker mode activated.  |
   |||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  |  No case conflicting files found.  |
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |   |   0m  0s | [test4tests](test4tests) |  The patch appears to include 1 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |  33m 23s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |   0m 38s |  |  trunk passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04  |
   | +1 :green_heart: |  compile  |   0m 33s |  |  trunk passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01  |
   | +1 :green_heart: |  checkstyle  |   0m 28s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   0m 39s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  16m 32s |  |  branch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 32s |  |  trunk passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04  |
   | +1 :green_heart: |  javadoc  |   0m 29s |  |  trunk passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01  |
   | +0 :ok: |  spotbugs  |   0m 59s |  |  Used deprecated FindBugs config; considering switching to SpotBugs.  |
   | +1 :green_heart: |  findbugs  |   0m 58s |  |  trunk passed  |
   | -0 :warning: |  patch  |   1m 19s |  |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   |||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   0m 31s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 29s |  |  the patch passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04  |
   | +1 :green_heart: |  javac  |   0m 29s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 25s |  |  the patch passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01  |
   | +1 :green_heart: |  javac  |   0m 25s |  |  the patch passed  |
   | -0 :warning: |  checkstyle  |   0m 19s | [/diff-checkstyle-hadoop-tools_hadoop-azure.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/12/artifact/out/diff-checkstyle-hadoop-tools_hadoop-azure.txt) |  hadoop-tools/hadoop-azure: The patch generated 5 new + 4 unchanged - 0 fixed = 9 total (was 4)  |
   | +1 :green_heart: |  mvnsite  |   0m 28s |  |  the patch passed  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  shadedclient  |  14m 51s |  |  patch has no errors when building and testing our client artifacts.  |
   | -1 :x: |  javadoc  |   0m 26s | [/diff-javadoc-javadoc-hadoop-tools_hadoop-azure-jdkUbuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/12/artifact/out/diff-javadoc-javadoc-hadoop-tools_hadoop-azure-jdkUbuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04.txt) |  hadoop-tools_hadoop-azure-jdkUbuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04 with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04 generated 3 new + 15 unchanged - 2 fixed = 18 total (was 17)  |
   | -1 :x: |  javadoc  |   0m 25s | [/diff-javadoc-javadoc-hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/12/artifact/out/diff-javadoc-javadoc-hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01.txt) |  hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01 with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01 generated 3 new + 15 unchanged - 2 fixed = 18 total (was 17)  |
   | -1 :x: |  findbugs  |   1m  1s | [/new-findbugs-hadoop-tools_hadoop-azure.html](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/12/artifact/out/new-findbugs-hadoop-tools_hadoop-azure.html) |  hadoop-tools/hadoop-azure generated 2 new + 0 unchanged - 0 fixed = 2 total (was 0)  |
   |||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   1m 26s |  |  hadoop-azure in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 34s |  |  The patch does not generate ASF License warnings.  |
   |  |   |  78m 10s |  |  |
   
   
   | Reason | Tests |
   |-------:|:------|
   | FindBugs | module:hadoop-tools/hadoop-azure |
   |  |  Inconsistent synchronization of org.apache.hadoop.fs.azurebfs.services.AbfsListStatusRemoteIterator.ioException; locked 66% of time  Unsynchronized access at AbfsListStatusRemoteIterator.java:66% of time  Unsynchronized access at AbfsListStatusRemoteIterator.java:[line 127] |
   |  |  Exceptional return value of java.util.concurrent.ArrayBlockingQueue.offer(Object) ignored in org.apache.hadoop.fs.azurebfs.services.AbfsListStatusRemoteIterator.asyncOp()  At AbfsListStatusRemoteIterator.java:ignored in org.apache.hadoop.fs.azurebfs.services.AbfsListStatusRemoteIterator.asyncOp()  At AbfsListStatusRemoteIterator.java:[line 128] |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/12/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/2548 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient findbugs checkstyle |
   | uname | Linux 2798434dd247 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 / 97f843de3a9 |
   | Default Java | Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01 |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/12/testReport/ |
   | Max. process+thread count | 668 (vs. ulimit of 5500) |
   | modules | C: hadoop-tools/hadoop-azure U: hadoop-tools/hadoop-azure |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2548/12/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