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 2020/06/18 05:45:29 UTC

[GitHub] [druid] jon-wei commented on a change in pull request #9898: support Aliyun OSS service as deep storage

jon-wei commented on a change in pull request #9898:
URL: https://github.com/apache/druid/pull/9898#discussion_r441968129



##########
File path: extensions-contrib/aliyun-oss-extensions/src/main/java/org/apache/druid/data/input/aliyun/OssEntity.java
##########
@@ -0,0 +1,88 @@
+/*
+ * 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.data.input.aliyun;
+
+import com.aliyun.oss.OSS;
+import com.aliyun.oss.OSSException;
+import com.aliyun.oss.model.GetObjectRequest;
+import com.aliyun.oss.model.OSSObject;
+import com.google.common.base.Predicate;
+import org.apache.druid.data.input.RetryingInputEntity;
+import org.apache.druid.data.input.impl.CloudObjectLocation;
+import org.apache.druid.java.util.common.ISE;
+import org.apache.druid.storage.aliyun.OssStorageDruidModule;
+import org.apache.druid.storage.aliyun.OssUtils;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.URI;
+
+public class OssEntity extends RetryingInputEntity
+{
+  private final OSS ossClient;
+  private final CloudObjectLocation object;
+
+  OssEntity(OSS ossClient, CloudObjectLocation coords)
+  {
+    this.ossClient = ossClient;
+    this.object = coords;
+  }
+
+  @Override
+  public URI getUri()
+  {
+    return object.toUri(OssStorageDruidModule.SCHEME);
+  }
+
+  @Override
+  protected InputStream readFrom(long offset) throws IOException
+  {
+    final GetObjectRequest request = new GetObjectRequest(object.getBucket(), object.getPath());
+    request.setRange(offset, -1 /*from offset to end*/);
+
+    try {
+      final OSSObject ossObject = ossClient.getObject(request);
+      if (ossObject == null) {
+        throw new ISE(
+            "Failed to get an s3 object for bucket[%s], key[%s], and start[%d]",

Review comment:
       s3 -> Aliyun OSS

##########
File path: extensions-contrib/aliyun-oss-extensions/src/main/java/org/apache/druid/storage/aliyun/ObjectSummaryIterator.java
##########
@@ -0,0 +1,163 @@
+/*
+ * 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.aliyun;
+
+import com.aliyun.oss.OSS;
+import com.aliyun.oss.OSSException;
+import com.aliyun.oss.model.ListObjectsRequest;
+import com.aliyun.oss.model.OSSObjectSummary;
+import com.aliyun.oss.model.ObjectListing;
+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 OssUtils#objectSummaryIterator}.
+ * <p>
+ * As required by the specification of that method, this iterator is computed incrementally in batches of
+ * {@code maxListLength}. The first call is made at the same time the iterator is constructed.
+ *
+ */
+public class ObjectSummaryIterator implements Iterator<OSSObjectSummary>

Review comment:
       Suggest renaming this to avoid having the same name as the S3 version

##########
File path: extensions-contrib/aliyun-oss-extensions/src/main/java/org/apache/druid/storage/aliyun/ObjectSummaryIterator.java
##########
@@ -0,0 +1,163 @@
+/*
+ * 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.aliyun;
+
+import com.aliyun.oss.OSS;
+import com.aliyun.oss.OSSException;
+import com.aliyun.oss.model.ListObjectsRequest;
+import com.aliyun.oss.model.OSSObjectSummary;
+import com.aliyun.oss.model.ObjectListing;
+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 OssUtils#objectSummaryIterator}.
+ * <p>
+ * As required by the specification of that method, this iterator is computed incrementally in batches of
+ * {@code maxListLength}. The first call is made at the same time the iterator is constructed.
+ *
+ */
+public class ObjectSummaryIterator implements Iterator<OSSObjectSummary>
+{
+  private final OSS client;
+  private final Iterator<URI> prefixesIterator;
+  private final int maxListingLength;
+
+  private ListObjectsRequest request;
+  private ObjectListing result;
+  private Iterator<OSSObjectSummary> objectSummaryIterator;
+  private OSSObjectSummary currentObjectSummary;
+
+  ObjectSummaryIterator(
+      final OSS client,
+      final Iterable<URI> prefixes,
+      final int maxListingLength
+  )
+  {
+    this.client = client;
+    this.prefixesIterator = prefixes.iterator();
+    this.maxListingLength = maxListingLength;
+
+    prepareNextRequest();
+    fetchNextBatch();
+    advanceObjectSummary();
+  }
+
+  @Override
+  public boolean hasNext()
+  {
+    return currentObjectSummary != null;
+  }
+
+  @Override
+  public OSSObjectSummary next()
+  {
+    if (currentObjectSummary == null) {
+      throw new NoSuchElementException();
+    }
+
+    final OSSObjectSummary retVal = currentObjectSummary;
+    advanceObjectSummary();
+    return retVal;
+  }
+
+  private void prepareNextRequest()
+  {
+    final URI currentUri = prefixesIterator.next();
+    final String currentBucket = currentUri.getAuthority();
+    final String currentPrefix = OssUtils.extractKey(currentUri);
+
+    request = new ListObjectsRequest(currentBucket, currentPrefix, null, null, maxListingLength);
+  }
+
+  private void fetchNextBatch()
+  {
+    try {
+      result = OssUtils.retry(() -> client.listObjects(request));
+      request.setMarker(result.getNextMarker());
+      objectSummaryIterator = result.getObjectSummaries().iterator();
+    }
+    catch (OSSException e) {
+      throw new RE(
+          e,
+          "Failed to get object summaries from S3 bucket[%s], prefix[%s]; S3 error: %s",
+          request.getBucketName(),
+          request.getPrefix(),
+          e.getMessage()
+      );
+    }
+    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();
+        // skips directories and empty objects
+        if (!isDirectoryPlaceholder(currentObjectSummary) && currentObjectSummary.getSize() > 0) {
+          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;
+  }
+
+  /**
+   * Checks if a given object is a directory placeholder and should be ignored.

Review comment:
       This should preserve the original javadoc about being adapted from jets3t code:
   
   ```
      * 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.
   ```
   
   Can you also add an entry noting the usage here? 
   https://github.com/apache/druid/blob/master/LICENSE#L276

##########
File path: extensions-contrib/aliyun-oss-extensions/src/main/java/org/apache/druid/data/input/aliyun/OssInputSource.java
##########
@@ -0,0 +1,178 @@
+/*
+ * 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.data.input.aliyun;
+
+import com.aliyun.oss.OSS;
+import com.aliyun.oss.model.OSSObjectSummary;
+import com.fasterxml.jackson.annotation.JacksonInject;
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.google.common.base.Preconditions;
+import com.google.common.base.Supplier;
+import com.google.common.base.Suppliers;
+import org.apache.druid.data.input.InputEntity;
+import org.apache.druid.data.input.InputFileAttribute;
+import org.apache.druid.data.input.InputSplit;
+import org.apache.druid.data.input.SplitHintSpec;
+import org.apache.druid.data.input.impl.CloudObjectInputSource;
+import org.apache.druid.data.input.impl.CloudObjectLocation;
+import org.apache.druid.data.input.impl.SplittableInputSource;
+import org.apache.druid.storage.aliyun.OssInputDataConfig;
+import org.apache.druid.storage.aliyun.OssStorageDruidModule;
+import org.apache.druid.storage.aliyun.OssUtils;
+import org.apache.druid.utils.Streams;
+
+import javax.annotation.Nonnull;
+import javax.annotation.Nullable;
+import java.net.URI;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Objects;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+public class OssInputSource extends CloudObjectInputSource
+{
+  private final Supplier<OSS> clientSupplier;
+  @JsonProperty("properties")
+  private final OssClientConfig inputSourceConfig;
+  private final OssInputDataConfig inputDataConfig;
+
+  /**
+   * Constructor for OssInputSource
+   *
+   * @param client            The default client built with all default configs
+   *                          from Guice. This injected singleton client is used when {@param inputSourceConfig}
+   *                          is not provided and hence
+   * @param inputDataConfig   Stores the configuration for options related to reading input data
+   * @param uris              User provided uris to read input data
+   * @param prefixes          User provided prefixes to read input data
+   * @param objects           User provided cloud objects values to read input data
+   * @param inputSourceConfig User provided properties for overriding the default aliyun-oss configuration
+   */
+  @JsonCreator
+  public OssInputSource(
+      @JacksonInject OSS client,
+      @JacksonInject OssInputDataConfig inputDataConfig,
+      @JsonProperty("uris") @Nullable List<URI> uris,
+      @JsonProperty("prefixes") @Nullable List<URI> prefixes,
+      @JsonProperty("objects") @Nullable List<CloudObjectLocation> objects,
+      @JsonProperty("properties") @Nullable OssClientConfig inputSourceConfig
+  )
+  {
+    super(OssStorageDruidModule.SCHEME, uris, prefixes, objects);
+    this.inputDataConfig = Preconditions.checkNotNull(inputDataConfig, "inputDataConfig");
+    Preconditions.checkNotNull(client, "client");
+    this.inputSourceConfig = inputSourceConfig;
+    this.clientSupplier = Suppliers.memoize(
+        () -> {
+          if (inputSourceConfig != null) {
+            return inputSourceConfig.buildClient();
+          } else {
+            return client;
+          }
+        }
+    );
+  }
+
+
+  @Nullable
+  @JsonProperty("properties")
+  public OssClientConfig getOssInputSourceConfig()
+  {
+    return inputSourceConfig;
+  }
+
+  @Override
+  protected InputEntity createEntity(CloudObjectLocation location)
+  {
+    return new OssEntity(clientSupplier.get(), location);
+  }
+
+  @Override
+  protected Stream<InputSplit<List<CloudObjectLocation>>> getPrefixesSplitStream(@Nonnull SplitHintSpec splitHintSpec)
+  {
+    final Iterator<List<OSSObjectSummary>> splitIterator = splitHintSpec.split(
+        getIterableObjectsFromPrefixes().iterator(),
+        object -> new InputFileAttribute(object.getSize())
+    );
+
+    return Streams.sequentialStreamFrom(splitIterator)
+                  .map(objects -> objects.stream()
+                                         .map(OssUtils::summaryToCloudObjectLocation)
+                                         .collect(Collectors.toList()))
+                  .map(InputSplit::new);
+  }
+
+  @Override
+  public SplittableInputSource<List<CloudObjectLocation>> withSplit(InputSplit<List<CloudObjectLocation>> split)
+  {
+    return new OssInputSource(
+        clientSupplier.get(),
+        inputDataConfig,
+        null,
+        null,
+        split.get(),
+        getOssInputSourceConfig()
+    );
+  }
+
+  @Override
+  public int hashCode()
+  {
+    return Objects.hash(super.hashCode(), inputSourceConfig);
+  }
+
+  @Override
+  public boolean equals(Object o)
+  {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+    if (!super.equals(o)) {
+      return false;
+    }
+    OssInputSource that = (OssInputSource) o;

Review comment:
       I think equals/hashcode should have uris, prefixes, etc.

##########
File path: extensions-contrib/aliyun-oss-extensions/src/main/java/org/apache/druid/storage/aliyun/OssStorageDruidModule.java
##########
@@ -0,0 +1,111 @@
+/*
+ * 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.aliyun;
+
+import com.aliyun.oss.OSS;
+import com.fasterxml.jackson.core.Version;
+import com.fasterxml.jackson.databind.Module;
+import com.google.common.collect.ImmutableList;
+import com.google.inject.Binder;
+import com.google.inject.Provides;
+import com.google.inject.multibindings.MapBinder;
+import org.apache.druid.data.SearchableVersionedDataFinder;
+import org.apache.druid.data.input.aliyun.OssClientConfig;
+import org.apache.druid.guice.Binders;
+import org.apache.druid.guice.JsonConfigProvider;
+import org.apache.druid.guice.LazySingleton;
+import org.apache.druid.initialization.DruidModule;
+
+import java.util.List;
+
+public class OssStorageDruidModule implements DruidModule
+{
+  public static final String SCHEME = "aliyun-oss";
+  public static final String SCHEME_S3N = "aliyun-oss_3n";
+  public static final String SCHEME_ZIP = "aliyun-oss_zip";
+
+  @Override
+  public List<? extends Module> getJacksonModules()
+  {
+    return ImmutableList.of(
+        new Module()
+        {
+          @Override
+          public String getModuleName()
+          {
+            return "DruidOss-" + System.identityHashCode(this);
+          }
+
+          @Override
+          public Version version()
+          {
+            return Version.unknownVersion();
+          }
+
+          @Override
+          public void setupModule(SetupContext context)
+          {
+            context.registerSubtypes(OssLoadSpec.class);
+          }
+        }
+    );
+  }
+
+  @Override
+  public void configure(Binder binder)
+  {
+    MapBinder.newMapBinder(binder, String.class, SearchableVersionedDataFinder.class)
+             .addBinding(SCHEME)
+             .to(OssTimestampVersionedDataFinder.class)
+             .in(LazySingleton.class);
+    MapBinder.newMapBinder(binder, String.class, SearchableVersionedDataFinder.class)
+             .addBinding(SCHEME_S3N)

Review comment:
       If there's only one aliyun URI scheme, this binding isn't needed

##########
File path: extensions-contrib/aliyun-oss-extensions/src/main/java/org/apache/druid/storage/aliyun/OssUtils.java
##########
@@ -0,0 +1,279 @@
+/*
+ * 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.aliyun;
+
+import com.aliyun.oss.OSS;
+import com.aliyun.oss.OSSException;
+import com.aliyun.oss.model.AccessControlList;
+import com.aliyun.oss.model.CannedAccessControlList;
+import com.aliyun.oss.model.DeleteObjectsRequest;
+import com.aliyun.oss.model.ListObjectsRequest;
+import com.aliyun.oss.model.OSSObjectSummary;
+import com.aliyun.oss.model.ObjectListing;
+import com.aliyun.oss.model.PutObjectRequest;
+import com.google.common.base.Joiner;
+import com.google.common.base.Predicate;
+import com.google.common.collect.ImmutableList;
+import org.apache.druid.data.input.impl.CloudObjectLocation;
+import org.apache.druid.java.util.common.ISE;
+import org.apache.druid.java.util.common.RetryUtils;
+import org.apache.druid.java.util.common.RetryUtils.Task;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.java.util.common.logger.Logger;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.URI;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+
+public class OssUtils
+{
+  private static final String SCHEME = OssStorageDruidModule.SCHEME;
+  private static final Joiner JOINER = Joiner.on("/").skipNulls();
+  private static final Logger log = new Logger(OssUtils.class);
+
+
+  static boolean isServiceExceptionRecoverable(OSSException ex)
+  {
+    final boolean isIOException = ex.getCause() instanceof IOException;
+    final boolean isTimeout = "RequestTimeout".equals(ex.getErrorCode());
+    final boolean badStatusCode = false; //ex. == 400 || ex.getStatusCode() == 403 || ex.getStatusCode() == 404;
+    return !badStatusCode && (isIOException || isTimeout);
+  }
+
+  public static final Predicate<Throwable> RETRYABLE = new Predicate<Throwable>()
+  {
+    @Override
+    public boolean apply(Throwable e)
+    {
+      if (e == null) {
+        return false;
+      } else if (e instanceof IOException) {
+        return true;
+      } else if (e instanceof OSSException) {
+        return isServiceExceptionRecoverable((OSSException) e);
+      } else {
+        return apply(e.getCause());
+      }
+    }
+  };
+
+  /**
+   * Retries aliyun-oss operations that fail due to io-related exceptions. Service-level exceptions (access denied, file not
+   * found, etc) are not retried.
+   */
+  static <T> T retry(Task<T> f) throws Exception
+  {
+    return RetryUtils.retry(f, RETRYABLE, RetryUtils.DEFAULT_MAX_TRIES);
+  }
+
+  static boolean isObjectInBucketIgnoringPermission(
+      OSS client,
+      String bucketName,
+      String objectKey
+  )
+  {
+    try {
+      return client.doesObjectExist(bucketName, objectKey);
+    }
+    catch (OSSException e) {
+      if (e.getErrorCode().equals("NoSuchKey")) {
+        // Object is inaccessible to current user, but does exist.
+        return true;
+      }
+      // Something else has gone wrong
+      throw e;
+    }
+  }
+
+  /**
+   * Create an iterator over a set of aliyun-oss objects specified by a set of prefixes.
+   * <p>
+   * For each provided prefix URI, the iterator will walk through all objects that are in the same bucket as the
+   * provided URI and whose keys start with that URI's path, except for directory placeholders (which will be
+   * ignored). The iterator is computed incrementally by calling {@link OSS#listObjects} for
+   * each prefix in batches of {@param maxListLength}. The first call is made at the same time the iterator is
+   * constructed.
+   */
+  public static Iterator<OSSObjectSummary> objectSummaryIterator(
+      final OSS client,
+      final Iterable<URI> prefixes,
+      final int maxListingLength
+  )
+  {
+    return new ObjectSummaryIterator(client, prefixes, maxListingLength);
+  }
+
+  /**
+   * Create an {@link URI} from the given {@link OSSObjectSummary}. The result URI is composed as below.
+   *
+   * <pre>
+   * {@code aliyun-oss://{BUCKET_NAME}/{OBJECT_KEY}}
+   * </pre>
+   */
+  public static URI summaryToUri(OSSObjectSummary object)
+  {
+    return summaryToCloudObjectLocation(object).toUri(SCHEME);
+  }
+
+  public static CloudObjectLocation summaryToCloudObjectLocation(OSSObjectSummary object)
+  {
+    return new CloudObjectLocation(object.getBucketName(), object.getKey());
+  }
+
+  static String constructSegmentPath(String baseKey, String storageDir)
+  {
+    return JOINER.join(
+        baseKey.isEmpty() ? null : baseKey,
+        storageDir
+    ) + "/index.zip";
+  }
+
+  static CannedAccessControlList grantFullControlToBucketOwner(OSS client, String bucket)
+  {
+    final AccessControlList acl = client.getBucketAcl(bucket);
+    return acl.getCannedACL();
+    //acl.grantAllPermissions(new Grant(new Grantee(acl.getOwner().getId()), Permission.FullControl));

Review comment:
       Please remove the unused code

##########
File path: extensions-contrib/aliyun-oss-extensions/src/main/java/org/apache/druid/firehose/aliyun/StaticOssFirehoseFactory.java
##########
@@ -0,0 +1,237 @@
+/*
+ * 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.firehose.aliyun;
+
+import com.aliyun.oss.OSS;
+import com.aliyun.oss.OSSException;
+import com.aliyun.oss.model.GetObjectRequest;
+import com.aliyun.oss.model.OSSObject;
+import com.aliyun.oss.model.OSSObjectSummary;
+import com.fasterxml.jackson.annotation.JacksonInject;
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.google.common.base.Preconditions;
+import com.google.common.base.Predicate;
+import org.apache.druid.data.input.FiniteFirehoseFactory;
+import org.apache.druid.data.input.InputSplit;
+import org.apache.druid.data.input.impl.StringInputRowParser;
+import org.apache.druid.data.input.impl.prefetch.PrefetchableTextFilesFirehoseFactory;
+import org.apache.druid.java.util.common.IAE;
+import org.apache.druid.java.util.common.ISE;
+import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.storage.aliyun.OssUtils;
+import org.apache.druid.utils.CompressionUtils;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.URI;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Objects;
+import java.util.stream.Collectors;
+
+/**
+ * Builds firehoses that read from a predefined list of aliyun-oss objects and then dry up.
+ */
+public class StaticOssFirehoseFactory extends PrefetchableTextFilesFirehoseFactory<URI>
+{
+  private static final Logger log = new Logger(StaticOssFirehoseFactory.class);
+  private static final int MAX_LISTING_LENGTH = 1024;
+
+  private final OSS client;
+  private final List<URI> uris;
+  private final List<URI> prefixes;
+
+  @JsonCreator
+  public StaticOssFirehoseFactory(
+      @JacksonInject OSS client,
+      @JsonProperty("uris") List<URI> uris,
+      @JsonProperty("prefixes") List<URI> prefixes,
+      @JsonProperty("maxCacheCapacityBytes") Long maxCacheCapacityBytes,
+      @JsonProperty("maxFetchCapacityBytes") Long maxFetchCapacityBytes,
+      @JsonProperty("prefetchTriggerBytes") Long prefetchTriggerBytes,
+      @JsonProperty("fetchTimeout") Long fetchTimeout,
+      @JsonProperty("maxFetchRetry") Integer maxFetchRetry
+  )
+  {
+    super(maxCacheCapacityBytes, maxFetchCapacityBytes, prefetchTriggerBytes, fetchTimeout, maxFetchRetry);
+    this.client = Preconditions.checkNotNull(client, "client");
+    this.uris = uris == null ? new ArrayList<>() : uris;
+    this.prefixes = prefixes == null ? new ArrayList<>() : prefixes;
+
+    if (!this.uris.isEmpty() && !this.prefixes.isEmpty()) {
+      throw new IAE("uris and prefixes cannot be used together");
+    }
+
+    if (this.uris.isEmpty() && this.prefixes.isEmpty()) {
+      throw new IAE("uris or prefixes must be specified");
+    }
+
+    for (final URI inputURI : this.uris) {
+      Preconditions.checkArgument("aliyun-oss".equals(inputURI.getScheme()), "input uri scheme == aliyun-oss (%s)", inputURI);

Review comment:
       Suggest making `"aliyun-oss"` a constant

##########
File path: extensions-contrib/aliyun-oss-extensions/src/main/java/org/apache/druid/storage/aliyun/OssDataSegmentPusher.java
##########
@@ -0,0 +1,131 @@
+/*
+ * 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.aliyun;
+
+import com.aliyun.oss.OSS;
+import com.aliyun.oss.OSSException;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import com.google.inject.Inject;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.java.util.emitter.EmittingLogger;
+import org.apache.druid.segment.SegmentUtils;
+import org.apache.druid.segment.loading.DataSegmentPusher;
+import org.apache.druid.timeline.DataSegment;
+import org.apache.druid.utils.CompressionUtils;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.URI;
+import java.util.List;
+import java.util.Map;
+
+public class OssDataSegmentPusher implements DataSegmentPusher
+{
+  private static final EmittingLogger log = new EmittingLogger(OssDataSegmentPusher.class);
+
+  private final OSS client;
+  private final OssStorageConfig config;
+
+  @Inject
+  public OssDataSegmentPusher(
+      OSS client,
+      OssStorageConfig config
+  )
+  {
+    this.client = client;
+    this.config = config;
+  }
+
+  @Override
+  public String getPathForHadoop()
+  {
+    return StringUtils.format("%s/%s", config.getBucket(), config.getPrefix());

Review comment:
       Have you tested this with a Hadoop cluster before? If so, cool. If not, that's fine, we can just document that Hadoop is untested with 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: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org