You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@druid.apache.org by GitBox <gi...@apache.org> on 2019/12/25 06:09:46 UTC

[GitHub] [druid] gianm opened a new pull request #9098: S3: Improvements to prefix listing (including fix for an infinite loop)

gianm opened a new pull request #9098: S3: Improvements to prefix listing (including fix for an infinite loop)
URL: https://github.com/apache/druid/pull/9098
 
 
   1) Fixes #9097, an infinite loop that occurs when more than one batch
      of objects is retrieved during a prefix listing.
   
   2) Removes the Access Denied fallback code added in #4444. I don't think
      the behavior is reasonable: its purpose is to fall back from a prefix
      listing to a single-object access, but it's only activated when the
      end user supplied a prefix, so it would be better to simply fail, so
      the end user knows that their request for a prefix-based load is not
      going to work. Presumably the end user can switch from supplying
      'prefixes' to supplying 'uris' if desired.
   
   3) Filters out directory placeholders when walking prefixes.
   
   4) Splits LazyObjectSummariesIterator into its own class and adds tests.

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [druid] suneet-s commented on a change in pull request #9098: S3: Improvements to prefix listing (including fix for an infinite loop)

Posted by GitBox <gi...@apache.org>.
suneet-s commented on a change in pull request #9098: S3: Improvements to prefix listing (including fix for an infinite loop)
URL: https://github.com/apache/druid/pull/9098#discussion_r361693808
 
 

 ##########
 File path: extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/LazyObjectSummariesIterator.java
 ##########
 @@ -0,0 +1,162 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.storage.s3;
