From 26f5bfb90ae5ccdbb36bf3d20ad37c2a6fd84afd Mon Sep 17 00:00:00 2001 From: Laksh Singla Date: Wed, 5 Jul 2023 10:41:44 +0530 Subject: [PATCH 01/28] initial commit --- .../storage/s3/output/S3StorageConnector.java | 15 +- .../druid/storage/StorageConnectorModule.java | 6 +- .../remote/ChunkingStorageConnector.java | 185 ++++++++++++++++++ 3 files changed, 203 insertions(+), 3 deletions(-) create mode 100644 processing/src/main/java/org/apache/druid/storage/remote/ChunkingStorageConnector.java diff --git a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3StorageConnector.java b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3StorageConnector.java index 81dfb5747621..a8cbc7ec095a 100644 --- a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3StorageConnector.java +++ b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3StorageConnector.java @@ -38,6 +38,7 @@ import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.storage.StorageConnector; +import org.apache.druid.storage.remote.ChunkingStorageConnector; import org.apache.druid.storage.s3.S3Utils; import org.apache.druid.storage.s3.ServerSideEncryptingAmazonS3; @@ -60,7 +61,7 @@ /** * In this implementation, all remote calls to aws s3 are retried {@link S3OutputConfig#getMaxRetry()} times. */ -public class S3StorageConnector implements StorageConnector +public class S3StorageConnector extends ChunkingStorageConnector { private static final Logger log = new Logger(S3StorageConnector.class); @@ -127,6 +128,18 @@ public InputStream readRange(String path, long from, long size) ); } + @Override + public BuildInputStreamParams buildInputParams(String path) + { + return null; + } + + @Override + public BuildInputStreamParams buildInputParams(String path, long from, long size) + { + return null; + } + private InputStream buildInputStream(GetObjectRequest getObjectRequest, String path) { // fetch the size of the whole object to make chunks diff --git a/processing/src/main/java/org/apache/druid/storage/StorageConnectorModule.java b/processing/src/main/java/org/apache/druid/storage/StorageConnectorModule.java index 9682ac01d88e..a0bf3a91f0cc 100644 --- a/processing/src/main/java/org/apache/druid/storage/StorageConnectorModule.java +++ b/processing/src/main/java/org/apache/druid/storage/StorageConnectorModule.java @@ -33,8 +33,10 @@ public class StorageConnectorModule implements DruidModule @Override public List getJacksonModules() { - return ImmutableList.of(new SimpleModule(StorageConnector.class.getSimpleName()).registerSubtypes( - LocalFileStorageConnectorProvider.class)); + return ImmutableList.of( + new SimpleModule(StorageConnector.class.getSimpleName()) + .registerSubtypes(LocalFileStorageConnectorProvider.class) + ); } @Override diff --git a/processing/src/main/java/org/apache/druid/storage/remote/ChunkingStorageConnector.java b/processing/src/main/java/org/apache/druid/storage/remote/ChunkingStorageConnector.java new file mode 100644 index 000000000000..f166139949ac --- /dev/null +++ b/processing/src/main/java/org/apache/druid/storage/remote/ChunkingStorageConnector.java @@ -0,0 +1,185 @@ +package org.apache.druid.storage.remote; + +import com.google.common.base.Predicate; +import com.google.common.base.Predicates; +import org.apache.commons.io.input.NullInputStream; +import org.apache.druid.data.input.impl.RetryingInputStream; +import org.apache.druid.data.input.impl.prefetch.ObjectOpenFunction; +import org.apache.druid.java.util.common.FileUtils; +import org.apache.druid.java.util.common.IAE; +import org.apache.druid.java.util.common.IOE; +import org.apache.druid.java.util.common.RE; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.storage.StorageConnector; + +import java.io.File; +import java.io.FileInputStream; +import java.io.FileNotFoundException; +import java.io.IOException; +import java.io.InputStream; +import java.io.SequenceInputStream; +import java.util.Enumeration; +import java.util.UUID; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicLong; +import java.util.function.Supplier; + +public abstract class ChunkingStorageConnector implements StorageConnector +{ + private static final long DOWNLOAD_MAX_CHUNK_SIZE = 100_000_000; + + public ChunkingStorageConnector() + { + + } + + @Override + public InputStream read(String path) throws IOException + { + return buildInputStream(buildInputParams(path)); + } + + @Override + public InputStream readRange(String path, long from, long size) throws IOException + { + return buildInputStream(buildInputParams(path, from, size)); + } + + public abstract BuildInputStreamParams buildInputParams(String path); + + public abstract BuildInputStreamParams buildInputParams(String path, long from, long size); + + private InputStream buildInputStream(BuildInputStreamParams params) + { + AtomicLong currentReadStartPosition = new AtomicLong(params.start); + long readEnd = params.end; + + AtomicBoolean isSequenceStreamClosed = new AtomicBoolean(false); + + return new SequenceInputStream( + + new Enumeration() + { + boolean initStream = false; + + @Override + public boolean hasMoreElements() + { + // Checking if the stream was already closed. If it was, then don't iterate over the remaining chunks + // SequenceInputStream's close method closes all the chunk streams in its close. Since we're opening them + // lazily, we don't need to close them. + if (isSequenceStreamClosed.get()) { + return false; + } + // Don't stop until the whole object is downloaded + return currentReadStartPosition.get() < readEnd; + } + + @Override + public InputStream nextElement() + { + if (!initStream) { + initStream = true; + return new NullInputStream(); + } + + File outFile = new File( + params.tempDirSupplier.get().getAbsolutePath(), + UUID.randomUUID().toString() + ); + + // exclusive + long currentReadEndPosition = Math.min( + currentReadStartPosition.get() + DOWNLOAD_MAX_CHUNK_SIZE, + readEnd + ); + + try { + if (!outFile.createNewFile()) { + throw new IOE( + StringUtils.format( + "Could not create temporary file [%s] for copying [%s]", + outFile.getAbsolutePath(), + params.cloudStoragePath + ) + ); + } + + FileUtils.copyLarge( + () -> new RetryingInputStream( + params.objectSupplier.getObject(currentReadStartPosition.get(), currentReadEndPosition), + params.objectOpenFunction, + params.retryCondition, + params.maxRetry + ), + outFile, + new byte[8 * 1024], + Predicates.alwaysFalse(), + 1, + StringUtils.format( + "Retrying copying of [%s] to [%s]", + params.cloudStoragePath, + outFile.getAbsolutePath() + ) + ); + } + catch (IOException e) { + throw new RE(e, StringUtils.format("Unable to copy [%s] to [%s]", params.cloudStoragePath, outFile)); + } + + try { + AtomicBoolean fileInputStreamClosed = new AtomicBoolean(false); + return new FileInputStream(outFile) + { + @Override + public void close() throws IOException + { + // close should be idempotent + if (fileInputStreamClosed.get()) { + return; + } + fileInputStreamClosed.set(true); + super.close(); + // since endPoint is inclusive in s3's get request API, the next currReadStart is endpoint + 1 + currentReadStartPosition.set(currentReadEndPosition); + if (!outFile.delete()) { + throw new RE("Cannot delete temp file [%s]", outFile); + } + } + + }; + } + catch (FileNotFoundException e) { + throw new RE(e, StringUtils.format("Unable to find temp file [%s]", outFile)); + } + } + } + ) + { + @Override + public void close() throws IOException + { + isSequenceStreamClosed.set(true); + super.close(); + } + }; + } + + // Start inclusive, end exclusive + public static class BuildInputStreamParams + { + public long start; + public long end; + public String cloudStoragePath; + public GetObjectFromRangeFunction objectSupplier; + public ObjectOpenFunction objectOpenFunction; + public Predicate retryCondition; + public int maxRetry; + public Supplier tempDirSupplier; + } + + public interface GetObjectFromRangeFunction + { + T getObject(long start, long end); + } +} From 2b9787a340466531d0d653c7a4e3e14426d6c1e0 Mon Sep 17 00:00:00 2001 From: Laksh Singla Date: Fri, 7 Jul 2023 00:32:49 +0530 Subject: [PATCH 02/28] classes n code --- .../druid/storage/google/output/GoogleOutputConfig.java | 5 +++++ .../druid/storage/google/output/GoogleStorageConnector.java | 5 +++++ .../storage/google/output/GoogleStorageConnectorModule.java | 5 +++++ .../google/output/GoogleStorageConnectorProvider.java | 5 +++++ 4 files changed, 20 insertions(+) create mode 100644 extensions-core/google-extensions/src/main/java/org/apache/druid/storage/google/output/GoogleOutputConfig.java create mode 100644 extensions-core/google-extensions/src/main/java/org/apache/druid/storage/google/output/GoogleStorageConnector.java create mode 100644 extensions-core/google-extensions/src/main/java/org/apache/druid/storage/google/output/GoogleStorageConnectorModule.java create mode 100644 extensions-core/google-extensions/src/main/java/org/apache/druid/storage/google/output/GoogleStorageConnectorProvider.java diff --git a/extensions-core/google-extensions/src/main/java/org/apache/druid/storage/google/output/GoogleOutputConfig.java b/extensions-core/google-extensions/src/main/java/org/apache/druid/storage/google/output/GoogleOutputConfig.java new file mode 100644 index 000000000000..ab093ff054e0 --- /dev/null +++ b/extensions-core/google-extensions/src/main/java/org/apache/druid/storage/google/output/GoogleOutputConfig.java @@ -0,0 +1,5 @@ +package org.apache.druid.storage.google.output; + +public class GoogleOutputConfig +{ +} diff --git a/extensions-core/google-extensions/src/main/java/org/apache/druid/storage/google/output/GoogleStorageConnector.java b/extensions-core/google-extensions/src/main/java/org/apache/druid/storage/google/output/GoogleStorageConnector.java new file mode 100644 index 000000000000..6ae9b06a1438 --- /dev/null +++ b/extensions-core/google-extensions/src/main/java/org/apache/druid/storage/google/output/GoogleStorageConnector.java @@ -0,0 +1,5 @@ +package org.apache.druid.storage.google.output; + +public class GoogleStorageConnector +{ +} diff --git a/extensions-core/google-extensions/src/main/java/org/apache/druid/storage/google/output/GoogleStorageConnectorModule.java b/extensions-core/google-extensions/src/main/java/org/apache/druid/storage/google/output/GoogleStorageConnectorModule.java new file mode 100644 index 000000000000..de9f7e2b785e --- /dev/null +++ b/extensions-core/google-extensions/src/main/java/org/apache/druid/storage/google/output/GoogleStorageConnectorModule.java @@ -0,0 +1,5 @@ +package org.apache.druid.storage.google.output; + +public class GoogleStorageConnectorModule +{ +} diff --git a/extensions-core/google-extensions/src/main/java/org/apache/druid/storage/google/output/GoogleStorageConnectorProvider.java b/extensions-core/google-extensions/src/main/java/org/apache/druid/storage/google/output/GoogleStorageConnectorProvider.java new file mode 100644 index 000000000000..bfe81a9339a6 --- /dev/null +++ b/extensions-core/google-extensions/src/main/java/org/apache/druid/storage/google/output/GoogleStorageConnectorProvider.java @@ -0,0 +1,5 @@ +package org.apache.druid.storage.google.output; + +public class GoogleStorageConnectorProvider +{ +} From d41e967617ec35fbc611ff4a89ffc71950e5cb57 Mon Sep 17 00:00:00 2001 From: Laksh Singla Date: Fri, 7 Jul 2023 22:54:54 +0530 Subject: [PATCH 03/28] add stubs for other classes --- .../google/output/GoogleOutputConfig.java | 38 ++++ .../google/output/GoogleStorageConnector.java | 76 +++++++- .../output/GoogleStorageConnectorModule.java | 19 ++ .../GoogleStorageConnectorProvider.java | 19 ++ .../storage/s3/output/S3StorageConnector.java | 8 +- .../remote/ChunkingStorageConnector.java | 47 ++--- .../ChunkingStorageConnectorParameters.java | 179 ++++++++++++++++++ 7 files changed, 349 insertions(+), 37 deletions(-) create mode 100644 processing/src/main/java/org/apache/druid/storage/remote/ChunkingStorageConnectorParameters.java diff --git a/extensions-core/google-extensions/src/main/java/org/apache/druid/storage/google/output/GoogleOutputConfig.java b/extensions-core/google-extensions/src/main/java/org/apache/druid/storage/google/output/GoogleOutputConfig.java index ab093ff054e0..d18e31273119 100644 --- a/extensions-core/google-extensions/src/main/java/org/apache/druid/storage/google/output/GoogleOutputConfig.java +++ b/extensions-core/google-extensions/src/main/java/org/apache/druid/storage/google/output/GoogleOutputConfig.java @@ -1,5 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + package org.apache.druid.storage.google.output; +import org.apache.druid.java.util.common.HumanReadableBytes; + +import java.io.File; + public class GoogleOutputConfig { + + String bucket; + String prefix; + File tempDir; + HumanReadableBytes chunkSize; + Integer maxRetry; + + public GoogleOutputConfig( + String bucket, + String prefix, + File tempDir, + HumanReadableBytes chunkSize, + Integer maxRetry + ) { + } } diff --git a/extensions-core/google-extensions/src/main/java/org/apache/druid/storage/google/output/GoogleStorageConnector.java b/extensions-core/google-extensions/src/main/java/org/apache/druid/storage/google/output/GoogleStorageConnector.java index 6ae9b06a1438..b819cf090daf 100644 --- a/extensions-core/google-extensions/src/main/java/org/apache/druid/storage/google/output/GoogleStorageConnector.java +++ b/extensions-core/google-extensions/src/main/java/org/apache/druid/storage/google/output/GoogleStorageConnector.java @@ -1,5 +1,79 @@ +/* + * 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.google.output; -public class GoogleStorageConnector +import com.google.api.services.storage.model.StorageObject; +import org.apache.druid.storage.remote.ChunkingStorageConnector; +import org.apache.druid.storage.remote.ChunkingStorageConnectorParameters; + +import java.io.IOException; +import java.io.OutputStream; +import java.util.Iterator; + +public class GoogleStorageConnector extends ChunkingStorageConnector { + @Override + public boolean pathExists(String path) + { + return false; + } + + @Override + public OutputStream write(String path) throws IOException + { + return null; + } + + @Override + public void deleteFile(String path) throws IOException + { + + } + + @Override + public void deleteFiles(Iterable paths) throws IOException + { + + } + + @Override + public void deleteRecursively(String path) throws IOException + { + + } + + @Override + public Iterator listDir(String dirName) throws IOException + { + return null; + } + + @Override + public ChunkingStorageConnectorParameters buildInputParams(String path) + { + return null; + } + + @Override + public ChunkingStorageConnectorParameters buildInputParams(String path, long from, long size) + { + return null; + } } diff --git a/extensions-core/google-extensions/src/main/java/org/apache/druid/storage/google/output/GoogleStorageConnectorModule.java b/extensions-core/google-extensions/src/main/java/org/apache/druid/storage/google/output/GoogleStorageConnectorModule.java index de9f7e2b785e..b3594730c2bd 100644 --- a/extensions-core/google-extensions/src/main/java/org/apache/druid/storage/google/output/GoogleStorageConnectorModule.java +++ b/extensions-core/google-extensions/src/main/java/org/apache/druid/storage/google/output/GoogleStorageConnectorModule.java @@ -1,3 +1,22 @@ +/* + * 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.google.output; public class GoogleStorageConnectorModule diff --git a/extensions-core/google-extensions/src/main/java/org/apache/druid/storage/google/output/GoogleStorageConnectorProvider.java b/extensions-core/google-extensions/src/main/java/org/apache/druid/storage/google/output/GoogleStorageConnectorProvider.java index bfe81a9339a6..1bb91b77e541 100644 --- a/extensions-core/google-extensions/src/main/java/org/apache/druid/storage/google/output/GoogleStorageConnectorProvider.java +++ b/extensions-core/google-extensions/src/main/java/org/apache/druid/storage/google/output/GoogleStorageConnectorProvider.java @@ -1,3 +1,22 @@ +/* + * 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.google.output; public class GoogleStorageConnectorProvider diff --git a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3StorageConnector.java b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3StorageConnector.java index a8cbc7ec095a..e238c12b1ddc 100644 --- a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3StorageConnector.java +++ b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3StorageConnector.java @@ -37,8 +37,8 @@ import org.apache.druid.java.util.common.RE; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.logger.Logger; -import org.apache.druid.storage.StorageConnector; import org.apache.druid.storage.remote.ChunkingStorageConnector; +import org.apache.druid.storage.remote.ChunkingStorageConnectorParameters; import org.apache.druid.storage.s3.S3Utils; import org.apache.druid.storage.s3.ServerSideEncryptingAmazonS3; @@ -61,7 +61,7 @@ /** * In this implementation, all remote calls to aws s3 are retried {@link S3OutputConfig#getMaxRetry()} times. */ -public class S3StorageConnector extends ChunkingStorageConnector +public class S3StorageConnector extends ChunkingStorageConnector { private static final Logger log = new Logger(S3StorageConnector.class); @@ -129,13 +129,13 @@ public InputStream readRange(String path, long from, long size) } @Override - public BuildInputStreamParams buildInputParams(String path) + public ChunkingStorageConnectorParameters buildInputParams(String path) { return null; } @Override - public BuildInputStreamParams buildInputParams(String path, long from, long size) + public ChunkingStorageConnectorParameters buildInputParams(String path, long from, long size) { return null; } diff --git a/processing/src/main/java/org/apache/druid/storage/remote/ChunkingStorageConnector.java b/processing/src/main/java/org/apache/druid/storage/remote/ChunkingStorageConnector.java index f166139949ac..d01a8eee3eb6 100644 --- a/processing/src/main/java/org/apache/druid/storage/remote/ChunkingStorageConnector.java +++ b/processing/src/main/java/org/apache/druid/storage/remote/ChunkingStorageConnector.java @@ -1,12 +1,9 @@ package org.apache.druid.storage.remote; -import com.google.common.base.Predicate; import com.google.common.base.Predicates; import org.apache.commons.io.input.NullInputStream; import org.apache.druid.data.input.impl.RetryingInputStream; -import org.apache.druid.data.input.impl.prefetch.ObjectOpenFunction; import org.apache.druid.java.util.common.FileUtils; -import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.IOE; import org.apache.druid.java.util.common.RE; import org.apache.druid.java.util.common.StringUtils; @@ -22,7 +19,6 @@ import java.util.UUID; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicLong; -import java.util.function.Supplier; public abstract class ChunkingStorageConnector implements StorageConnector { @@ -40,19 +36,19 @@ public InputStream read(String path) throws IOException } @Override - public InputStream readRange(String path, long from, long size) throws IOException + public InputStream readRange(String path, long from, long size) { return buildInputStream(buildInputParams(path, from, size)); } - public abstract BuildInputStreamParams buildInputParams(String path); + public abstract ChunkingStorageConnectorParameters buildInputParams(String path); - public abstract BuildInputStreamParams buildInputParams(String path, long from, long size); + public abstract ChunkingStorageConnectorParameters buildInputParams(String path, long from, long size); - private InputStream buildInputStream(BuildInputStreamParams params) + private InputStream buildInputStream(ChunkingStorageConnectorParameters params) { - AtomicLong currentReadStartPosition = new AtomicLong(params.start); - long readEnd = params.end; + AtomicLong currentReadStartPosition = new AtomicLong(params.getStart()); + long readEnd = params.getEnd(); AtomicBoolean isSequenceStreamClosed = new AtomicBoolean(false); @@ -84,7 +80,7 @@ public InputStream nextElement() } File outFile = new File( - params.tempDirSupplier.get().getAbsolutePath(), + params.getTempDirSupplier().get().getAbsolutePath(), UUID.randomUUID().toString() ); @@ -100,17 +96,17 @@ public InputStream nextElement() StringUtils.format( "Could not create temporary file [%s] for copying [%s]", outFile.getAbsolutePath(), - params.cloudStoragePath + params.getCloudStoragePath() ) ); } FileUtils.copyLarge( - () -> new RetryingInputStream( - params.objectSupplier.getObject(currentReadStartPosition.get(), currentReadEndPosition), - params.objectOpenFunction, - params.retryCondition, - params.maxRetry + () -> new RetryingInputStream<>( + params.getObjectSupplier().getObject(currentReadStartPosition.get(), currentReadEndPosition), + params.getObjectOpenFunction(), + params.getRetryCondition(), + params.getMaxRetry() ), outFile, new byte[8 * 1024], @@ -118,13 +114,13 @@ public InputStream nextElement() 1, StringUtils.format( "Retrying copying of [%s] to [%s]", - params.cloudStoragePath, + params.getCloudStoragePath(), outFile.getAbsolutePath() ) ); } catch (IOException e) { - throw new RE(e, StringUtils.format("Unable to copy [%s] to [%s]", params.cloudStoragePath, outFile)); + throw new RE(e, StringUtils.format("Unable to copy [%s] to [%s]", params.getCloudStoragePath(), outFile)); } try { @@ -165,19 +161,6 @@ public void close() throws IOException }; } - // Start inclusive, end exclusive - public static class BuildInputStreamParams - { - public long start; - public long end; - public String cloudStoragePath; - public GetObjectFromRangeFunction objectSupplier; - public ObjectOpenFunction objectOpenFunction; - public Predicate retryCondition; - public int maxRetry; - public Supplier tempDirSupplier; - } - public interface GetObjectFromRangeFunction { T getObject(long start, long end); diff --git a/processing/src/main/java/org/apache/druid/storage/remote/ChunkingStorageConnectorParameters.java b/processing/src/main/java/org/apache/druid/storage/remote/ChunkingStorageConnectorParameters.java new file mode 100644 index 000000000000..ec46a3f21d84 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/storage/remote/ChunkingStorageConnectorParameters.java @@ -0,0 +1,179 @@ +/* + * 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.remote; + +import com.google.common.base.Preconditions; +import com.google.common.base.Predicate; +import org.apache.druid.data.input.impl.prefetch.ObjectOpenFunction; + +import java.io.File; +import java.util.function.Supplier; + +// Start inclusive, end exclusive +public class ChunkingStorageConnectorParameters +{ + private final long start; + private final long end; + private final String cloudStoragePath; + private final ChunkingStorageConnector.GetObjectFromRangeFunction objectSupplier; + private final ObjectOpenFunction objectOpenFunction; + private final Predicate retryCondition; + private final int maxRetry; + private final Supplier tempDirSupplier; + + public ChunkingStorageConnectorParameters( + long start, + long end, + String cloudStoragePath, + ChunkingStorageConnector.GetObjectFromRangeFunction objectSupplier, + ObjectOpenFunction objectOpenFunction, + Predicate retryCondition, + int maxRetry, + Supplier tempDirSupplier + ) + { + this.start = start; + this.end = end; + this.cloudStoragePath = cloudStoragePath; + this.objectSupplier = objectSupplier; + this.objectOpenFunction = objectOpenFunction; + this.retryCondition = retryCondition; + this.maxRetry = maxRetry; + this.tempDirSupplier = tempDirSupplier; + } + + public long getStart() + { + return start; + } + + public long getEnd() + { + return end; + } + + public String getCloudStoragePath() + { + return cloudStoragePath; + } + + public ChunkingStorageConnector.GetObjectFromRangeFunction getObjectSupplier() + { + return objectSupplier; + } + + public ObjectOpenFunction getObjectOpenFunction() + { + return objectOpenFunction; + } + + public Predicate getRetryCondition() + { + return retryCondition; + } + + public int getMaxRetry() + { + return maxRetry; + } + + public Supplier getTempDirSupplier() + { + return tempDirSupplier; + } + + private static class Builder + { + private long start; + private long end; + private String cloudStoragePath; + private ChunkingStorageConnector.GetObjectFromRangeFunction objectSupplier; + private ObjectOpenFunction objectOpenFunction; + private Predicate retryCondition; + private int maxRetry; + private Supplier tempDirSupplier; + + + public Builder start(long start) + { + this.start = start; + return this; + } + + public Builder end(long end) + { + this.end = end; + return this; + } + + public Builder cloudStoragePath(String cloudStoragePath) + { + this.cloudStoragePath = cloudStoragePath; + return this; + } + + public Builder objectSupplier(ChunkingStorageConnector.GetObjectFromRangeFunction objectSupplier) + { + this.objectSupplier = objectSupplier; + return this; + } + + public Builder objectOpenFunction(ObjectOpenFunction objectOpenFunction) + { + this.objectOpenFunction = objectOpenFunction; + return this; + } + + public Builder retryCondition(Predicate retryCondition) + { + this.retryCondition = retryCondition; + return this; + } + + public Builder maxRetry(int maxRetry) + { + this.maxRetry = maxRetry; + return this; + } + + public Builder tempDirSupplier(Supplier tempDirSupplier) + { + this.tempDirSupplier = tempDirSupplier; + return this; + } + + ChunkingStorageConnectorParameters build() + { + Preconditions.checkArgument(start >= 0, "'start' not provided or an incorrect value [%s] passed", start); + Preconditions.checkArgument(end >= 0, "'end' not provided or an incorrect value [%s] passed", end); + Preconditions.checkArgument(maxRetry >= 0, "'maxRetry' not provided or an incorrect value [%s] passed", maxRetry); + return new ChunkingStorageConnectorParameters( + start, + end, + Preconditions.checkNotNull(cloudStoragePath, "'cloudStoragePath' not supplied"), + Preconditions.checkNotNull(objectSupplier, "'objectSupplier' not supplied"), + Preconditions.checkNotNull(objectOpenFunction, "'objectOpenFunction' not supplied"), + Preconditions.checkNotNull(retryCondition, "'retryCondition' not supplied"), + maxRetry, + Preconditions.checkNotNull(tempDirSupplier, "'tempDirSupplier' not supplied") + ); + } + } +} From 71c50fabb43cbf524cc04bbf3acfba38ec685971 Mon Sep 17 00:00:00 2001 From: Laksh Singla Date: Wed, 12 Jul 2023 15:26:37 +0530 Subject: [PATCH 04/28] goc changes stash --- .../google/output/GoogleOutputConfig.java | 87 ++++++++++++++++--- 1 file changed, 76 insertions(+), 11 deletions(-) diff --git a/extensions-core/google-extensions/src/main/java/org/apache/druid/storage/google/output/GoogleOutputConfig.java b/extensions-core/google-extensions/src/main/java/org/apache/druid/storage/google/output/GoogleOutputConfig.java index d18e31273119..9df02d8876fb 100644 --- a/extensions-core/google-extensions/src/main/java/org/apache/druid/storage/google/output/GoogleOutputConfig.java +++ b/extensions-core/google-extensions/src/main/java/org/apache/druid/storage/google/output/GoogleOutputConfig.java @@ -19,25 +19,90 @@ package org.apache.druid.storage.google.output; +import com.fasterxml.jackson.annotation.JsonProperty; import org.apache.druid.java.util.common.HumanReadableBytes; import java.io.File; +import java.util.Objects; public class GoogleOutputConfig { - String bucket; - String prefix; - File tempDir; - HumanReadableBytes chunkSize; - Integer maxRetry; + @JsonProperty + private final String bucket; + + @JsonProperty + private final String prefix; + + @JsonProperty + private final File tempDir; + + @JsonProperty + private final HumanReadableBytes chunkSize; + + @JsonProperty + private final Integer maxRetry; public GoogleOutputConfig( - String bucket, - String prefix, - File tempDir, - HumanReadableBytes chunkSize, - Integer maxRetry - ) { + final String bucket, + final String prefix, + final File tempDir, + final HumanReadableBytes chunkSize, + final Integer maxRetry + ) + { + this.bucket = bucket; + this.prefix = prefix; + this.tempDir = tempDir; + this.chunkSize = chunkSize; + this.maxRetry = maxRetry; + } + + public String getBucket() + { + return bucket; + } + + public String getPrefix() + { + return prefix; + } + + public File getTempDir() + { + return tempDir; + } + + public HumanReadableBytes getChunkSize() + { + return chunkSize; + } + + public Integer getMaxRetry() + { + return maxRetry; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + GoogleOutputConfig that = (GoogleOutputConfig) o; + return Objects.equals(bucket, that.bucket) + && Objects.equals(prefix, that.prefix) + && Objects.equals(tempDir, that.tempDir) + && Objects.equals(chunkSize, that.chunkSize) + && Objects.equals(maxRetry, that.maxRetry); + } + + @Override + public int hashCode() + { + return Objects.hash(bucket, prefix, tempDir, chunkSize, maxRetry); } } From d07b740fae425c006210b792ccb974645a0fe0f6 Mon Sep 17 00:00:00 2001 From: Laksh Singla Date: Fri, 14 Jul 2023 11:54:52 +0530 Subject: [PATCH 05/28] version before batch delete --- .../druid/storage/google/GoogleUtils.java | 2 +- .../google/output/GoogleStorageConnector.java | 53 ++++++++++++++++++- .../output/GoogleStorageConnectorModule.java | 24 ++++++++- .../GoogleStorageConnectorProvider.java | 42 ++++++++++++++- 4 files changed, 117 insertions(+), 4 deletions(-) diff --git a/extensions-core/google-extensions/src/main/java/org/apache/druid/storage/google/GoogleUtils.java b/extensions-core/google-extensions/src/main/java/org/apache/druid/storage/google/GoogleUtils.java index 25b4f3286ea7..5b2727abb61b 100644 --- a/extensions-core/google-extensions/src/main/java/org/apache/druid/storage/google/GoogleUtils.java +++ b/extensions-core/google-extensions/src/main/java/org/apache/druid/storage/google/GoogleUtils.java @@ -45,7 +45,7 @@ public static boolean isRetryable(Throwable t) return t instanceof IOException; } - static T retryGoogleCloudStorageOperation(RetryUtils.Task f) throws Exception + public static T retryGoogleCloudStorageOperation(RetryUtils.Task f) throws Exception { return RetryUtils.retry(f, GOOGLE_RETRY, RetryUtils.DEFAULT_MAX_TRIES); } diff --git a/extensions-core/google-extensions/src/main/java/org/apache/druid/storage/google/output/GoogleStorageConnector.java b/extensions-core/google-extensions/src/main/java/org/apache/druid/storage/google/output/GoogleStorageConnector.java index b819cf090daf..6bb28682b85e 100644 --- a/extensions-core/google-extensions/src/main/java/org/apache/druid/storage/google/output/GoogleStorageConnector.java +++ b/extensions-core/google-extensions/src/main/java/org/apache/druid/storage/google/output/GoogleStorageConnector.java @@ -20,6 +20,11 @@ package org.apache.druid.storage.google.output; import com.google.api.services.storage.model.StorageObject; +import com.google.common.base.Joiner; +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.storage.google.GoogleInputDataConfig; +import org.apache.druid.storage.google.GoogleStorage; +import org.apache.druid.storage.google.GoogleUtils; import org.apache.druid.storage.remote.ChunkingStorageConnector; import org.apache.druid.storage.remote.ChunkingStorageConnectorParameters; @@ -29,10 +34,31 @@ public class GoogleStorageConnector extends ChunkingStorageConnector { + + private static final String DELIM = "/"; + private static final Joiner JOINER = Joiner.on(DELIM).skipNulls(); + private static final Logger log = new Logger(GoogleStorageConnector.class); + + private final GoogleStorage storage; + private final GoogleOutputConfig config; + private final GoogleInputDataConfig inputDataConfig; + + public GoogleStorageConnector( + GoogleStorage storage, + GoogleOutputConfig config, + GoogleInputDataConfig inputDataConfig + ) + { + this.storage = storage; + this.config = config; + this.inputDataConfig = inputDataConfig; + } + + @Override public boolean pathExists(String path) { - return false; + return storage.exists(config.getBucket(), objectPath(path)); } @Override @@ -44,13 +70,33 @@ public OutputStream write(String path) throws IOException @Override public void deleteFile(String path) throws IOException { + try { + final String fullPath = objectPath(path); + log.debug("Deleting file at bucket: [%s], path: [%s]", config.getBucket(), fullPath); + GoogleUtils.retryGoogleCloudStorageOperation( + () -> { + storage.delete(config.getBucket(), fullPath); + return null; + } + ); + } + catch (Exception e) { + log.error("Error occurred while deleting file at path [%s]. Error: [%s]", path, e.getMessage()); + throw new IOException(e); + } } @Override public void deleteFiles(Iterable paths) throws IOException { + int currentItemSize = 0; + + for (String path : paths) { + + + } } @Override @@ -76,4 +122,9 @@ public ChunkingStorageConnectorParameters buildInputParams(String { return null; } + + private String objectPath(String path) + { + return JOINER.join(config.getPrefix(), path); + } } diff --git a/extensions-core/google-extensions/src/main/java/org/apache/druid/storage/google/output/GoogleStorageConnectorModule.java b/extensions-core/google-extensions/src/main/java/org/apache/druid/storage/google/output/GoogleStorageConnectorModule.java index b3594730c2bd..875e9fde46e3 100644 --- a/extensions-core/google-extensions/src/main/java/org/apache/druid/storage/google/output/GoogleStorageConnectorModule.java +++ b/extensions-core/google-extensions/src/main/java/org/apache/druid/storage/google/output/GoogleStorageConnectorModule.java @@ -19,6 +19,28 @@ package org.apache.druid.storage.google.output; -public class GoogleStorageConnectorModule +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 GoogleStorageConnectorModule implements DruidModule { + + @Override + public List getJacksonModules() + { + return Collections.singletonList( + new SimpleModule(GoogleStorageConnectorModule.class.getSimpleName()) + .registerSubtypes(GoogleStorageConnectorProvider.class) + ); + } + + @Override + public void configure(Binder binder) + { + } } diff --git a/extensions-core/google-extensions/src/main/java/org/apache/druid/storage/google/output/GoogleStorageConnectorProvider.java b/extensions-core/google-extensions/src/main/java/org/apache/druid/storage/google/output/GoogleStorageConnectorProvider.java index 1bb91b77e541..0746914fe861 100644 --- a/extensions-core/google-extensions/src/main/java/org/apache/druid/storage/google/output/GoogleStorageConnectorProvider.java +++ b/extensions-core/google-extensions/src/main/java/org/apache/druid/storage/google/output/GoogleStorageConnectorProvider.java @@ -19,6 +19,46 @@ package org.apache.druid.storage.google.output; -public class GoogleStorageConnectorProvider +import com.fasterxml.jackson.annotation.JacksonInject; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeName; +import com.google.common.base.Preconditions; +import org.apache.druid.java.util.common.HumanReadableBytes; +import org.apache.druid.storage.StorageConnector; +import org.apache.druid.storage.StorageConnectorProvider; +import org.apache.druid.storage.google.GoogleInputDataConfig; +import org.apache.druid.storage.google.GoogleStorage; +import org.apache.druid.storage.google.GoogleStorageDruidModule; + +import java.io.File; + +@JsonTypeName(GoogleStorageDruidModule.SCHEME) +public class GoogleStorageConnectorProvider extends GoogleOutputConfig implements StorageConnectorProvider { + + private final GoogleStorage storage; + private final GoogleInputDataConfig inputDataConfig; + + @JsonCreator + public GoogleStorageConnectorProvider( + @JacksonInject GoogleStorage storage, + @JacksonInject GoogleInputDataConfig inputDataConfig, + @JsonProperty(value = "bucket", required = true) final String bucket, + @JsonProperty(value = "prefix", required = true) final String prefix, + @JsonProperty(value = "tempDir", required = true) final File tempDir, + @JsonProperty(value = "chunkSize") final HumanReadableBytes chunkSize, + @JsonProperty(value = "maxRetry") final Integer maxRetry + ) + { + super(bucket, prefix, tempDir, chunkSize, maxRetry); + this.storage = Preconditions.checkNotNull(storage, "google client must be provided"); + this.inputDataConfig = Preconditions.checkNotNull(inputDataConfig, "google input data config must be provided"); + } + + @Override + public StorageConnector get() + { + return new GoogleStorageConnector(storage, this, inputDataConfig); + } } From 7c482c55b111baedae2eeb0ff818468ed29d1ab4 Mon Sep 17 00:00:00 2001 From: Laksh Singla Date: Wed, 19 Jul 2023 08:59:56 +0530 Subject: [PATCH 06/28] storage connector final --- .../druid/storage/google/GoogleStorage.java | 27 ++- .../google/output/GoogleInputRange.java | 81 +++++++++ .../google/output/GoogleOutputConfig.java | 35 +++- .../google/output/GoogleStorageConnector.java | 172 ++++++++++++++++-- .../GoogleStorageConnectorProvider.java | 3 +- .../remote/ChunkingStorageConnector.java | 23 ++- .../ChunkingStorageConnectorParameters.java | 4 +- 7 files changed, 318 insertions(+), 27 deletions(-) create mode 100644 extensions-core/google-extensions/src/main/java/org/apache/druid/storage/google/output/GoogleInputRange.java diff --git a/extensions-core/google-extensions/src/main/java/org/apache/druid/storage/google/GoogleStorage.java b/extensions-core/google-extensions/src/main/java/org/apache/druid/storage/google/GoogleStorage.java index f181d08f443c..168bdb7069da 100644 --- a/extensions-core/google-extensions/src/main/java/org/apache/druid/storage/google/GoogleStorage.java +++ b/extensions-core/google-extensions/src/main/java/org/apache/druid/storage/google/GoogleStorage.java @@ -20,10 +20,13 @@ package org.apache.druid.storage.google; import com.google.api.client.http.AbstractInputStreamContent; +import com.google.api.client.http.HttpHeaders; +import com.google.api.client.http.HttpResponse; import com.google.api.services.storage.Storage; import com.google.api.services.storage.Storage.Objects.Get; import com.google.api.services.storage.model.StorageObject; import com.google.common.base.Supplier; +import org.apache.druid.java.util.common.StringUtils; import java.io.IOException; import java.io.InputStream; @@ -36,7 +39,7 @@ public class GoogleStorage * if we have a Storage instead of a supplier of it, it can cause unnecessary config validation * against Google storage even when it's not used at all. To perform the config validation * only when it is actually used, we use a supplier. - * + *

* See OmniDataSegmentKiller for how DataSegmentKillers are initialized. */ private final Supplier storage; @@ -59,6 +62,26 @@ public InputStream get(final String bucket, final String path) throws IOExceptio return get(bucket, path, 0); } + public InputStream getUsingRangeHeaders(final String bucket, final String path, long start, long end) + throws IOException + { + final Get get = storage.get().objects().get(bucket, path); + HttpHeaders httpHeaders = new HttpHeaders(); + + String rangeString = StringUtils.format("bytes %d-%d/*", start, end); + + httpHeaders.setRange(rangeString); + get.setRequestHeaders(httpHeaders); + + HttpResponse httpResponse = get.executeUsingHead(); + String responseRangeString = httpResponse.getHeaders().getRange(); + if (!responseRangeString.equals(rangeString)) { + throw new IOException("Unable to fetch the correct range"); + } + + return get.executeMediaAsInputStream(); + } + public InputStream get(final String bucket, final String path, long start) throws IOException { final Get get = storage.get().objects().get(bucket, path); @@ -86,7 +109,7 @@ public boolean exists(final String bucket, final String path) return false; } } - + public long size(final String bucket, final String path) throws IOException { return storage.get().objects().get(bucket, path).execute().getSize().longValue(); diff --git a/extensions-core/google-extensions/src/main/java/org/apache/druid/storage/google/output/GoogleInputRange.java b/extensions-core/google-extensions/src/main/java/org/apache/druid/storage/google/output/GoogleInputRange.java new file mode 100644 index 000000000000..ee0d9a58f880 --- /dev/null +++ b/extensions-core/google-extensions/src/main/java/org/apache/druid/storage/google/output/GoogleInputRange.java @@ -0,0 +1,81 @@ +/* + * 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.google.output; + +import java.util.Objects; + +public class GoogleInputRange +{ + private final long start; + private final long size; + private final String bucket; + private final String path; + + public GoogleInputRange(long start, long size, String bucket, String path) + { + this.start = start; + this.size = size; + this.bucket = bucket; + this.path = path; + } + + public long getStart() + { + return start; + } + + public long getSize() + { + return size; + } + + public String getBucket() + { + return bucket; + } + + public String getPath() + { + return path; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + GoogleInputRange that = (GoogleInputRange) o; + return start == that.start + && size == that.size + && Objects.equals(bucket, that.bucket) + && Objects.equals(path, that.path); + } + + @Override + public int hashCode() + { + return Objects.hash(start, size, bucket, path); + } +} + diff --git a/extensions-core/google-extensions/src/main/java/org/apache/druid/storage/google/output/GoogleOutputConfig.java b/extensions-core/google-extensions/src/main/java/org/apache/druid/storage/google/output/GoogleOutputConfig.java index 9df02d8876fb..4fa91c1c911a 100644 --- a/extensions-core/google-extensions/src/main/java/org/apache/druid/storage/google/output/GoogleOutputConfig.java +++ b/extensions-core/google-extensions/src/main/java/org/apache/druid/storage/google/output/GoogleOutputConfig.java @@ -21,7 +21,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import org.apache.druid.java.util.common.HumanReadableBytes; +import org.apache.druid.java.util.common.RetryUtils; +import javax.annotation.Nullable; import java.io.File; import java.util.Objects; @@ -38,24 +40,35 @@ public class GoogleOutputConfig private final File tempDir; @JsonProperty - private final HumanReadableBytes chunkSize; + private boolean chunkedDownloads = false; @JsonProperty - private final Integer maxRetry; + private HumanReadableBytes chunkSize = new HumanReadableBytes("100MiB"); + + @JsonProperty + private int maxRetry = RetryUtils.DEFAULT_MAX_TRIES; public GoogleOutputConfig( final String bucket, final String prefix, final File tempDir, - final HumanReadableBytes chunkSize, - final Integer maxRetry + @Nullable final Boolean chunkedDownloads, + @Nullable final HumanReadableBytes chunkSize, + @Nullable final Integer maxRetry ) { this.bucket = bucket; this.prefix = prefix; this.tempDir = tempDir; - this.chunkSize = chunkSize; - this.maxRetry = maxRetry; + if (chunkedDownloads != null) { + this.chunkedDownloads = chunkedDownloads; + } + if (chunkSize != null) { + this.chunkSize = chunkSize; + } + if (maxRetry != null) { + this.maxRetry = maxRetry; + } } public String getBucket() @@ -73,6 +86,11 @@ public File getTempDir() return tempDir; } + public Boolean isChunkedDownloads() + { + return chunkedDownloads; + } + public HumanReadableBytes getChunkSize() { return chunkSize; @@ -96,6 +114,7 @@ public boolean equals(Object o) return Objects.equals(bucket, that.bucket) && Objects.equals(prefix, that.prefix) && Objects.equals(tempDir, that.tempDir) + && Objects.equals(chunkedDownloads, that.chunkedDownloads) && Objects.equals(chunkSize, that.chunkSize) && Objects.equals(maxRetry, that.maxRetry); } @@ -103,6 +122,8 @@ public boolean equals(Object o) @Override public int hashCode() { - return Objects.hash(bucket, prefix, tempDir, chunkSize, maxRetry); + return Objects.hash(bucket, prefix, tempDir, chunkedDownloads, chunkSize, maxRetry); } + + } diff --git a/extensions-core/google-extensions/src/main/java/org/apache/druid/storage/google/output/GoogleStorageConnector.java b/extensions-core/google-extensions/src/main/java/org/apache/druid/storage/google/output/GoogleStorageConnector.java index 6bb28682b85e..3f0bba093b6a 100644 --- a/extensions-core/google-extensions/src/main/java/org/apache/druid/storage/google/output/GoogleStorageConnector.java +++ b/extensions-core/google-extensions/src/main/java/org/apache/druid/storage/google/output/GoogleStorageConnector.java @@ -19,20 +19,33 @@ package org.apache.druid.storage.google.output; +import com.google.api.client.http.InputStreamContent; import com.google.api.services.storage.model.StorageObject; import com.google.common.base.Joiner; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Iterators; +import com.google.common.io.ByteStreams; +import org.apache.druid.data.input.impl.CloudObjectLocation; +import org.apache.druid.data.input.impl.prefetch.ObjectOpenFunction; +import org.apache.druid.java.util.common.RE; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.storage.google.GoogleInputDataConfig; import org.apache.druid.storage.google.GoogleStorage; +import org.apache.druid.storage.google.GoogleStorageDruidModule; import org.apache.druid.storage.google.GoogleUtils; import org.apache.druid.storage.remote.ChunkingStorageConnector; import org.apache.druid.storage.remote.ChunkingStorageConnectorParameters; +import java.io.File; +import java.io.FileOutputStream; import java.io.IOException; +import java.io.InputStream; import java.io.OutputStream; +import java.nio.file.Files; import java.util.Iterator; +import java.util.UUID; -public class GoogleStorageConnector extends ChunkingStorageConnector +public class GoogleStorageConnector extends ChunkingStorageConnector { private static final String DELIM = "/"; @@ -64,7 +77,58 @@ public boolean pathExists(String path) @Override public OutputStream write(String path) throws IOException { - return null; + File tempFile = new File(JOINER.join(config.getTempDir(), UUID.randomUUID().toString())); + + return new OutputStream() + { + + final OutputStream delegate = new FileOutputStream(tempFile); + @Override + public void write(int b) throws IOException + { + delegate.write(b); + } + + @Override + public void write(byte[] b) throws IOException + { + delegate.write(b); + } + + @Override + public void write(byte[] b, int off, int len) throws IOException + { + delegate.write(b, off, len); + } + + @Override + public void flush() throws IOException + { + delegate.flush(); + } + + @Override + public void close() throws IOException + { + delegate.close(); + + try (final InputStream fileInputStream = Files.newInputStream(tempFile.toPath())) { + InputStreamContent mediaContent = new InputStreamContent("application/octet-stream", fileInputStream); + GoogleUtils.retryGoogleCloudStorageOperation( + () -> { + storage.insert(config.getBucket(), objectPath(path), mediaContent); + return null; + } + ); + } + catch (Exception e) { + throw new IOException(e); + } + if (!tempFile.delete()) { + throw new RE("Unable to delete the temp file [%s]", tempFile); + } + } + }; } @Override @@ -90,37 +154,119 @@ public void deleteFile(String path) throws IOException @Override public void deleteFiles(Iterable paths) throws IOException { - int currentItemSize = 0; - - for (String path : paths) { - - + deleteFile(objectPath(path)); } } @Override public void deleteRecursively(String path) throws IOException { + try { + GoogleUtils.deleteObjectsInPath( + storage, + inputDataConfig, + config.getBucket(), + objectPath(config.getPrefix()), + p -> true + ); + } + catch (Exception e) { + throw new IOException(e); + } + } + + @Override + public Iterator listDir(String dirName) + { + final String fullPath = objectPath(dirName); + Iterator storageObjects = GoogleUtils.lazyFetchingStorageObjectsIterator( + storage, + ImmutableList.of(new CloudObjectLocation(config.getBucket(), fullPath) + .toUri(GoogleStorageDruidModule.SCHEME_GS)).iterator(), + inputDataConfig.getMaxListingLength() + ); + + return Iterators.transform( + storageObjects, + storageObject -> { + String[] split = storageObject.getName().split(fullPath, 2); + if (split.length > 1) { + return split[1]; + } else { + return ""; + } + } + ); + } + + @Override + public InputStream read(String path) throws IOException + { + if (config.isChunkedDownloads()) { + return super.read(path); + } + return storage.get(config.getBucket(), objectPath(path)); } @Override - public Iterator listDir(String dirName) throws IOException + public InputStream readRange(String path, long from, long size) throws IOException { - return null; + if (config.isChunkedDownloads()) { + return super.readRange(path, from, size); + } + + return ByteStreams.limit(storage.get(config.getBucket(), objectPath(path), from), size); } @Override - public ChunkingStorageConnectorParameters buildInputParams(String path) + public ChunkingStorageConnectorParameters buildInputParams(String path) throws IOException { - return null; + long size = storage.size(config.getBucket(), objectPath(path)); + return buildInputParams(path, 0, size); } @Override - public ChunkingStorageConnectorParameters buildInputParams(String path, long from, long size) + public ChunkingStorageConnectorParameters buildInputParams(String path, long from, long size) { - return null; + ChunkingStorageConnectorParameters.Builder builder = new ChunkingStorageConnectorParameters.Builder<>(); + builder.start(from); + builder.end(from + size); + builder.cloudStoragePath(objectPath(path)); + builder.tempDirSupplier(config::getTempDir); + builder.maxRetry(config.getMaxRetry()); + builder.retryCondition(GoogleUtils.GOOGLE_RETRY); + builder.objectSupplier(((start, end) -> new GoogleInputRange(start, end - start, config.getBucket(), objectPath(path)))); + builder.objectOpenFunction(new ObjectOpenFunction() + { + @Override + public InputStream open(GoogleInputRange googleInputRange) throws IOException + { + long rangeEnd = googleInputRange.getStart() + googleInputRange.getSize() - 1; + return storage.getUsingRangeHeaders( + googleInputRange.getBucket(), + googleInputRange.getPath(), + googleInputRange.getStart(), + rangeEnd + ); + } + + @Override + public InputStream open(GoogleInputRange googleInputRange, long offset) throws IOException + { + long rangeStart = googleInputRange.getStart() + offset; + long rangeEnd = googleInputRange.getStart() + googleInputRange.getSize() - 1; + return storage.getUsingRangeHeaders( + googleInputRange.getBucket(), + googleInputRange.getPath(), + rangeStart, + rangeEnd + ); + } + }); + + return builder.build(); } private String objectPath(String path) diff --git a/extensions-core/google-extensions/src/main/java/org/apache/druid/storage/google/output/GoogleStorageConnectorProvider.java b/extensions-core/google-extensions/src/main/java/org/apache/druid/storage/google/output/GoogleStorageConnectorProvider.java index 0746914fe861..37dd1c1234f9 100644 --- a/extensions-core/google-extensions/src/main/java/org/apache/druid/storage/google/output/GoogleStorageConnectorProvider.java +++ b/extensions-core/google-extensions/src/main/java/org/apache/druid/storage/google/output/GoogleStorageConnectorProvider.java @@ -47,11 +47,12 @@ public GoogleStorageConnectorProvider( @JsonProperty(value = "bucket", required = true) final String bucket, @JsonProperty(value = "prefix", required = true) final String prefix, @JsonProperty(value = "tempDir", required = true) final File tempDir, + @JsonProperty(value = "chunkedDownloads") final Boolean chunkedDownloads, @JsonProperty(value = "chunkSize") final HumanReadableBytes chunkSize, @JsonProperty(value = "maxRetry") final Integer maxRetry ) { - super(bucket, prefix, tempDir, chunkSize, maxRetry); + super(bucket, prefix, tempDir, chunkedDownloads, chunkSize, maxRetry); this.storage = Preconditions.checkNotNull(storage, "google client must be provided"); this.inputDataConfig = Preconditions.checkNotNull(inputDataConfig, "google input data config must be provided"); } diff --git a/processing/src/main/java/org/apache/druid/storage/remote/ChunkingStorageConnector.java b/processing/src/main/java/org/apache/druid/storage/remote/ChunkingStorageConnector.java index d01a8eee3eb6..d628ee87f4b8 100644 --- a/processing/src/main/java/org/apache/druid/storage/remote/ChunkingStorageConnector.java +++ b/processing/src/main/java/org/apache/druid/storage/remote/ChunkingStorageConnector.java @@ -1,3 +1,22 @@ +/* + * 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.remote; import com.google.common.base.Predicates; @@ -36,12 +55,12 @@ public InputStream read(String path) throws IOException } @Override - public InputStream readRange(String path, long from, long size) + public InputStream readRange(String path, long from, long size) throws IOException { return buildInputStream(buildInputParams(path, from, size)); } - public abstract ChunkingStorageConnectorParameters buildInputParams(String path); + public abstract ChunkingStorageConnectorParameters buildInputParams(String path) throws IOException; public abstract ChunkingStorageConnectorParameters buildInputParams(String path, long from, long size); diff --git a/processing/src/main/java/org/apache/druid/storage/remote/ChunkingStorageConnectorParameters.java b/processing/src/main/java/org/apache/druid/storage/remote/ChunkingStorageConnectorParameters.java index ec46a3f21d84..1e2440d11132 100644 --- a/processing/src/main/java/org/apache/druid/storage/remote/ChunkingStorageConnectorParameters.java +++ b/processing/src/main/java/org/apache/druid/storage/remote/ChunkingStorageConnectorParameters.java @@ -99,7 +99,7 @@ public Supplier getTempDirSupplier() return tempDirSupplier; } - private static class Builder + public static class Builder { private long start; private long end; @@ -159,7 +159,7 @@ public Builder tempDirSupplier(Supplier tempDirSupplier) return this; } - ChunkingStorageConnectorParameters build() + public ChunkingStorageConnectorParameters build() { Preconditions.checkArgument(start >= 0, "'start' not provided or an incorrect value [%s] passed", start); Preconditions.checkArgument(end >= 0, "'end' not provided or an incorrect value [%s] passed", end); From 5730c354fea66e62576432de481404cb9d36cc3a Mon Sep 17 00:00:00 2001 From: Laksh Singla Date: Wed, 19 Jul 2023 10:45:32 +0530 Subject: [PATCH 07/28] cleanup S3 storage connector --- .../storage/s3/output/S3StorageConnector.java | 205 +++--------------- 1 file changed, 36 insertions(+), 169 deletions(-) diff --git a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3StorageConnector.java b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3StorageConnector.java index e238c12b1ddc..a68ed9c1c00c 100644 --- a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3StorageConnector.java +++ b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3StorageConnector.java @@ -27,13 +27,9 @@ import com.google.common.base.Predicates; import com.google.common.collect.ImmutableList; import com.google.common.collect.Iterators; -import org.apache.commons.io.input.NullInputStream; import org.apache.druid.data.input.impl.CloudObjectLocation; -import org.apache.druid.data.input.impl.RetryingInputStream; import org.apache.druid.data.input.impl.prefetch.ObjectOpenFunction; import org.apache.druid.java.util.common.FileUtils; -import org.apache.druid.java.util.common.IAE; -import org.apache.druid.java.util.common.IOE; import org.apache.druid.java.util.common.RE; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.logger.Logger; @@ -43,20 +39,12 @@ import org.apache.druid.storage.s3.ServerSideEncryptingAmazonS3; import javax.annotation.Nonnull; -import java.io.File; -import java.io.FileInputStream; -import java.io.FileNotFoundException; import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; -import java.io.SequenceInputStream; import java.util.ArrayList; -import java.util.Enumeration; import java.util.Iterator; import java.util.List; -import java.util.UUID; -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicLong; /** * In this implementation, all remote calls to aws s3 are retried {@link S3OutputConfig#getMaxRetry()} times. @@ -70,7 +58,6 @@ public class S3StorageConnector extends ChunkingStorageConnector buildInputParams(String path) { - if (from < 0 || size < 0) { - throw new IAE( - "Invalid arguments for reading %s. from = %d, readSize = %d", - objectPath(path), - from, - size + long size; + try { + size = S3Utils.retryS3Operation( + () -> this.s3Client.getObjectMetadata(config.getBucket(), objectPath(path)).getInstanceLength(), + config.getMaxRetry() ); } - return buildInputStream( - new GetObjectRequest(config.getBucket(), objectPath(path)).withRange(from, from + size - 1), - path - ); - } - - @Override - public ChunkingStorageConnectorParameters buildInputParams(String path) - { - return null; + catch (Exception e) { + throw new RuntimeException(e); + } + return buildInputParams(path, 0, size); } @Override public ChunkingStorageConnectorParameters buildInputParams(String path, long from, long size) { - return null; - } - - private InputStream buildInputStream(GetObjectRequest getObjectRequest, String path) - { - // fetch the size of the whole object to make chunks - long readEnd; - AtomicLong currReadStart = new AtomicLong(0); - if (getObjectRequest.getRange() != null) { - currReadStart.set(getObjectRequest.getRange()[0]); - readEnd = getObjectRequest.getRange()[1] + 1; - } else { - try { - readEnd = S3Utils.retryS3Operation( - () -> this.s3Client.getObjectMetadata(config.getBucket(), objectPath(path)).getInstanceLength(), - config.getMaxRetry() - ); - } - catch (Exception e) { - throw new RuntimeException(e); - } - } - AtomicBoolean isSequenceStreamClosed = new AtomicBoolean(false); - - // build a sequence input stream from chunks - return new SequenceInputStream(new Enumeration() + ChunkingStorageConnectorParameters.Builder builder = new ChunkingStorageConnectorParameters.Builder<>(); + builder.start(from); + builder.end(from + size); + builder.cloudStoragePath(objectPath(path)); + builder.tempDirSupplier(config::getTempDir); + builder.maxRetry(config.getMaxRetry()); + builder.retryCondition(S3Utils.S3RETRY); + builder.objectSupplier((start, end) -> new GetObjectRequest(config.getBucket(), objectPath(path)).withRange(start, end - 1)); + builder.objectOpenFunction(new ObjectOpenFunction() { - boolean initStream = false; @Override - public boolean hasMoreElements() + public InputStream open(GetObjectRequest object) { - // checking if the stream was already closed. If it was, then don't iterate over the remaining chunks - // SequenceInputStream's close method closes all the chunk streams in its close. Since we're opening them - // lazily, we don't need to close them. - if (isSequenceStreamClosed.get()) { - return false; - } - // don't stop until the whole object is downloaded - return currReadStart.get() < readEnd; - } - - @Override - public InputStream nextElement() - { - // since Sequence input stream calls nextElement in the constructor, we start chunking as soon as we call read. - // to avoid that we pass a nullInputStream for the first iteration. - if (!initStream) { - initStream = true; - return new NullInputStream(); - } - File outFile = new File(config.getTempDir().getAbsolutePath(), UUID.randomUUID().toString()); - // in a single chunk, only download a maximum of DOWNLOAD_MAX_CHUNK_SIZE - long endPoint = Math.min(currReadStart.get() + DOWNLOAD_MAX_CHUNK_SIZE, readEnd) - 1; try { - if (!outFile.createNewFile()) { - throw new IOE( - StringUtils.format( - "Could not create temporary file [%s] for copying [%s]", - outFile.getAbsolutePath(), - objectPath(path) - ) - ); - } - FileUtils.copyLarge( - () -> new RetryingInputStream<>( - new GetObjectRequest( - config.getBucket(), - objectPath(path) - ).withRange(currReadStart.get(), endPoint), - new ObjectOpenFunction() - { - @Override - public InputStream open(GetObjectRequest object) - { - try { - return S3Utils.retryS3Operation( - () -> s3Client.getObject(object).getObjectContent(), - config.getMaxRetry() - ); - } - catch (Exception e) { - throw new RuntimeException(e); - } - } - - @Override - public InputStream open(GetObjectRequest object, long offset) - { - if (object.getRange() != null) { - long[] oldRange = object.getRange(); - object.setRange(oldRange[0] + offset, oldRange[1]); - } else { - object.setRange(offset); - } - return open(object); - } - }, - S3Utils.S3RETRY, - config.getMaxRetry() - ), - outFile, - new byte[8 * 1024], - Predicates.alwaysFalse(), - 1, - StringUtils.format("Retrying copying of [%s] to [%s]", objectPath(path), outFile.getAbsolutePath()) + return S3Utils.retryS3Operation( + () -> s3Client.getObject(object).getObjectContent(), + config.getMaxRetry() ); } - catch (IOException e) { - throw new RE(e, StringUtils.format("Unable to copy [%s] to [%s]", objectPath(path), outFile)); - } - try { - AtomicBoolean isClosed = new AtomicBoolean(false); - return new FileInputStream(outFile) - { - @Override - public void close() throws IOException - { - // close should be idempotent - if (isClosed.get()) { - return; - } - isClosed.set(true); - super.close(); - // since endPoint is inclusive in s3's get request API, the next currReadStart is endpoint + 1 - currReadStart.set(endPoint + 1); - if (!outFile.delete()) { - throw new RE("Cannot delete temp file [%s]", outFile); - } - } - }; - } - catch (FileNotFoundException e) { - throw new RE(e, StringUtils.format("Unable to find temp file [%s]", outFile)); + catch (Exception e) { + throw new RuntimeException(e); } } - }) - { + @Override - public void close() throws IOException + public InputStream open(GetObjectRequest object, long offset) { - isSequenceStreamClosed.set(true); - super.close(); + if (object.getRange() != null) { + long[] oldRange = object.getRange(); + object.setRange(oldRange[0] + offset, oldRange[1]); + } else { + object.setRange(offset); + } + return open(object); } - }; + }); + return builder.build(); } @Override From b119af8cf98f1a8ca14cc2d11a4c349eb2f10d88 Mon Sep 17 00:00:00 2001 From: Laksh Singla Date: Wed, 19 Jul 2023 18:04:05 +0530 Subject: [PATCH 08/28] change byte format --- .../java/org/apache/druid/storage/google/GoogleStorage.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/extensions-core/google-extensions/src/main/java/org/apache/druid/storage/google/GoogleStorage.java b/extensions-core/google-extensions/src/main/java/org/apache/druid/storage/google/GoogleStorage.java index 168bdb7069da..e45be09e03cc 100644 --- a/extensions-core/google-extensions/src/main/java/org/apache/druid/storage/google/GoogleStorage.java +++ b/extensions-core/google-extensions/src/main/java/org/apache/druid/storage/google/GoogleStorage.java @@ -68,7 +68,7 @@ public InputStream getUsingRangeHeaders(final String bucket, final String path, final Get get = storage.get().objects().get(bucket, path); HttpHeaders httpHeaders = new HttpHeaders(); - String rangeString = StringUtils.format("bytes %d-%d/*", start, end); + String rangeString = StringUtils.format("bytes=%d-%d", start, end); httpHeaders.setRange(rangeString); get.setRequestHeaders(httpHeaders); From 8448be4c131b8ae96495a48260bdaa5f7e3ed6af Mon Sep 17 00:00:00 2001 From: Laksh Singla Date: Tue, 25 Jul 2023 22:20:14 +0530 Subject: [PATCH 09/28] add azure files --- .../druid/storage/azure/AzureStorage.java | 62 +++++- .../azure/AzureStorageDruidModule.java | 2 +- .../storage/azure/output/AzureInputRange.java | 80 +++++++ .../azure/output/AzureOutputConfig.java | 124 +++++++++++ .../azure/output/AzureStorageConnector.java | 205 ++++++++++++++++++ .../output/AzureStorageConnectorModule.java | 46 ++++ .../output/AzureStorageConnectorProvider.java | 59 +++++ .../output/RetryableAzureOutputStream.java | 159 ++++++++++++++ 8 files changed, 734 insertions(+), 3 deletions(-) create mode 100644 extensions-core/azure-extensions/src/main/java/org/apache/druid/storage/azure/output/AzureInputRange.java create mode 100644 extensions-core/azure-extensions/src/main/java/org/apache/druid/storage/azure/output/AzureOutputConfig.java create mode 100644 extensions-core/azure-extensions/src/main/java/org/apache/druid/storage/azure/output/AzureStorageConnector.java create mode 100644 extensions-core/azure-extensions/src/main/java/org/apache/druid/storage/azure/output/AzureStorageConnectorModule.java create mode 100644 extensions-core/azure-extensions/src/main/java/org/apache/druid/storage/azure/output/AzureStorageConnectorProvider.java create mode 100644 extensions-core/azure-extensions/src/main/java/org/apache/druid/storage/azure/output/RetryableAzureOutputStream.java diff --git a/extensions-core/azure-extensions/src/main/java/org/apache/druid/storage/azure/AzureStorage.java b/extensions-core/azure-extensions/src/main/java/org/apache/druid/storage/azure/AzureStorage.java index 4b1539d35019..be236f3e8f13 100644 --- a/extensions-core/azure-extensions/src/main/java/org/apache/druid/storage/azure/AzureStorage.java +++ b/extensions-core/azure-extensions/src/main/java/org/apache/druid/storage/azure/AzureStorage.java @@ -24,18 +24,22 @@ import com.microsoft.azure.storage.ResultContinuation; import com.microsoft.azure.storage.ResultSegment; import com.microsoft.azure.storage.StorageException; +import com.microsoft.azure.storage.blob.BlobDeleteBatchOperation; import com.microsoft.azure.storage.blob.BlobListingDetails; import com.microsoft.azure.storage.blob.CloudBlob; import com.microsoft.azure.storage.blob.CloudBlobClient; import com.microsoft.azure.storage.blob.CloudBlobContainer; import com.microsoft.azure.storage.blob.CloudBlockBlob; import com.microsoft.azure.storage.blob.ListBlobItem; +import org.apache.druid.java.util.common.RE; import org.apache.druid.java.util.common.logger.Logger; +import javax.annotation.Nullable; import java.io.File; import java.io.FileInputStream; import java.io.IOException; import java.io.InputStream; +import java.io.OutputStream; import java.net.URISyntaxException; import java.util.ArrayList; import java.util.EnumSet; @@ -74,7 +78,7 @@ public List emptyCloudBlobDirectory(final String containerName, final St List deletedFiles = new ArrayList<>(); CloudBlobContainer container = getOrCreateCloudBlobContainer(containerName); - for (ListBlobItem blobItem : container.listBlobs(virtualDirPath, true, null, null, null)) { + for (ListBlobItem blobItem : container.listBlobs(virtualDirPath, USE_FLAT_BLOB_LISTING, null, null, null)) { CloudBlob cloudBlob = (CloudBlob) blobItem; log.info("Removing file[%s] from Azure.", cloudBlob.getName()); if (cloudBlob.deleteIfExists()) { @@ -98,6 +102,27 @@ public void uploadBlob(final File file, final String containerName, final String } } + public OutputStream getBlockBlobOutputStream( + final String containerName, + final String blobPath, + @Nullable final Integer streamWriteSizeBytes + ) throws URISyntaxException, StorageException + { + CloudBlobContainer container = getOrCreateCloudBlobContainer(containerName); + CloudBlockBlob blockBlobReference = container.getBlockBlobReference(blobPath); + + if (blockBlobReference.exists()) { + throw new RE("Reference already exists"); + } + + if (streamWriteSizeBytes != null) { + blockBlobReference.setStreamWriteSizeInBytes(streamWriteSizeBytes); + } + + return blockBlobReference.openOutputStream(); + + } + public CloudBlob getBlobReferenceWithAttributes(final String containerName, final String blobPath) throws URISyntaxException, StorageException { @@ -120,11 +145,44 @@ public InputStream getBlobInputStream(final String containerName, final String b public InputStream getBlobInputStream(long offset, final String containerName, final String blobPath) throws URISyntaxException, StorageException + { + return getBlobInputStream(offset, null, containerName, blobPath); + } + + public InputStream getBlobInputStream(long offset, Long length, final String containerName, final String blobPath) + throws URISyntaxException, StorageException { CloudBlobContainer container = getOrCreateCloudBlobContainer(containerName); - return container.getBlockBlobReference(blobPath).openInputStream(offset, null, null, null, null); + return container.getBlockBlobReference(blobPath).openInputStream(offset, length, null, null, null); } + public void batchDeleteFiles(String containerName, Iterable paths) + throws URISyntaxException, StorageException + { + CloudBlobContainer cloudBlobContainer = getOrCreateCloudBlobContainer(containerName); + BlobDeleteBatchOperation blobDeleteBatchOperation = new BlobDeleteBatchOperation(); + for (String path : paths) { + CloudBlob blobReference = getOrCreateCloudBlobContainer(containerName).getBlockBlobReference(path); + blobDeleteBatchOperation.addSubOperation(blobReference); + } + cloudBlobClient.get().executeBatch(blobDeleteBatchOperation); + } + + public List listDir(final String containerName, final String virtualDirPath) + throws StorageException, URISyntaxException + { + List files = new ArrayList<>(); + CloudBlobContainer container = getOrCreateCloudBlobContainer(containerName); + + for (ListBlobItem blobItem : container.listBlobs(virtualDirPath, USE_FLAT_BLOB_LISTING, null, null, null)) { + CloudBlob cloudBlob = (CloudBlob) blobItem; + files.add(cloudBlob.getName()); + } + + return files; + } + + public boolean getBlobExists(String container, String blobPath) throws URISyntaxException, StorageException { return getOrCreateCloudBlobContainer(container).getBlockBlobReference(blobPath).exists(); diff --git a/extensions-core/azure-extensions/src/main/java/org/apache/druid/storage/azure/AzureStorageDruidModule.java b/extensions-core/azure-extensions/src/main/java/org/apache/druid/storage/azure/AzureStorageDruidModule.java index f2f973f11abe..674e451de51a 100644 --- a/extensions-core/azure-extensions/src/main/java/org/apache/druid/storage/azure/AzureStorageDruidModule.java +++ b/extensions-core/azure-extensions/src/main/java/org/apache/druid/storage/azure/AzureStorageDruidModule.java @@ -51,7 +51,7 @@ public class AzureStorageDruidModule implements DruidModule { - static final String SCHEME = "azure"; + public static final String SCHEME = "azure"; public static final String STORAGE_CONNECTION_STRING_WITH_KEY = "DefaultEndpointsProtocol=%s;AccountName=%s;AccountKey=%s"; public static final String diff --git a/extensions-core/azure-extensions/src/main/java/org/apache/druid/storage/azure/output/AzureInputRange.java b/extensions-core/azure-extensions/src/main/java/org/apache/druid/storage/azure/output/AzureInputRange.java new file mode 100644 index 000000000000..07a86f871d58 --- /dev/null +++ b/extensions-core/azure-extensions/src/main/java/org/apache/druid/storage/azure/output/AzureInputRange.java @@ -0,0 +1,80 @@ +/* + * 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.azure.output; + +import java.util.Objects; + +public class AzureInputRange +{ + private final long start; + private final long size; + private final String bucket; + private final String path; + + public AzureInputRange(long start, long size, String bucket, String path) + { + this.start = start; + this.size = size; + this.bucket = bucket; + this.path = path; + } + + public long getStart() + { + return start; + } + + public long getSize() + { + return size; + } + + public String getBucket() + { + return bucket; + } + + public String getPath() + { + return path; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + AzureInputRange that = (AzureInputRange) o; + return start == that.start + && size == that.size + && Objects.equals(bucket, that.bucket) + && Objects.equals(path, that.path); + } + + @Override + public int hashCode() + { + return Objects.hash(start, size, bucket, path); + } +} diff --git a/extensions-core/azure-extensions/src/main/java/org/apache/druid/storage/azure/output/AzureOutputConfig.java b/extensions-core/azure-extensions/src/main/java/org/apache/druid/storage/azure/output/AzureOutputConfig.java new file mode 100644 index 000000000000..0365c0c1d354 --- /dev/null +++ b/extensions-core/azure-extensions/src/main/java/org/apache/druid/storage/azure/output/AzureOutputConfig.java @@ -0,0 +1,124 @@ +/* + * 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.azure.output; + +import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.druid.java.util.common.HumanReadableBytes; +import org.apache.druid.java.util.common.RetryUtils; + +import javax.annotation.Nullable; +import java.io.File; +import java.util.Objects; + +public class AzureOutputConfig +{ + @JsonProperty + private final String container; + + @JsonProperty + private final String prefix; + + @JsonProperty + private final File tempDir; + + @JsonProperty + private final HumanReadableBytes chunkSize; + + private static final HumanReadableBytes DEFAULT_CHUNK_SIZE = new HumanReadableBytes("100MiB"); + + @JsonProperty + private final int maxRetry; + + public AzureOutputConfig( + final String container, + final String prefix, + final File tempDir, + @Nullable final HumanReadableBytes chunkSize, + @Nullable final Integer maxRetry + ) + { + this.container = container; + this.prefix = prefix; + this.tempDir = tempDir; + this.chunkSize = chunkSize != null ? chunkSize : DEFAULT_CHUNK_SIZE; + this.maxRetry = maxRetry != null ? maxRetry : RetryUtils.DEFAULT_MAX_TRIES; + } + + + public String getContainer() + { + return container; + } + + public String getPrefix() + { + return prefix; + } + + public File getTempDir() + { + return tempDir; + } + + public HumanReadableBytes getChunkSize() + { + return chunkSize; + } + + public int getMaxRetry() + { + return maxRetry; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + AzureOutputConfig that = (AzureOutputConfig) o; + return maxRetry == that.maxRetry + && Objects.equals(container, that.container) + && Objects.equals(prefix, that.prefix) + && Objects.equals(tempDir, that.tempDir) + && Objects.equals(chunkSize, that.chunkSize); + } + + @Override + public int hashCode() + { + return Objects.hash(container, prefix, tempDir, chunkSize, maxRetry); + } + + @Override + public String toString() + { + return "AzureOutputConfig{" + + "container='" + container + '\'' + + ", prefix='" + prefix + '\'' + + ", tempDir=" + tempDir + + ", chunkSize=" + chunkSize + + ", maxRetry=" + maxRetry + + '}'; + } +} diff --git a/extensions-core/azure-extensions/src/main/java/org/apache/druid/storage/azure/output/AzureStorageConnector.java b/extensions-core/azure-extensions/src/main/java/org/apache/druid/storage/azure/output/AzureStorageConnector.java new file mode 100644 index 000000000000..9b57b81fbc17 --- /dev/null +++ b/extensions-core/azure-extensions/src/main/java/org/apache/druid/storage/azure/output/AzureStorageConnector.java @@ -0,0 +1,205 @@ +/* + * 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.azure.output; + +import com.google.common.base.Joiner; +import com.google.common.collect.Iterables; +import com.microsoft.azure.storage.StorageException; +import org.apache.druid.data.input.impl.prefetch.ObjectOpenFunction; +import org.apache.druid.storage.azure.AzureStorage; +import org.apache.druid.storage.azure.AzureUtils; +import org.apache.druid.storage.remote.ChunkingStorageConnector; +import org.apache.druid.storage.remote.ChunkingStorageConnectorParameters; + +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.net.URISyntaxException; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; + +public class AzureStorageConnector extends ChunkingStorageConnector +{ + + private static final String DELIM = "/"; + private static final Joiner JOINER = Joiner.on(DELIM).skipNulls(); + + private final AzureOutputConfig config; + private final AzureStorage azureStorage; + + public AzureStorageConnector( + final AzureOutputConfig config, + final AzureStorage azureStorage + ) + { + this.config = config; + this.azureStorage = azureStorage; + } + + @Override + public ChunkingStorageConnectorParameters buildInputParams(String path) throws IOException + { + try { + return buildInputParams(path, 0, azureStorage.getBlobLength(config.getContainer(), objectPath(path))); + } + catch (URISyntaxException | StorageException e) { + throw new IOException(e); + } + } + + @Override + public ChunkingStorageConnectorParameters buildInputParams(String path, long from, long size) + { + ChunkingStorageConnectorParameters.Builder parameters = new ChunkingStorageConnectorParameters.Builder<>(); + parameters.tempDirSupplier(config::getTempDir); + parameters.maxRetry(config.getMaxRetry()); + parameters.cloudStoragePath(objectPath(path)); + parameters.retryCondition(AzureUtils.AZURE_RETRY); + parameters.start(from); + parameters.end(from + size); + parameters.objectSupplier((start, end) -> new AzureInputRange( + start, + end - start, + config.getContainer(), + objectPath(path) + )); + parameters.objectOpenFunction( + new ObjectOpenFunction() + { + @Override + public InputStream open(AzureInputRange inputRange) throws IOException + { + try { + return azureStorage.getBlobInputStream( + inputRange.getStart(), + inputRange.getSize(), + inputRange.getBucket(), + inputRange.getPath() + ); + } + catch (URISyntaxException | StorageException e) { + throw new IOException(e); + } + } + + @Override + public InputStream open(AzureInputRange inputRange, long offset) throws IOException + { + AzureInputRange newInputRange = new AzureInputRange( + inputRange.getStart() + offset, + Math.max(inputRange.getSize() - offset, 0), + inputRange.getBucket(), + inputRange.getPath() + ); + return open(newInputRange); + } + } + ); + + return parameters.build(); + } + + @Override + public boolean pathExists(String path) throws IOException + { + try { + return azureStorage.getBlobExists(config.getContainer(), objectPath(path)); + } + catch (URISyntaxException | StorageException e) { + throw new IOException(e); + } + } + + @Override + public OutputStream write(String path) throws IOException + { + try { + return azureStorage.getBlockBlobOutputStream( + config.getContainer(), + objectPath(path), + config.getChunkSize().getBytesInInt() + ); + } + catch (URISyntaxException | StorageException e) { + throw new IOException(e); + } + } + + @Override + public void deleteFile(String path) throws IOException + { + try { + azureStorage.batchDeleteFiles(config.getContainer(), Collections.singletonList(objectPath(path))); + } + catch (URISyntaxException | StorageException e) { + throw new IOException(e); + } + } + + @Override + public void deleteFiles(Iterable paths) throws IOException + { + try { + azureStorage.batchDeleteFiles(config.getContainer(), Iterables.transform(paths, this::objectPath)); + } + catch (StorageException | URISyntaxException e) { + throw new IOException(e); + } + } + + @Override + public void deleteRecursively(String path) throws IOException + { + try { + azureStorage.emptyCloudBlobDirectory(config.getContainer(), objectPath(path)); + } + catch (StorageException | URISyntaxException e) { + throw new IOException(e); + } + } + + @Override + public Iterator listDir(String dirName) throws IOException + { + final String prefixBasePath = objectPath(dirName); + List paths; + try { + paths = azureStorage.listDir(config.getContainer(), prefixBasePath); + } + catch (StorageException | URISyntaxException e) { + throw new IOException(e); + } + + return paths.stream().map(path -> { + String[] size = path.split(prefixBasePath, 2); + if (size.length > 1) { + return size[1]; + } else { + return ""; + } + }).iterator(); + } + + private String objectPath(String path) + { + return JOINER.join(config.getPrefix(), path); + } +} diff --git a/extensions-core/azure-extensions/src/main/java/org/apache/druid/storage/azure/output/AzureStorageConnectorModule.java b/extensions-core/azure-extensions/src/main/java/org/apache/druid/storage/azure/output/AzureStorageConnectorModule.java new file mode 100644 index 000000000000..b2cdda0eb29a --- /dev/null +++ b/extensions-core/azure-extensions/src/main/java/org/apache/druid/storage/azure/output/AzureStorageConnectorModule.java @@ -0,0 +1,46 @@ +/* + * 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.azure.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 AzureStorageConnectorModule implements DruidModule +{ + @Override + public List getJacksonModules() + { + return Collections.singletonList( + new SimpleModule(AzureStorageConnectorModule.class.getSimpleName()) + .registerSubtypes(AzureStorageConnectorProvider.class) + ); + } + + @Override + public void configure(Binder binder) + { + + } +} diff --git a/extensions-core/azure-extensions/src/main/java/org/apache/druid/storage/azure/output/AzureStorageConnectorProvider.java b/extensions-core/azure-extensions/src/main/java/org/apache/druid/storage/azure/output/AzureStorageConnectorProvider.java new file mode 100644 index 000000000000..4264801f4acf --- /dev/null +++ b/extensions-core/azure-extensions/src/main/java/org/apache/druid/storage/azure/output/AzureStorageConnectorProvider.java @@ -0,0 +1,59 @@ +/* + * 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.azure.output; + +import com.fasterxml.jackson.annotation.JacksonInject; +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.HumanReadableBytes; +import org.apache.druid.storage.StorageConnector; +import org.apache.druid.storage.StorageConnectorProvider; +import org.apache.druid.storage.azure.AzureStorage; +import org.apache.druid.storage.azure.AzureStorageDruidModule; + +import javax.annotation.Nullable; +import java.io.File; + +@JsonTypeName(AzureStorageDruidModule.SCHEME) +public class AzureStorageConnectorProvider extends AzureOutputConfig implements StorageConnectorProvider +{ + + @JacksonInject + AzureStorage azureStorage; + + @JsonCreator + public AzureStorageConnectorProvider( + @JsonProperty(value = "container", required = true) String container, + @JsonProperty(value = "prefix", required = true) String prefix, + @JsonProperty(value = "tempDir", required = true) File tempDir, + @JsonProperty(value = "chunkSize") @Nullable HumanReadableBytes chunkSize, + @JsonProperty(value = "maxRetry") @Nullable Integer maxRetry + ) + { + super(container, prefix, tempDir, chunkSize, maxRetry); + } + + @Override + public StorageConnector get() + { + return new AzureStorageConnector(this, azureStorage); + } +} diff --git a/extensions-core/azure-extensions/src/main/java/org/apache/druid/storage/azure/output/RetryableAzureOutputStream.java b/extensions-core/azure-extensions/src/main/java/org/apache/druid/storage/azure/output/RetryableAzureOutputStream.java new file mode 100644 index 000000000000..bfb73a7bdbf7 --- /dev/null +++ b/extensions-core/azure-extensions/src/main/java/org/apache/druid/storage/azure/output/RetryableAzureOutputStream.java @@ -0,0 +1,159 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * 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.azure.output; + +import com.google.common.io.CountingOutputStream; +import it.unimi.dsi.fastutil.io.FastBufferedOutputStream; +import org.apache.druid.java.util.common.io.Closer; +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.storage.azure.AzureStorage; + +import java.io.Closeable; +import java.io.File; +import java.io.FileNotFoundException; +import java.io.FileOutputStream; +import java.io.IOException; +import java.io.OutputStream; +import java.util.Objects; + +public class RetryableAzureOutputStream extends OutputStream +{ + + private static final Logger log = new Logger(RetryableAzureOutputStream.class); + + private boolean error = false; + private boolean closed = false; + + private final AzureStorage azureStorage; + private final AzureOutputConfig config; + private final File chunkStorePath; + private final long chunkSize; + + private Chunk currentChunk; + private int nextChunkId = 1; + private int numChunksPushed; + + public RetryableAzureOutputStream( + AzureStorage azureStorage, + AzureOutputConfig config + ) + { + this.azureStorage = azureStorage; + this.config = config; + } + + @Override + public void write(int b) throws IOException + { + + } + + @Override + public void write(byte[] b, int off, int len) throws IOException + { + super.write(b, off, len); + } + + @Override + public void flush() throws IOException + { + super.flush(); + } + + @Override + public void close() throws IOException + { + if (closed) { + return; + } + + closed = true; + Closer closer = Closer.create(); + } + + private static class Chunk implements Closeable + { + private final int id; + private final File file; + private final CountingOutputStream outputStream; + private boolean closed = false; + + private Chunk(int id, File file) throws FileNotFoundException + { + this.id = id; + this.file = file; + this.outputStream = new CountingOutputStream(new FastBufferedOutputStream(new FileOutputStream(file))); + } + + private long length() + { + return outputStream.getCount(); + } + + private boolean delete() + { + return file.delete(); + } + + private String absolutePath() + { + return file.getAbsolutePath(); + } + + @Override + public void close() throws IOException + { + if (closed) { + return; + } + closed = true; + outputStream.close(); + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + Chunk chunk = (Chunk) o; + return id == chunk.id; + } + + @Override + public int hashCode() + { + return Objects.hash(id); + } + + @Override + public String toString() + { + return "Chunk{" + + "id=" + id + + ", file=" + absolutePath() + + ", length=" + length() + + '}'; + } + } +} From 7953f73cf6704ebf9899bf9ad34c887fe8250362 Mon Sep 17 00:00:00 2001 From: Laksh Singla Date: Wed, 26 Jul 2023 10:32:31 +0530 Subject: [PATCH 10/28] revert gcs changes --- .../druid/storage/google/GoogleStorage.java | 27 ++----------------- .../druid/storage/google/GoogleUtils.java | 2 +- 2 files changed, 3 insertions(+), 26 deletions(-) diff --git a/extensions-core/google-extensions/src/main/java/org/apache/druid/storage/google/GoogleStorage.java b/extensions-core/google-extensions/src/main/java/org/apache/druid/storage/google/GoogleStorage.java index e45be09e03cc..f181d08f443c 100644 --- a/extensions-core/google-extensions/src/main/java/org/apache/druid/storage/google/GoogleStorage.java +++ b/extensions-core/google-extensions/src/main/java/org/apache/druid/storage/google/GoogleStorage.java @@ -20,13 +20,10 @@ package org.apache.druid.storage.google; import com.google.api.client.http.AbstractInputStreamContent; -import com.google.api.client.http.HttpHeaders; -import com.google.api.client.http.HttpResponse; import com.google.api.services.storage.Storage; import com.google.api.services.storage.Storage.Objects.Get; import com.google.api.services.storage.model.StorageObject; import com.google.common.base.Supplier; -import org.apache.druid.java.util.common.StringUtils; import java.io.IOException; import java.io.InputStream; @@ -39,7 +36,7 @@ public class GoogleStorage * if we have a Storage instead of a supplier of it, it can cause unnecessary config validation * against Google storage even when it's not used at all. To perform the config validation * only when it is actually used, we use a supplier. - *

+ * * See OmniDataSegmentKiller for how DataSegmentKillers are initialized. */ private final Supplier storage; @@ -62,26 +59,6 @@ public InputStream get(final String bucket, final String path) throws IOExceptio return get(bucket, path, 0); } - public InputStream getUsingRangeHeaders(final String bucket, final String path, long start, long end) - throws IOException - { - final Get get = storage.get().objects().get(bucket, path); - HttpHeaders httpHeaders = new HttpHeaders(); - - String rangeString = StringUtils.format("bytes=%d-%d", start, end); - - httpHeaders.setRange(rangeString); - get.setRequestHeaders(httpHeaders); - - HttpResponse httpResponse = get.executeUsingHead(); - String responseRangeString = httpResponse.getHeaders().getRange(); - if (!responseRangeString.equals(rangeString)) { - throw new IOException("Unable to fetch the correct range"); - } - - return get.executeMediaAsInputStream(); - } - public InputStream get(final String bucket, final String path, long start) throws IOException { final Get get = storage.get().objects().get(bucket, path); @@ -109,7 +86,7 @@ public boolean exists(final String bucket, final String path) return false; } } - + public long size(final String bucket, final String path) throws IOException { return storage.get().objects().get(bucket, path).execute().getSize().longValue(); diff --git a/extensions-core/google-extensions/src/main/java/org/apache/druid/storage/google/GoogleUtils.java b/extensions-core/google-extensions/src/main/java/org/apache/druid/storage/google/GoogleUtils.java index 5b2727abb61b..25b4f3286ea7 100644 --- a/extensions-core/google-extensions/src/main/java/org/apache/druid/storage/google/GoogleUtils.java +++ b/extensions-core/google-extensions/src/main/java/org/apache/druid/storage/google/GoogleUtils.java @@ -45,7 +45,7 @@ public static boolean isRetryable(Throwable t) return t instanceof IOException; } - public static T retryGoogleCloudStorageOperation(RetryUtils.Task f) throws Exception + static T retryGoogleCloudStorageOperation(RetryUtils.Task f) throws Exception { return RetryUtils.retry(f, GOOGLE_RETRY, RetryUtils.DEFAULT_MAX_TRIES); } From f657137ba5ccee5c1a5ac7fedc0ff22cd357b6a5 Mon Sep 17 00:00:00 2001 From: Laksh Singla Date: Wed, 26 Jul 2023 10:33:49 +0530 Subject: [PATCH 11/28] remove files --- .../google/output/GoogleInputRange.java | 81 ----- .../google/output/GoogleOutputConfig.java | 129 -------- .../google/output/GoogleStorageConnector.java | 276 ------------------ .../output/GoogleStorageConnectorModule.java | 46 --- .../GoogleStorageConnectorProvider.java | 65 ----- 5 files changed, 597 deletions(-) delete mode 100644 extensions-core/google-extensions/src/main/java/org/apache/druid/storage/google/output/GoogleInputRange.java delete mode 100644 extensions-core/google-extensions/src/main/java/org/apache/druid/storage/google/output/GoogleOutputConfig.java delete mode 100644 extensions-core/google-extensions/src/main/java/org/apache/druid/storage/google/output/GoogleStorageConnector.java delete mode 100644 extensions-core/google-extensions/src/main/java/org/apache/druid/storage/google/output/GoogleStorageConnectorModule.java delete mode 100644 extensions-core/google-extensions/src/main/java/org/apache/druid/storage/google/output/GoogleStorageConnectorProvider.java diff --git a/extensions-core/google-extensions/src/main/java/org/apache/druid/storage/google/output/GoogleInputRange.java b/extensions-core/google-extensions/src/main/java/org/apache/druid/storage/google/output/GoogleInputRange.java deleted file mode 100644 index ee0d9a58f880..000000000000 --- a/extensions-core/google-extensions/src/main/java/org/apache/druid/storage/google/output/GoogleInputRange.java +++ /dev/null @@ -1,81 +0,0 @@ -/* - * 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.google.output; - -import java.util.Objects; - -public class GoogleInputRange -{ - private final long start; - private final long size; - private final String bucket; - private final String path; - - public GoogleInputRange(long start, long size, String bucket, String path) - { - this.start = start; - this.size = size; - this.bucket = bucket; - this.path = path; - } - - public long getStart() - { - return start; - } - - public long getSize() - { - return size; - } - - public String getBucket() - { - return bucket; - } - - public String getPath() - { - return path; - } - - @Override - public boolean equals(Object o) - { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - GoogleInputRange that = (GoogleInputRange) o; - return start == that.start - && size == that.size - && Objects.equals(bucket, that.bucket) - && Objects.equals(path, that.path); - } - - @Override - public int hashCode() - { - return Objects.hash(start, size, bucket, path); - } -} - diff --git a/extensions-core/google-extensions/src/main/java/org/apache/druid/storage/google/output/GoogleOutputConfig.java b/extensions-core/google-extensions/src/main/java/org/apache/druid/storage/google/output/GoogleOutputConfig.java deleted file mode 100644 index 4fa91c1c911a..000000000000 --- a/extensions-core/google-extensions/src/main/java/org/apache/druid/storage/google/output/GoogleOutputConfig.java +++ /dev/null @@ -1,129 +0,0 @@ -/* - * 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.google.output; - -import com.fasterxml.jackson.annotation.JsonProperty; -import org.apache.druid.java.util.common.HumanReadableBytes; -import org.apache.druid.java.util.common.RetryUtils; - -import javax.annotation.Nullable; -import java.io.File; -import java.util.Objects; - -public class GoogleOutputConfig -{ - - @JsonProperty - private final String bucket; - - @JsonProperty - private final String prefix; - - @JsonProperty - private final File tempDir; - - @JsonProperty - private boolean chunkedDownloads = false; - - @JsonProperty - private HumanReadableBytes chunkSize = new HumanReadableBytes("100MiB"); - - @JsonProperty - private int maxRetry = RetryUtils.DEFAULT_MAX_TRIES; - - public GoogleOutputConfig( - final String bucket, - final String prefix, - final File tempDir, - @Nullable final Boolean chunkedDownloads, - @Nullable final HumanReadableBytes chunkSize, - @Nullable final Integer maxRetry - ) - { - this.bucket = bucket; - this.prefix = prefix; - this.tempDir = tempDir; - if (chunkedDownloads != null) { - this.chunkedDownloads = chunkedDownloads; - } - if (chunkSize != null) { - this.chunkSize = chunkSize; - } - if (maxRetry != null) { - this.maxRetry = maxRetry; - } - } - - public String getBucket() - { - return bucket; - } - - public String getPrefix() - { - return prefix; - } - - public File getTempDir() - { - return tempDir; - } - - public Boolean isChunkedDownloads() - { - return chunkedDownloads; - } - - public HumanReadableBytes getChunkSize() - { - return chunkSize; - } - - public Integer getMaxRetry() - { - return maxRetry; - } - - @Override - public boolean equals(Object o) - { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - GoogleOutputConfig that = (GoogleOutputConfig) o; - return Objects.equals(bucket, that.bucket) - && Objects.equals(prefix, that.prefix) - && Objects.equals(tempDir, that.tempDir) - && Objects.equals(chunkedDownloads, that.chunkedDownloads) - && Objects.equals(chunkSize, that.chunkSize) - && Objects.equals(maxRetry, that.maxRetry); - } - - @Override - public int hashCode() - { - return Objects.hash(bucket, prefix, tempDir, chunkedDownloads, chunkSize, maxRetry); - } - - -} diff --git a/extensions-core/google-extensions/src/main/java/org/apache/druid/storage/google/output/GoogleStorageConnector.java b/extensions-core/google-extensions/src/main/java/org/apache/druid/storage/google/output/GoogleStorageConnector.java deleted file mode 100644 index 3f0bba093b6a..000000000000 --- a/extensions-core/google-extensions/src/main/java/org/apache/druid/storage/google/output/GoogleStorageConnector.java +++ /dev/null @@ -1,276 +0,0 @@ -/* - * 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.google.output; - -import com.google.api.client.http.InputStreamContent; -import com.google.api.services.storage.model.StorageObject; -import com.google.common.base.Joiner; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.Iterators; -import com.google.common.io.ByteStreams; -import org.apache.druid.data.input.impl.CloudObjectLocation; -import org.apache.druid.data.input.impl.prefetch.ObjectOpenFunction; -import org.apache.druid.java.util.common.RE; -import org.apache.druid.java.util.common.logger.Logger; -import org.apache.druid.storage.google.GoogleInputDataConfig; -import org.apache.druid.storage.google.GoogleStorage; -import org.apache.druid.storage.google.GoogleStorageDruidModule; -import org.apache.druid.storage.google.GoogleUtils; -import org.apache.druid.storage.remote.ChunkingStorageConnector; -import org.apache.druid.storage.remote.ChunkingStorageConnectorParameters; - -import java.io.File; -import java.io.FileOutputStream; -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import java.nio.file.Files; -import java.util.Iterator; -import java.util.UUID; - -public class GoogleStorageConnector extends ChunkingStorageConnector -{ - - private static final String DELIM = "/"; - private static final Joiner JOINER = Joiner.on(DELIM).skipNulls(); - private static final Logger log = new Logger(GoogleStorageConnector.class); - - private final GoogleStorage storage; - private final GoogleOutputConfig config; - private final GoogleInputDataConfig inputDataConfig; - - public GoogleStorageConnector( - GoogleStorage storage, - GoogleOutputConfig config, - GoogleInputDataConfig inputDataConfig - ) - { - this.storage = storage; - this.config = config; - this.inputDataConfig = inputDataConfig; - } - - - @Override - public boolean pathExists(String path) - { - return storage.exists(config.getBucket(), objectPath(path)); - } - - @Override - public OutputStream write(String path) throws IOException - { - File tempFile = new File(JOINER.join(config.getTempDir(), UUID.randomUUID().toString())); - - return new OutputStream() - { - - final OutputStream delegate = new FileOutputStream(tempFile); - @Override - public void write(int b) throws IOException - { - delegate.write(b); - } - - @Override - public void write(byte[] b) throws IOException - { - delegate.write(b); - } - - @Override - public void write(byte[] b, int off, int len) throws IOException - { - delegate.write(b, off, len); - } - - @Override - public void flush() throws IOException - { - delegate.flush(); - } - - @Override - public void close() throws IOException - { - delegate.close(); - - try (final InputStream fileInputStream = Files.newInputStream(tempFile.toPath())) { - InputStreamContent mediaContent = new InputStreamContent("application/octet-stream", fileInputStream); - GoogleUtils.retryGoogleCloudStorageOperation( - () -> { - storage.insert(config.getBucket(), objectPath(path), mediaContent); - return null; - } - ); - } - catch (Exception e) { - throw new IOException(e); - } - if (!tempFile.delete()) { - throw new RE("Unable to delete the temp file [%s]", tempFile); - } - } - }; - } - - @Override - public void deleteFile(String path) throws IOException - { - try { - final String fullPath = objectPath(path); - log.debug("Deleting file at bucket: [%s], path: [%s]", config.getBucket(), fullPath); - - GoogleUtils.retryGoogleCloudStorageOperation( - () -> { - storage.delete(config.getBucket(), fullPath); - return null; - } - ); - } - catch (Exception e) { - log.error("Error occurred while deleting file at path [%s]. Error: [%s]", path, e.getMessage()); - throw new IOException(e); - } - } - - @Override - public void deleteFiles(Iterable paths) throws IOException - { - for (String path : paths) { - deleteFile(objectPath(path)); - } - } - - @Override - public void deleteRecursively(String path) throws IOException - { - try { - GoogleUtils.deleteObjectsInPath( - storage, - inputDataConfig, - config.getBucket(), - objectPath(config.getPrefix()), - p -> true - ); - } - catch (Exception e) { - throw new IOException(e); - } - } - - @Override - public Iterator listDir(String dirName) - { - final String fullPath = objectPath(dirName); - Iterator storageObjects = GoogleUtils.lazyFetchingStorageObjectsIterator( - storage, - ImmutableList.of(new CloudObjectLocation(config.getBucket(), fullPath) - .toUri(GoogleStorageDruidModule.SCHEME_GS)).iterator(), - inputDataConfig.getMaxListingLength() - ); - - return Iterators.transform( - storageObjects, - storageObject -> { - String[] split = storageObject.getName().split(fullPath, 2); - if (split.length > 1) { - return split[1]; - } else { - return ""; - } - } - ); - } - - @Override - public InputStream read(String path) throws IOException - { - if (config.isChunkedDownloads()) { - return super.read(path); - } - - return storage.get(config.getBucket(), objectPath(path)); - } - - @Override - public InputStream readRange(String path, long from, long size) throws IOException - { - if (config.isChunkedDownloads()) { - return super.readRange(path, from, size); - } - - return ByteStreams.limit(storage.get(config.getBucket(), objectPath(path), from), size); - } - - @Override - public ChunkingStorageConnectorParameters buildInputParams(String path) throws IOException - { - long size = storage.size(config.getBucket(), objectPath(path)); - return buildInputParams(path, 0, size); - } - - @Override - public ChunkingStorageConnectorParameters buildInputParams(String path, long from, long size) - { - ChunkingStorageConnectorParameters.Builder builder = new ChunkingStorageConnectorParameters.Builder<>(); - builder.start(from); - builder.end(from + size); - builder.cloudStoragePath(objectPath(path)); - builder.tempDirSupplier(config::getTempDir); - builder.maxRetry(config.getMaxRetry()); - builder.retryCondition(GoogleUtils.GOOGLE_RETRY); - builder.objectSupplier(((start, end) -> new GoogleInputRange(start, end - start, config.getBucket(), objectPath(path)))); - builder.objectOpenFunction(new ObjectOpenFunction() - { - @Override - public InputStream open(GoogleInputRange googleInputRange) throws IOException - { - long rangeEnd = googleInputRange.getStart() + googleInputRange.getSize() - 1; - return storage.getUsingRangeHeaders( - googleInputRange.getBucket(), - googleInputRange.getPath(), - googleInputRange.getStart(), - rangeEnd - ); - } - - @Override - public InputStream open(GoogleInputRange googleInputRange, long offset) throws IOException - { - long rangeStart = googleInputRange.getStart() + offset; - long rangeEnd = googleInputRange.getStart() + googleInputRange.getSize() - 1; - return storage.getUsingRangeHeaders( - googleInputRange.getBucket(), - googleInputRange.getPath(), - rangeStart, - rangeEnd - ); - } - }); - - return builder.build(); - } - - private String objectPath(String path) - { - return JOINER.join(config.getPrefix(), path); - } -} diff --git a/extensions-core/google-extensions/src/main/java/org/apache/druid/storage/google/output/GoogleStorageConnectorModule.java b/extensions-core/google-extensions/src/main/java/org/apache/druid/storage/google/output/GoogleStorageConnectorModule.java deleted file mode 100644 index 875e9fde46e3..000000000000 --- a/extensions-core/google-extensions/src/main/java/org/apache/druid/storage/google/output/GoogleStorageConnectorModule.java +++ /dev/null @@ -1,46 +0,0 @@ -/* - * 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.google.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 GoogleStorageConnectorModule implements DruidModule -{ - - @Override - public List getJacksonModules() - { - return Collections.singletonList( - new SimpleModule(GoogleStorageConnectorModule.class.getSimpleName()) - .registerSubtypes(GoogleStorageConnectorProvider.class) - ); - } - - @Override - public void configure(Binder binder) - { - } -} diff --git a/extensions-core/google-extensions/src/main/java/org/apache/druid/storage/google/output/GoogleStorageConnectorProvider.java b/extensions-core/google-extensions/src/main/java/org/apache/druid/storage/google/output/GoogleStorageConnectorProvider.java deleted file mode 100644 index 37dd1c1234f9..000000000000 --- a/extensions-core/google-extensions/src/main/java/org/apache/druid/storage/google/output/GoogleStorageConnectorProvider.java +++ /dev/null @@ -1,65 +0,0 @@ -/* - * 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.google.output; - -import com.fasterxml.jackson.annotation.JacksonInject; -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; -import com.fasterxml.jackson.annotation.JsonTypeName; -import com.google.common.base.Preconditions; -import org.apache.druid.java.util.common.HumanReadableBytes; -import org.apache.druid.storage.StorageConnector; -import org.apache.druid.storage.StorageConnectorProvider; -import org.apache.druid.storage.google.GoogleInputDataConfig; -import org.apache.druid.storage.google.GoogleStorage; -import org.apache.druid.storage.google.GoogleStorageDruidModule; - -import java.io.File; - -@JsonTypeName(GoogleStorageDruidModule.SCHEME) -public class GoogleStorageConnectorProvider extends GoogleOutputConfig implements StorageConnectorProvider -{ - - private final GoogleStorage storage; - private final GoogleInputDataConfig inputDataConfig; - - @JsonCreator - public GoogleStorageConnectorProvider( - @JacksonInject GoogleStorage storage, - @JacksonInject GoogleInputDataConfig inputDataConfig, - @JsonProperty(value = "bucket", required = true) final String bucket, - @JsonProperty(value = "prefix", required = true) final String prefix, - @JsonProperty(value = "tempDir", required = true) final File tempDir, - @JsonProperty(value = "chunkedDownloads") final Boolean chunkedDownloads, - @JsonProperty(value = "chunkSize") final HumanReadableBytes chunkSize, - @JsonProperty(value = "maxRetry") final Integer maxRetry - ) - { - super(bucket, prefix, tempDir, chunkedDownloads, chunkSize, maxRetry); - this.storage = Preconditions.checkNotNull(storage, "google client must be provided"); - this.inputDataConfig = Preconditions.checkNotNull(inputDataConfig, "google input data config must be provided"); - } - - @Override - public StorageConnector get() - { - return new GoogleStorageConnector(storage, this, inputDataConfig); - } -} From cffbb7e0f41012acf14f4c5daf12542e013bb4d1 Mon Sep 17 00:00:00 2001 From: Laksh Singla Date: Wed, 26 Jul 2023 10:45:34 +0530 Subject: [PATCH 12/28] remove RetryableAzureOutputStream since that is already done in the azure lib --- .../output/RetryableAzureOutputStream.java | 159 ------------------ ...rg.apache.druid.initialization.DruidModule | 1 + 2 files changed, 1 insertion(+), 159 deletions(-) delete mode 100644 extensions-core/azure-extensions/src/main/java/org/apache/druid/storage/azure/output/RetryableAzureOutputStream.java diff --git a/extensions-core/azure-extensions/src/main/java/org/apache/druid/storage/azure/output/RetryableAzureOutputStream.java b/extensions-core/azure-extensions/src/main/java/org/apache/druid/storage/azure/output/RetryableAzureOutputStream.java deleted file mode 100644 index bfb73a7bdbf7..000000000000 --- a/extensions-core/azure-extensions/src/main/java/org/apache/druid/storage/azure/output/RetryableAzureOutputStream.java +++ /dev/null @@ -1,159 +0,0 @@ -/* - * 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.azure.output; - -import com.google.common.io.CountingOutputStream; -import it.unimi.dsi.fastutil.io.FastBufferedOutputStream; -import org.apache.druid.java.util.common.io.Closer; -import org.apache.druid.java.util.common.logger.Logger; -import org.apache.druid.storage.azure.AzureStorage; - -import java.io.Closeable; -import java.io.File; -import java.io.FileNotFoundException; -import java.io.FileOutputStream; -import java.io.IOException; -import java.io.OutputStream; -import java.util.Objects; - -public class RetryableAzureOutputStream extends OutputStream -{ - - private static final Logger log = new Logger(RetryableAzureOutputStream.class); - - private boolean error = false; - private boolean closed = false; - - private final AzureStorage azureStorage; - private final AzureOutputConfig config; - private final File chunkStorePath; - private final long chunkSize; - - private Chunk currentChunk; - private int nextChunkId = 1; - private int numChunksPushed; - - public RetryableAzureOutputStream( - AzureStorage azureStorage, - AzureOutputConfig config - ) - { - this.azureStorage = azureStorage; - this.config = config; - } - - @Override - public void write(int b) throws IOException - { - - } - - @Override - public void write(byte[] b, int off, int len) throws IOException - { - super.write(b, off, len); - } - - @Override - public void flush() throws IOException - { - super.flush(); - } - - @Override - public void close() throws IOException - { - if (closed) { - return; - } - - closed = true; - Closer closer = Closer.create(); - } - - private static class Chunk implements Closeable - { - private final int id; - private final File file; - private final CountingOutputStream outputStream; - private boolean closed = false; - - private Chunk(int id, File file) throws FileNotFoundException - { - this.id = id; - this.file = file; - this.outputStream = new CountingOutputStream(new FastBufferedOutputStream(new FileOutputStream(file))); - } - - private long length() - { - return outputStream.getCount(); - } - - private boolean delete() - { - return file.delete(); - } - - private String absolutePath() - { - return file.getAbsolutePath(); - } - - @Override - public void close() throws IOException - { - if (closed) { - return; - } - closed = true; - outputStream.close(); - } - - @Override - public boolean equals(Object o) - { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - Chunk chunk = (Chunk) o; - return id == chunk.id; - } - - @Override - public int hashCode() - { - return Objects.hash(id); - } - - @Override - public String toString() - { - return "Chunk{" + - "id=" + id + - ", file=" + absolutePath() + - ", length=" + length() + - '}'; - } - } -} diff --git a/extensions-core/azure-extensions/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule b/extensions-core/azure-extensions/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule index 298f1d39d173..a801f540d396 100644 --- a/extensions-core/azure-extensions/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule +++ b/extensions-core/azure-extensions/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule @@ -13,4 +13,5 @@ # See the License for the specific language governing permissions and # limitations under the License. +org.apache.druid.storage.azure.output.AzureStorageConnectorModule org.apache.druid.storage.azure.AzureStorageDruidModule From efb3b85ef670e987e332ada030b1b8fc3524a801 Mon Sep 17 00:00:00 2001 From: Laksh Singla Date: Fri, 28 Jul 2023 00:59:14 +0530 Subject: [PATCH 13/28] docs --- .../druid/storage/remote/ChunkingStorageConnector.java | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/processing/src/main/java/org/apache/druid/storage/remote/ChunkingStorageConnector.java b/processing/src/main/java/org/apache/druid/storage/remote/ChunkingStorageConnector.java index d628ee87f4b8..cf7ee641ebda 100644 --- a/processing/src/main/java/org/apache/druid/storage/remote/ChunkingStorageConnector.java +++ b/processing/src/main/java/org/apache/druid/storage/remote/ChunkingStorageConnector.java @@ -39,6 +39,12 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicLong; +/** + * An abstract implementation of the storage connectors that download the file from the remote storage in chunks + * and presents the downloaded chunks as a single {@link InputStream} for the consumers of the connector. + * This implementation benefits over keeping the InputStream to the remote source open since we don't require the + * connection to be open for the entire duration + */ public abstract class ChunkingStorageConnector implements StorageConnector { private static final long DOWNLOAD_MAX_CHUNK_SIZE = 100_000_000; From 27826362c7dca1bbc230bb4aa8b23abc20a7ca7c Mon Sep 17 00:00:00 2001 From: Laksh Singla Date: Tue, 1 Aug 2023 10:42:01 +0530 Subject: [PATCH 14/28] add tests, comments, validation --- .../azure/output/AzureOutputConfig.java | 40 ++++- .../azure/output/AzureOutputConfigTest.java | 63 +++++++ .../azure/output/AzureOutputSerdeTest.java | 155 ++++++++++++++++++ .../remote/ChunkingStorageConnector.java | 14 +- .../ChunkingStorageConnectorParameters.java | 81 ++++++++- ...hunkingStorageConnectorParametersTest.java | 43 +++++ 6 files changed, 384 insertions(+), 12 deletions(-) create mode 100644 extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/output/AzureOutputConfigTest.java create mode 100644 extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/output/AzureOutputSerdeTest.java create mode 100644 processing/src/test/java/org/apache/druid/storage/remote/ChunkingStorageConnectorParametersTest.java diff --git a/extensions-core/azure-extensions/src/main/java/org/apache/druid/storage/azure/output/AzureOutputConfig.java b/extensions-core/azure-extensions/src/main/java/org/apache/druid/storage/azure/output/AzureOutputConfig.java index 0365c0c1d354..2daea595589d 100644 --- a/extensions-core/azure-extensions/src/main/java/org/apache/druid/storage/azure/output/AzureOutputConfig.java +++ b/extensions-core/azure-extensions/src/main/java/org/apache/druid/storage/azure/output/AzureOutputConfig.java @@ -20,6 +20,7 @@ package org.apache.druid.storage.azure.output; import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.druid.error.InvalidInput; import org.apache.druid.java.util.common.HumanReadableBytes; import org.apache.druid.java.util.common.RetryUtils; @@ -41,17 +42,25 @@ public class AzureOutputConfig @JsonProperty private final HumanReadableBytes chunkSize; - private static final HumanReadableBytes DEFAULT_CHUNK_SIZE = new HumanReadableBytes("100MiB"); + private static final HumanReadableBytes DEFAULT_CHUNK_SIZE = new HumanReadableBytes("4MiB"); + + // Minimum limit is self-imposed, so that chunks are appropriately sized, and we don't spend a lot of time downloading + // the part of the blobs + private static final long AZURE_MIN_CHUNK_SIZE_BYTES = new HumanReadableBytes("256KiB").getBytes(); + + // Maximum limit is imposed by Azure, on the size of one block blob + private static final long AZURE_MAX_CHUNK_SIZE_BYTES = new HumanReadableBytes("4000MiB").getBytes(); + @JsonProperty private final int maxRetry; public AzureOutputConfig( - final String container, - final String prefix, - final File tempDir, - @Nullable final HumanReadableBytes chunkSize, - @Nullable final Integer maxRetry + @JsonProperty(value = "container", required = true) String container, + @JsonProperty(value = "prefix", required = true) String prefix, + @JsonProperty(value = "tempDir", required = true) File tempDir, + @JsonProperty(value = "chunkSize") @Nullable HumanReadableBytes chunkSize, + @JsonProperty(value = "maxRetry") @Nullable Integer maxRetry ) { this.container = container; @@ -59,6 +68,7 @@ public AzureOutputConfig( this.tempDir = tempDir; this.chunkSize = chunkSize != null ? chunkSize : DEFAULT_CHUNK_SIZE; this.maxRetry = maxRetry != null ? maxRetry : RetryUtils.DEFAULT_MAX_TRIES; + validateFields(); } @@ -87,6 +97,24 @@ public int getMaxRetry() return maxRetry; } + private void validateFields() + { + if (chunkSize.getBytes() < AZURE_MIN_CHUNK_SIZE_BYTES || chunkSize.getBytes() > AZURE_MAX_CHUNK_SIZE_BYTES) { + throw InvalidInput.exception( + "'chunkSize' [%d] bytes to the AzureConfig should be between [%d] bytes and [%d] bytes", + chunkSize.getBytes(), + AZURE_MIN_CHUNK_SIZE_BYTES, + AZURE_MAX_CHUNK_SIZE_BYTES + ); + } + + if (!tempDir.canRead() || !tempDir.canWrite()) { + throw InvalidInput.exception("Cannot read or write on the 'tempDir' [%s]. " + + "Please provide a different path to store the intermediate contents of AzureStorageConnector" + ); + } + } + @Override public boolean equals(Object o) { diff --git a/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/output/AzureOutputConfigTest.java b/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/output/AzureOutputConfigTest.java new file mode 100644 index 000000000000..b3e9ec836257 --- /dev/null +++ b/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/output/AzureOutputConfigTest.java @@ -0,0 +1,63 @@ +/* + * 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.azure.output; + + +import org.apache.druid.error.DruidException; +import org.apache.druid.java.util.common.HumanReadableBytes; +import org.junit.Assert; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +import java.io.File; +import java.io.IOException; + +public class AzureOutputConfigTest +{ + + @Rule + public final TemporaryFolder temporaryFolder = new TemporaryFolder(); + + private static final String CONTAINER = "container"; + private static final String PREFIX = "prefix"; + private static final int MAX_RETRY_COUNT = 0; + + @Test + public void testTooLargeChunkSize() + { + HumanReadableBytes chunkSize = new HumanReadableBytes("4001MiB"); + Assert.assertThrows( + DruidException.class, + () -> new AzureOutputConfig(CONTAINER, PREFIX, temporaryFolder.newFolder(), chunkSize, MAX_RETRY_COUNT) + ); + } + + @Test + public void testTempDirectoryNotWritable() throws IOException + { + File tempDir = temporaryFolder.newFolder(); + tempDir.setWritable(false); + Assert.assertThrows( + DruidException.class, + () -> new AzureOutputConfig(CONTAINER, PREFIX, tempDir, null, MAX_RETRY_COUNT) + ); + } +} diff --git a/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/output/AzureOutputSerdeTest.java b/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/output/AzureOutputSerdeTest.java new file mode 100644 index 000000000000..a00b21269a6c --- /dev/null +++ b/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/output/AzureOutputSerdeTest.java @@ -0,0 +1,155 @@ +/* + * 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.azure.output; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.exc.MismatchedInputException; +import com.fasterxml.jackson.databind.exc.ValueInstantiationException; +import org.apache.druid.java.util.common.HumanReadableBytes; +import org.apache.druid.java.util.common.StringUtils; +import org.junit.Assert; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; + +import java.io.File; +import java.io.IOException; + +public class AzureOutputSerdeTest +{ + + private static final ObjectMapper MAPPER = new ObjectMapper(); + + @Rule + public ExpectedException expectedException = ExpectedException.none(); + + + @Test + public void sanity() throws IOException + { + String json = jsonStringReadyForAssert("{\n" + + " \"container\": \"TEST\",\n" + + " \"prefix\": \"abc\",\n" + + " \"tempDir\": \"/tmp\",\n" + + " \"chunkSize\":104857600,\n" + + " \"maxRetry\": 2\n" + + "}\n"); + + AzureOutputConfig azureOutputConfig = new AzureOutputConfig( + "TEST", + "abc", + new File("/tmp"), + HumanReadableBytes.valueOf(HumanReadableBytes.parse("100Mib")), + 2 + ); + + Assert.assertEquals( + json, + MAPPER.writeValueAsString(azureOutputConfig) + ); + + Assert.assertEquals(azureOutputConfig, MAPPER.readValue(json, AzureOutputConfig.class)); + } + + @Test + public void noPrefix() throws JsonProcessingException + { + String json = jsonStringReadyForAssert("{\n" + + " \"container\": \"TEST\",\n" + + " \"tempDir\": \"/tmp\",\n" + + " \"chunkSize\":104857600,\n" + + " \"maxRetry\": 2\n" + + "}\n"); + expectedException.expect(MismatchedInputException.class); + expectedException.expectMessage("Missing required creator property 'prefix'"); + MAPPER.readValue(json, AzureOutputConfig.class); + } + + @Test + public void nocontainer() throws JsonProcessingException + { + String json = jsonStringReadyForAssert("{\n" + + " \"prefix\": \"abc\",\n" + + " \"tempDir\": \"/tmp\",\n" + + " \"chunkSize\":104857600,\n" + + " \"maxRetry\": 2\n" + + "}\n"); + expectedException.expect(MismatchedInputException.class); + expectedException.expectMessage("Missing required creator property 'container'"); + MAPPER.readValue(json, AzureOutputConfig.class); + } + + @Test + public void noTempDir() throws JsonProcessingException + { + String json = jsonStringReadyForAssert("{\n" + + " \"prefix\": \"abc\",\n" + + " \"container\": \"TEST\",\n" + + " \"chunkSize\":104857600,\n" + + " \"maxRetry\": 2\n" + + "}\n"); + expectedException.expect(MismatchedInputException.class); + expectedException.expectMessage("Missing required creator property 'tempDir'"); + MAPPER.readValue(json, AzureOutputConfig.class); + } + + @Test + public void leastArguments() throws JsonProcessingException + { + String json = jsonStringReadyForAssert("{\n" + + " \"tempDir\": \"/tmp\",\n" + + " \"prefix\": \"abc\",\n" + + " \"container\": \"TEST\"\n" + + "}\n"); + + AzureOutputConfig azureOutputConfig = new AzureOutputConfig( + "TEST", + "abc", + new File("/tmp"), + null, + null + ); + Assert.assertEquals(azureOutputConfig, MAPPER.readValue(json, AzureOutputConfig.class)); + } + + + @Test + public void testChunkValidation() throws JsonProcessingException + { + + String json = jsonStringReadyForAssert("{\n" + + " \"prefix\": \"abc\",\n" + + " \"container\": \"TEST\",\n" + + " \"tempDir\": \"/tmp\",\n" + + " \"chunkSize\":104,\n" + + " \"maxRetry\": 2\n" + + "}\n"); + expectedException.expect(ValueInstantiationException.class); + expectedException.expectMessage("'chunkSize' [104] bytes to the AzureConfig should be between " + + "[262144] bytes and [4194304000] bytes"); + MAPPER.readValue(json, AzureOutputConfig.class); + } + + private static String jsonStringReadyForAssert(String input) + { + return StringUtils.removeChar(StringUtils.removeChar(input, '\n'), ' '); + } +} diff --git a/processing/src/main/java/org/apache/druid/storage/remote/ChunkingStorageConnector.java b/processing/src/main/java/org/apache/druid/storage/remote/ChunkingStorageConnector.java index cf7ee641ebda..e18058a7e27a 100644 --- a/processing/src/main/java/org/apache/druid/storage/remote/ChunkingStorageConnector.java +++ b/processing/src/main/java/org/apache/druid/storage/remote/ChunkingStorageConnector.java @@ -43,11 +43,12 @@ * An abstract implementation of the storage connectors that download the file from the remote storage in chunks * and presents the downloaded chunks as a single {@link InputStream} for the consumers of the connector. * This implementation benefits over keeping the InputStream to the remote source open since we don't require the - * connection to be open for the entire duration + * connection to be open for the entire duration. + * Checkout {@link ChunkingStorageConnectorParameters} to see the inputs required to support chunking */ public abstract class ChunkingStorageConnector implements StorageConnector { - private static final long DOWNLOAD_MAX_CHUNK_SIZE = 100_000_000; + private static final long DOWNLOAD_MAX_CHUNK_SIZE_BYTES = 100_000_000; public ChunkingStorageConnector() { @@ -61,7 +62,7 @@ public InputStream read(String path) throws IOException } @Override - public InputStream readRange(String path, long from, long size) throws IOException + public InputStream readRange(String path, long from, long size) { return buildInputStream(buildInputParams(path, from, size)); } @@ -72,7 +73,10 @@ public InputStream readRange(String path, long from, long size) throws IOExcepti private InputStream buildInputStream(ChunkingStorageConnectorParameters params) { - AtomicLong currentReadStartPosition = new AtomicLong(params.getStart()); + // Position from where the read needs to be resumed + final AtomicLong currentReadStartPosition = new AtomicLong(params.getStart()); + + // Final position, exclusive long readEnd = params.getEnd(); AtomicBoolean isSequenceStreamClosed = new AtomicBoolean(false); @@ -111,7 +115,7 @@ public InputStream nextElement() // exclusive long currentReadEndPosition = Math.min( - currentReadStartPosition.get() + DOWNLOAD_MAX_CHUNK_SIZE, + currentReadStartPosition.get() + DOWNLOAD_MAX_CHUNK_SIZE_BYTES, readEnd ); diff --git a/processing/src/main/java/org/apache/druid/storage/remote/ChunkingStorageConnectorParameters.java b/processing/src/main/java/org/apache/druid/storage/remote/ChunkingStorageConnectorParameters.java index 1e2440d11132..9e599cc29e08 100644 --- a/processing/src/main/java/org/apache/druid/storage/remote/ChunkingStorageConnectorParameters.java +++ b/processing/src/main/java/org/apache/druid/storage/remote/ChunkingStorageConnectorParameters.java @@ -24,18 +24,58 @@ import org.apache.druid.data.input.impl.prefetch.ObjectOpenFunction; import java.io.File; +import java.util.Objects; import java.util.function.Supplier; -// Start inclusive, end exclusive +/** + * POJO for storing the parameters required to support chunking of the downloads by {@link ChunkingStorageConnector}. + * The implementations of the {@link ChunkingStorageConnector} should essentially provide a way to build this object, + * which contains the information required to support chunking. + * Therefore, to a call of {@link org.apache.druid.storage.StorageConnector#readRange(String, long, long)}, the + * implementations of the chunking storage connectors would fetch the required chunks using the information present in + * this POJO. + */ public class ChunkingStorageConnectorParameters { + /** + * Starting point from where to begin reading the cloud object. This is inclusive. + */ private final long start; + + /** + * Ending point till where to end reading the cloud object. This is exclusive. + */ private final long end; + + /** + * Absolute storage path of the cloud object. + */ private final String cloudStoragePath; + + /** + * Given a range (start inclusive, end exclusive), fetch the object which represents the provided range of the remote + * object + */ private final ChunkingStorageConnector.GetObjectFromRangeFunction objectSupplier; + + /** + * Fetching function, which opens the input stream to the range provided by the given object + */ private final ObjectOpenFunction objectOpenFunction; + + /** + * Condition to initiate a retry if downloading the chunk errors out + */ private final Predicate retryCondition; + + /** + * Max number of retries while reading the storage connector + */ private final int maxRetry; + + /** + * Temporary directory where the chunks are stored + */ private final Supplier tempDirSupplier; public ChunkingStorageConnectorParameters( @@ -99,6 +139,45 @@ public Supplier getTempDirSupplier() return tempDirSupplier; } + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + ChunkingStorageConnectorParameters that = (ChunkingStorageConnectorParameters) o; + return start == that.start && + end == that.end && + maxRetry == that.maxRetry && + Objects.equals(cloudStoragePath, that.cloudStoragePath) && + Objects.equals(objectSupplier, that.objectSupplier) && + Objects.equals(objectOpenFunction, that.objectOpenFunction) && + Objects.equals(retryCondition, that.retryCondition) && + Objects.equals(tempDirSupplier, that.tempDirSupplier); + } + + @Override + public int hashCode() + { + return Objects.hash( + start, + end, + cloudStoragePath, + objectSupplier, + objectOpenFunction, + retryCondition, + maxRetry, + tempDirSupplier + ); + } + + /** + * Builder for {@link ChunkingStorageConnectorParameters}. Performs null checks and asserts preconditions before + * building the instance + */ public static class Builder { private long start; diff --git a/processing/src/test/java/org/apache/druid/storage/remote/ChunkingStorageConnectorParametersTest.java b/processing/src/test/java/org/apache/druid/storage/remote/ChunkingStorageConnectorParametersTest.java new file mode 100644 index 000000000000..aad4a2577b64 --- /dev/null +++ b/processing/src/test/java/org/apache/druid/storage/remote/ChunkingStorageConnectorParametersTest.java @@ -0,0 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.storage.remote; + +import nl.jqno.equalsverifier.EqualsVerifier; +import org.junit.Assert; +import org.junit.Test; + +public class ChunkingStorageConnectorParametersTest +{ + @Test + public void testEquals() + { + EqualsVerifier.forClass(ChunkingStorageConnectorParameters.class) + .usingGetClass() + .verify(); + } + + @Test + public void testVerifyParameters() + { + ChunkingStorageConnectorParameters.Builder builder = new ChunkingStorageConnectorParameters.Builder<>(); + builder.start(-1); + Assert.assertThrows(IllegalArgumentException.class, builder::build); + } +} From 86ad32a6dfcd275533ff1fedd4cf0daa395a37fb Mon Sep 17 00:00:00 2001 From: Laksh Singla Date: Wed, 2 Aug 2023 11:40:11 +0530 Subject: [PATCH 15/28] add coverage --- .../output/AzureStorageConnectorTest.java | 121 ++++++++++++++++++ 1 file changed, 121 insertions(+) create mode 100644 extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/output/AzureStorageConnectorTest.java diff --git a/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/output/AzureStorageConnectorTest.java b/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/output/AzureStorageConnectorTest.java new file mode 100644 index 000000000000..1b34649c5a5a --- /dev/null +++ b/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/output/AzureStorageConnectorTest.java @@ -0,0 +1,121 @@ +/* + * 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.azure.output; + +import com.microsoft.azure.storage.StorageException; +import org.apache.druid.storage.StorageConnector; +import org.apache.druid.storage.azure.AzureStorage; +import org.easymock.Capture; +import org.easymock.EasyMock; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +import java.io.IOException; +import java.net.URISyntaxException; + +public class AzureStorageConnectorTest +{ + + private static final String CONTAINER = "CONTAINER"; + private static final String PREFIX = "P/R/E/F/I/X"; + public static final String TEST_FILE = "test.csv"; + + @Rule + public TemporaryFolder temporaryFolder = new TemporaryFolder(); + + private StorageConnector storageConnector; + private final AzureStorage azureStorage = EasyMock.createMock(AzureStorage.class); + + @Before + public void setup() throws IOException + { + storageConnector = new AzureStorageConnector( + new AzureOutputConfig(CONTAINER, PREFIX, temporaryFolder.newFolder(), null, null), + azureStorage + ); + } + + + @Test + public void testPathExistsSuccess() throws URISyntaxException, StorageException, IOException + { + final Capture bucket = Capture.newInstance(); + final Capture path = Capture.newInstance(); + EasyMock.reset(azureStorage); + EasyMock.expect(azureStorage.getBlobExists(EasyMock.capture(bucket), EasyMock.capture(path))).andReturn(true); + EasyMock.replay(azureStorage); + Assert.assertTrue(storageConnector.pathExists(TEST_FILE)); + Assert.assertEquals(CONTAINER, bucket.getValue()); + Assert.assertEquals(PREFIX + "/" + TEST_FILE, path.getValue()); + EasyMock.verify(azureStorage); + } + + @Test + public void testPathExistsErrors() throws URISyntaxException, StorageException, IOException + { + final Capture bucket = Capture.newInstance(); + final Capture path = Capture.newInstance(); + EasyMock.reset(azureStorage); + EasyMock.expect(azureStorage.getBlobExists(EasyMock.capture(bucket), EasyMock.capture(path))).andReturn(true); + EasyMock.replay(azureStorage); + Assert.assertTrue(storageConnector.pathExists(TEST_FILE)); + Assert.assertEquals(CONTAINER, bucket.getValue()); + Assert.assertEquals(PREFIX + "/" + TEST_FILE, path.getValue()); + EasyMock.verify(azureStorage); + } + + @Test + public void testPathExistsNotFound() throws URISyntaxException, StorageException, IOException + { + final Capture bucket = Capture.newInstance(); + final Capture path = Capture.newInstance(); + EasyMock.reset(azureStorage); + EasyMock.expect(azureStorage.getBlobExists(EasyMock.capture(bucket), EasyMock.capture(path))).andReturn(false); + EasyMock.replay(azureStorage); + Assert.assertFalse(storageConnector.pathExists(TEST_FILE)); + Assert.assertEquals(CONTAINER, bucket.getValue()); + Assert.assertEquals(PREFIX + "/" + TEST_FILE, path.getValue()); + EasyMock.verify(azureStorage); + } + + @Test + public void testRead() {} + + @Test + public void testReadRange() {} + + @Test + public void testDeleteSinglePath() {} + + @Test + public void testDeleteMultiplePaths() {} + + @Test + public void testDeleteRecursive() + { + + } + + @Test + public void testListDir() {} +} \ No newline at end of file From d2b68f32f6b90c3e654e1ca638f4d2d1e88cabe1 Mon Sep 17 00:00:00 2001 From: Laksh Singla Date: Thu, 3 Aug 2023 16:26:47 +0530 Subject: [PATCH 16/28] more test coverage, review --- .../druid/storage/azure/AzureStorage.java | 2 +- .../azure/output/AzureOutputConfigTest.java | 5 +- .../azure/output/AzureOutputSerdeTest.java | 6 - .../output/AzureStorageConnectorTest.java | 119 ++++++++++++++---- .../remote/ChunkingStorageConnector.java | 5 - .../ChunkingStorageConnectorParameters.java | 1 + 6 files changed, 102 insertions(+), 36 deletions(-) diff --git a/extensions-core/azure-extensions/src/main/java/org/apache/druid/storage/azure/AzureStorage.java b/extensions-core/azure-extensions/src/main/java/org/apache/druid/storage/azure/AzureStorage.java index be236f3e8f13..bd94f61bdbdd 100644 --- a/extensions-core/azure-extensions/src/main/java/org/apache/druid/storage/azure/AzureStorage.java +++ b/extensions-core/azure-extensions/src/main/java/org/apache/druid/storage/azure/AzureStorage.java @@ -162,7 +162,7 @@ public void batchDeleteFiles(String containerName, Iterable paths) CloudBlobContainer cloudBlobContainer = getOrCreateCloudBlobContainer(containerName); BlobDeleteBatchOperation blobDeleteBatchOperation = new BlobDeleteBatchOperation(); for (String path : paths) { - CloudBlob blobReference = getOrCreateCloudBlobContainer(containerName).getBlockBlobReference(path); + CloudBlob blobReference = cloudBlobContainer.getBlockBlobReference(path); blobDeleteBatchOperation.addSubOperation(blobReference); } cloudBlobClient.get().executeBatch(blobDeleteBatchOperation); diff --git a/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/output/AzureOutputConfigTest.java b/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/output/AzureOutputConfigTest.java index b3e9ec836257..318164755991 100644 --- a/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/output/AzureOutputConfigTest.java +++ b/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/output/AzureOutputConfigTest.java @@ -22,6 +22,7 @@ import org.apache.druid.error.DruidException; import org.apache.druid.java.util.common.HumanReadableBytes; +import org.apache.druid.java.util.common.ISE; import org.junit.Assert; import org.junit.Rule; import org.junit.Test; @@ -54,7 +55,9 @@ public void testTooLargeChunkSize() public void testTempDirectoryNotWritable() throws IOException { File tempDir = temporaryFolder.newFolder(); - tempDir.setWritable(false); + if (!tempDir.setWritable(false)) { + throw new ISE("Unable to change the permission of temp folder for %s", this.getClass().getName()); + } Assert.assertThrows( DruidException.class, () -> new AzureOutputConfig(CONTAINER, PREFIX, tempDir, null, MAX_RETRY_COUNT) diff --git a/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/output/AzureOutputSerdeTest.java b/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/output/AzureOutputSerdeTest.java index a00b21269a6c..6d8e6120acab 100644 --- a/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/output/AzureOutputSerdeTest.java +++ b/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/output/AzureOutputSerdeTest.java @@ -26,9 +26,7 @@ import org.apache.druid.java.util.common.HumanReadableBytes; import org.apache.druid.java.util.common.StringUtils; import org.junit.Assert; -import org.junit.Rule; import org.junit.Test; -import org.junit.rules.ExpectedException; import java.io.File; import java.io.IOException; @@ -38,10 +36,6 @@ public class AzureOutputSerdeTest private static final ObjectMapper MAPPER = new ObjectMapper(); - @Rule - public ExpectedException expectedException = ExpectedException.none(); - - @Test public void sanity() throws IOException { diff --git a/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/output/AzureStorageConnectorTest.java b/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/output/AzureStorageConnectorTest.java index 1b34649c5a5a..538dd68d00e5 100644 --- a/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/output/AzureStorageConnectorTest.java +++ b/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/output/AzureStorageConnectorTest.java @@ -19,7 +19,10 @@ package org.apache.druid.storage.azure.output; +import com.google.api.client.util.Lists; +import com.google.common.collect.ImmutableList; import com.microsoft.azure.storage.StorageException; +import org.apache.commons.io.IOUtils; import org.apache.druid.storage.StorageConnector; import org.apache.druid.storage.azure.AzureStorage; import org.easymock.Capture; @@ -31,7 +34,11 @@ import org.junit.rules.TemporaryFolder; import java.io.IOException; +import java.io.InputStream; import java.net.URISyntaxException; +import java.nio.charset.StandardCharsets; +import java.util.Collections; +import java.util.List; public class AzureStorageConnectorTest { @@ -70,20 +77,6 @@ public void testPathExistsSuccess() throws URISyntaxException, StorageException, EasyMock.verify(azureStorage); } - @Test - public void testPathExistsErrors() throws URISyntaxException, StorageException, IOException - { - final Capture bucket = Capture.newInstance(); - final Capture path = Capture.newInstance(); - EasyMock.reset(azureStorage); - EasyMock.expect(azureStorage.getBlobExists(EasyMock.capture(bucket), EasyMock.capture(path))).andReturn(true); - EasyMock.replay(azureStorage); - Assert.assertTrue(storageConnector.pathExists(TEST_FILE)); - Assert.assertEquals(CONTAINER, bucket.getValue()); - Assert.assertEquals(PREFIX + "/" + TEST_FILE, path.getValue()); - EasyMock.verify(azureStorage); - } - @Test public void testPathExistsNotFound() throws URISyntaxException, StorageException, IOException { @@ -99,23 +92,103 @@ public void testPathExistsNotFound() throws URISyntaxException, StorageException } @Test - public void testRead() {} + public void testRead() throws URISyntaxException, StorageException, IOException + { + EasyMock.reset(azureStorage); - @Test - public void testReadRange() {} + String data = "test"; + EasyMock.expect(azureStorage.getBlobLength(EasyMock.anyString(), EasyMock.anyString())) + .andReturn(4L); + EasyMock.expect( + azureStorage.getBlobInputStream( + EasyMock.anyLong(), + EasyMock.anyLong(), + EasyMock.anyString(), + EasyMock.anyString() + ) + ).andReturn(IOUtils.toInputStream(data, StandardCharsets.UTF_8)); - @Test - public void testDeleteSinglePath() {} + EasyMock.replay(azureStorage); + InputStream is = storageConnector.read(TEST_FILE); + byte[] dataBytes = new byte[data.length()]; + Assert.assertEquals(data.length(), is.read(dataBytes)); + Assert.assertEquals(-1, is.read()); + Assert.assertEquals(data, new String(dataBytes, StandardCharsets.UTF_8)); + + EasyMock.reset(azureStorage); + } @Test - public void testDeleteMultiplePaths() {} + public void testReadRange() throws URISyntaxException, StorageException, IOException + { + String data = "test"; + + for (int start = 0; start < data.length(); ++start) { + for (long length = 1; length <= data.length() - start; ++length) { + String dataQueried = data.substring(start, start + ((Long) length).intValue()); + EasyMock.reset(azureStorage); + EasyMock.expect(azureStorage.getBlobInputStream( + EasyMock.anyLong(), + EasyMock.anyLong(), + EasyMock.anyString(), + EasyMock.anyString() + )) + .andReturn(IOUtils.toInputStream(dataQueried, StandardCharsets.UTF_8)); + EasyMock.replay(azureStorage); + + InputStream is = storageConnector.readRange(TEST_FILE, start, length); + byte[] dataBytes = new byte[((Long) length).intValue()]; + Assert.assertEquals(length, is.read(dataBytes)); + Assert.assertEquals(-1, is.read()); + Assert.assertEquals(dataQueried, new String(dataBytes, StandardCharsets.UTF_8)); + EasyMock.reset(azureStorage); + } + } + } @Test - public void testDeleteRecursive() + public void testDeleteSinglePath() throws URISyntaxException, StorageException, IOException { + EasyMock.reset(azureStorage); + Capture containerCapture = EasyMock.newCapture(); + Capture> pathsCapture = EasyMock.newCapture(); + azureStorage.batchDeleteFiles(EasyMock.capture(containerCapture), EasyMock.capture(pathsCapture)); + EasyMock.replay(azureStorage); + storageConnector.deleteFile(TEST_FILE); + Assert.assertEquals(CONTAINER, containerCapture.getValue()); + Assert.assertEquals(Collections.singletonList(PREFIX + "/" + TEST_FILE), pathsCapture.getValue()); + EasyMock.reset(azureStorage); + } + @Test + public void testDeleteMultiplePaths() throws URISyntaxException, StorageException, IOException + { + EasyMock.reset(azureStorage); + Capture containerCapture = EasyMock.newCapture(); + Capture> pathsCapture = EasyMock.newCapture(); + azureStorage.batchDeleteFiles(EasyMock.capture(containerCapture), EasyMock.capture(pathsCapture)); + EasyMock.replay(azureStorage); + storageConnector.deleteFiles(ImmutableList.of(TEST_FILE + "_1.part", TEST_FILE + "_2.part")); + Assert.assertEquals(CONTAINER, containerCapture.getValue()); + Assert.assertEquals( + ImmutableList.of( + PREFIX + "/" + TEST_FILE + "_1.part", + PREFIX + "/" + TEST_FILE + "_2.part" + ), + Lists.newArrayList(pathsCapture.getValue()) + ); + EasyMock.reset(azureStorage); } @Test - public void testListDir() {} -} \ No newline at end of file + public void testListDir() throws URISyntaxException, StorageException, IOException + { + EasyMock.reset(azureStorage); + EasyMock.expect(azureStorage.listDir(EasyMock.anyString(), EasyMock.anyString())) + .andReturn(ImmutableList.of(PREFIX + "/x/y/z/" + TEST_FILE, PREFIX + "/p/q/r/" + TEST_FILE)); + EasyMock.replay(azureStorage); + List ret = Lists.newArrayList(storageConnector.listDir("")); + Assert.assertEquals(ImmutableList.of("x/y/z/" + TEST_FILE, "p/q/r/" + TEST_FILE), ret); + EasyMock.reset(azureStorage); + } +} diff --git a/processing/src/main/java/org/apache/druid/storage/remote/ChunkingStorageConnector.java b/processing/src/main/java/org/apache/druid/storage/remote/ChunkingStorageConnector.java index e18058a7e27a..5da08df65150 100644 --- a/processing/src/main/java/org/apache/druid/storage/remote/ChunkingStorageConnector.java +++ b/processing/src/main/java/org/apache/druid/storage/remote/ChunkingStorageConnector.java @@ -50,11 +50,6 @@ public abstract class ChunkingStorageConnector implements StorageConnector { private static final long DOWNLOAD_MAX_CHUNK_SIZE_BYTES = 100_000_000; - public ChunkingStorageConnector() - { - - } - @Override public InputStream read(String path) throws IOException { diff --git a/processing/src/main/java/org/apache/druid/storage/remote/ChunkingStorageConnectorParameters.java b/processing/src/main/java/org/apache/druid/storage/remote/ChunkingStorageConnectorParameters.java index 9e599cc29e08..03f5ecad1b13 100644 --- a/processing/src/main/java/org/apache/druid/storage/remote/ChunkingStorageConnectorParameters.java +++ b/processing/src/main/java/org/apache/druid/storage/remote/ChunkingStorageConnectorParameters.java @@ -242,6 +242,7 @@ public ChunkingStorageConnectorParameters build() { Preconditions.checkArgument(start >= 0, "'start' not provided or an incorrect value [%s] passed", start); Preconditions.checkArgument(end >= 0, "'end' not provided or an incorrect value [%s] passed", end); + Preconditions.checkArgument(start <= end, "'start' should not be greater than 'end'"); Preconditions.checkArgument(maxRetry >= 0, "'maxRetry' not provided or an incorrect value [%s] passed", maxRetry); return new ChunkingStorageConnectorParameters( start, From ef18323c11cd45a1c92766681b7c062f11fc93e4 Mon Sep 17 00:00:00 2001 From: Laksh Singla Date: Thu, 3 Aug 2023 16:32:33 +0530 Subject: [PATCH 17/28] tests fix --- .../azure/output/AzureOutputSerdeTest.java | 17 ++++------------- 1 file changed, 4 insertions(+), 13 deletions(-) diff --git a/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/output/AzureOutputSerdeTest.java b/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/output/AzureOutputSerdeTest.java index 6d8e6120acab..7d5ebaeb5751 100644 --- a/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/output/AzureOutputSerdeTest.java +++ b/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/output/AzureOutputSerdeTest.java @@ -72,9 +72,7 @@ public void noPrefix() throws JsonProcessingException + " \"chunkSize\":104857600,\n" + " \"maxRetry\": 2\n" + "}\n"); - expectedException.expect(MismatchedInputException.class); - expectedException.expectMessage("Missing required creator property 'prefix'"); - MAPPER.readValue(json, AzureOutputConfig.class); + Assert.assertThrows(MismatchedInputException.class, () -> MAPPER.readValue(json, AzureOutputConfig.class)); } @Test @@ -86,9 +84,7 @@ public void nocontainer() throws JsonProcessingException + " \"chunkSize\":104857600,\n" + " \"maxRetry\": 2\n" + "}\n"); - expectedException.expect(MismatchedInputException.class); - expectedException.expectMessage("Missing required creator property 'container'"); - MAPPER.readValue(json, AzureOutputConfig.class); + Assert.assertThrows(MismatchedInputException.class, () -> MAPPER.readValue(json, AzureOutputConfig.class)); } @Test @@ -100,9 +96,7 @@ public void noTempDir() throws JsonProcessingException + " \"chunkSize\":104857600,\n" + " \"maxRetry\": 2\n" + "}\n"); - expectedException.expect(MismatchedInputException.class); - expectedException.expectMessage("Missing required creator property 'tempDir'"); - MAPPER.readValue(json, AzureOutputConfig.class); + Assert.assertThrows(MismatchedInputException.class, () -> MAPPER.readValue(json, AzureOutputConfig.class)); } @Test @@ -136,10 +130,7 @@ public void testChunkValidation() throws JsonProcessingException + " \"chunkSize\":104,\n" + " \"maxRetry\": 2\n" + "}\n"); - expectedException.expect(ValueInstantiationException.class); - expectedException.expectMessage("'chunkSize' [104] bytes to the AzureConfig should be between " - + "[262144] bytes and [4194304000] bytes"); - MAPPER.readValue(json, AzureOutputConfig.class); + Assert.assertThrows(ValueInstantiationException.class, () -> MAPPER.readValue(json, AzureOutputConfig.class)); } private static String jsonStringReadyForAssert(String input) From f87266d15424eea2a2bc771e6e1a4d7c4dfcf683 Mon Sep 17 00:00:00 2001 From: Laksh Singla Date: Thu, 3 Aug 2023 17:34:23 +0530 Subject: [PATCH 18/28] fix import --- .../druid/storage/azure/output/AzureStorageConnectorTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/output/AzureStorageConnectorTest.java b/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/output/AzureStorageConnectorTest.java index 538dd68d00e5..eab0b92472c0 100644 --- a/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/output/AzureStorageConnectorTest.java +++ b/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/output/AzureStorageConnectorTest.java @@ -19,8 +19,8 @@ package org.apache.druid.storage.azure.output; -import com.google.api.client.util.Lists; import com.google.common.collect.ImmutableList; +import com.google.common.collect.Lists; import com.microsoft.azure.storage.StorageException; import org.apache.commons.io.IOUtils; import org.apache.druid.storage.StorageConnector; From 9cc6bf3fa49fa9c909537b5eeb0bf69315e95367 Mon Sep 17 00:00:00 2001 From: Laksh Singla Date: Fri, 4 Aug 2023 11:06:59 +0530 Subject: [PATCH 19/28] add more tests --- .../remote/ChunkingStorageConnector.java | 18 ++- ...hunkingStorageConnectorParametersTest.java | 25 +++- .../remote/ChunkingStorageConnectorTest.java | 87 +++++++++++ .../storage/remote/TestStorageConnector.java | 135 ++++++++++++++++++ 4 files changed, 261 insertions(+), 4 deletions(-) create mode 100644 processing/src/test/java/org/apache/druid/storage/remote/ChunkingStorageConnectorTest.java create mode 100644 processing/src/test/java/org/apache/druid/storage/remote/TestStorageConnector.java diff --git a/processing/src/main/java/org/apache/druid/storage/remote/ChunkingStorageConnector.java b/processing/src/main/java/org/apache/druid/storage/remote/ChunkingStorageConnector.java index 5da08df65150..49505e16996b 100644 --- a/processing/src/main/java/org/apache/druid/storage/remote/ChunkingStorageConnector.java +++ b/processing/src/main/java/org/apache/druid/storage/remote/ChunkingStorageConnector.java @@ -50,6 +50,20 @@ public abstract class ChunkingStorageConnector implements StorageConnector { private static final long DOWNLOAD_MAX_CHUNK_SIZE_BYTES = 100_000_000; + private final long chunkSizeBytes; + + public ChunkingStorageConnector() + { + this(DOWNLOAD_MAX_CHUNK_SIZE_BYTES); + } + + public ChunkingStorageConnector( + final long chunkSizeBytes + ) + { + this.chunkSizeBytes = chunkSizeBytes; + } + @Override public InputStream read(String path) throws IOException { @@ -108,9 +122,8 @@ public InputStream nextElement() UUID.randomUUID().toString() ); - // exclusive long currentReadEndPosition = Math.min( - currentReadStartPosition.get() + DOWNLOAD_MAX_CHUNK_SIZE_BYTES, + currentReadStartPosition.get() + chunkSizeBytes, readEnd ); @@ -160,7 +173,6 @@ public void close() throws IOException } fileInputStreamClosed.set(true); super.close(); - // since endPoint is inclusive in s3's get request API, the next currReadStart is endpoint + 1 currentReadStartPosition.set(currentReadEndPosition); if (!outFile.delete()) { throw new RE("Cannot delete temp file [%s]", outFile); diff --git a/processing/src/test/java/org/apache/druid/storage/remote/ChunkingStorageConnectorParametersTest.java b/processing/src/test/java/org/apache/druid/storage/remote/ChunkingStorageConnectorParametersTest.java index aad4a2577b64..d8b879da433a 100644 --- a/processing/src/test/java/org/apache/druid/storage/remote/ChunkingStorageConnectorParametersTest.java +++ b/processing/src/test/java/org/apache/druid/storage/remote/ChunkingStorageConnectorParametersTest.java @@ -19,10 +19,13 @@ package org.apache.druid.storage.remote; +import com.google.common.base.Predicates; import nl.jqno.equalsverifier.EqualsVerifier; import org.junit.Assert; import org.junit.Test; +import java.io.File; + public class ChunkingStorageConnectorParametersTest { @Test @@ -34,10 +37,30 @@ public void testEquals() } @Test - public void testVerifyParameters() + public void testIncorrectParameters() { ChunkingStorageConnectorParameters.Builder builder = new ChunkingStorageConnectorParameters.Builder<>(); builder.start(-1); Assert.assertThrows(IllegalArgumentException.class, builder::build); } + + @Test + public void testCorrectParameters() + { + ChunkingStorageConnectorParameters.Builder builder = new ChunkingStorageConnectorParameters.Builder<>(); + builder.start(0); + builder.end(10); + builder.objectSupplier((start, end) -> null); + builder.objectOpenFunction(obj -> null); + builder.maxRetry(10); + builder.cloudStoragePath("/path"); + builder.retryCondition(Predicates.alwaysTrue()); + builder.tempDirSupplier(() -> new File("/tmp")); + ChunkingStorageConnectorParameters parameters = builder.build(); + Assert.assertEquals(0, parameters.getStart()); + Assert.assertEquals(10, parameters.getEnd()); + Assert.assertEquals(10, parameters.getMaxRetry()); + Assert.assertEquals("/path", parameters.getCloudStoragePath()); + Assert.assertEquals("/tmp", parameters.getTempDirSupplier().get().getAbsolutePath()); + } } diff --git a/processing/src/test/java/org/apache/druid/storage/remote/ChunkingStorageConnectorTest.java b/processing/src/test/java/org/apache/druid/storage/remote/ChunkingStorageConnectorTest.java new file mode 100644 index 000000000000..ccadab3a8844 --- /dev/null +++ b/processing/src/test/java/org/apache/druid/storage/remote/ChunkingStorageConnectorTest.java @@ -0,0 +1,87 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.storage.remote; + +import com.google.common.collect.ImmutableList; +import org.apache.commons.io.IOUtils; +import org.apache.druid.storage.StorageConnector; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +import java.io.IOException; +import java.io.InputStream; +import java.nio.charset.StandardCharsets; +import java.util.List; + +public class ChunkingStorageConnectorTest +{ + + @Rule + public TemporaryFolder temporaryFolder = new TemporaryFolder(); + + private StorageConnector storageConnector; + + @Before + public void setup() throws IOException + { + storageConnector = new TestStorageConnector(temporaryFolder.newFolder()); + } + + @Test + public void testRead() throws IOException + { + InputStream is = storageConnector.read(""); + byte[] dataBytes = IOUtils.toByteArray(is); + Assert.assertEquals(TestStorageConnector.DATA, new String(dataBytes, StandardCharsets.UTF_8)); + } + + @Test + public void testReadRange() throws IOException + { + + List ranges = ImmutableList.of( + TestStorageConnector.CHUNK_SIZE_BYTES, + TestStorageConnector.CHUNK_SIZE_BYTES * 2, + TestStorageConnector.CHUNK_SIZE_BYTES * 7, + TestStorageConnector.CHUNK_SIZE_BYTES + 1, + TestStorageConnector.CHUNK_SIZE_BYTES + 2, + TestStorageConnector.CHUNK_SIZE_BYTES + 3 + ); + + List startPositions = ImmutableList.of(0, 25, 37, TestStorageConnector.DATA.length() - 10); + + for (int range : ranges) { + for (int startPosition : startPositions) { + int limitedRange = startPosition + range > TestStorageConnector.DATA.length() + ? TestStorageConnector.DATA.length() - startPosition + : range; + InputStream is = storageConnector.readRange("", startPosition, limitedRange); + byte[] dataBytes = IOUtils.toByteArray(is); + Assert.assertEquals( + TestStorageConnector.DATA.substring(startPosition, startPosition + limitedRange), + new String(dataBytes, StandardCharsets.UTF_8) + ); + } + } + } +} diff --git a/processing/src/test/java/org/apache/druid/storage/remote/TestStorageConnector.java b/processing/src/test/java/org/apache/druid/storage/remote/TestStorageConnector.java new file mode 100644 index 000000000000..e5757a949265 --- /dev/null +++ b/processing/src/test/java/org/apache/druid/storage/remote/TestStorageConnector.java @@ -0,0 +1,135 @@ +/* + * 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.remote; + +import com.google.common.base.Predicates; +import org.apache.commons.io.IOUtils; +import org.apache.druid.data.input.impl.prefetch.ObjectOpenFunction; + +import java.io.File; +import java.io.InputStream; +import java.io.OutputStream; +import java.nio.charset.StandardCharsets; +import java.util.Iterator; + +public class TestStorageConnector extends ChunkingStorageConnector +{ + + public static final String DATA = "This is some random data text. This should be returned in chunks by the methods, " + + "however the connector should reassemble it as a single stream of text"; + + public static final int CHUNK_SIZE_BYTES = 4; + + private final File tempDir; + + public TestStorageConnector( + final File tempDir + ) + { + super(CHUNK_SIZE_BYTES); + this.tempDir = tempDir; + } + + @Override + public ChunkingStorageConnectorParameters buildInputParams(String path) + { + return buildInputParams(path, 0, DATA.length()); + } + + @Override + public ChunkingStorageConnectorParameters buildInputParams( + String path, + long from, + long size + ) + { + ChunkingStorageConnectorParameters.Builder builder = new ChunkingStorageConnectorParameters.Builder<>(); + builder.start(from); + builder.end(from + size); + builder.cloudStoragePath(path); + builder.tempDirSupplier(() -> tempDir); + builder.retryCondition(Predicates.alwaysFalse()); + builder.maxRetry(2); + builder.objectSupplier((start, end) -> new InputRange((int) start, (int) end)); + builder.objectOpenFunction(new ObjectOpenFunction() + { + @Override + public InputStream open(InputRange ir) + { + return IOUtils.toInputStream(DATA.substring(ir.start, ir.end), StandardCharsets.UTF_8); + } + + @Override + public InputStream open(InputRange ir, long offset) + { + return open(new InputRange(ir.start + (int) offset, ir.end)); + } + }); + return builder.build(); + } + + @Override + public boolean pathExists(String path) + { + throw new UnsupportedOperationException(); + } + + @Override + public OutputStream write(String path) + { + throw new UnsupportedOperationException(); + } + + @Override + public void deleteFile(String path) + { + throw new UnsupportedOperationException(); + } + + @Override + public void deleteFiles(Iterable paths) + { + throw new UnsupportedOperationException(); + } + + @Override + public void deleteRecursively(String path) + { + throw new UnsupportedOperationException(); + } + + @Override + public Iterator listDir(String dirName) + { + throw new UnsupportedOperationException(); + } + + public static class InputRange + { + private final int start; + private final int end; + + public InputRange(int start, int end) + { + this.start = start; + this.end = end; + } + } +} From 7e138ef6d1b5d3449d2eab38a8153729c152816a Mon Sep 17 00:00:00 2001 From: Laksh Singla Date: Fri, 4 Aug 2023 16:11:02 +0530 Subject: [PATCH 20/28] fixup tests --- .../druid/storage/azure/output/AzureOutputSerdeTest.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/output/AzureOutputSerdeTest.java b/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/output/AzureOutputSerdeTest.java index 7d5ebaeb5751..ecf99666ce70 100644 --- a/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/output/AzureOutputSerdeTest.java +++ b/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/output/AzureOutputSerdeTest.java @@ -64,7 +64,7 @@ public void sanity() throws IOException } @Test - public void noPrefix() throws JsonProcessingException + public void noPrefix() { String json = jsonStringReadyForAssert("{\n" + " \"container\": \"TEST\",\n" @@ -76,7 +76,7 @@ public void noPrefix() throws JsonProcessingException } @Test - public void nocontainer() throws JsonProcessingException + public void noContainer() { String json = jsonStringReadyForAssert("{\n" + " \"prefix\": \"abc\",\n" @@ -88,7 +88,7 @@ public void nocontainer() throws JsonProcessingException } @Test - public void noTempDir() throws JsonProcessingException + public void noTempDir() { String json = jsonStringReadyForAssert("{\n" + " \"prefix\": \"abc\",\n" @@ -120,7 +120,7 @@ public void leastArguments() throws JsonProcessingException @Test - public void testChunkValidation() throws JsonProcessingException + public void testChunkValidation() { String json = jsonStringReadyForAssert("{\n" From eafe83641ab0ff8c1009f667a8a956b1ee8ebaa0 Mon Sep 17 00:00:00 2001 From: Laksh Singla Date: Sun, 6 Aug 2023 21:29:16 +0530 Subject: [PATCH 21/28] more tests --- .../AzureStorageConnectorProviderTest.java | 148 ++++++++++++++++++ 1 file changed, 148 insertions(+) create mode 100644 extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/output/AzureStorageConnectorProviderTest.java diff --git a/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/output/AzureStorageConnectorProviderTest.java b/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/output/AzureStorageConnectorProviderTest.java new file mode 100644 index 000000000000..50a856c71255 --- /dev/null +++ b/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/output/AzureStorageConnectorProviderTest.java @@ -0,0 +1,148 @@ +/* + * 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.azure.output; + +import com.fasterxml.jackson.databind.InjectableValues; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.inject.Injector; +import com.google.inject.Key; +import com.google.inject.ProvisionException; +import com.google.inject.name.Names; +import org.apache.druid.guice.JsonConfigProvider; +import org.apache.druid.guice.LazySingleton; +import org.apache.druid.guice.StartupInjectorBuilder; +import org.apache.druid.storage.StorageConnector; +import org.apache.druid.storage.StorageConnectorModule; +import org.apache.druid.storage.StorageConnectorProvider; +import org.apache.druid.storage.azure.AzureStorage; +import org.apache.druid.storage.azure.AzureStorageDruidModule; +import org.easymock.EasyMock; +import org.junit.Assert; +import org.junit.Test; + +import java.io.File; +import java.util.Properties; + +public class AzureStorageConnectorProviderTest +{ + private static final String CUSTOM_NAMESPACE = "custom"; + + @Test + public void createAzureStorageFactoryWithRequiredProperties() + { + + final Properties properties = new Properties(); + properties.setProperty(CUSTOM_NAMESPACE + ".type", "azure"); + properties.setProperty(CUSTOM_NAMESPACE + ".container", "container"); + properties.setProperty(CUSTOM_NAMESPACE + ".prefix", "prefix"); + properties.setProperty(CUSTOM_NAMESPACE + ".tempDir", "/tmp"); + StorageConnectorProvider s3StorageConnectorProvider = getStorageConnectorProvider(properties); + + Assert.assertTrue(s3StorageConnectorProvider instanceof AzureStorageConnectorProvider); + Assert.assertTrue(s3StorageConnectorProvider.get() instanceof AzureStorageConnector); + Assert.assertEquals("container", ((AzureStorageConnectorProvider) s3StorageConnectorProvider).getContainer()); + Assert.assertEquals("prefix", ((AzureStorageConnectorProvider) s3StorageConnectorProvider).getPrefix()); + Assert.assertEquals(new File("/tmp"), ((AzureStorageConnectorProvider) s3StorageConnectorProvider).getTempDir()); + + } + + @Test + public void createAzureStorageFactoryWithMissingPrefix() + { + + final Properties properties = new Properties(); + properties.setProperty(CUSTOM_NAMESPACE + ".type", "s3"); + properties.setProperty(CUSTOM_NAMESPACE + ".container", "container"); + properties.setProperty(CUSTOM_NAMESPACE + ".tempDir", "/tmp"); + Assert.assertThrows( + "Missing required creator property 'prefix'", + ProvisionException.class, + () -> getStorageConnectorProvider(properties) + ); + } + + + @Test + public void createAzureStorageFactoryWithMissingContainer() + { + + final Properties properties = new Properties(); + properties.setProperty(CUSTOM_NAMESPACE + ".type", "azure"); + properties.setProperty(CUSTOM_NAMESPACE + ".prefix", "prefix"); + properties.setProperty(CUSTOM_NAMESPACE + ".tempDir", "/tmp"); + Assert.assertThrows( + "Missing required creator property 'container'", + ProvisionException.class, + () -> getStorageConnectorProvider(properties) + ); + } + + @Test + public void createAzureStorageFactoryWithMissingTempDir() + { + + final Properties properties = new Properties(); + properties.setProperty(CUSTOM_NAMESPACE + ".type", "azure"); + properties.setProperty(CUSTOM_NAMESPACE + ".container", "container"); + properties.setProperty(CUSTOM_NAMESPACE + ".prefix", "prefix"); + + Assert.assertThrows( + "Missing required creator property 'tempDir'", + ProvisionException.class, + () -> getStorageConnectorProvider(properties) + ); + } + + private StorageConnectorProvider getStorageConnectorProvider(Properties properties) + { + StartupInjectorBuilder startupInjectorBuilder = new StartupInjectorBuilder().add( + new AzureStorageDruidModule(), + new StorageConnectorModule(), + new AzureStorageConnectorModule(), + binder -> { + JsonConfigProvider.bind( + binder, + CUSTOM_NAMESPACE, + StorageConnectorProvider.class, + Names.named(CUSTOM_NAMESPACE) + ); + + binder.bind(Key.get(StorageConnector.class, Names.named(CUSTOM_NAMESPACE))) + .toProvider(Key.get(StorageConnectorProvider.class, Names.named(CUSTOM_NAMESPACE))) + .in(LazySingleton.class); + } + ).withProperties(properties); + + Injector injector = startupInjectorBuilder.build(); + injector.getInstance(ObjectMapper.class).registerModules(new AzureStorageConnectorModule().getJacksonModules()); + injector.getInstance(ObjectMapper.class).setInjectableValues( + new InjectableValues.Std() + .addValue( + AzureStorage.class, + EasyMock.mock(AzureStorage.class) + )); + + + return injector.getInstance(Key.get( + StorageConnectorProvider.class, + Names.named(CUSTOM_NAMESPACE) + )); + } +} From 47aeae6ed019ca25ef536540ee37b06f9b212fab Mon Sep 17 00:00:00 2001 From: Laksh Singla Date: Mon, 7 Aug 2023 03:10:37 +0530 Subject: [PATCH 22/28] more coverage --- extensions-core/azure-extensions/pom.xml | 13 ++- .../druid/storage/azure/AzureStorage.java | 82 ++++++++++++++++--- .../azure/output/AzureStorageConnector.java | 24 ++++-- .../druid/storage/azure/AzureStorageTest.java | 71 ++++++++++++++++ .../azure/output/AzureInputRangeTest.java | 34 ++++++++ .../output/AzureStorageConnectorTest.java | 22 +++-- 6 files changed, 220 insertions(+), 26 deletions(-) create mode 100644 extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/AzureStorageTest.java create mode 100644 extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/output/AzureInputRangeTest.java diff --git a/extensions-core/azure-extensions/pom.xml b/extensions-core/azure-extensions/pom.xml index aa87fc353ccc..80969765af31 100644 --- a/extensions-core/azure-extensions/pom.xml +++ b/extensions-core/azure-extensions/pom.xml @@ -92,7 +92,7 @@ com.google.inject.extensions guice-assistedinject - ${guice.version} + provided com.fasterxml.jackson.core @@ -152,6 +152,17 @@ equalsverifier test + + org.mockito + mockito-core + ${mockito.version} + test + + + org.mockito + mockito-inline + test + diff --git a/extensions-core/azure-extensions/src/main/java/org/apache/druid/storage/azure/AzureStorage.java b/extensions-core/azure-extensions/src/main/java/org/apache/druid/storage/azure/AzureStorage.java index bd94f61bdbdd..0ca0e3cba70e 100644 --- a/extensions-core/azure-extensions/src/main/java/org/apache/druid/storage/azure/AzureStorage.java +++ b/extensions-core/azure-extensions/src/main/java/org/apache/druid/storage/azure/AzureStorage.java @@ -23,13 +23,16 @@ import com.google.common.base.Supplier; import com.microsoft.azure.storage.ResultContinuation; import com.microsoft.azure.storage.ResultSegment; +import com.microsoft.azure.storage.RetryExponentialRetry; import com.microsoft.azure.storage.StorageException; import com.microsoft.azure.storage.blob.BlobDeleteBatchOperation; import com.microsoft.azure.storage.blob.BlobListingDetails; +import com.microsoft.azure.storage.blob.BlobRequestOptions; import com.microsoft.azure.storage.blob.CloudBlob; import com.microsoft.azure.storage.blob.CloudBlobClient; import com.microsoft.azure.storage.blob.CloudBlobContainer; import com.microsoft.azure.storage.blob.CloudBlockBlob; +import com.microsoft.azure.storage.blob.DeleteSnapshotsOption; import com.microsoft.azure.storage.blob.ListBlobItem; import org.apache.druid.java.util.common.RE; import org.apache.druid.java.util.common.logger.Logger; @@ -52,6 +55,9 @@ public class AzureStorage { private static final boolean USE_FLAT_BLOB_LISTING = true; + // Default value from Azure library + private static final int DELTA_BACKOFF_MS = 30_000; + private static final Logger log = new Logger(AzureStorage.class); /** @@ -71,17 +77,31 @@ public AzureStorage( { this.cloudBlobClient = cloudBlobClient; } - + public List emptyCloudBlobDirectory(final String containerName, final String virtualDirPath) throws StorageException, URISyntaxException + { + return emptyCloudBlobDirectory(containerName, virtualDirPath, null); + } + + public List emptyCloudBlobDirectory(final String containerName, final String virtualDirPath, final Integer maxAttempts) + throws StorageException, URISyntaxException { List deletedFiles = new ArrayList<>(); CloudBlobContainer container = getOrCreateCloudBlobContainer(containerName); - for (ListBlobItem blobItem : container.listBlobs(virtualDirPath, USE_FLAT_BLOB_LISTING, null, null, null)) { + Iterable blobItems = container.listBlobs( + virtualDirPath, + USE_FLAT_BLOB_LISTING, + null, + getRequestOptionsWithRetry(maxAttempts), + null + ); + + for (ListBlobItem blobItem : blobItems) { CloudBlob cloudBlob = (CloudBlob) blobItem; - log.info("Removing file[%s] from Azure.", cloudBlob.getName()); - if (cloudBlob.deleteIfExists()) { + log.debug("Removing file[%s] from Azure.", cloudBlob.getName()); + if (cloudBlob.deleteIfExists(DeleteSnapshotsOption.NONE, null, getRequestOptionsWithRetry(maxAttempts), null)) { deletedFiles.add(cloudBlob.getName()); } } @@ -105,7 +125,8 @@ public void uploadBlob(final File file, final String containerName, final String public OutputStream getBlockBlobOutputStream( final String containerName, final String blobPath, - @Nullable final Integer streamWriteSizeBytes + @Nullable final Integer streamWriteSizeBytes, + Integer maxAttempts ) throws URISyntaxException, StorageException { CloudBlobContainer container = getOrCreateCloudBlobContainer(containerName); @@ -119,7 +140,7 @@ public OutputStream getBlockBlobOutputStream( blockBlobReference.setStreamWriteSizeInBytes(streamWriteSizeBytes); } - return blockBlobReference.openOutputStream(); + return blockBlobReference.openOutputStream(null, getRequestOptionsWithRetry(maxAttempts), null); } @@ -151,13 +172,27 @@ public InputStream getBlobInputStream(long offset, final String containerName, f public InputStream getBlobInputStream(long offset, Long length, final String containerName, final String blobPath) throws URISyntaxException, StorageException + { + return getBlobInputStream(offset, length, containerName, blobPath, null); + } + + public InputStream getBlobInputStream(long offset, Long length, final String containerName, final String blobPath, Integer maxAttempts) + throws URISyntaxException, StorageException { CloudBlobContainer container = getOrCreateCloudBlobContainer(containerName); - return container.getBlockBlobReference(blobPath).openInputStream(offset, length, null, null, null); + return container.getBlockBlobReference(blobPath) + .openInputStream(offset, length, null, getRequestOptionsWithRetry(maxAttempts), null); } + public void batchDeleteFiles(String containerName, Iterable paths) throws URISyntaxException, StorageException + { + batchDeleteFiles(containerName, paths, null); + } + + public void batchDeleteFiles(String containerName, Iterable paths, Integer maxAttempts) + throws URISyntaxException, StorageException { CloudBlobContainer cloudBlobContainer = getOrCreateCloudBlobContainer(containerName); BlobDeleteBatchOperation blobDeleteBatchOperation = new BlobDeleteBatchOperation(); @@ -165,16 +200,23 @@ public void batchDeleteFiles(String containerName, Iterable paths) CloudBlob blobReference = cloudBlobContainer.getBlockBlobReference(path); blobDeleteBatchOperation.addSubOperation(blobReference); } - cloudBlobClient.get().executeBatch(blobDeleteBatchOperation); + cloudBlobClient.get().executeBatch(blobDeleteBatchOperation, getRequestOptionsWithRetry(maxAttempts), null); } public List listDir(final String containerName, final String virtualDirPath) + throws URISyntaxException, StorageException + { + return listDir(containerName, virtualDirPath, null); + } + + public List listDir(final String containerName, final String virtualDirPath, final Integer maxAttempts) throws StorageException, URISyntaxException { List files = new ArrayList<>(); CloudBlobContainer container = getOrCreateCloudBlobContainer(containerName); - for (ListBlobItem blobItem : container.listBlobs(virtualDirPath, USE_FLAT_BLOB_LISTING, null, null, null)) { + for (ListBlobItem blobItem : + container.listBlobs(virtualDirPath, USE_FLAT_BLOB_LISTING, null, getRequestOptionsWithRetry(maxAttempts), null)) { CloudBlob cloudBlob = (CloudBlob) blobItem; files.add(cloudBlob.getName()); } @@ -182,10 +224,28 @@ public List listDir(final String containerName, final String virtualDirP return files; } - public boolean getBlobExists(String container, String blobPath) throws URISyntaxException, StorageException { - return getOrCreateCloudBlobContainer(container).getBlockBlobReference(blobPath).exists(); + return getBlobExists(container, blobPath, null); + } + + + public boolean getBlobExists(String container, String blobPath, Integer maxAttempts) + throws URISyntaxException, StorageException + { + return getOrCreateCloudBlobContainer(container).getBlockBlobReference(blobPath) + .exists(null, getRequestOptionsWithRetry(maxAttempts), null); + } + + @Nullable + private BlobRequestOptions getRequestOptionsWithRetry(Integer maxAttempts) + { + if (maxAttempts == null) { + return null; + } + BlobRequestOptions requestOptions = new BlobRequestOptions(); + requestOptions.setRetryPolicyFactory(new RetryExponentialRetry(DELTA_BACKOFF_MS, maxAttempts)); + return requestOptions; } @VisibleForTesting diff --git a/extensions-core/azure-extensions/src/main/java/org/apache/druid/storage/azure/output/AzureStorageConnector.java b/extensions-core/azure-extensions/src/main/java/org/apache/druid/storage/azure/output/AzureStorageConnector.java index 9b57b81fbc17..4b088a661ffc 100644 --- a/extensions-core/azure-extensions/src/main/java/org/apache/druid/storage/azure/output/AzureStorageConnector.java +++ b/extensions-core/azure-extensions/src/main/java/org/apache/druid/storage/azure/output/AzureStorageConnector.java @@ -92,7 +92,8 @@ public InputStream open(AzureInputRange inputRange) throws IOException inputRange.getStart(), inputRange.getSize(), inputRange.getBucket(), - inputRange.getPath() + inputRange.getPath(), + config.getMaxRetry() ); } catch (URISyntaxException | StorageException e) { @@ -121,7 +122,7 @@ public InputStream open(AzureInputRange inputRange, long offset) throws IOExcept public boolean pathExists(String path) throws IOException { try { - return azureStorage.getBlobExists(config.getContainer(), objectPath(path)); + return azureStorage.getBlobExists(config.getContainer(), objectPath(path), config.getMaxRetry()); } catch (URISyntaxException | StorageException e) { throw new IOException(e); @@ -135,7 +136,8 @@ public OutputStream write(String path) throws IOException return azureStorage.getBlockBlobOutputStream( config.getContainer(), objectPath(path), - config.getChunkSize().getBytesInInt() + config.getChunkSize().getBytesInInt(), + config.getMaxRetry() ); } catch (URISyntaxException | StorageException e) { @@ -147,7 +149,11 @@ public OutputStream write(String path) throws IOException public void deleteFile(String path) throws IOException { try { - azureStorage.batchDeleteFiles(config.getContainer(), Collections.singletonList(objectPath(path))); + azureStorage.batchDeleteFiles( + config.getContainer(), + Collections.singletonList(objectPath(path)), + config.getMaxRetry() + ); } catch (URISyntaxException | StorageException e) { throw new IOException(e); @@ -158,7 +164,11 @@ public void deleteFile(String path) throws IOException public void deleteFiles(Iterable paths) throws IOException { try { - azureStorage.batchDeleteFiles(config.getContainer(), Iterables.transform(paths, this::objectPath)); + azureStorage.batchDeleteFiles( + config.getContainer(), + Iterables.transform(paths, this::objectPath), + config.getMaxRetry() + ); } catch (StorageException | URISyntaxException e) { throw new IOException(e); @@ -169,7 +179,7 @@ public void deleteFiles(Iterable paths) throws IOException public void deleteRecursively(String path) throws IOException { try { - azureStorage.emptyCloudBlobDirectory(config.getContainer(), objectPath(path)); + azureStorage.emptyCloudBlobDirectory(config.getContainer(), objectPath(path), config.getMaxRetry()); } catch (StorageException | URISyntaxException e) { throw new IOException(e); @@ -182,7 +192,7 @@ public Iterator listDir(String dirName) throws IOException final String prefixBasePath = objectPath(dirName); List paths; try { - paths = azureStorage.listDir(config.getContainer(), prefixBasePath); + paths = azureStorage.listDir(config.getContainer(), prefixBasePath, config.getMaxRetry()); } catch (StorageException | URISyntaxException e) { throw new IOException(e); diff --git a/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/AzureStorageTest.java b/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/AzureStorageTest.java new file mode 100644 index 000000000000..9ae08546401a --- /dev/null +++ b/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/AzureStorageTest.java @@ -0,0 +1,71 @@ +/* + * 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.azure; + +import com.google.common.collect.ImmutableList; +import com.microsoft.azure.storage.StorageException; +import com.microsoft.azure.storage.blob.CloudBlobClient; +import com.microsoft.azure.storage.blob.CloudBlobContainer; +import com.microsoft.azure.storage.blob.CloudBlockBlob; +import com.microsoft.azure.storage.blob.ListBlobItem; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import org.mockito.ArgumentMatchers; +import org.mockito.Mockito; + +import java.net.URI; +import java.net.URISyntaxException; +import java.util.List; + +public class AzureStorageTest +{ + + AzureStorage azureStorage; + CloudBlobClient cloudBlobClient = Mockito.mock(CloudBlobClient.class); + CloudBlobContainer cloudBlobContainer = Mockito.mock(CloudBlobContainer.class); + + @Before + public void setup() throws URISyntaxException, StorageException + { + Mockito.doReturn(cloudBlobContainer).when(cloudBlobClient).getContainerReference(ArgumentMatchers.anyString()); + azureStorage = new AzureStorage(() -> cloudBlobClient); + } + + @Test + public void testListDir() throws URISyntaxException, StorageException + { + List listBlobItems = ImmutableList.of( + new CloudBlockBlob(new URI("azure://dummy.com/container/blobName")) + ); + + Mockito.doReturn(listBlobItems).when(cloudBlobContainer).listBlobs( + ArgumentMatchers.anyString(), + ArgumentMatchers.anyBoolean(), + ArgumentMatchers.any(), + ArgumentMatchers.any(), + ArgumentMatchers.any() + + ); + Assert.assertEquals(ImmutableList.of("blobName"), azureStorage.listDir("test", "")); + + } +} + diff --git a/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/output/AzureInputRangeTest.java b/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/output/AzureInputRangeTest.java new file mode 100644 index 000000000000..4753132d1c48 --- /dev/null +++ b/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/output/AzureInputRangeTest.java @@ -0,0 +1,34 @@ +/* + * 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.azure.output; + +import nl.jqno.equalsverifier.EqualsVerifier; +import org.junit.Test; + +public class AzureInputRangeTest +{ + @Test + public void testEquals() + { + EqualsVerifier.forClass(AzureInputRange.class) + .usingGetClass() + .verify(); + } +} diff --git a/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/output/AzureStorageConnectorTest.java b/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/output/AzureStorageConnectorTest.java index eab0b92472c0..c8358575b0ce 100644 --- a/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/output/AzureStorageConnectorTest.java +++ b/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/output/AzureStorageConnectorTest.java @@ -69,7 +69,8 @@ public void testPathExistsSuccess() throws URISyntaxException, StorageException, final Capture bucket = Capture.newInstance(); final Capture path = Capture.newInstance(); EasyMock.reset(azureStorage); - EasyMock.expect(azureStorage.getBlobExists(EasyMock.capture(bucket), EasyMock.capture(path))).andReturn(true); + EasyMock.expect(azureStorage.getBlobExists(EasyMock.capture(bucket), EasyMock.capture(path), EasyMock.anyInt())) + .andReturn(true); EasyMock.replay(azureStorage); Assert.assertTrue(storageConnector.pathExists(TEST_FILE)); Assert.assertEquals(CONTAINER, bucket.getValue()); @@ -83,7 +84,8 @@ public void testPathExistsNotFound() throws URISyntaxException, StorageException final Capture bucket = Capture.newInstance(); final Capture path = Capture.newInstance(); EasyMock.reset(azureStorage); - EasyMock.expect(azureStorage.getBlobExists(EasyMock.capture(bucket), EasyMock.capture(path))).andReturn(false); + EasyMock.expect(azureStorage.getBlobExists(EasyMock.capture(bucket), EasyMock.capture(path), EasyMock.anyInt())) + .andReturn(false); EasyMock.replay(azureStorage); Assert.assertFalse(storageConnector.pathExists(TEST_FILE)); Assert.assertEquals(CONTAINER, bucket.getValue()); @@ -104,7 +106,8 @@ public void testRead() throws URISyntaxException, StorageException, IOException EasyMock.anyLong(), EasyMock.anyLong(), EasyMock.anyString(), - EasyMock.anyString() + EasyMock.anyString(), + EasyMock.anyInt() ) ).andReturn(IOUtils.toInputStream(data, StandardCharsets.UTF_8)); @@ -131,7 +134,8 @@ public void testReadRange() throws URISyntaxException, StorageException, IOExcep EasyMock.anyLong(), EasyMock.anyLong(), EasyMock.anyString(), - EasyMock.anyString() + EasyMock.anyString(), + EasyMock.anyInt() )) .andReturn(IOUtils.toInputStream(dataQueried, StandardCharsets.UTF_8)); EasyMock.replay(azureStorage); @@ -152,7 +156,11 @@ public void testDeleteSinglePath() throws URISyntaxException, StorageException, EasyMock.reset(azureStorage); Capture containerCapture = EasyMock.newCapture(); Capture> pathsCapture = EasyMock.newCapture(); - azureStorage.batchDeleteFiles(EasyMock.capture(containerCapture), EasyMock.capture(pathsCapture)); + azureStorage.batchDeleteFiles( + EasyMock.capture(containerCapture), + EasyMock.capture(pathsCapture), + EasyMock.anyInt() + ); EasyMock.replay(azureStorage); storageConnector.deleteFile(TEST_FILE); Assert.assertEquals(CONTAINER, containerCapture.getValue()); @@ -166,7 +174,7 @@ public void testDeleteMultiplePaths() throws URISyntaxException, StorageExceptio EasyMock.reset(azureStorage); Capture containerCapture = EasyMock.newCapture(); Capture> pathsCapture = EasyMock.newCapture(); - azureStorage.batchDeleteFiles(EasyMock.capture(containerCapture), EasyMock.capture(pathsCapture)); + azureStorage.batchDeleteFiles(EasyMock.capture(containerCapture), EasyMock.capture(pathsCapture), EasyMock.anyInt()); EasyMock.replay(azureStorage); storageConnector.deleteFiles(ImmutableList.of(TEST_FILE + "_1.part", TEST_FILE + "_2.part")); Assert.assertEquals(CONTAINER, containerCapture.getValue()); @@ -184,7 +192,7 @@ public void testDeleteMultiplePaths() throws URISyntaxException, StorageExceptio public void testListDir() throws URISyntaxException, StorageException, IOException { EasyMock.reset(azureStorage); - EasyMock.expect(azureStorage.listDir(EasyMock.anyString(), EasyMock.anyString())) + EasyMock.expect(azureStorage.listDir(EasyMock.anyString(), EasyMock.anyString(), EasyMock.anyInt())) .andReturn(ImmutableList.of(PREFIX + "/x/y/z/" + TEST_FILE, PREFIX + "/p/q/r/" + TEST_FILE)); EasyMock.replay(azureStorage); List ret = Lists.newArrayList(storageConnector.listDir("")); From 08cecb0fa5c0732bb569ff04d03ac6963f2aaa08 Mon Sep 17 00:00:00 2001 From: Laksh Singla Date: Tue, 8 Aug 2023 10:43:53 +0530 Subject: [PATCH 23/28] refactor, add comments --- .../data/input/azure/AzureInputSource.java | 2 +- .../druid/storage/azure/AzureByteSource.java | 2 +- .../storage/azure/AzureDataSegmentPusher.java | 2 +- .../druid/storage/azure/AzureStorage.java | 41 ++++++------- .../druid/storage/azure/AzureTaskLogs.java | 8 +-- .../storage/azure/output/AzureInputRange.java | 33 +++++++--- .../azure/output/AzureOutputConfig.java | 3 + .../azure/output/AzureStorageConnector.java | 14 +++-- .../storage/azure/AzureByteSourceTest.java | 6 +- .../azure/AzureDataSegmentPullerTest.java | 8 +-- .../azure/AzureDataSegmentPusherTest.java | 12 ++-- .../storage/azure/AzureTaskLogsTest.java | 60 +++++++++---------- .../output/AzureStorageConnectorTest.java | 10 ++-- .../remote/ChunkingStorageConnector.java | 13 +++- 14 files changed, 124 insertions(+), 90 deletions(-) diff --git a/extensions-core/azure-extensions/src/main/java/org/apache/druid/data/input/azure/AzureInputSource.java b/extensions-core/azure-extensions/src/main/java/org/apache/druid/data/input/azure/AzureInputSource.java index f68ddfa9011a..6d0e60fe873b 100644 --- a/extensions-core/azure-extensions/src/main/java/org/apache/druid/data/input/azure/AzureInputSource.java +++ b/extensions-core/azure-extensions/src/main/java/org/apache/druid/data/input/azure/AzureInputSource.java @@ -139,7 +139,7 @@ public Iterator getDescriptorIteratorForPrefixes(List pre public long getObjectSize(CloudObjectLocation location) { try { - final CloudBlob blobWithAttributes = storage.getBlobReferenceWithAttributes( + final CloudBlob blobWithAttributes = storage.getBlockBlobReferenceWithAttributes( location.getBucket(), location.getPath() ); diff --git a/extensions-core/azure-extensions/src/main/java/org/apache/druid/storage/azure/AzureByteSource.java b/extensions-core/azure-extensions/src/main/java/org/apache/druid/storage/azure/AzureByteSource.java index a8461de0e884..91af1140cb5f 100644 --- a/extensions-core/azure-extensions/src/main/java/org/apache/druid/storage/azure/AzureByteSource.java +++ b/extensions-core/azure-extensions/src/main/java/org/apache/druid/storage/azure/AzureByteSource.java @@ -60,7 +60,7 @@ public InputStream openStream() throws IOException public InputStream openStream(long offset) throws IOException { try { - return azureStorage.getBlobInputStream(offset, containerName, blobPath); + return azureStorage.getBlockBlobInputStream(offset, containerName, blobPath); } catch (StorageException | URISyntaxException e) { if (AzureUtils.AZURE_RETRY.apply(e)) { diff --git a/extensions-core/azure-extensions/src/main/java/org/apache/druid/storage/azure/AzureDataSegmentPusher.java b/extensions-core/azure-extensions/src/main/java/org/apache/druid/storage/azure/AzureDataSegmentPusher.java index 1e46239fd21a..9f97256b1da8 100644 --- a/extensions-core/azure-extensions/src/main/java/org/apache/druid/storage/azure/AzureDataSegmentPusher.java +++ b/extensions-core/azure-extensions/src/main/java/org/apache/druid/storage/azure/AzureDataSegmentPusher.java @@ -183,7 +183,7 @@ DataSegment uploadDataSegment( ) throws StorageException, IOException, URISyntaxException { - azureStorage.uploadBlob(compressedSegmentData, segmentConfig.getContainer(), azurePath); + azureStorage.uploadBlockBlob(compressedSegmentData, segmentConfig.getContainer(), azurePath); final DataSegment outSegment = segment .withSize(size) diff --git a/extensions-core/azure-extensions/src/main/java/org/apache/druid/storage/azure/AzureStorage.java b/extensions-core/azure-extensions/src/main/java/org/apache/druid/storage/azure/AzureStorage.java index 0ca0e3cba70e..fc1a128e11e4 100644 --- a/extensions-core/azure-extensions/src/main/java/org/apache/druid/storage/azure/AzureStorage.java +++ b/extensions-core/azure-extensions/src/main/java/org/apache/druid/storage/azure/AzureStorage.java @@ -77,7 +77,7 @@ public AzureStorage( { this.cloudBlobClient = cloudBlobClient; } - + public List emptyCloudBlobDirectory(final String containerName, final String virtualDirPath) throws StorageException, URISyntaxException { @@ -113,7 +113,7 @@ public List emptyCloudBlobDirectory(final String containerName, final St return deletedFiles; } - public void uploadBlob(final File file, final String containerName, final String blobPath) + public void uploadBlockBlob(final File file, final String containerName, final String blobPath) throws IOException, StorageException, URISyntaxException { CloudBlobContainer container = getOrCreateCloudBlobContainer(containerName); @@ -144,7 +144,7 @@ public OutputStream getBlockBlobOutputStream( } - public CloudBlob getBlobReferenceWithAttributes(final String containerName, final String blobPath) + public CloudBlob getBlockBlobReferenceWithAttributes(final String containerName, final String blobPath) throws URISyntaxException, StorageException { final CloudBlockBlob blobReference = getOrCreateCloudBlobContainer(containerName).getBlockBlobReference(blobPath); @@ -152,31 +152,31 @@ public CloudBlob getBlobReferenceWithAttributes(final String containerName, fina return blobReference; } - public long getBlobLength(final String containerName, final String blobPath) + public long getBlockBlobLength(final String containerName, final String blobPath) throws URISyntaxException, StorageException { - return getBlobReferenceWithAttributes(containerName, blobPath).getProperties().getLength(); + return getBlockBlobReferenceWithAttributes(containerName, blobPath).getProperties().getLength(); } - public InputStream getBlobInputStream(final String containerName, final String blobPath) + public InputStream getBlockBlobInputStream(final String containerName, final String blobPath) throws URISyntaxException, StorageException { - return getBlobInputStream(0L, containerName, blobPath); + return getBlockBlobInputStream(0L, containerName, blobPath); } - public InputStream getBlobInputStream(long offset, final String containerName, final String blobPath) + public InputStream getBlockBlobInputStream(long offset, final String containerName, final String blobPath) throws URISyntaxException, StorageException { - return getBlobInputStream(offset, null, containerName, blobPath); + return getBlockBlobInputStream(offset, null, containerName, blobPath); } - public InputStream getBlobInputStream(long offset, Long length, final String containerName, final String blobPath) + public InputStream getBlockBlobInputStream(long offset, Long length, final String containerName, final String blobPath) throws URISyntaxException, StorageException { - return getBlobInputStream(offset, length, containerName, blobPath, null); + return getBlockBlobInputStream(offset, length, containerName, blobPath, null); } - public InputStream getBlobInputStream(long offset, Long length, final String containerName, final String blobPath, Integer maxAttempts) + public InputStream getBlockBlobInputStream(long offset, Long length, final String containerName, final String blobPath, Integer maxAttempts) throws URISyntaxException, StorageException { CloudBlobContainer container = getOrCreateCloudBlobContainer(containerName); @@ -184,13 +184,6 @@ public InputStream getBlobInputStream(long offset, Long length, final String con .openInputStream(offset, length, null, getRequestOptionsWithRetry(maxAttempts), null); } - - public void batchDeleteFiles(String containerName, Iterable paths) - throws URISyntaxException, StorageException - { - batchDeleteFiles(containerName, paths, null); - } - public void batchDeleteFiles(String containerName, Iterable paths, Integer maxAttempts) throws URISyntaxException, StorageException { @@ -224,19 +217,23 @@ public List listDir(final String containerName, final String virtualDirP return files; } - public boolean getBlobExists(String container, String blobPath) throws URISyntaxException, StorageException + public boolean getBlockBlobExists(String container, String blobPath) throws URISyntaxException, StorageException { - return getBlobExists(container, blobPath, null); + return getBlockBlobExists(container, blobPath, null); } - public boolean getBlobExists(String container, String blobPath, Integer maxAttempts) + public boolean getBlockBlobExists(String container, String blobPath, Integer maxAttempts) throws URISyntaxException, StorageException { return getOrCreateCloudBlobContainer(container).getBlockBlobReference(blobPath) .exists(null, getRequestOptionsWithRetry(maxAttempts), null); } + /** + * If maxAttempts is provided, this method returns request options with retry built in. + * Retry backoff is exponential backoff, with maxAttempts set to the one provided + */ @Nullable private BlobRequestOptions getRequestOptionsWithRetry(Integer maxAttempts) { diff --git a/extensions-core/azure-extensions/src/main/java/org/apache/druid/storage/azure/AzureTaskLogs.java b/extensions-core/azure-extensions/src/main/java/org/apache/druid/storage/azure/AzureTaskLogs.java index 9bfda5ab349f..5e6880c14ed2 100644 --- a/extensions-core/azure-extensions/src/main/java/org/apache/druid/storage/azure/AzureTaskLogs.java +++ b/extensions-core/azure-extensions/src/main/java/org/apache/druid/storage/azure/AzureTaskLogs.java @@ -95,7 +95,7 @@ private void pushTaskFile(final File logFile, String taskKey) try { AzureUtils.retryAzureOperation( () -> { - azureStorage.uploadBlob(logFile, config.getContainer(), taskKey); + azureStorage.uploadBlockBlob(logFile, config.getContainer(), taskKey); return null; }, config.getMaxTries() @@ -129,12 +129,12 @@ private Optional streamTaskFile(final String taskid, final long off { final String container = config.getContainer(); try { - if (!azureStorage.getBlobExists(container, taskKey)) { + if (!azureStorage.getBlockBlobExists(container, taskKey)) { return Optional.absent(); } try { final long start; - final long length = azureStorage.getBlobLength(container, taskKey); + final long length = azureStorage.getBlockBlobLength(container, taskKey); if (offset > 0 && offset < length) { start = offset; @@ -144,7 +144,7 @@ private Optional streamTaskFile(final String taskid, final long off start = 0; } - InputStream stream = azureStorage.getBlobInputStream(container, taskKey); + InputStream stream = azureStorage.getBlockBlobInputStream(container, taskKey); stream.skip(start); return Optional.of(stream); diff --git a/extensions-core/azure-extensions/src/main/java/org/apache/druid/storage/azure/output/AzureInputRange.java b/extensions-core/azure-extensions/src/main/java/org/apache/druid/storage/azure/output/AzureInputRange.java index 07a86f871d58..4803dc8b297b 100644 --- a/extensions-core/azure-extensions/src/main/java/org/apache/druid/storage/azure/output/AzureInputRange.java +++ b/extensions-core/azure-extensions/src/main/java/org/apache/druid/storage/azure/output/AzureInputRange.java @@ -21,18 +21,37 @@ import java.util.Objects; +/** + * Represents a chunk of the Azure blob + */ public class AzureInputRange { + + /** + * Starting location in the blob stream + */ private final long start; + + /** + * Size of the blob stream that this object represents + */ private final long size; - private final String bucket; + + /** + * Container where the blob resides + */ + private final String container; + + /** + * Absolute path of the blob + */ private final String path; - public AzureInputRange(long start, long size, String bucket, String path) + public AzureInputRange(long start, long size, String container, String path) { this.start = start; this.size = size; - this.bucket = bucket; + this.container = container; this.path = path; } @@ -46,9 +65,9 @@ public long getSize() return size; } - public String getBucket() + public String getContainer() { - return bucket; + return container; } public String getPath() @@ -68,13 +87,13 @@ public boolean equals(Object o) AzureInputRange that = (AzureInputRange) o; return start == that.start && size == that.size - && Objects.equals(bucket, that.bucket) + && Objects.equals(container, that.container) && Objects.equals(path, that.path); } @Override public int hashCode() { - return Objects.hash(start, size, bucket, path); + return Objects.hash(start, size, container, path); } } diff --git a/extensions-core/azure-extensions/src/main/java/org/apache/druid/storage/azure/output/AzureOutputConfig.java b/extensions-core/azure-extensions/src/main/java/org/apache/druid/storage/azure/output/AzureOutputConfig.java index 2daea595589d..6d5cc06be872 100644 --- a/extensions-core/azure-extensions/src/main/java/org/apache/druid/storage/azure/output/AzureOutputConfig.java +++ b/extensions-core/azure-extensions/src/main/java/org/apache/druid/storage/azure/output/AzureOutputConfig.java @@ -28,6 +28,9 @@ import java.io.File; import java.util.Objects; +/** + * Configuration of the Azure storage connector + */ public class AzureOutputConfig { @JsonProperty diff --git a/extensions-core/azure-extensions/src/main/java/org/apache/druid/storage/azure/output/AzureStorageConnector.java b/extensions-core/azure-extensions/src/main/java/org/apache/druid/storage/azure/output/AzureStorageConnector.java index 4b088a661ffc..657043797e03 100644 --- a/extensions-core/azure-extensions/src/main/java/org/apache/druid/storage/azure/output/AzureStorageConnector.java +++ b/extensions-core/azure-extensions/src/main/java/org/apache/druid/storage/azure/output/AzureStorageConnector.java @@ -36,6 +36,10 @@ import java.util.Iterator; import java.util.List; +/** + * Implementation of the storage connector that facilitates reading and writing from Azure's blob storage. + * This extends the {@link ChunkingStorageConnector} so that the downloads are in a chunked manner. + */ public class AzureStorageConnector extends ChunkingStorageConnector { @@ -58,7 +62,7 @@ public AzureStorageConnector( public ChunkingStorageConnectorParameters buildInputParams(String path) throws IOException { try { - return buildInputParams(path, 0, azureStorage.getBlobLength(config.getContainer(), objectPath(path))); + return buildInputParams(path, 0, azureStorage.getBlockBlobLength(config.getContainer(), objectPath(path))); } catch (URISyntaxException | StorageException e) { throw new IOException(e); @@ -88,10 +92,10 @@ public ChunkingStorageConnectorParameters buildInputParams(Stri public InputStream open(AzureInputRange inputRange) throws IOException { try { - return azureStorage.getBlobInputStream( + return azureStorage.getBlockBlobInputStream( inputRange.getStart(), inputRange.getSize(), - inputRange.getBucket(), + inputRange.getContainer(), inputRange.getPath(), config.getMaxRetry() ); @@ -107,7 +111,7 @@ public InputStream open(AzureInputRange inputRange, long offset) throws IOExcept AzureInputRange newInputRange = new AzureInputRange( inputRange.getStart() + offset, Math.max(inputRange.getSize() - offset, 0), - inputRange.getBucket(), + inputRange.getContainer(), inputRange.getPath() ); return open(newInputRange); @@ -122,7 +126,7 @@ public InputStream open(AzureInputRange inputRange, long offset) throws IOExcept public boolean pathExists(String path) throws IOException { try { - return azureStorage.getBlobExists(config.getContainer(), objectPath(path), config.getMaxRetry()); + return azureStorage.getBlockBlobExists(config.getContainer(), objectPath(path), config.getMaxRetry()); } catch (URISyntaxException | StorageException e) { throw new IOException(e); diff --git a/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/AzureByteSourceTest.java b/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/AzureByteSourceTest.java index 649001cc3577..f54ef2e40361 100644 --- a/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/AzureByteSourceTest.java +++ b/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/AzureByteSourceTest.java @@ -41,7 +41,7 @@ public void test_openStream_withoutOffset_succeeds() throws IOException, URISynt AzureStorage azureStorage = createMock(AzureStorage.class); InputStream stream = createMock(InputStream.class); - EasyMock.expect(azureStorage.getBlobInputStream(NO_OFFSET, containerName, blobPath)).andReturn(stream); + EasyMock.expect(azureStorage.getBlockBlobInputStream(NO_OFFSET, containerName, blobPath)).andReturn(stream); replayAll(); @@ -60,7 +60,7 @@ public void test_openStream_withOffset_succeeds() throws IOException, URISyntaxE AzureStorage azureStorage = createMock(AzureStorage.class); InputStream stream = createMock(InputStream.class); - EasyMock.expect(azureStorage.getBlobInputStream(OFFSET, containerName, blobPath)).andReturn(stream); + EasyMock.expect(azureStorage.getBlockBlobInputStream(OFFSET, containerName, blobPath)).andReturn(stream); replayAll(); @@ -78,7 +78,7 @@ public void openStreamWithRecoverableErrorTest() throws URISyntaxException, Stor final String blobPath = "/path/to/file"; AzureStorage azureStorage = createMock(AzureStorage.class); - EasyMock.expect(azureStorage.getBlobInputStream(NO_OFFSET, containerName, blobPath)).andThrow( + EasyMock.expect(azureStorage.getBlockBlobInputStream(NO_OFFSET, containerName, blobPath)).andThrow( new StorageException( "", "", diff --git a/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/AzureDataSegmentPullerTest.java b/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/AzureDataSegmentPullerTest.java index fc984349628a..13820072cb7b 100644 --- a/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/AzureDataSegmentPullerTest.java +++ b/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/AzureDataSegmentPullerTest.java @@ -62,7 +62,7 @@ public void test_getSegmentFiles_success() final InputStream zipStream = new FileInputStream(pulledFile); EasyMock.expect(byteSourceFactory.create(CONTAINER_NAME, BLOB_PATH)).andReturn(new AzureByteSource(azureStorage, CONTAINER_NAME, BLOB_PATH)); - EasyMock.expect(azureStorage.getBlobInputStream(0L, CONTAINER_NAME, BLOB_PATH)).andReturn(zipStream); + EasyMock.expect(azureStorage.getBlockBlobInputStream(0L, CONTAINER_NAME, BLOB_PATH)).andReturn(zipStream); replayAll(); @@ -94,7 +94,7 @@ public void test_getSegmentFiles_blobPathIsHadoop_success() final InputStream zipStream = new FileInputStream(pulledFile); EasyMock.expect(byteSourceFactory.create(CONTAINER_NAME, BLOB_PATH)).andReturn(new AzureByteSource(azureStorage, CONTAINER_NAME, BLOB_PATH)); - EasyMock.expect(azureStorage.getBlobInputStream(0L, CONTAINER_NAME, BLOB_PATH)).andReturn(zipStream); + EasyMock.expect(azureStorage.getBlockBlobInputStream(0L, CONTAINER_NAME, BLOB_PATH)).andReturn(zipStream); replayAll(); @@ -123,7 +123,7 @@ public void test_getSegmentFiles_nonRecoverableErrorRaisedWhenPullingSegmentFile final File outDir = FileUtils.createTempDir(); try { EasyMock.expect(byteSourceFactory.create(CONTAINER_NAME, BLOB_PATH)).andReturn(new AzureByteSource(azureStorage, CONTAINER_NAME, BLOB_PATH)); - EasyMock.expect(azureStorage.getBlobInputStream(0L, CONTAINER_NAME, BLOB_PATH)).andThrow( + EasyMock.expect(azureStorage.getBlockBlobInputStream(0L, CONTAINER_NAME, BLOB_PATH)).andThrow( new URISyntaxException( "error", "error", @@ -155,7 +155,7 @@ public void test_getSegmentFiles_recoverableErrorRaisedWhenPullingSegmentFiles_d final File outDir = FileUtils.createTempDir(); try { EasyMock.expect(byteSourceFactory.create(CONTAINER_NAME, BLOB_PATH)).andReturn(new AzureByteSource(azureStorage, CONTAINER_NAME, BLOB_PATH)); - EasyMock.expect(azureStorage.getBlobInputStream(0L, CONTAINER_NAME, BLOB_PATH)).andThrow( + EasyMock.expect(azureStorage.getBlockBlobInputStream(0L, CONTAINER_NAME, BLOB_PATH)).andThrow( new StorageException(null, null, 0, null, null) ).atLeastOnce(); diff --git a/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/AzureDataSegmentPusherTest.java b/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/AzureDataSegmentPusherTest.java index f3e65c7923f5..b18fabbc3dae 100644 --- a/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/AzureDataSegmentPusherTest.java +++ b/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/AzureDataSegmentPusherTest.java @@ -115,7 +115,7 @@ public void test_push_nonUniquePathNoPrefix_succeeds() throws Exception Files.write(DATA, tmp); String azurePath = pusher.getAzurePath(SEGMENT_TO_PUSH, useUniquePath); - azureStorage.uploadBlob(EasyMock.anyObject(File.class), EasyMock.eq(CONTAINER_NAME), EasyMock.eq(azurePath)); + azureStorage.uploadBlockBlob(EasyMock.anyObject(File.class), EasyMock.eq(CONTAINER_NAME), EasyMock.eq(azurePath)); EasyMock.expectLastCall(); replayAll(); @@ -145,7 +145,7 @@ public void test_push_nonUniquePathWithPrefix_succeeds() throws Exception Files.write(DATA, tmp); String azurePath = pusher.getAzurePath(SEGMENT_TO_PUSH, useUniquePath); - azureStorage.uploadBlob( + azureStorage.uploadBlockBlob( EasyMock.anyObject(File.class), EasyMock.eq(CONTAINER_NAME), EasyMock.eq(PREFIX + "/" + azurePath) @@ -178,7 +178,7 @@ public void test_push_uniquePathNoPrefix_succeeds() throws Exception Files.write(DATA, tmp); String azurePath = pusher.getAzurePath(SEGMENT_TO_PUSH, useUniquePath); - azureStorage.uploadBlob( + azureStorage.uploadBlockBlob( EasyMock.anyObject(File.class), EasyMock.eq(CONTAINER_NAME), EasyMock.matches(UNIQUE_MATCHER_NO_PREFIX) @@ -211,7 +211,7 @@ public void test_push_uniquePath_succeeds() throws Exception Files.write(DATA, tmp); String azurePath = pusher.getAzurePath(SEGMENT_TO_PUSH, useUniquePath); - azureStorage.uploadBlob( + azureStorage.uploadBlockBlob( EasyMock.anyObject(File.class), EasyMock.eq(CONTAINER_NAME), EasyMock.matches(UNIQUE_MATCHER_PREFIX) @@ -245,7 +245,7 @@ public void test_push_exception_throwsException() throws Exception final long size = DATA.length; String azurePath = pusher.getAzurePath(SEGMENT_TO_PUSH, useUniquePath); - azureStorage.uploadBlob(EasyMock.anyObject(File.class), EasyMock.eq(CONTAINER_NAME), EasyMock.eq(azurePath)); + azureStorage.uploadBlockBlob(EasyMock.anyObject(File.class), EasyMock.eq(CONTAINER_NAME), EasyMock.eq(azurePath)); EasyMock.expectLastCall().andThrow(new URISyntaxException("", "")); replayAll(); @@ -284,7 +284,7 @@ public void uploadDataSegmentTest() throws StorageException, IOException, URISyn final File compressedSegmentData = new File("index.zip"); final String azurePath = pusher.getAzurePath(DATA_SEGMENT, false); - azureStorage.uploadBlob(compressedSegmentData, CONTAINER_NAME, azurePath); + azureStorage.uploadBlockBlob(compressedSegmentData, CONTAINER_NAME, azurePath); EasyMock.expectLastCall(); replayAll(); diff --git a/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/AzureTaskLogsTest.java b/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/AzureTaskLogsTest.java index 297545e4cad0..2575793176e3 100644 --- a/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/AzureTaskLogsTest.java +++ b/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/AzureTaskLogsTest.java @@ -97,7 +97,7 @@ public void test_PushTaskLog_uploadsBlob() throws Exception try { final File logFile = new File(tmpDir, "log"); - azureStorage.uploadBlob(logFile, CONTAINER, PREFIX + "/" + TASK_ID + "/log"); + azureStorage.uploadBlockBlob(logFile, CONTAINER, PREFIX + "/" + TASK_ID + "/log"); EasyMock.expectLastCall(); replayAll(); @@ -119,7 +119,7 @@ public void test_PushTaskLog_exception_rethrowsException() throws Exception try { final File logFile = new File(tmpDir, "log"); - azureStorage.uploadBlob(logFile, CONTAINER, PREFIX + "/" + TASK_ID + "/log"); + azureStorage.uploadBlockBlob(logFile, CONTAINER, PREFIX + "/" + TASK_ID + "/log"); EasyMock.expectLastCall().andThrow(new IOException()); replayAll(); @@ -141,7 +141,7 @@ public void test_PushTaskReports_uploadsBlob() throws Exception try { final File logFile = new File(tmpDir, "log"); - azureStorage.uploadBlob(logFile, CONTAINER, PREFIX + "/" + TASK_ID + "/report.json"); + azureStorage.uploadBlockBlob(logFile, CONTAINER, PREFIX + "/" + TASK_ID + "/report.json"); EasyMock.expectLastCall(); replayAll(); @@ -163,7 +163,7 @@ public void test_PushTaskStatus_uploadsBlob() throws Exception try { final File logFile = new File(tmpDir, "status.json"); - azureStorage.uploadBlob(logFile, CONTAINER, PREFIX + "/" + TASK_ID + "/status.json"); + azureStorage.uploadBlockBlob(logFile, CONTAINER, PREFIX + "/" + TASK_ID + "/status.json"); EasyMock.expectLastCall(); replayAll(); @@ -185,7 +185,7 @@ public void test_PushTaskReports_exception_rethrowsException() throws Exception try { final File logFile = new File(tmpDir, "log"); - azureStorage.uploadBlob(logFile, CONTAINER, PREFIX + "/" + TASK_ID + "/report.json"); + azureStorage.uploadBlockBlob(logFile, CONTAINER, PREFIX + "/" + TASK_ID + "/report.json"); EasyMock.expectLastCall().andThrow(new IOException()); replayAll(); @@ -205,9 +205,9 @@ public void testStreamTaskLogWithoutOffset() throws Exception final String testLog = "hello this is a log"; final String blobPath = PREFIX + "/" + TASK_ID + "/log"; - EasyMock.expect(azureStorage.getBlobExists(CONTAINER, blobPath)).andReturn(true); - EasyMock.expect(azureStorage.getBlobLength(CONTAINER, blobPath)).andReturn((long) testLog.length()); - EasyMock.expect(azureStorage.getBlobInputStream(CONTAINER, blobPath)).andReturn( + EasyMock.expect(azureStorage.getBlockBlobExists(CONTAINER, blobPath)).andReturn(true); + EasyMock.expect(azureStorage.getBlockBlobLength(CONTAINER, blobPath)).andReturn((long) testLog.length()); + EasyMock.expect(azureStorage.getBlockBlobInputStream(CONTAINER, blobPath)).andReturn( new ByteArrayInputStream(testLog.getBytes(StandardCharsets.UTF_8))); @@ -228,9 +228,9 @@ public void testStreamTaskLogWithPositiveOffset() throws Exception final String testLog = "hello this is a log"; final String blobPath = PREFIX + "/" + TASK_ID + "/log"; - EasyMock.expect(azureStorage.getBlobExists(CONTAINER, blobPath)).andReturn(true); - EasyMock.expect(azureStorage.getBlobLength(CONTAINER, blobPath)).andReturn((long) testLog.length()); - EasyMock.expect(azureStorage.getBlobInputStream(CONTAINER, blobPath)).andReturn( + EasyMock.expect(azureStorage.getBlockBlobExists(CONTAINER, blobPath)).andReturn(true); + EasyMock.expect(azureStorage.getBlockBlobLength(CONTAINER, blobPath)).andReturn((long) testLog.length()); + EasyMock.expect(azureStorage.getBlockBlobInputStream(CONTAINER, blobPath)).andReturn( new ByteArrayInputStream(testLog.getBytes(StandardCharsets.UTF_8))); @@ -251,9 +251,9 @@ public void testStreamTaskLogWithNegative() throws Exception final String testLog = "hello this is a log"; final String blobPath = PREFIX + "/" + TASK_ID + "/log"; - EasyMock.expect(azureStorage.getBlobExists(CONTAINER, blobPath)).andReturn(true); - EasyMock.expect(azureStorage.getBlobLength(CONTAINER, blobPath)).andReturn((long) testLog.length()); - EasyMock.expect(azureStorage.getBlobInputStream(CONTAINER, blobPath)).andReturn( + EasyMock.expect(azureStorage.getBlockBlobExists(CONTAINER, blobPath)).andReturn(true); + EasyMock.expect(azureStorage.getBlockBlobLength(CONTAINER, blobPath)).andReturn((long) testLog.length()); + EasyMock.expect(azureStorage.getBlockBlobInputStream(CONTAINER, blobPath)).andReturn( new ByteArrayInputStream(StringUtils.toUtf8(testLog))); @@ -274,9 +274,9 @@ public void test_streamTaskReports_blobExists_succeeds() throws Exception final String testLog = "hello this is a log"; final String blobPath = PREFIX + "/" + TASK_ID + "/report.json"; - EasyMock.expect(azureStorage.getBlobExists(CONTAINER, blobPath)).andReturn(true); - EasyMock.expect(azureStorage.getBlobLength(CONTAINER, blobPath)).andReturn((long) testLog.length()); - EasyMock.expect(azureStorage.getBlobInputStream(CONTAINER, blobPath)).andReturn( + EasyMock.expect(azureStorage.getBlockBlobExists(CONTAINER, blobPath)).andReturn(true); + EasyMock.expect(azureStorage.getBlockBlobLength(CONTAINER, blobPath)).andReturn((long) testLog.length()); + EasyMock.expect(azureStorage.getBlockBlobInputStream(CONTAINER, blobPath)).andReturn( new ByteArrayInputStream(testLog.getBytes(StandardCharsets.UTF_8))); @@ -297,7 +297,7 @@ public void test_streamTaskReports_blobDoesNotExist_returnsAbsent() throws Excep final String testLog = "hello this is a log"; final String blobPath = PREFIX + "/" + TASK_ID_NOT_FOUND + "/report.json"; - EasyMock.expect(azureStorage.getBlobExists(CONTAINER, blobPath)).andReturn(false); + EasyMock.expect(azureStorage.getBlockBlobExists(CONTAINER, blobPath)).andReturn(false); replayAll(); @@ -315,9 +315,9 @@ public void test_streamTaskReports_exceptionWhenGettingStream_throwsException() final String testLog = "hello this is a log"; final String blobPath = PREFIX + "/" + TASK_ID + "/report.json"; - EasyMock.expect(azureStorage.getBlobExists(CONTAINER, blobPath)).andReturn(true); - EasyMock.expect(azureStorage.getBlobLength(CONTAINER, blobPath)).andReturn((long) testLog.length()); - EasyMock.expect(azureStorage.getBlobInputStream(CONTAINER, blobPath)).andThrow( + EasyMock.expect(azureStorage.getBlockBlobExists(CONTAINER, blobPath)).andReturn(true); + EasyMock.expect(azureStorage.getBlockBlobLength(CONTAINER, blobPath)).andReturn((long) testLog.length()); + EasyMock.expect(azureStorage.getBlockBlobInputStream(CONTAINER, blobPath)).andThrow( new URISyntaxException("", "")); @@ -336,7 +336,7 @@ public void test_streamTaskReports_exceptionWhenCheckingBlobExistence_throwsExce final String testLog = "hello this is a log"; final String blobPath = PREFIX + "/" + TASK_ID + "/report.json"; - EasyMock.expect(azureStorage.getBlobExists(CONTAINER, blobPath)).andThrow(new URISyntaxException("", "")); + EasyMock.expect(azureStorage.getBlockBlobExists(CONTAINER, blobPath)).andThrow(new URISyntaxException("", "")); replayAll(); @@ -351,9 +351,9 @@ public void test_streamTaskStatus_blobExists_succeeds() throws Exception final String taskStatus = "{}"; final String blobPath = PREFIX + "/" + TASK_ID + "/status.json"; - EasyMock.expect(azureStorage.getBlobExists(CONTAINER, blobPath)).andReturn(true); - EasyMock.expect(azureStorage.getBlobLength(CONTAINER, blobPath)).andReturn((long) taskStatus.length()); - EasyMock.expect(azureStorage.getBlobInputStream(CONTAINER, blobPath)).andReturn( + EasyMock.expect(azureStorage.getBlockBlobExists(CONTAINER, blobPath)).andReturn(true); + EasyMock.expect(azureStorage.getBlockBlobLength(CONTAINER, blobPath)).andReturn((long) taskStatus.length()); + EasyMock.expect(azureStorage.getBlockBlobInputStream(CONTAINER, blobPath)).andReturn( new ByteArrayInputStream(taskStatus.getBytes(StandardCharsets.UTF_8))); @@ -372,7 +372,7 @@ public void test_streamTaskStatus_blobExists_succeeds() throws Exception public void test_streamTaskStatus_blobDoesNotExist_returnsAbsent() throws Exception { final String blobPath = PREFIX + "/" + TASK_ID_NOT_FOUND + "/status.json"; - EasyMock.expect(azureStorage.getBlobExists(CONTAINER, blobPath)).andReturn(false); + EasyMock.expect(azureStorage.getBlockBlobExists(CONTAINER, blobPath)).andReturn(false); replayAll(); @@ -390,9 +390,9 @@ public void test_streamTaskStatus_exceptionWhenGettingStream_throwsException() t final String taskStatus = "{}"; final String blobPath = PREFIX + "/" + TASK_ID + "/status.json"; - EasyMock.expect(azureStorage.getBlobExists(CONTAINER, blobPath)).andReturn(true); - EasyMock.expect(azureStorage.getBlobLength(CONTAINER, blobPath)).andReturn((long) taskStatus.length()); - EasyMock.expect(azureStorage.getBlobInputStream(CONTAINER, blobPath)).andThrow( + EasyMock.expect(azureStorage.getBlockBlobExists(CONTAINER, blobPath)).andReturn(true); + EasyMock.expect(azureStorage.getBlockBlobLength(CONTAINER, blobPath)).andReturn((long) taskStatus.length()); + EasyMock.expect(azureStorage.getBlockBlobInputStream(CONTAINER, blobPath)).andThrow( new URISyntaxException("", "")); @@ -409,7 +409,7 @@ public void test_streamTaskStatus_exceptionWhenGettingStream_throwsException() t public void test_streamTaskStatus_exceptionWhenCheckingBlobExistence_throwsException() throws Exception { final String blobPath = PREFIX + "/" + TASK_ID + "/status.json"; - EasyMock.expect(azureStorage.getBlobExists(CONTAINER, blobPath)).andThrow(new URISyntaxException("", "")); + EasyMock.expect(azureStorage.getBlockBlobExists(CONTAINER, blobPath)).andThrow(new URISyntaxException("", "")); replayAll(); diff --git a/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/output/AzureStorageConnectorTest.java b/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/output/AzureStorageConnectorTest.java index c8358575b0ce..f8592c32eaf8 100644 --- a/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/output/AzureStorageConnectorTest.java +++ b/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/output/AzureStorageConnectorTest.java @@ -69,7 +69,7 @@ public void testPathExistsSuccess() throws URISyntaxException, StorageException, final Capture bucket = Capture.newInstance(); final Capture path = Capture.newInstance(); EasyMock.reset(azureStorage); - EasyMock.expect(azureStorage.getBlobExists(EasyMock.capture(bucket), EasyMock.capture(path), EasyMock.anyInt())) + EasyMock.expect(azureStorage.getBlockBlobExists(EasyMock.capture(bucket), EasyMock.capture(path), EasyMock.anyInt())) .andReturn(true); EasyMock.replay(azureStorage); Assert.assertTrue(storageConnector.pathExists(TEST_FILE)); @@ -84,7 +84,7 @@ public void testPathExistsNotFound() throws URISyntaxException, StorageException final Capture bucket = Capture.newInstance(); final Capture path = Capture.newInstance(); EasyMock.reset(azureStorage); - EasyMock.expect(azureStorage.getBlobExists(EasyMock.capture(bucket), EasyMock.capture(path), EasyMock.anyInt())) + EasyMock.expect(azureStorage.getBlockBlobExists(EasyMock.capture(bucket), EasyMock.capture(path), EasyMock.anyInt())) .andReturn(false); EasyMock.replay(azureStorage); Assert.assertFalse(storageConnector.pathExists(TEST_FILE)); @@ -99,10 +99,10 @@ public void testRead() throws URISyntaxException, StorageException, IOException EasyMock.reset(azureStorage); String data = "test"; - EasyMock.expect(azureStorage.getBlobLength(EasyMock.anyString(), EasyMock.anyString())) + EasyMock.expect(azureStorage.getBlockBlobLength(EasyMock.anyString(), EasyMock.anyString())) .andReturn(4L); EasyMock.expect( - azureStorage.getBlobInputStream( + azureStorage.getBlockBlobInputStream( EasyMock.anyLong(), EasyMock.anyLong(), EasyMock.anyString(), @@ -130,7 +130,7 @@ public void testReadRange() throws URISyntaxException, StorageException, IOExcep for (long length = 1; length <= data.length() - start; ++length) { String dataQueried = data.substring(start, start + ((Long) length).intValue()); EasyMock.reset(azureStorage); - EasyMock.expect(azureStorage.getBlobInputStream( + EasyMock.expect(azureStorage.getBlockBlobInputStream( EasyMock.anyLong(), EasyMock.anyLong(), EasyMock.anyString(), diff --git a/processing/src/main/java/org/apache/druid/storage/remote/ChunkingStorageConnector.java b/processing/src/main/java/org/apache/druid/storage/remote/ChunkingStorageConnector.java index 49505e16996b..5d181b724881 100644 --- a/processing/src/main/java/org/apache/druid/storage/remote/ChunkingStorageConnector.java +++ b/processing/src/main/java/org/apache/druid/storage/remote/ChunkingStorageConnector.java @@ -48,8 +48,19 @@ */ public abstract class ChunkingStorageConnector implements StorageConnector { + /** + * Default size for chunking of the storage connector. Set to 100MBs to keep the chunk size small relative to the + * total frame size, while also preventing a large number of calls to the remote storage. While fetching a single + * file, 100MBs would be required in the disk space. + */ private static final long DOWNLOAD_MAX_CHUNK_SIZE_BYTES = 100_000_000; + /** + * Default fetch buffer size while copying from the remote location to the download file. Set to default sizing given + * in the {@link org.apache.commons.io.IOUtils} + */ + private static final int FETCH_BUFFER_SIZE_BYTES = 8 * 1024; + private final long chunkSizeBytes; public ChunkingStorageConnector() @@ -146,7 +157,7 @@ public InputStream nextElement() params.getMaxRetry() ), outFile, - new byte[8 * 1024], + new byte[FETCH_BUFFER_SIZE_BYTES], Predicates.alwaysFalse(), 1, StringUtils.format( From b458bd603e3635707c5307e763ac729d767041bd Mon Sep 17 00:00:00 2001 From: Laksh Singla Date: Tue, 8 Aug 2023 11:16:43 +0530 Subject: [PATCH 24/28] docs --- docs/multi-stage-query/reference.md | 29 ++++++++++++++++++++++------- 1 file changed, 22 insertions(+), 7 deletions(-) diff --git a/docs/multi-stage-query/reference.md b/docs/multi-stage-query/reference.md index 592aed9a2a98..78575bcf32c8 100644 --- a/docs/multi-stage-query/reference.md +++ b/docs/multi-stage-query/reference.md @@ -349,20 +349,35 @@ SQL-based ingestion supports using durable storage to store intermediate files t ### Durable storage configurations -The following common service properties control how durable storage behaves: +Durable storage is supported on Amazon S3 storage and Microsoft's Azure storage. There are a few common configurations that controls the behaviour for both the services as documented below. Apart from the common configurations, +there are a few properties specific to each storage that must be set. + +Common properties to configure the behaviour of durable storage |Parameter |Default | Description | |-------------------|----------------------------------------|----------------------| -|`druid.msq.intermediate.storage.enable` | true | Required. Whether to enable durable storage for the cluster. For more information about enabling durable storage, see [Durable storage](../operations/durable-storage.md).| -|`druid.msq.intermediate.storage.type` | `s3` for Amazon S3 | Required. The type of storage to use. `s3` is the only supported storage type. | -|`druid.msq.intermediate.storage.bucket` | n/a | The S3 bucket to store intermediate files. | -|`druid.msq.intermediate.storage.prefix` | n/a | S3 prefix to store intermediate stage results. Provide a unique value for the prefix. Don't share the same prefix between clusters. If the location includes other files or directories, then they will get cleaned up as well. | -|`druid.msq.intermediate.storage.tempDir`| n/a | Required. Directory path on the local disk to temporarily store intermediate stage results. | +|`druid.msq.intermediate.storage.enable` | false | Whether to enable durable storage for the cluster. Set it to true to enable durable storage. For more information about enabling durable storage, see [Durable storage](../operations/durable-storage.md).| +|`druid.msq.intermediate.storage.type` | n/a | Required. The type of storage to use. Set it to `s3` for S3 and `azure` for Azure | +|`druid.msq.intermediate.storage.tempDir`| n/a | Required. Directory path on the local disk to store temporary files required while uploading and downloading the data | |`druid.msq.intermediate.storage.maxRetry` | 10 | Optional. Defines the max number times to attempt S3 API calls to avoid failures due to transient errors. | |`druid.msq.intermediate.storage.chunkSize` | 100MiB | Optional. Defines the size of each chunk to temporarily store in `druid.msq.intermediate.storage.tempDir`. The chunk size must be between 5 MiB and 5 GiB. A large chunk size reduces the API calls made to the durable storage, however it requires more disk space to store the temporary chunks. Druid uses a default of 100MiB if the value is not provided.| +Following properties need to be set in addition to the common properties to enable durable storage on S3 + +|Parameter |Default | Description | +|-------------------|----------------------------------------|----------------------| +|`druid.msq.intermediate.storage.bucket` | n/a | Required. The S3 bucket where the files are uploaded to and download from | +|`druid.msq.intermediate.storage.prefix` | n/a | Required. Path prepended to all the paths uploaded to the bucket to namespace the connector's files. Provide a unique value for the prefix and donot share the same prefix between different clusters. If the location includes other files or directories, then they might get cleaned up as well. | + +Following properties must be set in addition to the common properties to enable durable storage on Azure. + +|Parameter |Default | Description | +|-------------------|----------------------------------------|----------------------| +|`druid.msq.intermediate.storage.container` | n/a | Required. The Azure container where the files are uploaded to and downloaded from. | +|`druid.msq.intermediate.storage.prefix` | n/a | Required. Path prepended to all the paths uploaded to the container to namespace the connector's files. Provide a unique value for the prefix and donot share the same prefix between different clusters. If the location includes other files or directories, then they might get cleaned up as well. | -In addition to the common service properties, there are certain properties that you configure on the Overlord specifically to clean up intermediate files: +Durable storage creates files on the remote storage and is cleaned up once the job no longer requires those files. However, due to failures causing abrupt exit of the tasks, these files might not get cleaned up. +Therefore, there are certain properties that you configure on the Overlord specifically to clean up intermediate files for the tasks that have completed and would no longer require these files: |Parameter |Default | Description | |-------------------|----------------------------------------|----------------------| From 850cb9425ddf1668756b641fcef907b4cd1261e2 Mon Sep 17 00:00:00 2001 From: Laksh Singla Date: Tue, 8 Aug 2023 12:13:44 +0530 Subject: [PATCH 25/28] spellcheck --- docs/multi-stage-query/reference.md | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/docs/multi-stage-query/reference.md b/docs/multi-stage-query/reference.md index 78575bcf32c8..e676d77f57b3 100644 --- a/docs/multi-stage-query/reference.md +++ b/docs/multi-stage-query/reference.md @@ -349,10 +349,10 @@ SQL-based ingestion supports using durable storage to store intermediate files t ### Durable storage configurations -Durable storage is supported on Amazon S3 storage and Microsoft's Azure storage. There are a few common configurations that controls the behaviour for both the services as documented below. Apart from the common configurations, +Durable storage is supported on Amazon S3 storage and Microsoft's Azure storage. There are a few common configurations that controls the behavior for both the services as documented below. Apart from the common configurations, there are a few properties specific to each storage that must be set. -Common properties to configure the behaviour of durable storage +Common properties to configure the behavior of durable storage |Parameter |Default | Description | |-------------------|----------------------------------------|----------------------| @@ -367,14 +367,14 @@ Following properties need to be set in addition to the common properties to enab |Parameter |Default | Description | |-------------------|----------------------------------------|----------------------| |`druid.msq.intermediate.storage.bucket` | n/a | Required. The S3 bucket where the files are uploaded to and download from | -|`druid.msq.intermediate.storage.prefix` | n/a | Required. Path prepended to all the paths uploaded to the bucket to namespace the connector's files. Provide a unique value for the prefix and donot share the same prefix between different clusters. If the location includes other files or directories, then they might get cleaned up as well. | +|`druid.msq.intermediate.storage.prefix` | n/a | Required. Path prepended to all the paths uploaded to the bucket to namespace the connector's files. Provide a unique value for the prefix and do not share the same prefix between different clusters. If the location includes other files or directories, then they might get cleaned up as well. | Following properties must be set in addition to the common properties to enable durable storage on Azure. |Parameter |Default | Description | |-------------------|----------------------------------------|----------------------| |`druid.msq.intermediate.storage.container` | n/a | Required. The Azure container where the files are uploaded to and downloaded from. | -|`druid.msq.intermediate.storage.prefix` | n/a | Required. Path prepended to all the paths uploaded to the container to namespace the connector's files. Provide a unique value for the prefix and donot share the same prefix between different clusters. If the location includes other files or directories, then they might get cleaned up as well. | +|`druid.msq.intermediate.storage.prefix` | n/a | Required. Path prepended to all the paths uploaded to the container to namespace the connector's files. Provide a unique value for the prefix and do not share the same prefix between different clusters. If the location includes other files or directories, then they might get cleaned up as well. | Durable storage creates files on the remote storage and is cleaned up once the job no longer requires those files. However, due to failures causing abrupt exit of the tasks, these files might not get cleaned up. Therefore, there are certain properties that you configure on the Overlord specifically to clean up intermediate files for the tasks that have completed and would no longer require these files: From 8639351914ddf01dec2a46ef9b6a64a27d5d3a89 Mon Sep 17 00:00:00 2001 From: Laksh Singla Date: Tue, 8 Aug 2023 17:57:38 +0530 Subject: [PATCH 26/28] create dir before checking for permissions --- .../druid/storage/azure/output/AzureOutputConfig.java | 2 +- .../storage/azure/output/AzureOutputConfigTest.java | 9 +++++++++ 2 files changed, 10 insertions(+), 1 deletion(-) diff --git a/extensions-core/azure-extensions/src/main/java/org/apache/druid/storage/azure/output/AzureOutputConfig.java b/extensions-core/azure-extensions/src/main/java/org/apache/druid/storage/azure/output/AzureOutputConfig.java index 6d5cc06be872..02e206162abc 100644 --- a/extensions-core/azure-extensions/src/main/java/org/apache/druid/storage/azure/output/AzureOutputConfig.java +++ b/extensions-core/azure-extensions/src/main/java/org/apache/druid/storage/azure/output/AzureOutputConfig.java @@ -111,7 +111,7 @@ private void validateFields() ); } - if (!tempDir.canRead() || !tempDir.canWrite()) { + if (!tempDir.mkdirs() || !tempDir.canRead() || !tempDir.canWrite()) { throw InvalidInput.exception("Cannot read or write on the 'tempDir' [%s]. " + "Please provide a different path to store the intermediate contents of AzureStorageConnector" ); diff --git a/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/output/AzureOutputConfigTest.java b/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/output/AzureOutputConfigTest.java index 318164755991..46edf2e46a30 100644 --- a/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/output/AzureOutputConfigTest.java +++ b/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/output/AzureOutputConfigTest.java @@ -58,9 +58,18 @@ public void testTempDirectoryNotWritable() throws IOException if (!tempDir.setWritable(false)) { throw new ISE("Unable to change the permission of temp folder for %s", this.getClass().getName()); } + //noinspection ResultOfObjectAllocationIgnored Assert.assertThrows( DruidException.class, () -> new AzureOutputConfig(CONTAINER, PREFIX, tempDir, null, MAX_RETRY_COUNT) ); } + + @Test + public void testTempDirectoryNotPresentButWritable() throws IOException + { + File tempDir = new File(temporaryFolder.newFolder() + "/notPresent1/notPresent2/notPresent3"); + //noinspection ResultOfObjectAllocationIgnored + new AzureOutputConfig(CONTAINER, PREFIX, tempDir, null, MAX_RETRY_COUNT); + } } From 575e2ecf7b0e9b82cd103bacc55c8493fe5e5a3f Mon Sep 17 00:00:00 2001 From: Laksh Singla Date: Tue, 8 Aug 2023 21:09:13 +0530 Subject: [PATCH 27/28] check fix --- .../azure/output/AzureOutputConfig.java | 21 +++++++++++++++---- .../azure/output/AzureOutputConfigTest.java | 10 +++++++++ 2 files changed, 27 insertions(+), 4 deletions(-) diff --git a/extensions-core/azure-extensions/src/main/java/org/apache/druid/storage/azure/output/AzureOutputConfig.java b/extensions-core/azure-extensions/src/main/java/org/apache/druid/storage/azure/output/AzureOutputConfig.java index 02e206162abc..bc5cdf6d7b8f 100644 --- a/extensions-core/azure-extensions/src/main/java/org/apache/druid/storage/azure/output/AzureOutputConfig.java +++ b/extensions-core/azure-extensions/src/main/java/org/apache/druid/storage/azure/output/AzureOutputConfig.java @@ -20,12 +20,15 @@ package org.apache.druid.storage.azure.output; import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.druid.error.DruidException; import org.apache.druid.error.InvalidInput; +import org.apache.druid.java.util.common.FileUtils; import org.apache.druid.java.util.common.HumanReadableBytes; import org.apache.druid.java.util.common.RetryUtils; import javax.annotation.Nullable; import java.io.File; +import java.io.IOException; import java.util.Objects; /** @@ -111,10 +114,20 @@ private void validateFields() ); } - if (!tempDir.mkdirs() || !tempDir.canRead() || !tempDir.canWrite()) { - throw InvalidInput.exception("Cannot read or write on the 'tempDir' [%s]. " - + "Please provide a different path to store the intermediate contents of AzureStorageConnector" - ); + try { + FileUtils.mkdirp(tempDir); + } + catch (IOException e) { + throw DruidException.forPersona(DruidException.Persona.ADMIN) + .ofCategory(DruidException.Category.RUNTIME_FAILURE) + .build(e, "Unable to create temporary directory [%s]", tempDir.getAbsolutePath()); + } + + if (!tempDir.canRead() || !tempDir.canWrite()) { + throw DruidException.forPersona(DruidException.Persona.ADMIN) + .ofCategory(DruidException.Category.RUNTIME_FAILURE) + .build("Cannot read or write on the 'tempDir' [%s]. " + + "Please provide a different path to store the intermediate contents of AzureStorageConnector"); } } diff --git a/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/output/AzureOutputConfigTest.java b/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/output/AzureOutputConfigTest.java index 46edf2e46a30..ab3104adf4ea 100644 --- a/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/output/AzureOutputConfigTest.java +++ b/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/output/AzureOutputConfigTest.java @@ -21,6 +21,7 @@ import org.apache.druid.error.DruidException; +import org.apache.druid.java.util.common.FileUtils; import org.apache.druid.java.util.common.HumanReadableBytes; import org.apache.druid.java.util.common.ISE; import org.junit.Assert; @@ -72,4 +73,13 @@ public void testTempDirectoryNotPresentButWritable() throws IOException //noinspection ResultOfObjectAllocationIgnored new AzureOutputConfig(CONTAINER, PREFIX, tempDir, null, MAX_RETRY_COUNT); } + + @Test + public void testTempDirectoryPresent() throws IOException + { + File tempDir = new File(temporaryFolder.newFolder() + "/notPresent1/notPresent2/notPresent3"); + FileUtils.mkdirp(tempDir); + //noinspection ResultOfObjectAllocationIgnored + new AzureOutputConfig(CONTAINER, PREFIX, tempDir, null, MAX_RETRY_COUNT); + } } From e1da797a95e39a9f87b8c7b6f747911daf921918 Mon Sep 17 00:00:00 2001 From: Laksh Singla Date: Tue, 8 Aug 2023 23:24:15 +0530 Subject: [PATCH 28/28] build fix --- .../druid/storage/azure/output/AzureOutputConfig.java | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/extensions-core/azure-extensions/src/main/java/org/apache/druid/storage/azure/output/AzureOutputConfig.java b/extensions-core/azure-extensions/src/main/java/org/apache/druid/storage/azure/output/AzureOutputConfig.java index bc5cdf6d7b8f..7af9c856c5f5 100644 --- a/extensions-core/azure-extensions/src/main/java/org/apache/druid/storage/azure/output/AzureOutputConfig.java +++ b/extensions-core/azure-extensions/src/main/java/org/apache/druid/storage/azure/output/AzureOutputConfig.java @@ -126,8 +126,11 @@ private void validateFields() if (!tempDir.canRead() || !tempDir.canWrite()) { throw DruidException.forPersona(DruidException.Persona.ADMIN) .ofCategory(DruidException.Category.RUNTIME_FAILURE) - .build("Cannot read or write on the 'tempDir' [%s]. " - + "Please provide a different path to store the intermediate contents of AzureStorageConnector"); + .build( + "Cannot read or write on the 'tempDir' [%s]. " + + "Please provide a different path to store the intermediate contents of AzureStorageConnector", + tempDir.getAbsolutePath() + ); } }