paul-rogers commented on code in PR #12874:
URL: https://github.com/apache/druid/pull/12874#discussion_r943683939


##########
extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/output/RetryableS3OutputStreamTest.java:
##########
@@ -0,0 +1,382 @@
+/*
+ * 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.output;
+
+import com.amazonaws.AmazonClientException;
+import com.amazonaws.SdkClientException;
+import com.amazonaws.services.s3.AmazonS3;
+import com.amazonaws.services.s3.model.AbortMultipartUploadRequest;
+import com.amazonaws.services.s3.model.CompleteMultipartUploadRequest;
+import com.amazonaws.services.s3.model.CompleteMultipartUploadResult;
+import com.amazonaws.services.s3.model.InitiateMultipartUploadRequest;
+import com.amazonaws.services.s3.model.InitiateMultipartUploadResult;
+import com.amazonaws.services.s3.model.PartETag;
+import com.amazonaws.services.s3.model.UploadPartRequest;
+import com.amazonaws.services.s3.model.UploadPartResult;
+import org.apache.druid.java.util.common.IOE;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.storage.s3.NoopServerSideEncryption;
+import org.apache.druid.storage.s3.ServerSideEncryptingAmazonS3;
+import org.easymock.EasyMock;
+import org.hamcrest.CoreMatchers;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+import org.junit.rules.TemporaryFolder;
+
+import javax.annotation.Nullable;
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.stream.Collectors;
+
+public class RetryableS3OutputStreamTest
+{
+  @Rule
+  public TemporaryFolder temporaryFolder = new TemporaryFolder();
+
+  @Rule
+  public ExpectedException expectedException = ExpectedException.none();
+
+  private final TestAmazonS3 s3 = new TestAmazonS3(0);
+  private final String path = "resultId";
+
+
+  private S3OutputConfig config;
+  private long maxResultsSize;
+  private long chunkSize;
+
+  @Before
+  public void setup() throws IOException
+  {
+    final File tempDir = temporaryFolder.newFolder();
+    chunkSize = 10L;
+    config = new S3OutputConfig()
+    {
+      @Override
+      public File getTempDir()
+      {
+        return tempDir;
+      }
+
+      @Override
+      public Long getChunkSize()
+      {
+        return chunkSize;
+      }
+
+      @Override
+      public long getMaxResultsSize()
+      {
+        return maxResultsSize;
+      }
+
+      @Override
+      public int getMaxTriesOnTransientError()
+      {
+        return 2;
+      }
+    };
+  }
+
+  @Test
+  public void testTooSmallChunkSize() throws IOException
+  {
+    maxResultsSize = 100_000_000_000L;
+    chunkSize = 9000_000L;
+
+    expectedException.expect(IllegalArgumentException.class);
+    expectedException.expectMessage(
+        "chunkSize[9000000] is too small for maxResultsSize[100000000000]. 
chunkSize should be at least [10000000]"
+    );
+    new RetryableS3OutputStream(config, s3, path).close();
+  }
+
+  @Test
+  public void testTooSmallChunkSizeMaxResultsSizeIsNotRetionalToMaxPartNum() 
throws IOException
+  {
+    maxResultsSize = 274_877_906_944L;
+    chunkSize = 2_7487_790;
+
+    expectedException.expect(IllegalArgumentException.class);
+    expectedException.expectMessage(
+        "chunkSize[27487790] is too small for maxResultsSize[274877906944]. 
chunkSize should be at least [27487791]"
+    );
+    new RetryableS3OutputStream(config, s3, path).close();
+  }
+
+  @Test
+  public void testTooLargeChunkSize() throws IOException
+  {
+    maxResultsSize = 1024L * 1024 * 1024 * 1024;
+    chunkSize = S3OutputConfig.S3_MULTIPART_UPLOAD_MAX_PART_SIZE_BYTES + 1;
+
+    expectedException.expect(IllegalArgumentException.class);
+    expectedException.expectMessage(
+        "chunkSize[5368709121] should be smaller than [5368709120]"
+    );
+    new RetryableS3OutputStream(config, s3, path).close();
+  }
+
+  @Test
+  public void testWriteAndHappy() throws IOException
+  {
+    maxResultsSize = 1000;
+    chunkSize = 10;
+    ByteBuffer bb = ByteBuffer.allocate(Integer.BYTES);
+    try (RetryableS3OutputStream out = new RetryableS3OutputStream(
+        config,
+        s3,
+        path,
+        false
+    )) {
+      for (int i = 0; i < 25; i++) {
+        bb.clear();
+        bb.putInt(i);
+        out.write(bb.array());
+      }
+    }
+    // each chunk is 10 bytes, so there should be 10 chunks.
+    Assert.assertEquals(10, s3.partRequests.size());
+    s3.assertCompleted(chunkSize, Integer.BYTES * 25);
+  }
+
+  @Test
+  public void testWriteSizeLargerThanConfiguredMaxChunkSizeShouldSucceed() 
throws IOException
+  {
+    maxResultsSize = 1000;
+    chunkSize = 10;
+    ByteBuffer bb = ByteBuffer.allocate(Integer.BYTES * 3);
+    try (RetryableS3OutputStream out = new RetryableS3OutputStream(
+        config,
+        s3,
+        path,
+        false
+    )) {
+      bb.clear();
+      bb.putInt(1);
+      bb.putInt(2);
+      bb.putInt(3);
+      out.write(bb.array());
+    }
+    // each chunk 10 bytes, so there should be 2 chunks.
+    Assert.assertEquals(2, s3.partRequests.size());
+    s3.assertCompleted(chunkSize, Integer.BYTES * 3);
+  }
+
+  @Test
+  public void testWriteSmallBufferShouldSucceed() throws IOException
+  {
+    maxResultsSize = 1000;
+    chunkSize = 128;
+    try (RetryableS3OutputStream out = new RetryableS3OutputStream(
+        config,
+        s3,
+        path,
+        false
+    )) {
+      for (int i = 0; i < 600; i++) {
+        out.write(i);
+      }
+    }
+    // each chunk 128 bytes, so there should be 5 chunks.
+    Assert.assertEquals(5, s3.partRequests.size());
+    s3.assertCompleted(chunkSize, 600);
+  }
+
+  @Test
+  public void testHitResultsSizeLimit() throws IOException
+  {
+    maxResultsSize = 50;
+    ByteBuffer bb = ByteBuffer.allocate(Integer.BYTES);
+    try (RetryableS3OutputStream out = new RetryableS3OutputStream(
+        config,
+        s3,
+        path,
+        false
+    )) {
+      for (int i = 0; i < 14; i++) {
+        bb.clear();
+        bb.putInt(i);
+        out.write(bb.array());
+      }
+
+      Assert.assertThrows(
+          "Exceeded max results size [50]",
+          IOException.class,
+          () -> {
+            bb.clear();
+            bb.putInt(14);
+            out.write(bb.array());
+          }
+      );
+    }
+
+    s3.assertAborted();
+  }
+
+  @Test
+  public void testSuccessToUploadAfterRetry() throws IOException
+  {
+    final TestAmazonS3 s3 = new TestAmazonS3(1);
+
+    maxResultsSize = 1000;
+    chunkSize = 10;
+    ByteBuffer bb = ByteBuffer.allocate(Integer.BYTES);
+    try (RetryableS3OutputStream out = new RetryableS3OutputStream(
+        config,
+        s3,
+        path,
+        false
+    )) {
+      for (int i = 0; i < 25; i++) {
+        bb.clear();
+        bb.putInt(i);
+        out.write(bb.array());
+      }
+    }
+    // each chunk is 10 bytes, so there should be 10 chunks.
+    Assert.assertEquals(10, s3.partRequests.size());
+    s3.assertCompleted(chunkSize, Integer.BYTES * 25);
+  }
+
+  @Test
+  public void testFailToUploadAfterRetries() throws IOException
+  {
+    final TestAmazonS3 s3 = new TestAmazonS3(3);
+
+    maxResultsSize = 1000;
+    ByteBuffer bb = ByteBuffer.allocate(Integer.BYTES);
+    try (RetryableS3OutputStream out = new RetryableS3OutputStream(
+        config,
+        s3,
+        path,
+        false
+    )) {
+      for (int i = 0; i < 2; i++) {
+        bb.clear();
+        bb.putInt(i);
+        out.write(bb.array());
+      }
+
+      expectedException.expect(RuntimeException.class);
+      
expectedException.expectCause(CoreMatchers.instanceOf(AmazonClientException.class));
+      expectedException.expectMessage("Upload failure test. Remaining failures 
[1]");
+      bb.clear();
+      bb.putInt(3);
+      out.write(bb.array());
+    }
+
+    s3.assertAborted();
+  }
+
+  private static class TestAmazonS3 extends ServerSideEncryptingAmazonS3
+  {
+    private final List<UploadPartRequest> partRequests = new ArrayList<>();
+
+    private int uploadFailuresLeft;
+    private boolean aborted = false;
+    @Nullable
+    private CompleteMultipartUploadRequest completeRequest;
+
+    private TestAmazonS3(int totalUploadFailures)
+    {
+      super(EasyMock.createMock(AmazonS3.class), new 
NoopServerSideEncryption());
+      this.uploadFailuresLeft = totalUploadFailures;
+    }
+
+    @Override
+    public InitiateMultipartUploadResult 
initiateMultipartUpload(InitiateMultipartUploadRequest request)
+        throws SdkClientException
+    {
+      InitiateMultipartUploadResult result = new 
InitiateMultipartUploadResult();
+      result.setUploadId("uploadId");
+      return result;
+    }
+
+    @Override
+    public UploadPartResult uploadPart(UploadPartRequest request) throws 
SdkClientException
+    {
+      if (uploadFailuresLeft > 0) {
+        throw new AmazonClientException(
+            new IOE("Upload failure test. Remaining failures [%s]", 
--uploadFailuresLeft)
+        );
+      }
+      partRequests.add(request);
+      UploadPartResult result = new UploadPartResult();
+      result.setETag(StringUtils.format("%s", request.getPartNumber()));
+      result.setPartNumber(request.getPartNumber());
+      return result;
+    }
+
+    @Override
+    public void abortMultipartUpload(AbortMultipartUploadRequest request) 
throws SdkClientException
+    {
+      aborted = true;

Review Comment:
   Due to the sensitivity of this term in some countries, can we perhaps use 
`canceled` instead?



##########
core/src/main/java/org/apache/druid/storage/StorageConnector.java:
##########
@@ -0,0 +1,103 @@
+/*
+ * 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;
+
+import org.apache.druid.guice.annotations.UnstableApi;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+
+/**
+ * Low level interface for interacting with different storage providers like 
S3, GCS, Azure and local file system.
+ * <p>
+ * For adding a new implementation of this interface in your extension extend 
{@link StorageConnectorProvider}.
+ * <p>
+ * For using the interface in your extension as a consumer, use 
JsonConfigProvider like:
+ * <ol>
+ * <li>{@code JsonConfigProvider.bind(binder, "druid,extension.custom.type", 
StorageConnectorProvider.class, Custom.class);}</li>
+ * <li>// bind the storage config provider {@code 
binder.bind(Key.get(StorageConnector.class, 
Custom.class)).toProvider(Key.get(StorageConnectorProvider.class, 
Custom.class)).in(LazySingleton.class);} </li>
+ * <li>// Use Named annotations to access the storageConnector instance in 
your custom extension.{@code @Custom StorageConnector storageConnector} </li>
+ * </ol>
+ * The final state of this inteface would have
+ * <ol>
+ * <li>Future Non blocking API's</li>
+ * <li>Offset based fetch</li>
+ * </ol>
+ */
+
+@UnstableApi
+public interface StorageConnector
+{
+
+  /**
+   * Check if the path exists in the underlying storage layer. Most 
implementations prepend the input path
+   * with a basePath.
+   *
+   * @param path
+   * @return true if path exists else false.
+   * @throws IOException
+   */
+  @SuppressWarnings("all")
+  boolean pathExists(String path) throws IOException;
+
+  /**
+   * Reads the data present at the path the underlying storage system. Most 
implementations prepend the input path
+   * with a basePath.
+   * The caller should take care of closing the stream when done or in case of 
error.
+   *
+   * @param path
+   * @return InputStream
+   * @throws IOException if the path is not present or the unable to read the 
data present on the path.
+   */
+  InputStream read(String path) throws IOException;
+
+  /**
+   * Open an {@link OutputStream} for writing data to the path in the 
underlying storage system.
+   * Most implementations prepend the input path with a basePath.
+   * Callers are adivised to namespace there files as there might be race 
conditions.
+   * The caller should take care of closing the stream when done or in case of 
error.
+   *
+   * @param path
+   * @return
+   * @throws IOException
+   */
+  OutputStream write(String path) throws IOException;

Review Comment:
   What are the semantics on a crash? Is the file left half written? Is it the 
responsibility of the caller to clean up such "orphan" files later?
   
   What are the semantics for replace? Do we support an atomic replace? Or, is 
that something that the caller must provide?
   
   I assume that, since this is a low-level API, the fancy stuff would be added 
by another layer on top. Correct?



##########
core/src/main/java/org/apache/druid/storage/StorageConnector.java:
##########
@@ -0,0 +1,103 @@
+/*
+ * 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;
+
+import org.apache.druid.guice.annotations.UnstableApi;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+
+/**
+ * Low level interface for interacting with different storage providers like 
S3, GCS, Azure and local file system.
+ * <p>
+ * For adding a new implementation of this interface in your extension extend 
{@link StorageConnectorProvider}.
+ * <p>
+ * For using the interface in your extension as a consumer, use 
JsonConfigProvider like:
+ * <ol>
+ * <li>{@code JsonConfigProvider.bind(binder, "druid,extension.custom.type", 
StorageConnectorProvider.class, Custom.class);}</li>
+ * <li>// bind the storage config provider {@code 
binder.bind(Key.get(StorageConnector.class, 
Custom.class)).toProvider(Key.get(StorageConnectorProvider.class, 
Custom.class)).in(LazySingleton.class);} </li>
+ * <li>// Use Named annotations to access the storageConnector instance in 
your custom extension.{@code @Custom StorageConnector storageConnector} </li>
+ * </ol>
+ * The final state of this inteface would have
+ * <ol>
+ * <li>Future Non blocking API's</li>
+ * <li>Offset based fetch</li>
+ * </ol>
+ */
+
+@UnstableApi
+public interface StorageConnector
+{
+
+  /**
+   * Check if the path exists in the underlying storage layer. Most 
implementations prepend the input path
+   * with a basePath.

Review Comment:
   Thanks much for providing Javadoc! It is tedious to write, but is _so_ 
useful to consumers of the API!
   
   This comment:
   
   > Most implementations prepend the input path with a basePath.
   
   Could be in the class Javadoc:
   
   > Most implementations provide a base path within the target file system. 
For example, for a local file system, the base path might be /tmp/druid. Each 
path given in this API is relative to that base path. The implementation 
computed the full path from the bast path and relative path provided in the 
call.
   
   Then, the info won't have to be repeated for every API method.



##########
core/src/main/java/org/apache/druid/storage/local/LocalFileStorageConnectorProvider.java:
##########
@@ -0,0 +1,58 @@
+/*
+ * 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.local;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonTypeName;
+import org.apache.druid.java.util.common.RE;
+import org.apache.druid.storage.StorageConnector;
+import org.apache.druid.storage.StorageConnectorProvider;
+
+import java.io.IOException;
+
+@JsonTypeName("local")
+public class LocalFileStorageConnectorProvider implements 
StorageConnectorProvider
+{
+  @JsonProperty
+  String basePath;
+
+  @JsonCreator
+  public LocalFileStorageConnectorProvider(String basePath)
+  {
+    this.basePath = basePath;
+  }
+
+  @Override
+  public StorageConnector get()
+  {
+    try {
+      return new LocalFileStorageConnector(basePath);
+    }
+    catch (IOException e) {
+      throw new RE(

Review Comment:
   `throw IAE` since the error is presumably related to the base path argument?



##########
core/src/test/java/org/apache/druid/storage/local/LocalFileStorageConnectorTest.java:
##########
@@ -0,0 +1,105 @@
+/*
+ * 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.local;
+
+import org.apache.druid.java.util.common.FileUtils;
+import org.apache.druid.storage.StorageConnector;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.UUID;
+
+public class LocalFileStorageConnectorTest
+{
+  private File tempDir = FileUtils.createTempDir();
+  private StorageConnector storageConnector = new 
LocalFileStorageConnectorProvider(tempDir.getPath()).get();
+
+  public LocalFileStorageConnectorTest()
+  {
+    tempDir.deleteOnExit();
+  }
+
+  @Test
+  public void sanityCheck() throws IOException
+  {
+    String uuid = UUID.randomUUID().toString();
+
+    //create file
+    createAndPopulateFile(uuid);
+
+    // check if file is created
+    Assert.assertTrue(storageConnector.pathExists(uuid));
+    Assert.assertTrue(new File(tempDir.getAbsolutePath() + "/" + 
uuid).exists());

Review Comment:
   ```java
   Assert.assertTrue(new File(tempDir, uuid).exists());
   ```
   
   Here and below.



##########
extensions-core/s3-extensions/pom.xml:
##########
@@ -116,6 +116,11 @@
       <artifactId>aws-java-sdk-sts</artifactId>
       <version>${aws.sdk.version}</version>
     </dependency>
+    <dependency>
+      <groupId>it.unimi.dsi</groupId>
+      <artifactId>fastutil-core</artifactId>
+      <scope>provided</scope>

Review Comment:
   Provided means that, at run time, in production, this jar will already be on 
the class path. Is that true in this case? Does core Druid or S3 already 
include this jar? If not, the scope has to be the default so that the jar is 
downloaded and put on the class path.



##########
extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3StorageConnectorModule.java:
##########
@@ -0,0 +1,44 @@
+/*
+ * 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.output;
+
+import com.fasterxml.jackson.databind.Module;
+import com.fasterxml.jackson.databind.module.SimpleModule;
+import com.google.inject.Binder;
+import org.apache.druid.initialization.DruidModule;
+
+import java.util.Collections;
+import java.util.List;
+
+public class S3StorageConnectorModule implements DruidModule
+{
+  @Override
+  public List<? extends Module> getJacksonModules()
+  {
+    return Collections.singletonList(
+        new 
SimpleModule(this.getClass().getSimpleName()).registerSubtypes(S3StorageConnectorProvider.class)
+    );
+  }
+
+  @Override
+  public void configure(Binder binder)
+  {
+  }

Review Comment:
   Thanks for the explanation. That hint about namespace based singleton clears 
up the confusion.



##########
core/src/main/java/org/apache/druid/storage/StorageConnector.java:
##########
@@ -0,0 +1,93 @@
+/*
+ * 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;
+
+import org.apache.druid.guice.annotations.UnstableApi;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+
+/**
+ * Low level interface for interacting with different storage providers like 
S3, GCS, Azure and local file system.
+ * For adding a new implementation of this interface in your extension extend 
{@link StorageConnectorProvider}.
+ * For using the interface in your extension as a consumer, use 
JsonConfigProvider like:
+ * 1. JsonConfigProvider.bind(binder, "druid,extension.custom.type", 
StorageConnectorProvider.class, Custom.class);
+ * // bind the storage config provider
+ * 2. binder.bind(Key.get(StorageConnector.class, 
Custom.class)).toProvider(Key.get(StorageConnectorProvider.class, 
Custom.class)).in(LazySingleton.class);
+ * // Use Named annotations to access the storageConnector instance in your 
custom extension.
+ * 3. @Custom StorageConnector storageConnector
+ * <p>
+ * The final state of this inteface would have
+ * * 1. Future Non blocking API's
+ * * 2. Offset based fetch
+ */
+
+@UnstableApi
+public interface StorageConnector
+{
+
+  /**
+   * Check if the relative path exists in the underlying storage layer.
+   *
+   * @param path
+   * @return true if path exists else false.
+   * @throws IOException
+   */
+  @SuppressWarnings("all")
+  boolean pathExists(String path) throws IOException;
+
+  /**
+   * Reads the data present at the relative path from the underlying storage 
system.
+   * The caller should take care of closing the stream when done or in case of 
error.
+   *
+   * @param path
+   * @return InputStream
+   * @throws IOException if the path is not present or the unable to read the 
data present on the path.
+   */
+  InputStream read(String path) throws IOException;
+
+  /**
+   * Open an {@link OutputStream} for writing data to the relative path in the 
underlying storage system.
+   * The caller should take care of closing the stream when done or in case of 
error.
+   *
+   * @param path
+   * @return
+   * @throws IOException
+   */
+  OutputStream write(String path) throws IOException;
+
+  /**
+   * Delete file present at the relative path.
+   *
+   * @param path
+   * @throws IOException
+   */
+  @SuppressWarnings("all")
+  void delete(String path) throws IOException;
+
+  /**
+   * Delete a directory pointed to by the path and also recursively deletes 
all files in said directory.
+   *
+   * @param path path
+   * @throws IOException
+   */
+  void deleteRecursively(String path) throws IOException;

Review Comment:
   Interfaces should not need to suppress warnings. The implementation can omit 
the throw clause:
   
   Interface:
   ```java
   void deleteFile(String path) throws IOException;
   ```
   
   S3 implementation:
   
   ```java
   public void deleteFile(String path) ...
   ```
   
   No `@SuppressWarnings` needed.



##########
core/src/main/java/org/apache/druid/storage/StorageConnector.java:
##########
@@ -0,0 +1,103 @@
+/*
+ * 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;
+
+import org.apache.druid.guice.annotations.UnstableApi;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+
+/**
+ * Low level interface for interacting with different storage providers like 
S3, GCS, Azure and local file system.
+ * <p>
+ * For adding a new implementation of this interface in your extension extend 
{@link StorageConnectorProvider}.
+ * <p>
+ * For using the interface in your extension as a consumer, use 
JsonConfigProvider like:
+ * <ol>
+ * <li>{@code JsonConfigProvider.bind(binder, "druid,extension.custom.type", 
StorageConnectorProvider.class, Custom.class);}</li>
+ * <li>// bind the storage config provider {@code 
binder.bind(Key.get(StorageConnector.class, 
Custom.class)).toProvider(Key.get(StorageConnectorProvider.class, 
Custom.class)).in(LazySingleton.class);} </li>
+ * <li>// Use Named annotations to access the storageConnector instance in 
your custom extension.{@code @Custom StorageConnector storageConnector} </li>
+ * </ol>

Review Comment:
   This example explains a lot! Thanks for providing it.
   
   Just a formatting nit:
   
   ```text
   <code><pre>
   JsonConfigProvider.bind(binder, "druid,extension.custom.type", 
StorageConnectorProvider.class, Custom.class);
   // bind the storage config provider
   binder.bind(Key.get(StorageConnector.class, 
Custom.class)).toProvider(Key.get(StorageConnectorProvider.class, 
Custom.class)).in(LazySingleton.class)
   // Use Named annotations to access the storageConnector instance in your 
custom extension.
   @Custom StorageConnector storageConnector
   </pre></code>
   ```
   
   Maybe add a note about:
   
   > configure the storage provider in the runtime.properties file:
   >
   > druid.extension.custom.type="s3"
   > druid.extension.custom.bucket="myBucket"
   > ...
   
   Also, "druid,extension..." should be "druid.extension" (dot rather than 
comma).
   



##########
core/src/main/java/org/apache/druid/storage/StorageConnector.java:
##########
@@ -0,0 +1,103 @@
+/*
+ * 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;
+
+import org.apache.druid.guice.annotations.UnstableApi;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+
+/**
+ * Low level interface for interacting with different storage providers like 
S3, GCS, Azure and local file system.
+ * <p>
+ * For adding a new implementation of this interface in your extension extend 
{@link StorageConnectorProvider}.
+ * <p>
+ * For using the interface in your extension as a consumer, use 
JsonConfigProvider like:
+ * <ol>
+ * <li>{@code JsonConfigProvider.bind(binder, "druid,extension.custom.type", 
StorageConnectorProvider.class, Custom.class);}</li>
+ * <li>// bind the storage config provider {@code 
binder.bind(Key.get(StorageConnector.class, 
Custom.class)).toProvider(Key.get(StorageConnectorProvider.class, 
Custom.class)).in(LazySingleton.class);} </li>
+ * <li>// Use Named annotations to access the storageConnector instance in 
your custom extension.{@code @Custom StorageConnector storageConnector} </li>
+ * </ol>
+ * The final state of this inteface would have
+ * <ol>
+ * <li>Future Non blocking API's</li>
+ * <li>Offset based fetch</li>
+ * </ol>
+ */
+
+@UnstableApi
+public interface StorageConnector
+{
+
+  /**
+   * Check if the path exists in the underlying storage layer. Most 
implementations prepend the input path
+   * with a basePath.
+   *
+   * @param path
+   * @return true if path exists else false.
+   * @throws IOException
+   */
+  @SuppressWarnings("all")

Review Comment:
   See comment about `@SuppressWarnings` below.



##########
core/src/main/java/org/apache/druid/storage/local/LocalFileStorageConnectorProvider.java:
##########
@@ -0,0 +1,58 @@
+/*
+ * 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.local;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonTypeName;
+import org.apache.druid.java.util.common.RE;
+import org.apache.druid.storage.StorageConnector;
+import org.apache.druid.storage.StorageConnectorProvider;
+
+import java.io.IOException;
+
+@JsonTypeName("local")
+public class LocalFileStorageConnectorProvider implements 
StorageConnectorProvider
+{
+  @JsonProperty
+  String basePath;
+
+  @JsonCreator
+  public LocalFileStorageConnectorProvider(String basePath)
+  {
+    this.basePath = basePath;
+  }
+
+  @Override
+  public StorageConnector get()
+  {
+    try {
+      return new LocalFileStorageConnector(basePath);
+    }
+    catch (IOException e) {
+      throw new RE(
+          e,
+          "Unable to create storage connector[%s] for basepath[%s]",

Review Comment:
   Suggestion:
   
   ```
   "Unable to create %s for base path [%s]"
   ```
   
   Desired result:
   
   ```
   Unable to create LocalFileStorageConnectorProvider for base path [~bob]
   ```



##########
core/src/main/java/org/apache/druid/storage/local/LocalFileStorageConnector.java:
##########
@@ -0,0 +1,120 @@
+/*
+ * 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.local;
+
+import org.apache.druid.java.util.common.FileUtils;
+import org.apache.druid.java.util.common.IAE;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.storage.StorageConnector;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+
+/**
+ * Implementation that uses local filesystem. All paths are appended with the 
base path, in such a way that its not visible
+ * to the users of this class.
+ */
+public class LocalFileStorageConnector implements StorageConnector
+{
+
+  private final String basePath;
+
+  public LocalFileStorageConnector(String basePath) throws IOException
+  {
+    this.basePath = basePath;
+    FileUtils.mkdirp(new File(basePath));
+  }
+
+  @Override
+  public boolean pathExists(String path)
+  {
+    return fileWithBasePath(path).exists();
+  }
+
+  /**
+   * Reads the file present as basePath + path. Will throw an IO exception in 
case the file is not present.
+   * Closing of the stream is the responsibility of the caller.
+   *
+   * @param path
+   * @return
+   * @throws IOException
+   */
+  @Override
+  public InputStream read(String path) throws IOException
+  {
+    return Files.newInputStream(fileWithBasePath(path).toPath());
+  }
+
+  /**
+   * Writes the file present with the materialized location as basePath + path.
+   * In case the parent directory does not exist, we create the parent dir 
recursively.
+   * Closing of the stream is the responsibility of the caller.
+   *
+   * @param path
+   * @return
+   * @throws IOException
+   */
+  @Override
+  public OutputStream write(String path) throws IOException
+  {
+    File toWrite = fileWithBasePath(path);
+    FileUtils.mkdirp(toWrite.getParentFile());
+    return Files.newOutputStream(toWrite.toPath());
+  }
+
+  /**
+   * Deletes the file present at the location basePath + path. Throws an 
excecption in case a dir is encountered.
+   *
+   * @param path
+   * @throws IOException
+   */
+  @Override
+  public void deleteFile(String path) throws IOException
+  {
+    File toDelete = fileWithBasePath(path);
+    if (toDelete.isDirectory()) {
+      throw new IAE(StringUtils.format(
+          "Found a directory on path[%s]. Please use deleteRecusively to 
delete dirs", path));
+    }
+    Files.delete(fileWithBasePath(path).toPath());
+  }
+
+  /**
+   * Deletes the files and sub dirs present at the basePath + dirName. Also 
removes the dirName
+   *
+   * @param dirName path
+   * @throws IOException
+   */
+  @Override
+  public void deleteRecursively(String dirName) throws IOException
+  {
+    FileUtils.deleteDirectory(fileWithBasePath(dirName));
+  }
+
+  private File fileWithBasePath(String path)
+  {
+    return new File(Paths.get(basePath, path).toString());

Review Comment:
   This works, but the following is simpler:
   
   ```java
   return new File(new File(basePath), path);
   ```
   
   In fact, we could just store `basePath` as a `File`, then we just need:
   
   ```java
   return new File(basePath, path);
   ```



##########
extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3OutputConfig.java:
##########
@@ -0,0 +1,95 @@
+/*
+ * 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.output;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+import org.apache.druid.java.util.common.HumanReadableBytes;
+import org.apache.druid.java.util.common.HumanReadableBytesRange;
+import org.apache.druid.java.util.common.RetryUtils;
+
+import javax.annotation.Nullable;
+import java.io.File;
+
+public class S3OutputConfig
+{
+  @JsonProperty
+  private String bucket;
+
+  @JsonProperty
+  private String prefix;
+
+  @JsonProperty
+  private File tempDir;
+
+  @Nullable
+  @JsonProperty
+  @HumanReadableBytesRange(
+      min = RetriableS3OutputStream.S3_MULTIPART_UPLOAD_MIN_PART_SIZE,
+      max = RetriableS3OutputStream.S3_MULTIPART_UPLOAD_MAX_PART_SIZE
+  ) // limits of s3 multipart upload
+  private HumanReadableBytes chunkSize;

Review Comment:
   Since the API here is generic, clients would find themselves needing to be 
aware of the difference between local, S3, GCP, etc. A good future enhancement 
is that clients just write data and the implementations handle any chunking. 
Maybe leave an issue for that also: the issue will become clear as we use 
implementations other than S3.



##########
core/src/test/java/org/apache/druid/storage/local/LocalFileStorageConnectorTest.java:
##########
@@ -0,0 +1,105 @@
+/*
+ * 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.local;
+
+import org.apache.druid.java.util.common.FileUtils;
+import org.apache.druid.storage.StorageConnector;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.UUID;
+
+public class LocalFileStorageConnectorTest
+{
+  private File tempDir = FileUtils.createTempDir();
+  private StorageConnector storageConnector = new 
LocalFileStorageConnectorProvider(tempDir.getPath()).get();
+
+  public LocalFileStorageConnectorTest()
+  {
+    tempDir.deleteOnExit();
+  }
+
+  @Test
+  public void sanityCheck() throws IOException
+  {
+    String uuid = UUID.randomUUID().toString();
+
+    //create file
+    createAndPopulateFile(uuid);
+
+    // check if file is created
+    Assert.assertTrue(storageConnector.pathExists(uuid));
+    Assert.assertTrue(new File(tempDir.getAbsolutePath() + "/" + 
uuid).exists());
+
+    // check contents
+    checkContents(uuid);
+
+    // delete file
+    storageConnector.deleteFile(uuid);
+    Assert.assertFalse(new File(tempDir.getAbsolutePath() + "/" + 
uuid).exists());
+  }
+
+  @Test
+  public void deleteRecursivelyTest() throws IOException
+  {
+    String topLeveldir = "top" + UUID.randomUUID();
+    String uuid_base = topLeveldir + "/" + UUID.randomUUID();

Review Comment:
   Nit: `topLevelDir` - consistent camel casing.



##########
extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3OutputConfig.java:
##########
@@ -0,0 +1,95 @@
+/*
+ * 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.output;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+import org.apache.druid.java.util.common.HumanReadableBytes;
+import org.apache.druid.java.util.common.HumanReadableBytesRange;
+import org.apache.druid.java.util.common.RetryUtils;
+
+import javax.annotation.Nullable;
+import java.io.File;
+
+public class S3OutputConfig
+{
+  @JsonProperty
+  private String bucket;
+
+  @JsonProperty
+  private String prefix;
+
+  @JsonProperty
+  private File tempDir;

Review Comment:
   Thanks for filing the issue.



##########
core/src/test/java/org/apache/druid/storage/local/LocalFileStorageConnectorTest.java:
##########
@@ -0,0 +1,105 @@
+/*
+ * 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.local;
+
+import org.apache.druid.java.util.common.FileUtils;
+import org.apache.druid.storage.StorageConnector;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.UUID;
+
+public class LocalFileStorageConnectorTest
+{
+  private File tempDir = FileUtils.createTempDir();

Review Comment:
   Would be great to use JUnit temporary files here. Is there a reason why they 
won't work? A bit of a pain for the reader to have to work out how temp dirs 
work in this test and why they have to be unlike other tests that use the JUnit 
mechanism.



##########
extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3OutputConfig.java:
##########
@@ -0,0 +1,95 @@
+/*
+ * 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.output;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+import org.apache.druid.java.util.common.HumanReadableBytes;
+import org.apache.druid.java.util.common.HumanReadableBytesRange;
+import org.apache.druid.java.util.common.RetryUtils;
+
+import javax.annotation.Nullable;
+import java.io.File;
+
+public class S3OutputConfig
+{
+  @JsonProperty
+  private String bucket;
+
+  @JsonProperty
+  private String prefix;
+
+  @JsonProperty
+  private File tempDir;
+
+  @Nullable
+  @JsonProperty
+  @HumanReadableBytesRange(
+      min = RetriableS3OutputStream.S3_MULTIPART_UPLOAD_MIN_PART_SIZE,
+      max = RetriableS3OutputStream.S3_MULTIPART_UPLOAD_MAX_PART_SIZE
+  ) // limits of s3 multipart upload
+  private HumanReadableBytes chunkSize;
+
+  /**
+   * Max size for each query results. This limit relies on the s3 multipart 
upload limits.
+   * See https://docs.aws.amazon.com/AmazonS3/latest/userguide/qfacts.html for 
more details.
+   *
+   * @see RetriableS3OutputStream
+   */
+  @JsonProperty
+  @HumanReadableBytesRange(min = 5L * 1024 * 1024, max = 5L * 1024 * 1024 * 
1024 * 1024)
+  private HumanReadableBytes maxResultsSize = new HumanReadableBytes("100MiB");

Review Comment:
   Since this is an S3 thing, can we link to the S3 docs so the reader know 
where to find the info?



##########
extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3OutputConfig.java:
##########
@@ -0,0 +1,95 @@
+/*
+ * 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.output;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+import org.apache.druid.java.util.common.HumanReadableBytes;
+import org.apache.druid.java.util.common.HumanReadableBytesRange;
+import org.apache.druid.java.util.common.RetryUtils;
+
+import javax.annotation.Nullable;
+import java.io.File;
+
+public class S3OutputConfig
+{
+  @JsonProperty
+  private String bucket;
+
+  @JsonProperty
+  private String prefix;
+
+  @JsonProperty
+  private File tempDir;

Review Comment:
   Over time, we should have a unified config system so that there is a single 
Druid temp dir, and each extension carves out a subdirectory within it. Else, 
it is up to the poor user to copy/paste the same path into multiple places in 
the configs. To us developers, each extension is its own thing. To the user, 
there is one Druid and one file system, they don't want to have to set a half 
dozen different "temp" directories.



##########
extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/ServerSideEncryptingAmazonS3.java:
##########
@@ -43,9 +51,10 @@
 
 /**
  * {@link AmazonS3} wrapper with {@link ServerSideEncryption}. Every {@link 
AmazonS3#putObject},
- * {@link AmazonS3#copyObject}, {@link AmazonS3#getObject}, and {@link 
AmazonS3#getObjectMetadata} methods should be
+ * {@link AmazonS3#copyObject}, {@link AmazonS3#getObject}, and {@link 
AmazonS3#getObjectMetadata},
+ * {@link AmazonS3#initiateMultipartUpload}, @{@link AmazonS3#uploadPart} 
methods should be
  * wrapped using ServerSideEncryption.
- *
+ * <p>

Review Comment:
   Note for future work: if encryption is needed, and there is only one right 
way to do it, then it should be done automatically by an 
`EncryptedS3StorageConnector` rather than forcing clients to add the code. 
Forcing clients to do that means clients have to be aware of the target file 
system, which is what this great new API tries to avoid.



##########
extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3OutputConfig.java:
##########
@@ -0,0 +1,95 @@
+/*
+ * 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.output;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+import org.apache.druid.java.util.common.HumanReadableBytes;
+import org.apache.druid.java.util.common.HumanReadableBytesRange;
+import org.apache.druid.java.util.common.RetryUtils;
+
+import javax.annotation.Nullable;
+import java.io.File;
+
+public class S3OutputConfig
+{
+  @JsonProperty
+  private String bucket;

Review Comment:
   Is putting credentials in the config file the correct solution? Should there 
be some form of credentials provider? In this day and age, folks usually don't 
put their actual S3 credentials in files that are presumably put in Git for 
configuration-as-code.



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

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to