+
+import com.amazonaws.services.s3.model.ListObjectsV2Request;
+import com.amazonaws.services.s3.model.ListObjectsV2Result;
+import com.amazonaws.services.s3.model.S3ObjectSummary;
+import org.apache.druid.java.util.common.RE;
+
+import java.net.URI;
+import java.util.Iterator;
+import java.util.NoSuchElementException;
+
+/**
+ * Iterator class used by {@link S3Utils#lazyObjectSummaryIterator}.
+ *
+ * Walks a set of prefixes, returning all objects underneath them except for directory placeholders.
+ */
+public class LazyObjectSummariesIterator implements Iterator<S3ObjectSummary>
+{
+  private final ServerSideEncryptingAmazonS3 s3Client;
+  private final Iterator<URI> prefixesIterator;
+  private final int maxListingLength;
+
+  private ListObjectsV2Request request;
+  private ListObjectsV2Result result;
+  private Iterator<S3ObjectSummary> objectSummaryIterator;
+  private S3ObjectSummary currentObjectSummary;
+
+  LazyObjectSummariesIterator(
+      final ServerSideEncryptingAmazonS3 s3Client,
+      final Iterable<URI> prefixes,
+      final int maxListingLength
+  )
+  {
+    this.s3Client = s3Client;
+    this.prefixesIterator = prefixes.iterator();
+    this.maxListingLength = maxListingLength;
+
+    prepareNextRequest();
+    fetchNextBatch();
+    advanceObjectSummary();
+  }
+
+  private void prepareNextRequest()
+  {
+    final URI currentUri = prefixesIterator.next();
+    final String currentBucket = currentUri.getAuthority();
+    final String currentPrefix = S3Utils.extractS3Key(currentUri);
+
+    request = new ListObjectsV2Request()
+        .withBucketName(currentBucket)
+        .withPrefix(currentPrefix)
+        .withMaxKeys(maxListingLength);
+  }
+
+  private void fetchNextBatch()
+  {
+    try {
+      result = S3Utils.retryS3Operation(() -> s3Client.listObjectsV2(request));
+      request.setContinuationToken(result.getNextContinuationToken());
+      objectSummaryIterator = result.getObjectSummaries().iterator();
+    }
+    catch (Exception e) {
+      throw new RE(
+          e,
+          "Failed to get object summaries from S3 bucket[%s], prefix[%s]",
+          request.getBucketName(),
+          request.getPrefix()
+      );
+    }
+  }
+
+  /**
+   * Advance objectSummaryIterator to the next non-placeholder, updating "currentObjectSummary".
+   */
+  private void advanceObjectSummary()
+  {
+    while (objectSummaryIterator.hasNext() || result.isTruncated() || prefixesIterator.hasNext()) {
+      while (objectSummaryIterator.hasNext()) {
+        currentObjectSummary = objectSummaryIterator.next();
+
+        if (!isDirectoryPlaceholder(currentObjectSummary)) {
+          return;
+        }
+      }
+
+      // Exhausted "objectSummaryIterator" without finding a non-placeholder.
+      if (result.isTruncated()) {
+        fetchNextBatch();
+      } else if (prefixesIterator.hasNext()) {
+        prepareNextRequest();
+        fetchNextBatch();
+      }
+    }
+
+    // Truly nothing left to read.
+    currentObjectSummary = null;
+  }
+
+  @Override
+  public boolean hasNext()
+  {
+    return currentObjectSummary != null;
+  }
+
+  @Override
+  public S3ObjectSummary next()
+  {
+    if (currentObjectSummary == null) {
+      throw new NoSuchElementException();
+    }
+
+    final S3ObjectSummary retVal = currentObjectSummary;
+    advanceObjectSummary();
+    return retVal;
+  }
+
+  /**
+   * Checks if a given object is a directory placeholder and should be ignored.
+   *
+   * Adapted from org.jets3t.service.model.StorageObject.isDirectoryPlaceholder(). Does not include the check for
+   * legacy JetS3t directory placeholder objects, since it is based on content-type, which isn't available in an
+   * S3ObjectSummary.
+   */
+  private static boolean isDirectoryPlaceholder(final S3ObjectSummary objectSummary)
+  {
+    // Recognize "standard" directory place-holder indications used by Amazon's AWS Console and Panic's Transmit.
+    if (objectSummary.getKey().endsWith("/") && objectSummary.getSize() == 0) {
+      return true;
+    }
+
+    // Recognize s3sync.rb directory placeholders by MD5/ETag value.
+    if ("d66759af42f282e1ba19144df2d405d0".equals(objectSummary.getETag())) {
 
 Review comment:
   nit: define magic constant at the top of class - did you find this from here - https://github.com/stephenh/s3fsr/blob/master/lib/s3fsr.rb#L116

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [druid] gianm merged pull request #9098: S3: Improvements to prefix listing (including fix for an infinite loop)

Posted by GitBox <gi...@apache.org>.
gianm merged pull request #9098: S3: Improvements to prefix listing (including fix for an infinite loop)
URL: https://github.com/apache/druid/pull/9098
 
 
   

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [druid] gianm commented on a change in pull request #9098: S3: Improvements to prefix listing (including fix for an infinite loop)

Posted by GitBox <gi...@apache.org>.
gianm commented on a change in pull request #9098: S3: Improvements to prefix listing (including fix for an infinite loop)
URL: https://github.com/apache/druid/pull/9098#discussion_r362067759
 
 

 ##########
 File path: extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/LazyObjectSummariesIterator.java
 ##########
 @@ -0,0 +1,162 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.storage.s3;
+
+import com.amazonaws.services.s3.model.ListObjectsV2Request;
+import com.amazonaws.services.s3.model.ListObjectsV2Result;
+import com.amazonaws.services.s3.model.S3ObjectSummary;
+import org.apache.druid.java.util.common.RE;
+
+import java.net.URI;
+import java.util.Iterator;
+import java.util.NoSuchElementException;
+
+/**
+ * Iterator class used by {@link S3Utils#lazyObjectSummaryIterator}.
+ *
+ * Walks a set of prefixes, returning all objects underneath them except for directory placeholders.
+ */
+public class LazyObjectSummariesIterator implements Iterator<S3ObjectSummary>
+{
+  private final ServerSideEncryptingAmazonS3 s3Client;
+  private final Iterator<URI> prefixesIterator;
+  private final int maxListingLength;
+
+  private ListObjectsV2Request request;
+  private ListObjectsV2Result result;
+  private Iterator<S3ObjectSummary> objectSummaryIterator;
+  private S3ObjectSummary currentObjectSummary;
+
+  LazyObjectSummariesIterator(
+      final ServerSideEncryptingAmazonS3 s3Client,
+      final Iterable<URI> prefixes,
+      final int maxListingLength
+  )
+  {
+    this.s3Client = s3Client;
+    this.prefixesIterator = prefixes.iterator();
+    this.maxListingLength = maxListingLength;
+
+    prepareNextRequest();
+    fetchNextBatch();
+    advanceObjectSummary();
+  }
+
+  private void prepareNextRequest()
+  {
+    final URI currentUri = prefixesIterator.next();
+    final String currentBucket = currentUri.getAuthority();
+    final String currentPrefix = S3Utils.extractS3Key(currentUri);
+
+    request = new ListObjectsV2Request()
+        .withBucketName(currentBucket)
+        .withPrefix(currentPrefix)
+        .withMaxKeys(maxListingLength);
+  }
+
+  private void fetchNextBatch()
+  {
+    try {
+      result = S3Utils.retryS3Operation(() -> s3Client.listObjectsV2(request));
+      request.setContinuationToken(result.getNextContinuationToken());
+      objectSummaryIterator = result.getObjectSummaries().iterator();
+    }
+    catch (Exception e) {
+      throw new RE(
+          e,
+          "Failed to get object summaries from S3 bucket[%s], prefix[%s]",
+          request.getBucketName(),
+          request.getPrefix()
+      );
+    }
+  }
+
+  /**
+   * Advance objectSummaryIterator to the next non-placeholder, updating "currentObjectSummary".
+   */
+  private void advanceObjectSummary()
+  {
+    while (objectSummaryIterator.hasNext() || result.isTruncated() || prefixesIterator.hasNext()) {
+      while (objectSummaryIterator.hasNext()) {
+        currentObjectSummary = objectSummaryIterator.next();
+
+        if (!isDirectoryPlaceholder(currentObjectSummary)) {
+          return;
+        }
+      }
+
+      // Exhausted "objectSummaryIterator" without finding a non-placeholder.
+      if (result.isTruncated()) {
+        fetchNextBatch();
+      } else if (prefixesIterator.hasNext()) {
+        prepareNextRequest();
+        fetchNextBatch();
+      }
+    }
+
+    // Truly nothing left to read.
+    currentObjectSummary = null;
+  }
+
+  @Override
+  public boolean hasNext()
+  {
+    return currentObjectSummary != null;
+  }
+
+  @Override
+  public S3ObjectSummary next()
+  {
+    if (currentObjectSummary == null) {
+      throw new NoSuchElementException();
+    }
+
+    final S3ObjectSummary retVal = currentObjectSummary;
+    advanceObjectSummary();
+    return retVal;
+  }
+
+  /**
+   * Checks if a given object is a directory placeholder and should be ignored.
+   *
+   * Adapted from org.jets3t.service.model.StorageObject.isDirectoryPlaceholder(). Does not include the check for
+   * legacy JetS3t directory placeholder objects, since it is based on content-type, which isn't available in an
+   * S3ObjectSummary.
+   */
+  private static boolean isDirectoryPlaceholder(final S3ObjectSummary objectSummary)
+  {
+    // Recognize "standard" directory place-holder indications used by Amazon's AWS Console and Panic's Transmit.
+    if (objectSummary.getKey().endsWith("/") && objectSummary.getSize() == 0) {
+      return true;
+    }
+
+    // Recognize s3sync.rb directory placeholders by MD5/ETag value.
+    if ("d66759af42f282e1ba19144df2d405d0".equals(objectSummary.getETag())) {
 
 Review comment:
   In that case, I'll leave it as is. Thanks for taking a look.

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [druid] suneet-s commented on a change in pull request #9098: S3: Improvements to prefix listing (including fix for an infinite loop)

Posted by GitBox <gi...@apache.org>.
suneet-s commented on a change in pull request #9098: S3: Improvements to prefix listing (including fix for an infinite loop)
URL: https://github.com/apache/druid/pull/9098#discussion_r361689896
 
 

 ##########
 File path: extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/LazyObjectSummariesIterator.java
 ##########
 @@ -0,0 +1,162 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.storage.s3;
+
+import com.amazonaws.services.s3.model.ListObjectsV2Request;
+import com.amazonaws.services.s3.model.ListObjectsV2Result;
+import com.amazonaws.services.s3.model.S3ObjectSummary;
+import org.apache.druid.java.util.common.RE;
+
+import java.net.URI;
+import java.util.Iterator;
+import java.util.NoSuchElementException;
+
+/**
+ * Iterator class used by {@link S3Utils#lazyObjectSummaryIterator}.
+ *
+ * Walks a set of prefixes, returning all objects underneath them except for directory placeholders.
+ */
+public class LazyObjectSummariesIterator implements Iterator<S3ObjectSummary>
+{
+  private final ServerSideEncryptingAmazonS3 s3Client;
+  private final Iterator<URI> prefixesIterator;
+  private final int maxListingLength;
+
+  private ListObjectsV2Request request;
+  private ListObjectsV2Result result;
+  private Iterator<S3ObjectSummary> objectSummaryIterator;
+  private S3ObjectSummary currentObjectSummary;
+
+  LazyObjectSummariesIterator(
+      final ServerSideEncryptingAmazonS3 s3Client,
+      final Iterable<URI> prefixes,
+      final int maxListingLength
+  )
+  {
+    this.s3Client = s3Client;
+    this.prefixesIterator = prefixes.iterator();
+    this.maxListingLength = maxListingLength;
+
+    prepareNextRequest();
+    fetchNextBatch();
+    advanceObjectSummary();
 
 Review comment:
   Does it matter if we make the first fetch in the constructor when the class indicates it is lazy?

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [druid] suneet-s commented on a change in pull request #9098: S3: Improvements to prefix listing (including fix for an infinite loop)

Posted by GitBox <gi...@apache.org>.
suneet-s commented on a change in pull request #9098: S3: Improvements to prefix listing (including fix for an infinite loop)
URL: https://github.com/apache/druid/pull/9098#discussion_r361689094
 
 

 ##########
 File path: extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/LazyObjectSummariesIterator.java
 ##########
 @@ -0,0 +1,162 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.storage.s3;
+
+import com.amazonaws.services.s3.model.ListObjectsV2Request;
+import com.amazonaws.services.s3.model.ListObjectsV2Result;
+import com.amazonaws.services.s3.model.S3ObjectSummary;
+import org.apache.druid.java.util.common.RE;
+
+import java.net.URI;
+import java.util.Iterator;
+import java.util.NoSuchElementException;
+
+/**
+ * Iterator class used by {@link S3Utils#lazyObjectSummaryIterator}.
+ *
+ * Walks a set of prefixes, returning all objects underneath them except for directory placeholders.
+ */
+public class LazyObjectSummariesIterator implements Iterator<S3ObjectSummary>
+{
+  private final ServerSideEncryptingAmazonS3 s3Client;
+  private final Iterator<URI> prefixesIterator;
+  private final int maxListingLength;
+
+  private ListObjectsV2Request request;
+  private ListObjectsV2Result result;
+  private Iterator<S3ObjectSummary> objectSummaryIterator;
+  private S3ObjectSummary currentObjectSummary;
+
+  LazyObjectSummariesIterator(
+      final ServerSideEncryptingAmazonS3 s3Client,
+      final Iterable<URI> prefixes,
+      final int maxListingLength
+  )
+  {
+    this.s3Client = s3Client;
+    this.prefixesIterator = prefixes.iterator();
+    this.maxListingLength = maxListingLength;
+
+    prepareNextRequest();
+    fetchNextBatch();
+    advanceObjectSummary();
+  }
+
+  private void prepareNextRequest()
+  {
+    final URI currentUri = prefixesIterator.next();
+    final String currentBucket = currentUri.getAuthority();
+    final String currentPrefix = S3Utils.extractS3Key(currentUri);
+
+    request = new ListObjectsV2Request()
+        .withBucketName(currentBucket)
+        .withPrefix(currentPrefix)
+        .withMaxKeys(maxListingLength);
+  }
+
+  private void fetchNextBatch()
+  {
+    try {
+      result = S3Utils.retryS3Operation(() -> s3Client.listObjectsV2(request));
+      request.setContinuationToken(result.getNextContinuationToken());
+      objectSummaryIterator = result.getObjectSummaries().iterator();
+    }
+    catch (Exception e) {
+      throw new RE(
+          e,
+          "Failed to get object summaries from S3 bucket[%s], prefix[%s]",
+          request.getBucketName(),
+          request.getPrefix()
+      );
+    }
+  }
+
+  /**
+   * Advance objectSummaryIterator to the next non-placeholder, updating "currentObjectSummary".
+   */
+  private void advanceObjectSummary()
+  {
+    while (objectSummaryIterator.hasNext() || result.isTruncated() || prefixesIterator.hasNext()) {
+      while (objectSummaryIterator.hasNext()) {
+        currentObjectSummary = objectSummaryIterator.next();
+
+        if (!isDirectoryPlaceholder(currentObjectSummary)) {
+          return;
+        }
+      }
+
+      // Exhausted "objectSummaryIterator" without finding a non-placeholder.
+      if (result.isTruncated()) {
+        fetchNextBatch();
+      } else if (prefixesIterator.hasNext()) {
+        prepareNextRequest();
+        fetchNextBatch();
+      }
+    }
+
+    // Truly nothing left to read.
+    currentObjectSummary = null;
+  }
+
+  @Override
+  public boolean hasNext()
+  {
+    return currentObjectSummary != null;
+  }
+
+  @Override
+  public S3ObjectSummary next()
+  {
+    if (currentObjectSummary == null) {
+      throw new NoSuchElementException();
+    }
+
+    final S3ObjectSummary retVal = currentObjectSummary;
+    advanceObjectSummary();
+    return retVal;
+  }
 
 Review comment:
   nit: public Override functions after the constructor, before private functions to help with readability 

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [druid] suneet-s commented on a change in pull request #9098: S3: Improvements to prefix listing (including fix for an infinite loop)

Posted by GitBox <gi...@apache.org>.
suneet-s commented on a change in pull request #9098: S3: Improvements to prefix listing (including fix for an infinite loop)
URL: https://github.com/apache/druid/pull/9098#discussion_r362041585
 
 

 ##########
 File path: extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/LazyObjectSummariesIterator.java
 ##########
 @@ -0,0 +1,162 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.storage.s3;
+
+import com.amazonaws.services.s3.model.ListObjectsV2Request;
+import com.amazonaws.services.s3.model.ListObjectsV2Result;
+import com.amazonaws.services.s3.model.S3ObjectSummary;
+import org.apache.druid.java.util.common.RE;
+
+import java.net.URI;
+import java.util.Iterator;
+import java.util.NoSuchElementException;
+
+/**
+ * Iterator class used by {@link S3Utils#lazyObjectSummaryIterator}.
+ *
+ * Walks a set of prefixes, returning all objects underneath them except for directory placeholders.
+ */
+public class LazyObjectSummariesIterator implements Iterator<S3ObjectSummary>
+{
+  private final ServerSideEncryptingAmazonS3 s3Client;
+  private final Iterator<URI> prefixesIterator;
+  private final int maxListingLength;
+
+  private ListObjectsV2Request request;
+  private ListObjectsV2Result result;
+  private Iterator<S3ObjectSummary> objectSummaryIterator;
+  private S3ObjectSummary currentObjectSummary;
+
+  LazyObjectSummariesIterator(
+      final ServerSideEncryptingAmazonS3 s3Client,
+      final Iterable<URI> prefixes,
+      final int maxListingLength
+  )
+  {
+    this.s3Client = s3Client;
+    this.prefixesIterator = prefixes.iterator();
+    this.maxListingLength = maxListingLength;
+
+    prepareNextRequest();
+    fetchNextBatch();
+    advanceObjectSummary();
+  }
+
+  private void prepareNextRequest()
+  {
+    final URI currentUri = prefixesIterator.next();
+    final String currentBucket = currentUri.getAuthority();
+    final String currentPrefix = S3Utils.extractS3Key(currentUri);
+
+    request = new ListObjectsV2Request()
+        .withBucketName(currentBucket)
+        .withPrefix(currentPrefix)
+        .withMaxKeys(maxListingLength);
+  }
+
+  private void fetchNextBatch()
+  {
+    try {
+      result = S3Utils.retryS3Operation(() -> s3Client.listObjectsV2(request));
+      request.setContinuationToken(result.getNextContinuationToken());
+      objectSummaryIterator = result.getObjectSummaries().iterator();
+    }
+    catch (Exception e) {
+      throw new RE(
+          e,
+          "Failed to get object summaries from S3 bucket[%s], prefix[%s]",
+          request.getBucketName(),
+          request.getPrefix()
+      );
+    }
+  }
+
+  /**
+   * Advance objectSummaryIterator to the next non-placeholder, updating "currentObjectSummary".
+   */
+  private void advanceObjectSummary()
+  {
+    while (objectSummaryIterator.hasNext() || result.isTruncated() || prefixesIterator.hasNext()) {
+      while (objectSummaryIterator.hasNext()) {
+        currentObjectSummary = objectSummaryIterator.next();
+
+        if (!isDirectoryPlaceholder(currentObjectSummary)) {
+          return;
+        }
+      }
+
+      // Exhausted "objectSummaryIterator" without finding a non-placeholder.
+      if (result.isTruncated()) {
+        fetchNextBatch();
+      } else if (prefixesIterator.hasNext()) {
+        prepareNextRequest();
+        fetchNextBatch();
+      }
+    }
+
+    // Truly nothing left to read.
+    currentObjectSummary = null;
+  }
+
+  @Override
+  public boolean hasNext()
+  {
+    return currentObjectSummary != null;
+  }
+
+  @Override
+  public S3ObjectSummary next()
+  {
+    if (currentObjectSummary == null) {
+      throw new NoSuchElementException();
+    }
+
+    final S3ObjectSummary retVal = currentObjectSummary;
+    advanceObjectSummary();
+    return retVal;
+  }
+
+  /**
+   * Checks if a given object is a directory placeholder and should be ignored.
+   *
+   * Adapted from org.jets3t.service.model.StorageObject.isDirectoryPlaceholder(). Does not include the check for
+   * legacy JetS3t directory placeholder objects, since it is based on content-type, which isn't available in an
+   * S3ObjectSummary.
+   */
+  private static boolean isDirectoryPlaceholder(final S3ObjectSummary objectSummary)
+  {
+    // Recognize "standard" directory place-holder indications used by Amazon's AWS Console and Panic's Transmit.
+    if (objectSummary.getKey().endsWith("/") && objectSummary.getSize() == 0) {
+      return true;
+    }
+
+    // Recognize s3sync.rb directory placeholders by MD5/ETag value.
+    if ("d66759af42f282e1ba19144df2d405d0".equals(objectSummary.getETag())) {
 
 Review comment:
   Wow I feel embarrassed for not reading that comment 😶
   
   Given that we're close to a release, I'd err on the side of caution and just leave this as you have it. I don't see this creating a maintenance burden, so the benefits for removing it is minimal.

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [druid] gianm commented on a change in pull request #9098: S3: Improvements to prefix listing (including fix for an infinite loop)

Posted by GitBox <gi...@apache.org>.
gianm commented on a change in pull request #9098: S3: Improvements to prefix listing (including fix for an infinite loop)
URL: https://github.com/apache/druid/pull/9098#discussion_r362036658
 
 

 ##########
 File path: extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/LazyObjectSummariesIterator.java
 ##########
 @@ -0,0 +1,162 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.storage.s3;
+
+import com.amazonaws.services.s3.model.ListObjectsV2Request;
+import com.amazonaws.services.s3.model.ListObjectsV2Result;
+import com.amazonaws.services.s3.model.S3ObjectSummary;
+import org.apache.druid.java.util.common.RE;
+
+import java.net.URI;
+import java.util.Iterator;
+import java.util.NoSuchElementException;
+
+/**
+ * Iterator class used by {@link S3Utils#lazyObjectSummaryIterator}.
+ *
+ * Walks a set of prefixes, returning all objects underneath them except for directory placeholders.
+ */
+public class LazyObjectSummariesIterator implements Iterator<S3ObjectSummary>
+{
+  private final ServerSideEncryptingAmazonS3 s3Client;
+  private final Iterator<URI> prefixesIterator;
+  private final int maxListingLength;
+
+  private ListObjectsV2Request request;
+  private ListObjectsV2Result result;
+  private Iterator<S3ObjectSummary> objectSummaryIterator;
+  private S3ObjectSummary currentObjectSummary;
+
+  LazyObjectSummariesIterator(
+      final ServerSideEncryptingAmazonS3 s3Client,
+      final Iterable<URI> prefixes,
+      final int maxListingLength
+  )
+  {
+    this.s3Client = s3Client;
+    this.prefixesIterator = prefixes.iterator();
+    this.maxListingLength = maxListingLength;
+
+    prepareNextRequest();
+    fetchNextBatch();
+    advanceObjectSummary();
+  }
+
+  private void prepareNextRequest()
+  {
+    final URI currentUri = prefixesIterator.next();
+    final String currentBucket = currentUri.getAuthority();
+    final String currentPrefix = S3Utils.extractS3Key(currentUri);
+
+    request = new ListObjectsV2Request()
+        .withBucketName(currentBucket)
+        .withPrefix(currentPrefix)
+        .withMaxKeys(maxListingLength);
+  }
+
+  private void fetchNextBatch()
+  {
+    try {
+      result = S3Utils.retryS3Operation(() -> s3Client.listObjectsV2(request));
+      request.setContinuationToken(result.getNextContinuationToken());
+      objectSummaryIterator = result.getObjectSummaries().iterator();
+    }
+    catch (Exception e) {
+      throw new RE(
+          e,
+          "Failed to get object summaries from S3 bucket[%s], prefix[%s]",
+          request.getBucketName(),
+          request.getPrefix()
+      );
+    }
+  }
+
+  /**
+   * Advance objectSummaryIterator to the next non-placeholder, updating "currentObjectSummary".
+   */
+  private void advanceObjectSummary()
+  {
+    while (objectSummaryIterator.hasNext() || result.isTruncated() || prefixesIterator.hasNext()) {
+      while (objectSummaryIterator.hasNext()) {
+        currentObjectSummary = objectSummaryIterator.next();
+
+        if (!isDirectoryPlaceholder(currentObjectSummary)) {
+          return;
+        }
+      }
+
+      // Exhausted "objectSummaryIterator" without finding a non-placeholder.
+      if (result.isTruncated()) {
+        fetchNextBatch();
+      } else if (prefixesIterator.hasNext()) {
+        prepareNextRequest();
+        fetchNextBatch();
+      }
+    }
+
+    // Truly nothing left to read.
+    currentObjectSummary = null;
+  }
+
+  @Override
+  public boolean hasNext()
+  {
+    return currentObjectSummary != null;
+  }
+
+  @Override
+  public S3ObjectSummary next()
+  {
+    if (currentObjectSummary == null) {
+      throw new NoSuchElementException();
+    }
+
+    final S3ObjectSummary retVal = currentObjectSummary;
+    advanceObjectSummary();
+    return retVal;
+  }
+
+  /**
+   * Checks if a given object is a directory placeholder and should be ignored.
+   *
+   * Adapted from org.jets3t.service.model.StorageObject.isDirectoryPlaceholder(). Does not include the check for
+   * legacy JetS3t directory placeholder objects, since it is based on content-type, which isn't available in an
+   * S3ObjectSummary.
+   */
+  private static boolean isDirectoryPlaceholder(final S3ObjectSummary objectSummary)
+  {
+    // Recognize "standard" directory place-holder indications used by Amazon's AWS Console and Panic's Transmit.
+    if (objectSummary.getKey().endsWith("/") && objectSummary.getSize() == 0) {
+      return true;
+    }
+
+    // Recognize s3sync.rb directory placeholders by MD5/ETag value.
+    if ("d66759af42f282e1ba19144df2d405d0".equals(objectSummary.getETag())) {
 
 Review comment:
   It's from `org.jets3t.service.model.StorageObject.isDirectoryPlaceholder` (mentioned in the javadoc for this method). Sources are at https://github.com/mondain/jets3t/blob/master/jets3t/src/main/java/org/jets3t/service/model/StorageObject.java.
   
   I did it this way for these two reasons:
   
   - Minimal diff from the original function, in case someone wants to compare them visually and see what we changed.
   - I didn't see a benefit to putting `"d66759af42f282e1ba19144df2d405d0"` into a constant: it won't change, it won't be used in any other locations, and it's adequately explained by the comment above the line it appears in, so the usual benefits of pulling magic values out into constants don't apply.
   
   Although actually, maybe this is moot, since it might be reasonable to just remove this block. I don't think we have any particular need to support the `s3sync.rb` style directory placeholders. The other two kinds are much more common. It looks like `s3sync.rb` hasn't been maintained in many years, according to http://s3sync.net/wiki.html. I am pretty ambivalent on this.
   
   What do you think is best?

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [druid] gianm commented on a change in pull request #9098: S3: Improvements to prefix listing (including fix for an infinite loop)

Posted by GitBox <gi...@apache.org>.
gianm commented on a change in pull request #9098: S3: Improvements to prefix listing (including fix for an infinite loop)
URL: https://github.com/apache/druid/pull/9098#discussion_r362036658
 
 

 ##########
 File path: extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/LazyObjectSummariesIterator.java
 ##########
 @@ -0,0 +1,162 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.storage.s3;
+
+import com.amazonaws.services.s3.model.ListObjectsV2Request;
+import com.amazonaws.services.s3.model.ListObjectsV2Result;
+import com.amazonaws.services.s3.model.S3ObjectSummary;
+import org.apache.druid.java.util.common.RE;
+
+import java.net.URI;
+import java.util.Iterator;
+import java.util.NoSuchElementException;
+
+/**
+ * Iterator class used by {@link S3Utils#lazyObjectSummaryIterator}.
+ *
+ * Walks a set of prefixes, returning all objects underneath them except for directory placeholders.
+ */
+public class LazyObjectSummariesIterator implements Iterator<S3ObjectSummary>
+{
+  private final ServerSideEncryptingAmazonS3 s3Client;
+  private final Iterator<URI> prefixesIterator;
+  private final int maxListingLength;
+
+  private ListObjectsV2Request request;
+  private ListObjectsV2Result result;
+  private Iterator<S3ObjectSummary> objectSummaryIterator;
+  private S3ObjectSummary currentObjectSummary;
+
+  LazyObjectSummariesIterator(
+      final ServerSideEncryptingAmazonS3 s3Client,
+      final Iterable<URI> prefixes,
+      final int maxListingLength
+  )
+  {
+    this.s3Client = s3Client;
+    this.prefixesIterator = prefixes.iterator();
+    this.maxListingLength = maxListingLength;
+
+    prepareNextRequest();
+    fetchNextBatch();
+    advanceObjectSummary();
+  }
+
+  private void prepareNextRequest()
+  {
+    final URI currentUri = prefixesIterator.next();
+    final String currentBucket = currentUri.getAuthority();
+    final String currentPrefix = S3Utils.extractS3Key(currentUri);
+
+    request = new ListObjectsV2Request()
+        .withBucketName(currentBucket)
+        .withPrefix(currentPrefix)
+        .withMaxKeys(maxListingLength);
+  }
+
+  private void fetchNextBatch()
+  {
+    try {
+      result = S3Utils.retryS3Operation(() -> s3Client.listObjectsV2(request));
+      request.setContinuationToken(result.getNextContinuationToken());
+      objectSummaryIterator = result.getObjectSummaries().iterator();
+    }
+    catch (Exception e) {
+      throw new RE(
+          e,
+          "Failed to get object summaries from S3 bucket[%s], prefix[%s]",
+          request.getBucketName(),
+          request.getPrefix()
+      );
+    }
+  }
+
+  /**
+   * Advance objectSummaryIterator to the next non-placeholder, updating "currentObjectSummary".
+   */
+  private void advanceObjectSummary()
+  {
+    while (objectSummaryIterator.hasNext() || result.isTruncated() || prefixesIterator.hasNext()) {
+      while (objectSummaryIterator.hasNext()) {
+        currentObjectSummary = objectSummaryIterator.next();
+
+        if (!isDirectoryPlaceholder(currentObjectSummary)) {
+          return;
+        }
+      }
+
+      // Exhausted "objectSummaryIterator" without finding a non-placeholder.
+      if (result.isTruncated()) {
+        fetchNextBatch();
+      } else if (prefixesIterator.hasNext()) {
+        prepareNextRequest();
+        fetchNextBatch();
+      }
+    }
+
+    // Truly nothing left to read.
+    currentObjectSummary = null;
+  }
+
+  @Override
+  public boolean hasNext()
+  {
+    return currentObjectSummary != null;
+  }
+
+  @Override
+  public S3ObjectSummary next()
+  {
+    if (currentObjectSummary == null) {
+      throw new NoSuchElementException();
+    }
+
+    final S3ObjectSummary retVal = currentObjectSummary;
+    advanceObjectSummary();
+    return retVal;
+  }
+
+  /**
+   * Checks if a given object is a directory placeholder and should be ignored.
+   *
+   * Adapted from org.jets3t.service.model.StorageObject.isDirectoryPlaceholder(). Does not include the check for
+   * legacy JetS3t directory placeholder objects, since it is based on content-type, which isn't available in an
+   * S3ObjectSummary.
+   */
+  private static boolean isDirectoryPlaceholder(final S3ObjectSummary objectSummary)
+  {
+    // Recognize "standard" directory place-holder indications used by Amazon's AWS Console and Panic's Transmit.
+    if (objectSummary.getKey().endsWith("/") && objectSummary.getSize() == 0) {
+      return true;
+    }
+
+    // Recognize s3sync.rb directory placeholders by MD5/ETag value.
+    if ("d66759af42f282e1ba19144df2d405d0".equals(objectSummary.getETag())) {
 
 Review comment:
   It's from `org.jets3t.service.model.StorageObject.isDirectoryPlaceholder` (mentioned in the javadoc for this method). Sources are at https://github.com/mondain/jets3t/blob/master/jets3t/src/main/java/org/jets3t/service/model/StorageObject.java.
   
   I did it this way for these two reasons:
   
   - Minimal diff from the original function, in case someone wants to compare them visually and see what we changed.
   - I didn't see a benefit to putting `"d66759af42f282e1ba19144df2d405d0"` into a constant: it won't change, it won't be used in any other locations, and it's adequately explained by the comment above the line it appears in, so the usual benefits of pulling magic values out into constants don't apply.
   
   Although actually, maybe this is moot, since it might be reasonable to just remove this block. I don't think we have any particular need to support the `s3sync.rb` style directory placeholders. The other two kinds are much more common. It looks like `s3sync.rb` hasn't been maintained in many years, according to http://s3sync.net/wiki.html.
   
   What do you think is best?

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [druid] gianm commented on a change in pull request #9098: S3: Improvements to prefix listing (including fix for an infinite loop)

Posted by GitBox <gi...@apache.org>.
gianm commented on a change in pull request #9098: S3: Improvements to prefix listing (including fix for an infinite loop)
URL: https://github.com/apache/druid/pull/9098#discussion_r362031593
 
 

 ##########
 File path: extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/LazyObjectSummariesIterator.java
 ##########
 @@ -0,0 +1,162 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.storage.s3;
+
+import com.amazonaws.services.s3.model.ListObjectsV2Request;
+import com.amazonaws.services.s3.model.ListObjectsV2Result;
+import com.amazonaws.services.s3.model.S3ObjectSummary;
+import org.apache.druid.java.util.common.RE;
+
+import java.net.URI;
+import java.util.Iterator;
+import java.util.NoSuchElementException;
+
+/**
+ * Iterator class used by {@link S3Utils#lazyObjectSummaryIterator}.
+ *
+ * Walks a set of prefixes, returning all objects underneath them except for directory placeholders.
+ */
+public class LazyObjectSummariesIterator implements Iterator<S3ObjectSummary>
+{
+  private final ServerSideEncryptingAmazonS3 s3Client;
+  private final Iterator<URI> prefixesIterator;
+  private final int maxListingLength;
+
+  private ListObjectsV2Request request;
+  private ListObjectsV2Result result;
+  private Iterator<S3ObjectSummary> objectSummaryIterator;
+  private S3ObjectSummary currentObjectSummary;
+
+  LazyObjectSummariesIterator(
+      final ServerSideEncryptingAmazonS3 s3Client,
+      final Iterable<URI> prefixes,
+      final int maxListingLength
+  )
+  {
+    this.s3Client = s3Client;
+    this.prefixesIterator = prefixes.iterator();
+    this.maxListingLength = maxListingLength;
+
+    prepareNextRequest();
+    fetchNextBatch();
+    advanceObjectSummary();
+  }
+
+  private void prepareNextRequest()
+  {
+    final URI currentUri = prefixesIterator.next();
+    final String currentBucket = currentUri.getAuthority();
+    final String currentPrefix = S3Utils.extractS3Key(currentUri);
+
+    request = new ListObjectsV2Request()
+        .withBucketName(currentBucket)
+        .withPrefix(currentPrefix)
+        .withMaxKeys(maxListingLength);
+  }
+
+  private void fetchNextBatch()
+  {
+    try {
+      result = S3Utils.retryS3Operation(() -> s3Client.listObjectsV2(request));
+      request.setContinuationToken(result.getNextContinuationToken());
+      objectSummaryIterator = result.getObjectSummaries().iterator();
+    }
+    catch (Exception e) {
+      throw new RE(
+          e,
+          "Failed to get object summaries from S3 bucket[%s], prefix[%s]",
+          request.getBucketName(),
+          request.getPrefix()
+      );
+    }
+  }
+
+  /**
+   * Advance objectSummaryIterator to the next non-placeholder, updating "currentObjectSummary".
+   */
+  private void advanceObjectSummary()
+  {
+    while (objectSummaryIterator.hasNext() || result.isTruncated() || prefixesIterator.hasNext()) {
+      while (objectSummaryIterator.hasNext()) {
+        currentObjectSummary = objectSummaryIterator.next();
+
+        if (!isDirectoryPlaceholder(currentObjectSummary)) {
+          return;
+        }
+      }
+
+      // Exhausted "objectSummaryIterator" without finding a non-placeholder.
+      if (result.isTruncated()) {
+        fetchNextBatch();
+      } else if (prefixesIterator.hasNext()) {
+        prepareNextRequest();
+        fetchNextBatch();
+      }
+    }
+
+    // Truly nothing left to read.
+    currentObjectSummary = null;
+  }
+
+  @Override
+  public boolean hasNext()
+  {
+    return currentObjectSummary != null;
+  }
+
+  @Override
+  public S3ObjectSummary next()
+  {
+    if (currentObjectSummary == null) {
+      throw new NoSuchElementException();
+    }
+
+    final S3ObjectSummary retVal = currentObjectSummary;
+    advanceObjectSummary();
+    return retVal;
+  }
 
 Review comment:
   I'll move it, because I agree with this particular point, but IMO style nits should either be encoded into an automated style checker or written down somewhere centrally agreed-upon. Otherwise they can create friction in the review process. Maybe we can come up with a nice way as a community to handle this.

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [druid] gianm commented on a change in pull request #9098: S3: Improvements to prefix listing (including fix for an infinite loop)

Posted by GitBox <gi...@apache.org>.
gianm commented on a change in pull request #9098: S3: Improvements to prefix listing (including fix for an infinite loop)
URL: https://github.com/apache/druid/pull/9098#discussion_r362036658
 
 

 ##########
 File path: extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/LazyObjectSummariesIterator.java
 ##########
 @@ -0,0 +1,162 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.storage.s3;
+
+import com.amazonaws.services.s3.model.ListObjectsV2Request;
+import com.amazonaws.services.s3.model.ListObjectsV2Result;
+import com.amazonaws.services.s3.model.S3ObjectSummary;
+import org.apache.druid.java.util.common.RE;
+
+import java.net.URI;
+import java.util.Iterator;
+import java.util.NoSuchElementException;
+
+/**
+ * Iterator class used by {@link S3Utils#lazyObjectSummaryIterator}.
+ *
+ * Walks a set of prefixes, returning all objects underneath them except for directory placeholders.
+ */
+public class LazyObjectSummariesIterator implements Iterator<S3ObjectSummary>
+{
+  private final ServerSideEncryptingAmazonS3 s3Client;
+  private final Iterator<URI> prefixesIterator;
+  private final int maxListingLength;
+
+  private ListObjectsV2Request request;
+  private ListObjectsV2Result result;
+  private Iterator<S3ObjectSummary> objectSummaryIterator;
+  private S3ObjectSummary currentObjectSummary;
+
+  LazyObjectSummariesIterator(
+      final ServerSideEncryptingAmazonS3 s3Client,
+      final Iterable<URI> prefixes,
+      final int maxListingLength
+  )
+  {
+    this.s3Client = s3Client;
+    this.prefixesIterator = prefixes.iterator();
+    this.maxListingLength = maxListingLength;
+
+    prepareNextRequest();
+    fetchNextBatch();
+    advanceObjectSummary();
+  }
+
+  private void prepareNextRequest()
+  {
+    final URI currentUri = prefixesIterator.next();
+    final String currentBucket = currentUri.getAuthority();
+    final String currentPrefix = S3Utils.extractS3Key(currentUri);
+
+    request = new ListObjectsV2Request()
+        .withBucketName(currentBucket)
+        .withPrefix(currentPrefix)
+        .withMaxKeys(maxListingLength);
+  }
+
+  private void fetchNextBatch()
+  {
+    try {
+      result = S3Utils.retryS3Operation(() -> s3Client.listObjectsV2(request));
+      request.setContinuationToken(result.getNextContinuationToken());
+      objectSummaryIterator = result.getObjectSummaries().iterator();
+    }
+    catch (Exception e) {
+      throw new RE(
+          e,
+          "Failed to get object summaries from S3 bucket[%s], prefix[%s]",
+          request.getBucketName(),
+          request.getPrefix()
+      );
+    }
+  }
+
+  /**
+   * Advance objectSummaryIterator to the next non-placeholder, updating "currentObjectSummary".
+   */
+  private void advanceObjectSummary()
+  {
+    while (objectSummaryIterator.hasNext() || result.isTruncated() || prefixesIterator.hasNext()) {
+      while (objectSummaryIterator.hasNext()) {
+        currentObjectSummary = objectSummaryIterator.next();
+
+        if (!isDirectoryPlaceholder(currentObjectSummary)) {
+          return;
+        }
+      }
+
+      // Exhausted "objectSummaryIterator" without finding a non-placeholder.
+      if (result.isTruncated()) {
+        fetchNextBatch();
+      } else if (prefixesIterator.hasNext()) {
+        prepareNextRequest();
+        fetchNextBatch();
+      }
+    }
+
+    // Truly nothing left to read.
+    currentObjectSummary = null;
+  }
+
+  @Override
+  public boolean hasNext()
+  {
+    return currentObjectSummary != null;
+  }
+
+  @Override
+  public S3ObjectSummary next()
+  {
+    if (currentObjectSummary == null) {
+      throw new NoSuchElementException();
+    }
+
+    final S3ObjectSummary retVal = currentObjectSummary;
+    advanceObjectSummary();
+    return retVal;
+  }
+
+  /**
+   * Checks if a given object is a directory placeholder and should be ignored.
+   *
+   * Adapted from org.jets3t.service.model.StorageObject.isDirectoryPlaceholder(). Does not include the check for
+   * legacy JetS3t directory placeholder objects, since it is based on content-type, which isn't available in an
+   * S3ObjectSummary.
+   */
+  private static boolean isDirectoryPlaceholder(final S3ObjectSummary objectSummary)
+  {
+    // Recognize "standard" directory place-holder indications used by Amazon's AWS Console and Panic's Transmit.
+    if (objectSummary.getKey().endsWith("/") && objectSummary.getSize() == 0) {
+      return true;
+    }
+
+    // Recognize s3sync.rb directory placeholders by MD5/ETag value.
+    if ("d66759af42f282e1ba19144df2d405d0".equals(objectSummary.getETag())) {
 
 Review comment:
   It's from `org.jets3t.service.model.StorageObject.isDirectoryPlaceholder` (mentioned in the javadoc for this method). Sources are at https://github.com/mondain/jets3t/blob/master/jets3t/src/main/java/org/jets3t/service/model/StorageObject.java.
   
   I did it this way for these two reasons:
   
   - Minimal diff from the original function, in case someone wants to compare them visually and see what we changed.
   - I didn't see a benefit to putting `"d66759af42f282e1ba19144df2d405d0"` into a constant: it won't change, it won't be used in any other locations, and it's adequately explained by the comment above the line it appears in, so the usual benefits of pulling magic values out into constants don't apply.
   
   Although actually, maybe this is moot, since it might be reasonable to just remove this block. I don't think we have any particular need to support the `s3sync.rb` style directory placeholders. The other two kinds are much more common. It looks like `s3sync.rb` hasn't been maintained in many years, according to http://s3sync.net/wiki.html.
   
   What do you think?

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [druid] gianm commented on a change in pull request #9098: S3: Improvements to prefix listing (including fix for an infinite loop)

Posted by GitBox <gi...@apache.org>.
gianm commented on a change in pull request #9098: S3: Improvements to prefix listing (including fix for an infinite loop)
URL: https://github.com/apache/druid/pull/9098#discussion_r362038684
 
 

 ##########
 File path: extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/LazyObjectSummariesIterator.java
 ##########
 @@ -0,0 +1,162 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.storage.s3;
+
+import com.amazonaws.services.s3.model.ListObjectsV2Request;
+import com.amazonaws.services.s3.model.ListObjectsV2Result;
+import com.amazonaws.services.s3.model.S3ObjectSummary;
+import org.apache.druid.java.util.common.RE;
+
+import java.net.URI;
+import java.util.Iterator;
+import java.util.NoSuchElementException;
+
+/**
+ * Iterator class used by {@link S3Utils#lazyObjectSummaryIterator}.
+ *
+ * Walks a set of prefixes, returning all objects underneath them except for directory placeholders.
+ */
+public class LazyObjectSummariesIterator implements Iterator<S3ObjectSummary>
+{
+  private final ServerSideEncryptingAmazonS3 s3Client;
+  private final Iterator<URI> prefixesIterator;
+  private final int maxListingLength;
+
+  private ListObjectsV2Request request;
+  private ListObjectsV2Result result;
+  private Iterator<S3ObjectSummary> objectSummaryIterator;
+  private S3ObjectSummary currentObjectSummary;
+
+  LazyObjectSummariesIterator(
+      final ServerSideEncryptingAmazonS3 s3Client,
+      final Iterable<URI> prefixes,
+      final int maxListingLength
+  )
+  {
+    this.s3Client = s3Client;
+    this.prefixesIterator = prefixes.iterator();
+    this.maxListingLength = maxListingLength;
+
+    prepareNextRequest();
+    fetchNextBatch();
+    advanceObjectSummary();
+  }
+
+  private void prepareNextRequest()
+  {
+    final URI currentUri = prefixesIterator.next();
+    final String currentBucket = currentUri.getAuthority();
+    final String currentPrefix = S3Utils.extractS3Key(currentUri);
+
+    request = new ListObjectsV2Request()
+        .withBucketName(currentBucket)
+        .withPrefix(currentPrefix)
+        .withMaxKeys(maxListingLength);
+  }
+
+  private void fetchNextBatch()
+  {
+    try {
+      result = S3Utils.retryS3Operation(() -> s3Client.listObjectsV2(request));
+      request.setContinuationToken(result.getNextContinuationToken());
+      objectSummaryIterator = result.getObjectSummaries().iterator();
+    }
+    catch (Exception e) {
+      throw new RE(
+          e,
+          "Failed to get object summaries from S3 bucket[%s], prefix[%s]",
 
 Review comment:
   It would appear if this thrown exception is fully logged, since its cause will be set to `e`, which has a message that includes the stack trace. But I agree it would be nicer to include the S3 error in _this_ exception's message rather than only as its cause. So I added it in the latest patch.

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [druid] gianm commented on a change in pull request #9098: S3: Improvements to prefix listing (including fix for an infinite loop)

Posted by GitBox <gi...@apache.org>.
gianm commented on a change in pull request #9098: S3: Improvements to prefix listing (including fix for an infinite loop)
URL: https://github.com/apache/druid/pull/9098#discussion_r362033570
 
 

 ##########
 File path: extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/LazyObjectSummariesIterator.java
 ##########
 @@ -0,0 +1,162 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.storage.s3;
+
+import com.amazonaws.services.s3.model.ListObjectsV2Request;
+import com.amazonaws.services.s3.model.ListObjectsV2Result;
+import com.amazonaws.services.s3.model.S3ObjectSummary;
+import org.apache.druid.java.util.common.RE;
+
+import java.net.URI;
+import java.util.Iterator;
+import java.util.NoSuchElementException;
+
+/**
+ * Iterator class used by {@link S3Utils#lazyObjectSummaryIterator}.
+ *
+ * Walks a set of prefixes, returning all objects underneath them except for directory placeholders.
+ */
+public class LazyObjectSummariesIterator implements Iterator<S3ObjectSummary>
+{
+  private final ServerSideEncryptingAmazonS3 s3Client;
+  private final Iterator<URI> prefixesIterator;
+  private final int maxListingLength;
+
+  private ListObjectsV2Request request;
+  private ListObjectsV2Result result;
+  private Iterator<S3ObjectSummary> objectSummaryIterator;
+  private S3ObjectSummary currentObjectSummary;
+
+  LazyObjectSummariesIterator(
+      final ServerSideEncryptingAmazonS3 s3Client,
+      final Iterable<URI> prefixes,
+      final int maxListingLength
+  )
+  {
+    this.s3Client = s3Client;
+    this.prefixesIterator = prefixes.iterator();
+    this.maxListingLength = maxListingLength;
+
+    prepareNextRequest();
+    fetchNextBatch();
+    advanceObjectSummary();
 
 Review comment:
   I agree. This patch didn't name the concept (it existed previously as `S3Utils.lazyFetchingObjectSummariesIterator`), but I don't mind changing it. It isn't required to be truly lazy, so I won't change its behavior, only its name.
   
   I made the following changes:
   
   - Renamed the iterator to `ObjectSummaryIterator`.
   - Renamed `S3Utils.lazyObjectSummaryIterator` to `S3Utils.objectSummaryIterator`.
   - Clarified the behavior in javadocs (first batch computed during iterator construction, later batches computed incrementally).

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [druid] gianm commented on issue #9098: S3: Improvements to prefix listing (including fix for an infinite loop)

Posted by GitBox <gi...@apache.org>.
gianm commented on issue #9098: S3: Improvements to prefix listing (including fix for an infinite loop)
URL: https://github.com/apache/druid/pull/9098#issuecomment-569732751
 
 
   @suneet-s thanks for taking a look. I pushed changes reflecting your comments except for https://github.com/apache/druid/pull/9098#discussion_r361693808, since I'm hoping to see what you think first on that one.

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [druid] suneet-s commented on a change in pull request #9098: S3: Improvements to prefix listing (including fix for an infinite loop)

Posted by GitBox <gi...@apache.org>.
suneet-s commented on a change in pull request #9098: S3: Improvements to prefix listing (including fix for an infinite loop)
URL: https://github.com/apache/druid/pull/9098#discussion_r361700268
 
 

 ##########
 File path: extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/LazyObjectSummariesIterator.java
 ##########
 @@ -0,0 +1,162 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.storage.s3;
+
+import com.amazonaws.services.s3.model.ListObjectsV2Request;
+import com.amazonaws.services.s3.model.ListObjectsV2Result;
+import com.amazonaws.services.s3.model.S3ObjectSummary;
+import org.apache.druid.java.util.common.RE;
+
+import java.net.URI;
+import java.util.Iterator;
+import java.util.NoSuchElementException;
+
+/**
+ * Iterator class used by {@link S3Utils#lazyObjectSummaryIterator}.
+ *
+ * Walks a set of prefixes, returning all objects underneath them except for directory placeholders.
+ */
+public class LazyObjectSummariesIterator implements Iterator<S3ObjectSummary>
+{
+  private final ServerSideEncryptingAmazonS3 s3Client;
+  private final Iterator<URI> prefixesIterator;
+  private final int maxListingLength;
+
+  private ListObjectsV2Request request;
+  private ListObjectsV2Result result;
+  private Iterator<S3ObjectSummary> objectSummaryIterator;
+  private S3ObjectSummary currentObjectSummary;
+
+  LazyObjectSummariesIterator(
+      final ServerSideEncryptingAmazonS3 s3Client,
+      final Iterable<URI> prefixes,
+      final int maxListingLength
+  )
+  {
+    this.s3Client = s3Client;
+    this.prefixesIterator = prefixes.iterator();
+    this.maxListingLength = maxListingLength;
+
+    prepareNextRequest();
+    fetchNextBatch();
+    advanceObjectSummary();
+  }
+
+  private void prepareNextRequest()
+  {
+    final URI currentUri = prefixesIterator.next();
+    final String currentBucket = currentUri.getAuthority();
+    final String currentPrefix = S3Utils.extractS3Key(currentUri);
+
+    request = new ListObjectsV2Request()
+        .withBucketName(currentBucket)
+        .withPrefix(currentPrefix)
+        .withMaxKeys(maxListingLength);
+  }
+
+  private void fetchNextBatch()
+  {
+    try {
+      result = S3Utils.retryS3Operation(() -> s3Client.listObjectsV2(request));
+      request.setContinuationToken(result.getNextContinuationToken());
+      objectSummaryIterator = result.getObjectSummaries().iterator();
+    }
+    catch (Exception e) {
+      throw new RE(
+          e,
+          "Failed to get object summaries from S3 bucket[%s], prefix[%s]",
 
 Review comment:
   This log message doesn't tell us if the error is because of permissions or something else. Would we want to expose that information in 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


With regards,
Apache Git Services

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