diff --git a/core/trino-main/pom.xml b/core/trino-main/pom.xml index 252cd20dcddc..0c94c78dcad9 100644 --- a/core/trino-main/pom.xml +++ b/core/trino-main/pom.xml @@ -376,6 +376,12 @@ + + io.trino + trino-exchange + test + + io.trino trino-parser diff --git a/core/trino-main/src/main/java/io/trino/server/testing/TestingTrinoServer.java b/core/trino-main/src/main/java/io/trino/server/testing/TestingTrinoServer.java index e4b0ab3dc19d..d8d76b446d46 100644 --- a/core/trino-main/src/main/java/io/trino/server/testing/TestingTrinoServer.java +++ b/core/trino-main/src/main/java/io/trino/server/testing/TestingTrinoServer.java @@ -73,7 +73,6 @@ import io.trino.server.ShutdownAction; import io.trino.server.security.CertificateAuthenticatorManager; import io.trino.server.security.ServerSecurityModule; -import io.trino.server.testing.exchange.LocalFileSystemExchangeManagerFactory; import io.trino.spi.ErrorType; import io.trino.spi.Plugin; import io.trino.spi.QueryId; @@ -177,6 +176,7 @@ public static Builder builder() private final MBeanServer mBeanServer; private final boolean coordinator; private final FailureInjector failureInjector; + private final ExchangeManagerRegistry exchangeManagerRegistry; public static class TestShutdownAction implements ShutdownAction @@ -344,17 +344,13 @@ private TestingTrinoServer( mBeanServer = injector.getInstance(MBeanServer.class); announcer = injector.getInstance(Announcer.class); failureInjector = injector.getInstance(FailureInjector.class); + exchangeManagerRegistry = injector.getInstance(ExchangeManagerRegistry.class); accessControl.setSystemAccessControls(systemAccessControls); EventListenerManager eventListenerManager = injector.getInstance(EventListenerManager.class); eventListeners.forEach(eventListenerManager::addEventListener); - ExchangeManagerRegistry exchangeManagerRegistry = injector.getInstance(ExchangeManagerRegistry.class); - exchangeManagerRegistry.addExchangeManagerFactory(new LocalFileSystemExchangeManagerFactory()); - exchangeManagerRegistry.loadExchangeManager("local", ImmutableMap.of( - "base-directory", System.getProperty("java.io.tmpdir") + "/trino-local-file-system-exchange-manager")); - announcer.forceAnnounce(); refreshNodes(); @@ -421,6 +417,11 @@ public CatalogName createCatalog(String catalogName, String connectorName, Map properties) + { + exchangeManagerRegistry.loadExchangeManager(name, properties); + } + public Path getBaseDataDir() { return baseDataDir; diff --git a/core/trino-main/src/main/java/io/trino/server/testing/exchange/LocalFileSystemExchange.java b/core/trino-main/src/main/java/io/trino/server/testing/exchange/LocalFileSystemExchange.java deleted file mode 100644 index f40b91307629..000000000000 --- a/core/trino-main/src/main/java/io/trino/server/testing/exchange/LocalFileSystemExchange.java +++ /dev/null @@ -1,295 +0,0 @@ -/* - * Licensed 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 io.trino.server.testing.exchange; - -import com.google.common.collect.ArrayListMultimap; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; -import com.google.common.collect.Multimap; -import io.trino.spi.exchange.Exchange; -import io.trino.spi.exchange.ExchangeContext; -import io.trino.spi.exchange.ExchangeSinkHandle; -import io.trino.spi.exchange.ExchangeSinkInstanceHandle; -import io.trino.spi.exchange.ExchangeSourceHandle; -import io.trino.spi.exchange.ExchangeSourceSplitter; -import io.trino.spi.exchange.ExchangeSourceStatistics; - -import javax.annotation.concurrent.GuardedBy; - -import java.io.IOException; -import java.io.UncheckedIOException; -import java.nio.file.Files; -import java.nio.file.Path; -import java.nio.file.Paths; -import java.util.HashSet; -import java.util.Iterator; -import java.util.List; -import java.util.Map; -import java.util.Optional; -import java.util.Set; -import java.util.concurrent.CompletableFuture; -import java.util.function.Predicate; -import java.util.regex.Matcher; -import java.util.regex.Pattern; -import java.util.stream.Stream; - -import static com.google.common.base.Preconditions.checkState; -import static com.google.common.base.Verify.verify; -import static com.google.common.collect.ImmutableList.toImmutableList; -import static com.google.common.io.MoreFiles.deleteRecursively; -import static com.google.common.io.RecursiveDeleteOption.ALLOW_INSECURE; -import static io.trino.server.testing.exchange.LocalFileSystemExchangeSink.COMMITTED_MARKER_FILE_NAME; -import static io.trino.server.testing.exchange.LocalFileSystemExchangeSink.DATA_FILE_SUFFIX; -import static java.nio.file.Files.createDirectories; -import static java.nio.file.Files.exists; -import static java.util.Objects.requireNonNull; -import static java.util.concurrent.CompletableFuture.completedFuture; - -public class LocalFileSystemExchange - implements Exchange -{ - private static final Pattern PARTITION_FILE_NAME_PATTERN = Pattern.compile("(\\d+)\\.data"); - - private final Path baseDirectory; - private final ExchangeContext exchangeContext; - private final int outputPartitionCount; - - @GuardedBy("this") - private final Set allSinks = new HashSet<>(); - @GuardedBy("this") - private final Set finishedSinks = new HashSet<>(); - @GuardedBy("this") - private boolean noMoreSinks; - - private final CompletableFuture> exchangeSourceHandlesFuture = new CompletableFuture<>(); - @GuardedBy("this") - private boolean exchangeSourceHandlesCreated; - - public LocalFileSystemExchange(Path baseDirectory, ExchangeContext exchangeContext, int outputPartitionCount) - { - this.baseDirectory = requireNonNull(baseDirectory, "baseDirectory is null"); - this.exchangeContext = requireNonNull(exchangeContext, "exchangeContext is null"); - this.outputPartitionCount = outputPartitionCount; - } - - public void initialize() - { - try { - createDirectories(getExchangeDirectory()); - } - catch (IOException e) { - throw new UncheckedIOException(e); - } - } - - @Override - public synchronized ExchangeSinkHandle addSink(int taskPartitionId) - { - LocalFileSystemExchangeSinkHandle sinkHandle = new LocalFileSystemExchangeSinkHandle( - exchangeContext.getQueryId(), - exchangeContext.getExchangeId(), - taskPartitionId); - allSinks.add(sinkHandle); - return sinkHandle; - } - - @Override - public void noMoreSinks() - { - synchronized (this) { - noMoreSinks = true; - } - checkInputReady(); - } - - @Override - public ExchangeSinkInstanceHandle instantiateSink(ExchangeSinkHandle sinkHandle, int taskAttemptId) - { - LocalFileSystemExchangeSinkHandle localFileSystemSinkHandle = (LocalFileSystemExchangeSinkHandle) sinkHandle; - Path outputDirectory = getExchangeDirectory() - .resolve(Integer.toString(localFileSystemSinkHandle.getTaskPartitionId())) - .resolve(Integer.toString(taskAttemptId)); - return new LocalFileSystemExchangeSinkInstanceHandle(localFileSystemSinkHandle, outputDirectory, outputPartitionCount); - } - - @Override - public void sinkFinished(ExchangeSinkInstanceHandle handle) - { - synchronized (this) { - LocalFileSystemExchangeSinkInstanceHandle localHandle = (LocalFileSystemExchangeSinkInstanceHandle) handle; - finishedSinks.add(localHandle.getSinkHandle()); - } - checkInputReady(); - } - - private void checkInputReady() - { - verify(!Thread.holdsLock(this)); - List exchangeSourceHandles = null; - synchronized (this) { - if (exchangeSourceHandlesCreated) { - return; - } - if (noMoreSinks && finishedSinks.containsAll(allSinks)) { - // input is ready, create exchange source handles - exchangeSourceHandles = createExchangeSourceHandles(); - exchangeSourceHandlesCreated = true; - } - } - if (exchangeSourceHandles != null) { - exchangeSourceHandlesFuture.complete(exchangeSourceHandles); - } - } - - private synchronized List createExchangeSourceHandles() - { - Multimap partitionFiles = ArrayListMultimap.create(); - for (LocalFileSystemExchangeSinkHandle sinkHandle : finishedSinks) { - Path committedAttemptPath = getCommittedAttemptPath(sinkHandle); - Map partitions = getCommittedPartitions(committedAttemptPath); - partitions.forEach((partition, file) -> partitionFiles.put(partition, file.toAbsolutePath().toString())); - } - - ImmutableList.Builder result = ImmutableList.builder(); - for (Integer partitionId : partitionFiles.keySet()) { - result.add(new LocalFileSystemExchangeSourceHandle(partitionId, ImmutableList.copyOf(partitionFiles.get(partitionId)))); - } - return result.build(); - } - - private Path getCommittedAttemptPath(LocalFileSystemExchangeSinkHandle sinkHandle) - { - Path sinkOutputBasePath = getExchangeDirectory() - .resolve(Integer.toString(sinkHandle.getTaskPartitionId())); - try { - List attemptPaths = listFiles(sinkOutputBasePath, Files::isDirectory); - checkState(!attemptPaths.isEmpty(), "no attempts found for sink %s", sinkHandle); - - List committedAttemptPaths = attemptPaths.stream() - .filter(LocalFileSystemExchange::isCommitted) - .collect(toImmutableList()); - checkState(!committedAttemptPaths.isEmpty(), "no committed attempts found for %s", sinkHandle); - - return committedAttemptPaths.get(0); - } - catch (IOException e) { - throw new UncheckedIOException(e); - } - } - - private static boolean isCommitted(Path attemptPath) - { - Path commitMarkerFilePath = attemptPath.resolve(COMMITTED_MARKER_FILE_NAME); - return Files.exists(commitMarkerFilePath); - } - - private static Map getCommittedPartitions(Path committedAttemptPath) - { - try { - List partitionFiles = listFiles(committedAttemptPath, path -> path.toString().endsWith(DATA_FILE_SUFFIX)); - ImmutableMap.Builder result = ImmutableMap.builder(); - for (Path partitionFile : partitionFiles) { - Matcher matcher = PARTITION_FILE_NAME_PATTERN.matcher(partitionFile.getFileName().toString()); - checkState(matcher.matches(), "unexpected partition file: %s", partitionFile); - int partitionId = Integer.parseInt(matcher.group(1)); - result.put(partitionId, partitionFile); - } - return result.build(); - } - catch (IOException e) { - throw new UncheckedIOException(e); - } - } - - private Path getExchangeDirectory() - { - return baseDirectory.resolve(exchangeContext.getQueryId() + "." + exchangeContext.getExchangeId()); - } - - @Override - public CompletableFuture> getSourceHandles() - { - return exchangeSourceHandlesFuture; - } - - @Override - public ExchangeSourceSplitter split(ExchangeSourceHandle handle, long targetSizeInBytes) - { - // always split for testing - LocalFileSystemExchangeSourceHandle localHandle = (LocalFileSystemExchangeSourceHandle) handle; - Iterator filesIterator = localHandle.getFiles().iterator(); - return new ExchangeSourceSplitter() - { - @Override - public CompletableFuture isBlocked() - { - return completedFuture(null); - } - - @Override - public Optional getNext() - { - if (filesIterator.hasNext()) { - return Optional.of(new LocalFileSystemExchangeSourceHandle(localHandle.getPartitionId(), ImmutableList.of(filesIterator.next()))); - } - return Optional.empty(); - } - - @Override - public void close() - { - } - }; - } - - @Override - public ExchangeSourceStatistics getExchangeSourceStatistics(ExchangeSourceHandle handle) - { - LocalFileSystemExchangeSourceHandle localHandle = (LocalFileSystemExchangeSourceHandle) handle; - long sizeInBytes = 0; - for (String file : localHandle.getFiles()) { - try { - sizeInBytes += Files.size(Paths.get(file)); - } - catch (IOException e) { - throw new UncheckedIOException(e); - } - } - return new ExchangeSourceStatistics(sizeInBytes); - } - - @Override - public void close() - { - try { - Path exchangeDirectory = getExchangeDirectory(); - if (exists(exchangeDirectory)) { - deleteRecursively(exchangeDirectory, ALLOW_INSECURE); - } - } - catch (IOException e) { - throw new UncheckedIOException(e); - } - } - - private static List listFiles(Path directory, Predicate predicate) - throws IOException - { - ImmutableList.Builder builder = ImmutableList.builder(); - try (Stream dir = Files.list(directory)) { - dir.filter(predicate).forEach(builder::add); - } - return builder.build(); - } -} diff --git a/core/trino-main/src/main/java/io/trino/server/testing/exchange/LocalFileSystemExchangeManager.java b/core/trino-main/src/main/java/io/trino/server/testing/exchange/LocalFileSystemExchangeManager.java deleted file mode 100644 index 3627ece30cd1..000000000000 --- a/core/trino-main/src/main/java/io/trino/server/testing/exchange/LocalFileSystemExchangeManager.java +++ /dev/null @@ -1,76 +0,0 @@ -/* - * Licensed 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 io.trino.server.testing.exchange; - -import io.trino.spi.exchange.Exchange; -import io.trino.spi.exchange.ExchangeContext; -import io.trino.spi.exchange.ExchangeManager; -import io.trino.spi.exchange.ExchangeSink; -import io.trino.spi.exchange.ExchangeSinkInstanceHandle; -import io.trino.spi.exchange.ExchangeSource; -import io.trino.spi.exchange.ExchangeSourceHandle; - -import java.io.IOException; -import java.io.UncheckedIOException; -import java.nio.file.Path; -import java.nio.file.Paths; -import java.util.List; - -import static com.google.common.collect.ImmutableList.toImmutableList; -import static java.nio.file.Files.createDirectories; -import static java.util.Objects.requireNonNull; - -public class LocalFileSystemExchangeManager - implements ExchangeManager -{ - private final Path baseDirectory; - - public LocalFileSystemExchangeManager(Path baseDirectory) - { - this.baseDirectory = requireNonNull(baseDirectory, "baseDirectory is null"); - } - - @Override - public Exchange createExchange(ExchangeContext context, int outputPartitionCount) - { - LocalFileSystemExchange exchange = new LocalFileSystemExchange(baseDirectory, context, outputPartitionCount); - exchange.initialize(); - return exchange; - } - - @Override - public ExchangeSink createSink(ExchangeSinkInstanceHandle handle, boolean preserveRecordsOrder) - { - LocalFileSystemExchangeSinkInstanceHandle localHandle = (LocalFileSystemExchangeSinkInstanceHandle) handle; - Path outputDirectory = localHandle.getOutputDirectory(); - try { - createDirectories(outputDirectory); - } - catch (IOException e) { - throw new UncheckedIOException(e); - } - return new LocalFileSystemExchangeSink(outputDirectory, localHandle.getOutputPartitionCount()); - } - - @Override - public ExchangeSource createSource(List handles) - { - List files = handles.stream() - .map(LocalFileSystemExchangeSourceHandle.class::cast) - .flatMap(handle -> handle.getFiles().stream()) - .map(Paths::get) - .collect(toImmutableList()); - return new LocalFileSystemExchangeSource(files); - } -} diff --git a/core/trino-main/src/main/java/io/trino/server/testing/exchange/LocalFileSystemExchangeSink.java b/core/trino-main/src/main/java/io/trino/server/testing/exchange/LocalFileSystemExchangeSink.java deleted file mode 100644 index d3f5e90dee43..000000000000 --- a/core/trino-main/src/main/java/io/trino/server/testing/exchange/LocalFileSystemExchangeSink.java +++ /dev/null @@ -1,169 +0,0 @@ -/* - * Licensed 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 io.trino.server.testing.exchange; - -import io.airlift.log.Logger; -import io.airlift.slice.OutputStreamSliceOutput; -import io.airlift.slice.Slice; -import io.airlift.slice.SliceOutput; -import io.airlift.units.DataSize; -import io.trino.spi.exchange.ExchangeSink; -import org.openjdk.jol.info.ClassLayout; - -import javax.annotation.concurrent.GuardedBy; - -import java.io.FileNotFoundException; -import java.io.FileOutputStream; -import java.io.IOException; -import java.io.UncheckedIOException; -import java.nio.file.Path; -import java.util.HashMap; -import java.util.Map; -import java.util.concurrent.CompletableFuture; - -import static com.google.common.base.Preconditions.checkArgument; -import static com.google.common.base.Preconditions.checkState; -import static com.google.common.io.MoreFiles.deleteRecursively; -import static com.google.common.io.RecursiveDeleteOption.ALLOW_INSECURE; -import static io.airlift.slice.SizeOf.estimatedSizeOf; -import static io.airlift.units.DataSize.Unit.KILOBYTE; -import static java.lang.Math.toIntExact; -import static java.nio.file.Files.createFile; -import static java.util.Objects.requireNonNull; -import static java.util.concurrent.CompletableFuture.completedFuture; -import static java.util.concurrent.CompletableFuture.failedFuture; - -public class LocalFileSystemExchangeSink - implements ExchangeSink -{ - private static final Logger log = Logger.get(LocalFileSystemExchangeSink.class); - - public static final String COMMITTED_MARKER_FILE_NAME = "committed"; - public static final String DATA_FILE_SUFFIX = ".data"; - - private static final int INSTANCE_SIZE = ClassLayout.parseClass(LocalFileSystemExchangeSink.class).instanceSize(); - private static final int INTEGER_INSTANCE_SIZE = ClassLayout.parseClass(Integer.class).instanceSize(); - - private static final int BUFFER_SIZE_IN_BYTES = toIntExact(DataSize.of(4, KILOBYTE).toBytes()); - - private final Path outputDirectory; - private final int outputPartitionCount; - - @GuardedBy("this") - private final Map outputs = new HashMap<>(); - @GuardedBy("this") - private boolean committed; - @GuardedBy("this") - private boolean closed; - - public LocalFileSystemExchangeSink(Path outputDirectory, int outputPartitionCount) - { - this.outputDirectory = requireNonNull(outputDirectory, "outputDirectory is null"); - this.outputPartitionCount = outputPartitionCount; - } - - @Override - public CompletableFuture isBlocked() - { - return NOT_BLOCKED; - } - - @Override - public synchronized void add(int partitionId, Slice data) - { - checkArgument(partitionId < outputPartitionCount, "partition id is expected to be less than %s: %s", outputPartitionCount, partitionId); - checkState(!committed, "already committed"); - if (closed) { - return; - } - SliceOutput output = outputs.computeIfAbsent(partitionId, this::createOutput); - output.writeInt(data.length()); - output.writeBytes(data); - } - - private SliceOutput createOutput(int partitionId) - { - Path outputPath = outputDirectory.resolve(partitionId + DATA_FILE_SUFFIX); - try { - return new OutputStreamSliceOutput(new FileOutputStream(outputPath.toFile()), BUFFER_SIZE_IN_BYTES); - } - catch (FileNotFoundException e) { - throw new UncheckedIOException(e); - } - } - - @Override - public synchronized long getMemoryUsage() - { - return INSTANCE_SIZE + - estimatedSizeOf(outputs, (ignored) -> INTEGER_INSTANCE_SIZE, SliceOutput::getRetainedSize); - } - - @Override - public synchronized CompletableFuture finish() - { - if (closed) { - return completedFuture(null); - } - try { - for (SliceOutput output : outputs.values()) { - try { - output.close(); - } - catch (IOException e) { - throw new UncheckedIOException(e); - } - } - outputs.clear(); - try { - createFile(outputDirectory.resolve(COMMITTED_MARKER_FILE_NAME)); - } - catch (IOException e) { - throw new UncheckedIOException(e); - } - } - catch (Throwable t) { - abort(); - return failedFuture(t); - } - committed = true; - closed = true; - return completedFuture(null); - } - - @Override - public synchronized CompletableFuture abort() - { - if (closed) { - return completedFuture(null); - } - closed = true; - for (SliceOutput output : outputs.values()) { - try { - output.close(); - } - catch (IOException e) { - log.warn(e, "Error closing output"); - } - } - outputs.clear(); - try { - deleteRecursively(outputDirectory, ALLOW_INSECURE); - } - catch (IOException e) { - log.warn(e, "Error cleaning output directory"); - } - return completedFuture(null); - } -} diff --git a/core/trino-main/src/main/java/io/trino/testing/LocalQueryRunner.java b/core/trino-main/src/main/java/io/trino/testing/LocalQueryRunner.java index 5e04d00fe6c5..8d27578a8422 100644 --- a/core/trino-main/src/main/java/io/trino/testing/LocalQueryRunner.java +++ b/core/trino-main/src/main/java/io/trino/testing/LocalQueryRunner.java @@ -847,6 +847,12 @@ public List createDrivers(@Language("SQL") String sql, OutputFactory out return createDrivers(defaultSession, sql, outputFactory, taskContext); } + @Override + public void loadExchangeManager(String name, Map properties) + { + exchangeManagerRegistry.loadExchangeManager(name, properties); + } + public List createDrivers(Session session, @Language("SQL") String sql, OutputFactory outputFactory, TaskContext taskContext) { Plan plan = createPlan(session, sql, WarningCollector.NOOP); diff --git a/core/trino-main/src/main/java/io/trino/testing/QueryRunner.java b/core/trino-main/src/main/java/io/trino/testing/QueryRunner.java index 0072d503a95f..cdc37516923b 100644 --- a/core/trino-main/src/main/java/io/trino/testing/QueryRunner.java +++ b/core/trino-main/src/main/java/io/trino/testing/QueryRunner.java @@ -104,6 +104,8 @@ void injectTaskFailure( InjectedFailureType injectionType, Optional errorType); + void loadExchangeManager(String name, Map properties); + class MaterializedResultWithPlan { private final MaterializedResult materializedResult; diff --git a/core/trino-main/src/test/java/io/trino/operator/TestDeduplicatingDirectExchangeBuffer.java b/core/trino-main/src/test/java/io/trino/operator/TestDeduplicatingDirectExchangeBuffer.java index addb29a37df7..6f78619449d2 100644 --- a/core/trino-main/src/test/java/io/trino/operator/TestDeduplicatingDirectExchangeBuffer.java +++ b/core/trino-main/src/test/java/io/trino/operator/TestDeduplicatingDirectExchangeBuffer.java @@ -26,7 +26,7 @@ import io.trino.execution.StageId; import io.trino.execution.TaskId; import io.trino.metadata.ExchangeHandleResolver; -import io.trino.server.testing.exchange.LocalFileSystemExchangeManagerFactory; +import io.trino.plugin.exchange.FileSystemExchangeManagerFactory; import io.trino.spi.QueryId; import io.trino.spi.TrinoException; import org.testng.annotations.AfterClass; @@ -65,9 +65,9 @@ public class TestDeduplicatingDirectExchangeBuffer public void beforeClass() { exchangeManagerRegistry = new ExchangeManagerRegistry(new ExchangeHandleResolver()); - exchangeManagerRegistry.addExchangeManagerFactory(new LocalFileSystemExchangeManagerFactory()); - exchangeManagerRegistry.loadExchangeManager("local", ImmutableMap.of( - "base-directory", System.getProperty("java.io.tmpdir") + "/trino-local-file-system-exchange-manager")); + exchangeManagerRegistry.addExchangeManagerFactory(new FileSystemExchangeManagerFactory()); + exchangeManagerRegistry.loadExchangeManager("filesystem", ImmutableMap.of( + "exchange.base-directory", System.getProperty("java.io.tmpdir") + "/trino-local-file-system-exchange-manager")); } @AfterClass(alwaysRun = true) diff --git a/core/trino-server/src/main/provisio/trino.xml b/core/trino-server/src/main/provisio/trino.xml index 48190e7b21a7..7c65bfb437f3 100644 --- a/core/trino-server/src/main/provisio/trino.xml +++ b/core/trino-server/src/main/provisio/trino.xml @@ -270,4 +270,10 @@ + + + + + + diff --git a/plugin/trino-exchange/pom.xml b/plugin/trino-exchange/pom.xml new file mode 100644 index 000000000000..f4220fd23c12 --- /dev/null +++ b/plugin/trino-exchange/pom.xml @@ -0,0 +1,180 @@ + + + 4.0.0 + + io.trino + trino-root + 371-SNAPSHOT + ../../pom.xml + + + trino-exchange + Trino - Exchange + trino-plugin + + + ${project.parent.basedir} + 2.17.102 + + + + + + software.amazon.awssdk + bom + pom + ${awsjavasdk.version} + import + + + + + + + io.airlift + bootstrap + + + + io.airlift + concurrent + + + + io.airlift + configuration + + + + io.airlift + log + + + + io.airlift + units + + + + com.google.code.findbugs + jsr305 + + + + com.google.guava + guava + + + + com.google.inject + guice + + + + javax.annotation + javax.annotation-api + + + + javax.inject + javax.inject + + + + javax.validation + validation-api + + + + org.reactivestreams + reactive-streams + 1.0.3 + + + + software.amazon.awssdk + auth + + + + software.amazon.awssdk + aws-core + + + + software.amazon.awssdk + regions + + + + software.amazon.awssdk + s3 + + + commons-logging + commons-logging + + + + + + software.amazon.awssdk + sdk-core + + + + software.amazon.awssdk + utils + + + + + io.trino + trino-spi + provided + + + + io.airlift + slice + provided + + + + com.fasterxml.jackson.core + jackson-annotations + provided + + + + org.openjdk.jol + jol-core + provided + + + + + io.trino + trino-testing-containers + test + + + + org.assertj + assertj-core + test + + + + org.testcontainers + testcontainers + test + + + + org.testng + testng + test + + + diff --git a/plugin/trino-exchange/src/main/java/io/trino/plugin/exchange/ExchangeStorageWriter.java b/plugin/trino-exchange/src/main/java/io/trino/plugin/exchange/ExchangeStorageWriter.java new file mode 100644 index 000000000000..29697214141a --- /dev/null +++ b/plugin/trino-exchange/src/main/java/io/trino/plugin/exchange/ExchangeStorageWriter.java @@ -0,0 +1,31 @@ +package io.trino.plugin.exchange; +/* + * Licensed 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. + */ + +import com.google.common.util.concurrent.ListenableFuture; +import io.airlift.slice.Slice; + +import javax.annotation.concurrent.NotThreadSafe; + +@NotThreadSafe +public interface ExchangeStorageWriter +{ + ListenableFuture write(Slice slice); + + ListenableFuture finish(); + + ListenableFuture abort(); + + long getRetainedSize(); +} diff --git a/plugin/trino-exchange/src/main/java/io/trino/plugin/exchange/FileStatus.java b/plugin/trino-exchange/src/main/java/io/trino/plugin/exchange/FileStatus.java new file mode 100644 index 000000000000..64dfc77636bf --- /dev/null +++ b/plugin/trino-exchange/src/main/java/io/trino/plugin/exchange/FileStatus.java @@ -0,0 +1,87 @@ +/* + * Licensed 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 io.trino.plugin.exchange; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import org.openjdk.jol.info.ClassLayout; + +import javax.annotation.concurrent.Immutable; + +import java.util.Objects; + +import static com.google.common.base.MoreObjects.toStringHelper; +import static io.airlift.slice.SizeOf.estimatedSizeOf; +import static java.util.Objects.requireNonNull; + +@Immutable +public class FileStatus +{ + private static final int INSTANCE_SIZE = ClassLayout.parseClass(FileStatus.class).instanceSize(); + + private final String filePath; + private final long fileSize; + + @JsonCreator + public FileStatus(@JsonProperty("filePath") String filePath, @JsonProperty("fileSize") long fileSize) + { + this.filePath = requireNonNull(filePath, "path is null"); + this.fileSize = fileSize; + } + + @JsonProperty + public String getFilePath() + { + return filePath; + } + + @JsonProperty + public long getFileSize() + { + return fileSize; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + FileStatus that = (FileStatus) o; + return filePath.equals(that.getFilePath()) && fileSize == that.getFileSize(); + } + + @Override + public int hashCode() + { + return Objects.hash(filePath, fileSize); + } + + @Override + public String toString() + { + return toStringHelper(this) + .add("filePath", filePath) + .add("fileSize", fileSize) + .toString(); + } + + public long getRetainedSizeInBytes() + { + return INSTANCE_SIZE + estimatedSizeOf(filePath); + } +} diff --git a/plugin/trino-exchange/src/main/java/io/trino/plugin/exchange/FileSystemExchange.java b/plugin/trino-exchange/src/main/java/io/trino/plugin/exchange/FileSystemExchange.java new file mode 100644 index 000000000000..ed9dc500673f --- /dev/null +++ b/plugin/trino-exchange/src/main/java/io/trino/plugin/exchange/FileSystemExchange.java @@ -0,0 +1,300 @@ +/* + * Licensed 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 io.trino.plugin.exchange; + +import com.google.common.collect.ArrayListMultimap; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Multimap; +import io.trino.spi.exchange.Exchange; +import io.trino.spi.exchange.ExchangeContext; +import io.trino.spi.exchange.ExchangeSinkHandle; +import io.trino.spi.exchange.ExchangeSinkInstanceHandle; +import io.trino.spi.exchange.ExchangeSourceHandle; +import io.trino.spi.exchange.ExchangeSourceSplitter; +import io.trino.spi.exchange.ExchangeSourceStatistics; + +import javax.annotation.concurrent.GuardedBy; +import javax.crypto.SecretKey; + +import java.io.File; +import java.io.IOException; +import java.io.UncheckedIOException; +import java.net.URI; +import java.security.Key; +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutorService; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +import static com.google.common.base.Preconditions.checkState; +import static com.google.common.base.Verify.verify; +import static com.google.common.collect.ImmutableList.toImmutableList; +import static io.trino.plugin.exchange.FileSystemExchangeManager.PATH_SEPARATOR; +import static io.trino.plugin.exchange.FileSystemExchangeSink.COMMITTED_MARKER_FILE_NAME; +import static io.trino.plugin.exchange.FileSystemExchangeSink.DATA_FILE_SUFFIX; +import static java.lang.String.format; +import static java.util.Objects.requireNonNull; +import static java.util.concurrent.CompletableFuture.completedFuture; +import static java.util.concurrent.CompletableFuture.supplyAsync; + +public class FileSystemExchange + implements Exchange +{ + private static final Pattern PARTITION_FILE_NAME_PATTERN = Pattern.compile("(\\d+)\\.data"); + + private final URI baseDirectory; + private final FileSystemExchangeStorage exchangeStorage; + private final ExchangeContext exchangeContext; + private final int outputPartitionCount; + private final Optional secretKey; + private final ExecutorService executor; + + @GuardedBy("this") + private final Set allSinks = new HashSet<>(); + @GuardedBy("this") + private final Set finishedSinks = new HashSet<>(); + @GuardedBy("this") + private boolean noMoreSinks; + @GuardedBy("this") + private boolean exchangeSourceHandlesCreationStarted; + + private final CompletableFuture> exchangeSourceHandlesFuture = new CompletableFuture<>(); + + public FileSystemExchange( + URI baseDirectory, + FileSystemExchangeStorage exchangeStorage, + ExchangeContext exchangeContext, + int outputPartitionCount, + Optional secretKey, + ExecutorService executor) + { + this.baseDirectory = requireNonNull(baseDirectory, "baseDirectory is null"); + this.exchangeStorage = requireNonNull(exchangeStorage, "exchangeStorage is null"); + this.exchangeContext = requireNonNull(exchangeContext, "exchangeContext is null"); + this.outputPartitionCount = outputPartitionCount; + this.secretKey = requireNonNull(secretKey, "secretKey is null"); + this.executor = requireNonNull(executor, "executor is null"); + } + + public void initialize() + { + try { + exchangeStorage.createDirectories(getExchangeDirectory()); + } + catch (IOException e) { + throw new UncheckedIOException(e); + } + } + + @Override + public synchronized ExchangeSinkHandle addSink(int taskPartition) + { + FileSystemExchangeSinkHandle sinkHandle = new FileSystemExchangeSinkHandle(taskPartition, secretKey.map(Key::getEncoded)); + allSinks.add(taskPartition); + return sinkHandle; + } + + @Override + public void noMoreSinks() + { + synchronized (this) { + noMoreSinks = true; + } + checkInputReady(); + } + + @Override + public ExchangeSinkInstanceHandle instantiateSink(ExchangeSinkHandle sinkHandle, int taskAttemptId) + { + FileSystemExchangeSinkHandle fileSystemExchangeSinkHandle = (FileSystemExchangeSinkHandle) sinkHandle; + URI outputDirectory = getExchangeDirectory() + .resolve(fileSystemExchangeSinkHandle.getPartitionId() + PATH_SEPARATOR) + .resolve(taskAttemptId + PATH_SEPARATOR); + try { + exchangeStorage.createDirectories(outputDirectory); + } + catch (IOException e) { + throw new UncheckedIOException(e); + } + + return new FileSystemExchangeSinkInstanceHandle(fileSystemExchangeSinkHandle, outputDirectory, outputPartitionCount); + } + + @Override + public void sinkFinished(ExchangeSinkInstanceHandle handle) + { + synchronized (this) { + FileSystemExchangeSinkInstanceHandle instanceHandle = (FileSystemExchangeSinkInstanceHandle) handle; + finishedSinks.add(instanceHandle.getSinkHandle().getPartitionId()); + } + checkInputReady(); + } + + private void checkInputReady() + { + verify(!Thread.holdsLock(this)); + CompletableFuture> exchangeSourceHandlesCreationFuture = null; + synchronized (this) { + if (exchangeSourceHandlesCreationStarted) { + return; + } + if (noMoreSinks && finishedSinks.containsAll(allSinks)) { + // input is ready, create exchange source handles + exchangeSourceHandlesCreationStarted = true; + exchangeSourceHandlesCreationFuture = supplyAsync(this::createExchangeSourceHandles, executor); + } + } + if (exchangeSourceHandlesCreationFuture != null) { + exchangeSourceHandlesCreationFuture.whenComplete((exchangeSourceHandles, throwable) -> { + if (throwable != null) { + exchangeSourceHandlesFuture.completeExceptionally(throwable); + } + else { + exchangeSourceHandlesFuture.complete(exchangeSourceHandles); + } + }); + } + } + + private List createExchangeSourceHandles() + { + Multimap partitionFiles = ArrayListMultimap.create(); + List finishedTaskPartitions; + synchronized (this) { + finishedTaskPartitions = ImmutableList.copyOf(finishedSinks); + } + for (Integer taskPartition : finishedTaskPartitions) { + URI committedAttemptPath = getCommittedAttemptPath(taskPartition); + Map partitions = getCommittedPartitions(committedAttemptPath); + partitions.forEach(partitionFiles::put); + } + + ImmutableList.Builder result = ImmutableList.builder(); + for (Integer partitionId : partitionFiles.keySet()) { + result.add(new FileSystemExchangeSourceHandle(partitionId, ImmutableList.copyOf(partitionFiles.get(partitionId)), secretKey.map(SecretKey::getEncoded))); + } + return result.build(); + } + + private URI getCommittedAttemptPath(Integer taskPartition) + { + URI sinkOutputBasePath = getExchangeDirectory().resolve(taskPartition + PATH_SEPARATOR); + try { + List attemptPaths = exchangeStorage.listDirectories(sinkOutputBasePath); + checkState(!attemptPaths.isEmpty(), "No attempts found under sink output path %s", sinkOutputBasePath); + + return attemptPaths.stream() + .filter(this::isCommitted) + .findFirst() + .orElseThrow(() -> new IllegalStateException(format("No committed attempts found under sink output path %s", sinkOutputBasePath))); + } + catch (IOException e) { + throw new UncheckedIOException(e); + } + } + + private boolean isCommitted(URI attemptPath) + { + URI commitMarkerFilePath = attemptPath.resolve(COMMITTED_MARKER_FILE_NAME); + try { + return exchangeStorage.exists(commitMarkerFilePath); + } + catch (IOException e) { + throw new UncheckedIOException(e); + } + } + + private Map getCommittedPartitions(URI committedAttemptPath) + { + try { + List partitionFiles = exchangeStorage.listFiles(committedAttemptPath) + .stream() + .filter(file -> file.getFilePath().endsWith(DATA_FILE_SUFFIX)) + .collect(toImmutableList()); + ImmutableMap.Builder result = ImmutableMap.builder(); + for (FileStatus partitionFile : partitionFiles) { + Matcher matcher = PARTITION_FILE_NAME_PATTERN.matcher(new File(partitionFile.getFilePath()).getName()); + checkState(matcher.matches(), "Unexpected partition file: %s", partitionFile); + int partitionId = Integer.parseInt(matcher.group(1)); + result.put(partitionId, partitionFile); + } + return result.build(); + } + catch (IOException e) { + throw new UncheckedIOException(e); + } + } + + private URI getExchangeDirectory() + { + return baseDirectory.resolve(exchangeContext.getQueryId() + "." + exchangeContext.getExchangeId() + PATH_SEPARATOR); + } + + @Override + public CompletableFuture> getSourceHandles() + { + return exchangeSourceHandlesFuture; + } + + @Override + public ExchangeSourceSplitter split(ExchangeSourceHandle handle, long targetSizeInBytes) + { + // Currently we only split at the file level, and external logic groups sources that are not large enough + FileSystemExchangeSourceHandle sourceHandle = (FileSystemExchangeSourceHandle) handle; + Iterator filesIterator = sourceHandle.getFiles().iterator(); + return new ExchangeSourceSplitter() + { + @Override + public CompletableFuture isBlocked() + { + return completedFuture(null); + } + + @Override + public Optional getNext() + { + if (filesIterator.hasNext()) { + return Optional.of(new FileSystemExchangeSourceHandle(sourceHandle.getPartitionId(), ImmutableList.of(filesIterator.next()), secretKey.map(SecretKey::getEncoded))); + } + return Optional.empty(); + } + + @Override + public void close() + { + } + }; + } + + @Override + public ExchangeSourceStatistics getExchangeSourceStatistics(ExchangeSourceHandle handle) + { + FileSystemExchangeSourceHandle sourceHandle = (FileSystemExchangeSourceHandle) handle; + long sizeInBytes = sourceHandle.getFiles().stream().mapToLong(FileStatus::getFileSize).sum(); + return new ExchangeSourceStatistics(sizeInBytes); + } + + @Override + public void close() + { + exchangeStorage.deleteRecursively(getExchangeDirectory()); + } +} diff --git a/plugin/trino-exchange/src/main/java/io/trino/plugin/exchange/FileSystemExchangeConfig.java b/plugin/trino-exchange/src/main/java/io/trino/plugin/exchange/FileSystemExchangeConfig.java new file mode 100644 index 000000000000..dee6b7c933a5 --- /dev/null +++ b/plugin/trino-exchange/src/main/java/io/trino/plugin/exchange/FileSystemExchangeConfig.java @@ -0,0 +1,64 @@ +/* + * Licensed 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 io.trino.plugin.exchange; + +import io.airlift.configuration.Config; + +import javax.validation.constraints.Min; +import javax.validation.constraints.NotNull; + +public class FileSystemExchangeConfig +{ + private String baseDirectory; + private boolean exchangeEncryptionEnabled; + private int exchangeSinkBufferPoolMinSize; + + @NotNull + public String getBaseDirectory() + { + return baseDirectory; + } + + @Config("exchange.base-directory") + public FileSystemExchangeConfig setBaseDirectory(String baseDirectory) + { + this.baseDirectory = baseDirectory; + return this; + } + + public boolean isExchangeEncryptionEnabled() + { + return exchangeEncryptionEnabled; + } + + @Config("exchange.encryption-enabled") + public FileSystemExchangeConfig setExchangeEncryptionEnabled(boolean exchangeEncryptionEnabled) + { + this.exchangeEncryptionEnabled = exchangeEncryptionEnabled; + return this; + } + + @Min(0) + public int getExchangeSinkBufferPoolMinSize() + { + return exchangeSinkBufferPoolMinSize; + } + + @Config("exchange.sink-buffer-pool-min-size") + public FileSystemExchangeConfig setExchangeSinkBufferPoolMinSize(int exchangeSinkBufferPoolMinSize) + { + this.exchangeSinkBufferPoolMinSize = exchangeSinkBufferPoolMinSize; + return this; + } +} diff --git a/plugin/trino-exchange/src/main/java/io/trino/plugin/exchange/FileSystemExchangeManager.java b/plugin/trino-exchange/src/main/java/io/trino/plugin/exchange/FileSystemExchangeManager.java new file mode 100644 index 000000000000..f21ca15b4272 --- /dev/null +++ b/plugin/trino-exchange/src/main/java/io/trino/plugin/exchange/FileSystemExchangeManager.java @@ -0,0 +1,120 @@ +/* + * Licensed 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 io.trino.plugin.exchange; + +import com.google.common.collect.ImmutableList; +import io.trino.spi.TrinoException; +import io.trino.spi.exchange.Exchange; +import io.trino.spi.exchange.ExchangeContext; +import io.trino.spi.exchange.ExchangeManager; +import io.trino.spi.exchange.ExchangeSink; +import io.trino.spi.exchange.ExchangeSinkInstanceHandle; +import io.trino.spi.exchange.ExchangeSource; +import io.trino.spi.exchange.ExchangeSourceHandle; + +import javax.crypto.KeyGenerator; +import javax.crypto.SecretKey; +import javax.crypto.spec.SecretKeySpec; +import javax.inject.Inject; + +import java.net.URI; +import java.security.NoSuchAlgorithmException; +import java.util.List; +import java.util.Optional; +import java.util.concurrent.ExecutorService; + +import static com.google.common.collect.ImmutableList.toImmutableList; +import static io.airlift.concurrent.Threads.daemonThreadsNamed; +import static io.trino.spi.StandardErrorCode.GENERIC_INTERNAL_ERROR; +import static java.util.Collections.nCopies; +import static java.util.Objects.requireNonNull; +import static java.util.concurrent.Executors.newCachedThreadPool; + +public class FileSystemExchangeManager + implements ExchangeManager +{ + public static final String PATH_SEPARATOR = "/"; + + private static final int KEY_BITS = 256; + + private final FileSystemExchangeStorage exchangeStorage; + private final URI baseDirectory; + private final boolean exchangeEncryptionEnabled; + private final int exchangeSinkBufferPoolMinSize; + private final ExecutorService executor; + + @Inject + public FileSystemExchangeManager(FileSystemExchangeStorage exchangeStorage, FileSystemExchangeConfig fileSystemExchangeConfig) + { + requireNonNull(fileSystemExchangeConfig, "fileSystemExchangeConfig is null"); + + this.exchangeStorage = requireNonNull(exchangeStorage, "exchangeStorage is null"); + String baseDirectory = requireNonNull(fileSystemExchangeConfig.getBaseDirectory(), "baseDirectory is null"); + if (!baseDirectory.endsWith(PATH_SEPARATOR)) { + // This is needed as URI's resolve method expects directories to end with '/' + baseDirectory += PATH_SEPARATOR; + } + this.baseDirectory = URI.create(baseDirectory); + this.exchangeEncryptionEnabled = fileSystemExchangeConfig.isExchangeEncryptionEnabled(); + this.exchangeSinkBufferPoolMinSize = fileSystemExchangeConfig.getExchangeSinkBufferPoolMinSize(); + this.executor = newCachedThreadPool(daemonThreadsNamed("exchange-source-handles-creation-%s")); + } + + @Override + public Exchange createExchange(ExchangeContext context, int outputPartitionCount) + { + Optional secretKey = Optional.empty(); + if (exchangeEncryptionEnabled) { + try { + KeyGenerator keyGenerator = KeyGenerator.getInstance("AES"); + keyGenerator.init(KEY_BITS); + secretKey = Optional.of(keyGenerator.generateKey()); + } + catch (NoSuchAlgorithmException e) { + throw new TrinoException(GENERIC_INTERNAL_ERROR, "Failed to generate new secret key: " + e.getMessage(), e); + } + } + FileSystemExchange exchange = new FileSystemExchange(baseDirectory, exchangeStorage, context, outputPartitionCount, secretKey, executor); + exchange.initialize(); + return exchange; + } + + @Override + public ExchangeSink createSink(ExchangeSinkInstanceHandle handle, boolean preserveRecordsOrder) + { + FileSystemExchangeSinkInstanceHandle instanceHandle = (FileSystemExchangeSinkInstanceHandle) handle; + return new FileSystemExchangeSink( + exchangeStorage, + instanceHandle.getOutputDirectory(), + instanceHandle.getOutputPartitionCount(), + instanceHandle.getSinkHandle().getSecretKey().map(key -> new SecretKeySpec(key, 0, key.length, "AES")), + exchangeSinkBufferPoolMinSize); + } + + @Override + public ExchangeSource createSource(List handles) + { + List files = handles.stream() + .map(FileSystemExchangeSourceHandle.class::cast) + .flatMap(handle -> handle.getFiles().stream()) + .map(fileStatus -> URI.create(fileStatus.getFilePath())) + .collect(toImmutableList()); + ImmutableList.Builder> secretKeys = ImmutableList.builder(); + for (ExchangeSourceHandle handle : handles) { + FileSystemExchangeSourceHandle sourceHandle = (FileSystemExchangeSourceHandle) handle; + secretKeys.addAll(nCopies(sourceHandle.getFiles().size(), sourceHandle.getSecretKey().map(key -> new SecretKeySpec(key, 0, key.length, "AES")))); + } + return new FileSystemExchangeSource(exchangeStorage, files, secretKeys.build()); + } +} diff --git a/core/trino-main/src/main/java/io/trino/server/testing/exchange/LocalFileSystemExchangeManagerFactory.java b/plugin/trino-exchange/src/main/java/io/trino/plugin/exchange/FileSystemExchangeManagerFactory.java similarity index 59% rename from core/trino-main/src/main/java/io/trino/server/testing/exchange/LocalFileSystemExchangeManagerFactory.java rename to plugin/trino-exchange/src/main/java/io/trino/plugin/exchange/FileSystemExchangeManagerFactory.java index 5b288dffe10d..e68956f101f4 100644 --- a/core/trino-main/src/main/java/io/trino/server/testing/exchange/LocalFileSystemExchangeManagerFactory.java +++ b/plugin/trino-exchange/src/main/java/io/trino/plugin/exchange/FileSystemExchangeManagerFactory.java @@ -11,49 +11,42 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.server.testing.exchange; +package io.trino.plugin.exchange; +import com.google.inject.Injector; +import io.airlift.bootstrap.Bootstrap; import io.trino.spi.exchange.ExchangeManager; import io.trino.spi.exchange.ExchangeManagerFactory; import io.trino.spi.exchange.ExchangeManagerHandleResolver; import io.trino.spi.exchange.ExchangeSinkInstanceHandle; import io.trino.spi.exchange.ExchangeSourceHandle; -import java.io.IOException; -import java.io.UncheckedIOException; -import java.nio.file.Files; -import java.nio.file.Path; -import java.nio.file.Paths; import java.util.Map; -public class LocalFileSystemExchangeManagerFactory +import static java.util.Objects.requireNonNull; + +public class FileSystemExchangeManagerFactory implements ExchangeManagerFactory { - private static final String BASE_DIRECTORY_PROPERTY = "base-directory"; - @Override public String getName() { - return "local"; + return "filesystem"; } @Override public ExchangeManager create(Map config) { - String configuredBaseDirectory = config.get(BASE_DIRECTORY_PROPERTY); - Path baseDirectory; - if (configuredBaseDirectory != null) { - baseDirectory = Paths.get(configuredBaseDirectory); - } - else { - try { - baseDirectory = Files.createTempDirectory("exchange-manager-"); - } - catch (IOException e) { - throw new UncheckedIOException(e); - } - } - return new LocalFileSystemExchangeManager(baseDirectory); + requireNonNull(config, "config is null"); + + Bootstrap app = new Bootstrap(new FileSystemExchangeModule()); + + Injector injector = app + .doNotInitializeLogging() + .setRequiredConfigurationProperties(config) + .initialize(); + + return injector.getInstance(FileSystemExchangeManager.class); } @Override @@ -64,13 +57,13 @@ public ExchangeManagerHandleResolver getHandleResolver() @Override public Class getExchangeSinkInstanceHandleClass() { - return LocalFileSystemExchangeSinkInstanceHandle.class; + return FileSystemExchangeSinkInstanceHandle.class; } @Override public Class getExchangeSourceHandleHandleClass() { - return LocalFileSystemExchangeSourceHandle.class; + return FileSystemExchangeSourceHandle.class; } }; } diff --git a/plugin/trino-exchange/src/main/java/io/trino/plugin/exchange/FileSystemExchangeModule.java b/plugin/trino-exchange/src/main/java/io/trino/plugin/exchange/FileSystemExchangeModule.java new file mode 100644 index 000000000000..e1db43be9c24 --- /dev/null +++ b/plugin/trino-exchange/src/main/java/io/trino/plugin/exchange/FileSystemExchangeModule.java @@ -0,0 +1,53 @@ +/* + * Licensed 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 io.trino.plugin.exchange; + +import com.google.common.collect.ImmutableSet; +import com.google.inject.Binder; +import com.google.inject.Scopes; +import io.airlift.configuration.AbstractConfigurationAwareModule; +import io.trino.plugin.exchange.local.LocalFileSystemExchangeStorage; +import io.trino.plugin.exchange.s3.ExchangeS3Config; +import io.trino.plugin.exchange.s3.S3FileSystemExchangeStorage; +import io.trino.spi.TrinoException; + +import java.net.URI; + +import static io.airlift.configuration.ConfigBinder.configBinder; +import static io.trino.spi.StandardErrorCode.NOT_SUPPORTED; +import static java.lang.String.format; +import static java.util.Objects.requireNonNull; + +public class FileSystemExchangeModule + extends AbstractConfigurationAwareModule +{ + @Override + protected void setup(Binder binder) + { + binder.bind(FileSystemExchangeManager.class).in(Scopes.SINGLETON); + + FileSystemExchangeConfig fileSystemExchangeConfig = buildConfigObject(FileSystemExchangeConfig.class); + String scheme = URI.create(requireNonNull(fileSystemExchangeConfig.getBaseDirectory(), "baseDirectory is null")).getScheme(); + if (scheme == null || scheme.equals("file")) { + binder.bind(FileSystemExchangeStorage.class).to(LocalFileSystemExchangeStorage.class).in(Scopes.SINGLETON); + } + else if (ImmutableSet.of("s3", "s3a", "s3n").contains(scheme)) { + binder.bind(FileSystemExchangeStorage.class).to(S3FileSystemExchangeStorage.class).in(Scopes.SINGLETON); + configBinder(binder).bindConfig(ExchangeS3Config.class); + } + else { + throw new TrinoException(NOT_SUPPORTED, format("Scheme %s is not supported as exchange spooling storage", scheme)); + } + } +} diff --git a/plugin/trino-exchange/src/main/java/io/trino/plugin/exchange/FileSystemExchangePlugin.java b/plugin/trino-exchange/src/main/java/io/trino/plugin/exchange/FileSystemExchangePlugin.java new file mode 100644 index 000000000000..04e5b08dc19c --- /dev/null +++ b/plugin/trino-exchange/src/main/java/io/trino/plugin/exchange/FileSystemExchangePlugin.java @@ -0,0 +1,28 @@ +/* + * Licensed 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 io.trino.plugin.exchange; + +import com.google.common.collect.ImmutableList; +import io.trino.spi.Plugin; +import io.trino.spi.exchange.ExchangeManagerFactory; + +public class FileSystemExchangePlugin + implements Plugin +{ + @Override + public Iterable getExchangeManagerFactories() + { + return ImmutableList.of(new FileSystemExchangeManagerFactory()); + } +} diff --git a/plugin/trino-exchange/src/main/java/io/trino/plugin/exchange/FileSystemExchangeSink.java b/plugin/trino-exchange/src/main/java/io/trino/plugin/exchange/FileSystemExchangeSink.java new file mode 100644 index 000000000000..d1cad117b24a --- /dev/null +++ b/plugin/trino-exchange/src/main/java/io/trino/plugin/exchange/FileSystemExchangeSink.java @@ -0,0 +1,374 @@ +/* + * Licensed 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 io.trino.plugin.exchange; + +import com.google.common.util.concurrent.FutureCallback; +import com.google.common.util.concurrent.Futures; +import com.google.common.util.concurrent.ListenableFuture; +import io.airlift.slice.SizeOf; +import io.airlift.slice.Slice; +import io.airlift.slice.SliceOutput; +import io.airlift.slice.Slices; +import io.trino.spi.exchange.ExchangeSink; +import org.openjdk.jol.info.ClassLayout; + +import javax.annotation.concurrent.GuardedBy; +import javax.annotation.concurrent.ThreadSafe; +import javax.crypto.SecretKey; + +import java.io.IOException; +import java.io.UncheckedIOException; +import java.net.URI; +import java.util.ArrayDeque; +import java.util.Map; +import java.util.Optional; +import java.util.Queue; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.atomic.AtomicReference; + +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.base.Throwables.throwIfUnchecked; +import static com.google.common.collect.ImmutableList.toImmutableList; +import static com.google.common.util.concurrent.MoreExecutors.directExecutor; +import static io.airlift.concurrent.MoreFutures.addExceptionCallback; +import static io.airlift.concurrent.MoreFutures.addSuccessCallback; +import static io.airlift.concurrent.MoreFutures.asVoid; +import static io.airlift.concurrent.MoreFutures.toCompletableFuture; +import static io.airlift.slice.SizeOf.estimatedSizeOf; +import static java.lang.Math.max; +import static java.lang.Math.min; +import static java.util.Objects.requireNonNull; +import static java.util.concurrent.CompletableFuture.completedFuture; +import static java.util.concurrent.CompletableFuture.failedFuture; + +@ThreadSafe +public class FileSystemExchangeSink + implements ExchangeSink +{ + public static final String COMMITTED_MARKER_FILE_NAME = "committed"; + public static final String DATA_FILE_SUFFIX = ".data"; + + private static final int INSTANCE_SIZE = ClassLayout.parseClass(FileSystemExchangeSink.class).instanceSize(); + + private final FileSystemExchangeStorage exchangeStorage; + private final URI outputDirectory; + private final int outputPartitionCount; + private final Optional secretKey; + private final BufferPool bufferPool; + + private final Map writersMap = new ConcurrentHashMap<>(); + private final AtomicReference failure = new AtomicReference<>(); + private volatile boolean closed; + + public FileSystemExchangeSink( + FileSystemExchangeStorage exchangeStorage, + URI outputDirectory, + int outputPartitionCount, + Optional secretKey, + int exchangeSinkBufferPoolMinSize) + { + this.exchangeStorage = requireNonNull(exchangeStorage, "exchangeStorage is null"); + this.outputDirectory = requireNonNull(outputDirectory, "outputDirectory is null"); + this.outputPartitionCount = outputPartitionCount; + this.secretKey = requireNonNull(secretKey, "secretKey is null"); + // double buffering to overlap computation and I/O + this.bufferPool = new BufferPool(max(outputPartitionCount * 2, exchangeSinkBufferPoolMinSize), exchangeStorage.getWriteBufferSize()); + } + + // The future returned by {@link #isBlocked()} should only be considered as a best-effort hint. + @Override + public CompletableFuture isBlocked() + { + return bufferPool.isBlocked(); + } + + @Override + public void add(int partitionId, Slice data) + { + throwIfFailed(); + + checkArgument(partitionId < outputPartitionCount, "partition id is expected to be less than %s: %s", outputPartitionCount, partitionId); + + // Ensure no new writers can be created after `closed` is set to true + BufferedStorageWriter writer; + synchronized (this) { + if (closed) { + return; + } + writer = writersMap.computeIfAbsent(partitionId, this::createWriter); + } + writer.write(data); + } + + private BufferedStorageWriter createWriter(int partitionId) + { + URI outputPath = outputDirectory.resolve(partitionId + DATA_FILE_SUFFIX); + try { + return new BufferedStorageWriter(exchangeStorage.createExchangeStorageWriter(outputPath, secretKey), bufferPool, failure); + } + catch (IOException e) { + throw new UncheckedIOException(e); + } + } + + @Override + public long getMemoryUsage() + { + return INSTANCE_SIZE + + bufferPool.getRetainedSize() + + estimatedSizeOf(writersMap, SizeOf::sizeOf, BufferedStorageWriter::getRetainedSize); + } + + @Override + public synchronized CompletableFuture finish() + { + if (closed) { + return failedFuture(new IllegalStateException("Exchange sink has already closed")); + } + + ListenableFuture finishFuture = asVoid(Futures.allAsList( + writersMap.values().stream().map(BufferedStorageWriter::finish).collect(toImmutableList()))); + addSuccessCallback(finishFuture, this::destroy); + finishFuture = Futures.transformAsync( + finishFuture, + ignored -> exchangeStorage.createEmptyFile(outputDirectory.resolve(COMMITTED_MARKER_FILE_NAME)), + directExecutor()); + Futures.addCallback(finishFuture, new FutureCallback<>() + { + @Override + public void onSuccess(Void result) + { + closed = true; + } + + @Override + public void onFailure(Throwable ignored) + { + abort(); + } + }, directExecutor()); + + return toCompletableFuture(finishFuture); + } + + @Override + public synchronized CompletableFuture abort() + { + if (closed) { + return completedFuture(null); + } + closed = true; + + ListenableFuture abortFuture = asVoid(Futures.allAsList( + writersMap.values().stream().map(BufferedStorageWriter::abort).collect(toImmutableList()))); + addSuccessCallback(abortFuture, this::destroy); + + return toCompletableFuture(Futures.transformAsync( + abortFuture, + ignored -> exchangeStorage.deleteRecursively(outputDirectory), + directExecutor())); + } + + private void throwIfFailed() + { + Throwable throwable = failure.get(); + if (throwable != null) { + throwIfUnchecked(throwable); + throw new RuntimeException(throwable); + } + } + + private void destroy() + { + writersMap.clear(); + bufferPool.close(); + } + + @ThreadSafe + private static class BufferedStorageWriter + { + private static final int INSTANCE_SIZE = ClassLayout.parseClass(BufferedStorageWriter.class).instanceSize(); + + private final ExchangeStorageWriter storageWriter; + private final BufferPool bufferPool; + private final AtomicReference failure; + + @GuardedBy("this") + private SliceOutput currentBuffer; + + public BufferedStorageWriter(ExchangeStorageWriter storageWriter, BufferPool bufferPool, AtomicReference failure) + { + this.storageWriter = requireNonNull(storageWriter, "storageWriter is null"); + this.bufferPool = requireNonNull(bufferPool, "bufferPool is null"); + this.failure = requireNonNull(failure, "failure is null"); + } + + public synchronized void write(Slice data) + { + writeInternal(Slices.wrappedIntArray(data.length())); + writeInternal(data); + } + + public synchronized ListenableFuture finish() + { + flushIfNeeded(true); + return storageWriter.finish(); + } + + public synchronized ListenableFuture abort() + { + return storageWriter.abort(); + } + + public synchronized long getRetainedSize() + { + return INSTANCE_SIZE + storageWriter.getRetainedSize(); + } + + private void writeInternal(Slice slice) + { + int position = 0; + while (position < slice.length()) { + if (currentBuffer == null) { + currentBuffer = bufferPool.take(); + if (currentBuffer == null) { + // buffer pool is closed + return; + } + } + int writableBytes = min(currentBuffer.writableBytes(), slice.length() - position); + currentBuffer.writeBytes(slice.getBytes(position, writableBytes)); + position += writableBytes; + + flushIfNeeded(false); + } + } + + private void flushIfNeeded(boolean finished) + { + SliceOutput buffer = currentBuffer; + if (!buffer.isWritable() || finished) { + if (!buffer.isWritable()) { + currentBuffer = null; + } + ListenableFuture writeFuture = storageWriter.write(buffer.slice()); + writeFuture.addListener(() -> bufferPool.offer(buffer), directExecutor()); + addExceptionCallback(writeFuture, throwable -> failure.compareAndSet(null, throwable)); + } + } + } + + @ThreadSafe + private static class BufferPool + { + private static final int INSTANCE_SIZE = ClassLayout.parseClass(BufferPool.class).instanceSize(); + + private final int numBuffers; + private final long bufferRetainedSize; + @GuardedBy("this") + private final Queue freeBuffersQueue; + @GuardedBy("this") + private CompletableFuture blockedFuture = new CompletableFuture<>(); + @GuardedBy("this") + private boolean closed; + + public BufferPool(int numBuffers, int writeBufferSize) + { + checkArgument(numBuffers >= 1, "numBuffers must be at least one"); + + this.numBuffers = numBuffers; + this.freeBuffersQueue = new ArrayDeque<>(numBuffers); + for (int i = 0; i < numBuffers; ++i) { + freeBuffersQueue.add(Slices.allocate(writeBufferSize).getOutput()); + } + this.bufferRetainedSize = freeBuffersQueue.peek().getRetainedSize(); + } + + public synchronized CompletableFuture isBlocked() + { + if (freeBuffersQueue.isEmpty()) { + if (blockedFuture.isDone()) { + blockedFuture = new CompletableFuture<>(); + } + return blockedFuture; + } + else { + return NOT_BLOCKED; + } + } + + public synchronized SliceOutput take() + { + while (true) { + if (closed) { + return null; + } + if (!freeBuffersQueue.isEmpty()) { + return freeBuffersQueue.poll(); + } + try { + wait(); + } + catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new RuntimeException(e); + } + } + } + + public void offer(SliceOutput buffer) + { + buffer.reset(); + + CompletableFuture completableFuture; + synchronized (this) { + if (closed) { + return; + } + completableFuture = blockedFuture; + freeBuffersQueue.add(buffer); + notify(); + } + + completableFuture.complete(null); + } + + public synchronized long getRetainedSize() + { + if (closed) { + return INSTANCE_SIZE; + } + else { + return INSTANCE_SIZE + numBuffers * bufferRetainedSize; + } + } + + public void close() + { + CompletableFuture completableFuture; + synchronized (this) { + if (closed) { + return; + } + closed = true; + notifyAll(); + completableFuture = blockedFuture; + freeBuffersQueue.clear(); + } + + completableFuture.complete(null); + } + } +} diff --git a/core/trino-main/src/main/java/io/trino/server/testing/exchange/LocalFileSystemExchangeSinkHandle.java b/plugin/trino-exchange/src/main/java/io/trino/plugin/exchange/FileSystemExchangeSinkHandle.java similarity index 50% rename from core/trino-main/src/main/java/io/trino/server/testing/exchange/LocalFileSystemExchangeSinkHandle.java rename to plugin/trino-exchange/src/main/java/io/trino/plugin/exchange/FileSystemExchangeSinkHandle.java index 48829306c73f..3704227f3dfe 100644 --- a/core/trino-main/src/main/java/io/trino/server/testing/exchange/LocalFileSystemExchangeSinkHandle.java +++ b/plugin/trino-exchange/src/main/java/io/trino/plugin/exchange/FileSystemExchangeSinkHandle.java @@ -11,53 +11,44 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.server.testing.exchange; +package io.trino.plugin.exchange; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import io.trino.spi.QueryId; -import io.trino.spi.exchange.ExchangeId; import io.trino.spi.exchange.ExchangeSinkHandle; +import java.util.Arrays; import java.util.Objects; +import java.util.Optional; import static com.google.common.base.MoreObjects.toStringHelper; import static java.util.Objects.requireNonNull; -public class LocalFileSystemExchangeSinkHandle +public class FileSystemExchangeSinkHandle implements ExchangeSinkHandle { - private final QueryId queryId; - private final ExchangeId exchangeId; - private final int taskPartitionId; + private final int partitionId; + private final Optional secretKey; @JsonCreator - public LocalFileSystemExchangeSinkHandle( - @JsonProperty("queryId") QueryId queryId, - @JsonProperty("exchangeId") ExchangeId exchangeId, - @JsonProperty("taskPartitionId") int taskPartitionId) + public FileSystemExchangeSinkHandle( + @JsonProperty("partitionId") int partitionId, + @JsonProperty("secretKey") Optional secretKey) { - this.queryId = requireNonNull(queryId, "queryId is null"); - this.exchangeId = requireNonNull(exchangeId, "exchangeId is null"); - this.taskPartitionId = taskPartitionId; + this.partitionId = partitionId; + this.secretKey = requireNonNull(secretKey, "secretKey is null"); } @JsonProperty - public QueryId getQueryId() + public int getPartitionId() { - return queryId; + return partitionId; } @JsonProperty - public ExchangeId getExchangeId() + public Optional getSecretKey() { - return exchangeId; - } - - @JsonProperty - public int getTaskPartitionId() - { - return taskPartitionId; + return secretKey; } @Override @@ -69,23 +60,27 @@ public boolean equals(Object o) if (o == null || getClass() != o.getClass()) { return false; } - LocalFileSystemExchangeSinkHandle that = (LocalFileSystemExchangeSinkHandle) o; - return taskPartitionId == that.taskPartitionId && Objects.equals(queryId, that.queryId) && Objects.equals(exchangeId, that.exchangeId); + FileSystemExchangeSinkHandle that = (FileSystemExchangeSinkHandle) o; + if (secretKey.isPresent() && that.secretKey.isPresent()) { + return partitionId == that.getPartitionId() && Arrays.equals(secretKey.get(), that.secretKey.get()); + } + else { + return partitionId == that.getPartitionId() && secretKey.isEmpty() && that.secretKey.isEmpty(); + } } @Override public int hashCode() { - return Objects.hash(queryId, exchangeId, taskPartitionId); + return Objects.hash(partitionId, secretKey); } @Override public String toString() { return toStringHelper(this) - .add("queryId", queryId) - .add("exchangeId", exchangeId) - .add("taskPartitionId", taskPartitionId) + .add("partitionId", partitionId) + .add("secretKey", secretKey.map(value -> "[REDACTED]")) .toString(); } } diff --git a/core/trino-main/src/main/java/io/trino/server/testing/exchange/LocalFileSystemExchangeSinkInstanceHandle.java b/plugin/trino-exchange/src/main/java/io/trino/plugin/exchange/FileSystemExchangeSinkInstanceHandle.java similarity index 72% rename from core/trino-main/src/main/java/io/trino/server/testing/exchange/LocalFileSystemExchangeSinkInstanceHandle.java rename to plugin/trino-exchange/src/main/java/io/trino/plugin/exchange/FileSystemExchangeSinkInstanceHandle.java index 0a99ea445571..0335c5716cb7 100644 --- a/core/trino-main/src/main/java/io/trino/server/testing/exchange/LocalFileSystemExchangeSinkInstanceHandle.java +++ b/plugin/trino-exchange/src/main/java/io/trino/plugin/exchange/FileSystemExchangeSinkInstanceHandle.java @@ -11,27 +11,27 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.server.testing.exchange; +package io.trino.plugin.exchange; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import io.trino.spi.exchange.ExchangeSinkInstanceHandle; -import java.nio.file.Path; +import java.net.URI; import static java.util.Objects.requireNonNull; -public class LocalFileSystemExchangeSinkInstanceHandle +public class FileSystemExchangeSinkInstanceHandle implements ExchangeSinkInstanceHandle { - private final LocalFileSystemExchangeSinkHandle sinkHandle; - private final Path outputDirectory; + private final FileSystemExchangeSinkHandle sinkHandle; + private final URI outputDirectory; private final int outputPartitionCount; @JsonCreator - public LocalFileSystemExchangeSinkInstanceHandle( - @JsonProperty("sinkHandle") LocalFileSystemExchangeSinkHandle sinkHandle, - @JsonProperty("outputDirectory") Path outputDirectory, + public FileSystemExchangeSinkInstanceHandle( + @JsonProperty("sinkHandle") FileSystemExchangeSinkHandle sinkHandle, + @JsonProperty("outputDirectory") URI outputDirectory, @JsonProperty("outputPartitionCount") int outputPartitionCount) { this.sinkHandle = requireNonNull(sinkHandle, "sinkHandle is null"); @@ -40,13 +40,13 @@ public LocalFileSystemExchangeSinkInstanceHandle( } @JsonProperty - public LocalFileSystemExchangeSinkHandle getSinkHandle() + public FileSystemExchangeSinkHandle getSinkHandle() { return sinkHandle; } @JsonProperty - public Path getOutputDirectory() + public URI getOutputDirectory() { return outputDirectory; } diff --git a/core/trino-main/src/main/java/io/trino/server/testing/exchange/LocalFileSystemExchangeSource.java b/plugin/trino-exchange/src/main/java/io/trino/plugin/exchange/FileSystemExchangeSource.java similarity index 65% rename from core/trino-main/src/main/java/io/trino/server/testing/exchange/LocalFileSystemExchangeSource.java rename to plugin/trino-exchange/src/main/java/io/trino/plugin/exchange/FileSystemExchangeSource.java index 3c9d2c721310..dfb2bbbd5ece 100644 --- a/core/trino-main/src/main/java/io/trino/server/testing/exchange/LocalFileSystemExchangeSource.java +++ b/plugin/trino-exchange/src/main/java/io/trino/plugin/exchange/FileSystemExchangeSource.java @@ -11,46 +11,50 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.server.testing.exchange; +package io.trino.plugin.exchange; import com.google.common.collect.ImmutableList; -import io.airlift.slice.InputStreamSliceInput; import io.airlift.slice.Slice; import io.airlift.slice.SliceInput; -import io.airlift.units.DataSize; import io.trino.spi.exchange.ExchangeSource; import javax.annotation.Nullable; import javax.annotation.concurrent.GuardedBy; +import javax.crypto.SecretKey; -import java.io.FileInputStream; -import java.io.FileNotFoundException; +import java.io.IOException; import java.io.UncheckedIOException; -import java.nio.file.Path; +import java.net.URI; import java.util.Iterator; import java.util.List; +import java.util.Optional; import java.util.concurrent.CompletableFuture; -import static io.airlift.units.DataSize.Unit.KILOBYTE; -import static java.lang.Math.toIntExact; +import static com.google.common.base.Preconditions.checkArgument; +import static java.lang.String.format; import static java.util.Objects.requireNonNull; -public class LocalFileSystemExchangeSource +public class FileSystemExchangeSource implements ExchangeSource { - private static final int BUFFER_SIZE_IN_BYTES = toIntExact(DataSize.of(4, KILOBYTE).toBytes()); - + private final FileSystemExchangeStorage exchangeStorage; + @GuardedBy("this") + private final Iterator files; @GuardedBy("this") - private final Iterator files; + private final Iterator> secretKeys; @GuardedBy("this") private SliceInput sliceInput; @GuardedBy("this") private boolean closed; - public LocalFileSystemExchangeSource(List files) + public FileSystemExchangeSource(FileSystemExchangeStorage exchangeStorage, List files, List> secretKeys) { - this.files = ImmutableList.copyOf(requireNonNull(files, "files is null")).iterator(); + this.exchangeStorage = requireNonNull(exchangeStorage, "exchangeStorage is null"); + checkArgument(requireNonNull(files, "files is null").size() == requireNonNull(secretKeys, "secretKeys is null").size(), + format("number of files (%d) doesn't match number of secretKeys (%d)", files.size(), secretKeys.size())); + this.files = ImmutableList.copyOf(files).iterator(); + this.secretKeys = ImmutableList.copyOf(secretKeys).iterator(); } @Override @@ -80,11 +84,13 @@ public synchronized Slice read() if (sliceInput == null) { if (files.hasNext()) { - Path file = files.next(); + // TODO: implement parallel read + URI file = files.next(); + Optional secretKey = secretKeys.next(); try { - sliceInput = new InputStreamSliceInput(new FileInputStream(file.toFile()), BUFFER_SIZE_IN_BYTES); + sliceInput = exchangeStorage.getSliceInput(file, secretKey); } - catch (FileNotFoundException e) { + catch (IOException e) { throw new UncheckedIOException(e); } } diff --git a/core/trino-main/src/main/java/io/trino/server/testing/exchange/LocalFileSystemExchangeSourceHandle.java b/plugin/trino-exchange/src/main/java/io/trino/plugin/exchange/FileSystemExchangeSourceHandle.java similarity index 59% rename from core/trino-main/src/main/java/io/trino/server/testing/exchange/LocalFileSystemExchangeSourceHandle.java rename to plugin/trino-exchange/src/main/java/io/trino/plugin/exchange/FileSystemExchangeSourceHandle.java index ee3f9ec0dc02..0ae44e542f4b 100644 --- a/core/trino-main/src/main/java/io/trino/server/testing/exchange/LocalFileSystemExchangeSourceHandle.java +++ b/plugin/trino-exchange/src/main/java/io/trino/plugin/exchange/FileSystemExchangeSourceHandle.java @@ -11,7 +11,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.server.testing.exchange; +package io.trino.plugin.exchange; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; @@ -20,26 +20,34 @@ import io.trino.spi.exchange.ExchangeSourceHandle; import org.openjdk.jol.info.ClassLayout; +import java.util.Arrays; import java.util.List; import java.util.Objects; +import java.util.Optional; import static com.google.common.base.MoreObjects.toStringHelper; import static io.airlift.slice.SizeOf.estimatedSizeOf; +import static io.airlift.slice.SizeOf.sizeOf; import static java.util.Objects.requireNonNull; -public class LocalFileSystemExchangeSourceHandle +public class FileSystemExchangeSourceHandle implements ExchangeSourceHandle { - private static final int INSTANCE_SIZE = ClassLayout.parseClass(LocalFileSystemExchangeSourceHandle.class).instanceSize(); + private static final int INSTANCE_SIZE = ClassLayout.parseClass(FileSystemExchangeSourceHandle.class).instanceSize(); private final int partitionId; - private final List files; + private final List files; + private final Optional secretKey; @JsonCreator - public LocalFileSystemExchangeSourceHandle(@JsonProperty("partitionId") int partitionId, @JsonProperty("files") List files) + public FileSystemExchangeSourceHandle( + @JsonProperty("partitionId") int partitionId, + @JsonProperty("files") List files, + @JsonProperty("secretKey") Optional secretKey) { this.partitionId = partitionId; this.files = ImmutableList.copyOf(requireNonNull(files, "files is null")); + this.secretKey = requireNonNull(secretKey, "secretKey is null"); } @Override @@ -49,17 +57,24 @@ public int getPartitionId() return partitionId; } + @Override + public long getRetainedSizeInBytes() + { + return INSTANCE_SIZE + + estimatedSizeOf(files, FileStatus::getRetainedSizeInBytes) + + sizeOf(secretKey, SizeOf::sizeOf); + } + @JsonProperty - public List getFiles() + public List getFiles() { return files; } - @Override - public long getRetainedSizeInBytes() + @JsonProperty + public Optional getSecretKey() { - return INSTANCE_SIZE - + estimatedSizeOf(files, SizeOf::estimatedSizeOf); + return secretKey; } @Override @@ -71,14 +86,19 @@ public boolean equals(Object o) if (o == null || getClass() != o.getClass()) { return false; } - LocalFileSystemExchangeSourceHandle that = (LocalFileSystemExchangeSourceHandle) o; - return partitionId == that.partitionId && Objects.equals(files, that.files); + FileSystemExchangeSourceHandle that = (FileSystemExchangeSourceHandle) o; + if (secretKey.isPresent() && that.secretKey.isPresent()) { + return partitionId == that.getPartitionId() && Arrays.equals(secretKey.get(), that.secretKey.get()); + } + else { + return partitionId == that.getPartitionId() && secretKey.isEmpty() && that.secretKey.isEmpty(); + } } @Override public int hashCode() { - return Objects.hash(partitionId, files); + return Objects.hash(partitionId, files, secretKey); } @Override @@ -87,6 +107,7 @@ public String toString() return toStringHelper(this) .add("partitionId", partitionId) .add("files", files) + .add("secretKey", secretKey.map(value -> "[REDACTED]")) .toString(); } } diff --git a/plugin/trino-exchange/src/main/java/io/trino/plugin/exchange/FileSystemExchangeStorage.java b/plugin/trino-exchange/src/main/java/io/trino/plugin/exchange/FileSystemExchangeStorage.java new file mode 100644 index 000000000000..d519817f7653 --- /dev/null +++ b/plugin/trino-exchange/src/main/java/io/trino/plugin/exchange/FileSystemExchangeStorage.java @@ -0,0 +1,49 @@ +package io.trino.plugin.exchange; +/* + * Licensed 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. + */ + +import com.google.common.util.concurrent.ListenableFuture; +import io.airlift.slice.SliceInput; + +import javax.crypto.SecretKey; + +import java.io.IOException; +import java.net.URI; +import java.util.List; +import java.util.Optional; + +public interface FileSystemExchangeStorage + extends AutoCloseable +{ + void createDirectories(URI dir) throws IOException; + + SliceInput getSliceInput(URI file, Optional secretKey) throws IOException; + + ExchangeStorageWriter createExchangeStorageWriter(URI file, Optional secretKey) throws IOException; + + boolean exists(URI file) throws IOException; + + ListenableFuture createEmptyFile(URI file); + + ListenableFuture deleteRecursively(URI dir); + + List listFiles(URI dir) throws IOException; + + List listDirectories(URI dir) throws IOException; + + int getWriteBufferSize(); + + @Override + void close() throws IOException; +} diff --git a/plugin/trino-exchange/src/main/java/io/trino/plugin/exchange/local/LocalFileSystemExchangeStorage.java b/plugin/trino-exchange/src/main/java/io/trino/plugin/exchange/local/LocalFileSystemExchangeStorage.java new file mode 100644 index 000000000000..bcb9155173b5 --- /dev/null +++ b/plugin/trino-exchange/src/main/java/io/trino/plugin/exchange/local/LocalFileSystemExchangeStorage.java @@ -0,0 +1,237 @@ +/* + * Licensed 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 io.trino.plugin.exchange.local; + +import com.google.common.collect.ImmutableList; +import com.google.common.io.MoreFiles; +import com.google.common.util.concurrent.ListenableFuture; +import io.airlift.slice.InputStreamSliceInput; +import io.airlift.slice.Slice; +import io.airlift.slice.SliceInput; +import io.airlift.units.DataSize; +import io.trino.plugin.exchange.ExchangeStorageWriter; +import io.trino.plugin.exchange.FileStatus; +import io.trino.plugin.exchange.FileSystemExchangeStorage; +import io.trino.spi.TrinoException; +import org.openjdk.jol.info.ClassLayout; + +import javax.crypto.Cipher; +import javax.crypto.CipherInputStream; +import javax.crypto.CipherOutputStream; +import javax.crypto.NoSuchPaddingException; +import javax.crypto.SecretKey; + +import java.io.FileInputStream; +import java.io.FileNotFoundException; +import java.io.FileOutputStream; +import java.io.IOException; +import java.io.OutputStream; +import java.net.URI; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.security.InvalidKeyException; +import java.security.NoSuchAlgorithmException; +import java.util.List; +import java.util.Optional; +import java.util.function.Predicate; +import java.util.stream.Stream; + +import static com.google.common.io.RecursiveDeleteOption.ALLOW_INSECURE; +import static com.google.common.util.concurrent.Futures.immediateFailedFuture; +import static com.google.common.util.concurrent.Futures.immediateVoidFuture; +import static io.airlift.units.DataSize.Unit.KILOBYTE; +import static io.trino.spi.StandardErrorCode.GENERIC_INTERNAL_ERROR; +import static java.lang.Math.toIntExact; +import static java.nio.file.Files.createFile; + +public class LocalFileSystemExchangeStorage + implements FileSystemExchangeStorage +{ + private static final int BUFFER_SIZE_IN_BYTES = toIntExact(DataSize.of(4, KILOBYTE).toBytes()); + + @Override + public void createDirectories(URI dir) + throws IOException + { + Files.createDirectories(Paths.get(dir.getPath())); + } + + @Override + public SliceInput getSliceInput(URI file, Optional secretKey) + throws IOException + { + if (secretKey.isPresent()) { + try { + Cipher cipher = Cipher.getInstance("AES"); + cipher.init(Cipher.DECRYPT_MODE, secretKey.get()); + return new InputStreamSliceInput(new CipherInputStream(new FileInputStream(Paths.get(file.getPath()).toFile()), cipher), BUFFER_SIZE_IN_BYTES); + } + catch (NoSuchAlgorithmException | NoSuchPaddingException | InvalidKeyException e) { + throw new TrinoException(GENERIC_INTERNAL_ERROR, "Failed to create CipherInputStream: " + e.getMessage(), e); + } + } + else { + return new InputStreamSliceInput(new FileInputStream(Paths.get(file.getPath()).toFile()), BUFFER_SIZE_IN_BYTES); + } + } + + @Override + public ExchangeStorageWriter createExchangeStorageWriter(URI file, Optional secretKey) + throws IOException + { + return new LocalExchangeStorageWriter(file, secretKey); + } + + @Override + public boolean exists(URI file) + { + return Files.exists(Paths.get(file.getPath())); + } + + @Override + public ListenableFuture createEmptyFile(URI file) + { + try { + createFile(Paths.get(file.getPath())); + } + catch (IOException | RuntimeException e) { + return immediateFailedFuture(e); + } + return immediateVoidFuture(); + } + + @Override + public ListenableFuture deleteRecursively(URI dir) + { + try { + MoreFiles.deleteRecursively(Paths.get(dir.getPath()), ALLOW_INSECURE); + } + catch (IOException | RuntimeException e) { + return immediateFailedFuture(e); + } + return immediateVoidFuture(); + } + + @Override + public List listFiles(URI dir) + throws IOException + { + ImmutableList.Builder builder = ImmutableList.builder(); + for (Path file : listPaths(dir, Files::isRegularFile)) { + builder.add(new FileStatus(file.toUri().toString(), Files.size(file))); + } + return builder.build(); + } + + @Override + public List listDirectories(URI dir) + throws IOException + { + ImmutableList.Builder builder = ImmutableList.builder(); + for (Path subDir : listPaths(dir, Files::isDirectory)) { + builder.add(subDir.toUri()); + } + return builder.build(); + } + + @Override + public int getWriteBufferSize() + { + return BUFFER_SIZE_IN_BYTES; + } + + @Override + public void close() + { + } + + private static List listPaths(URI directory, Predicate predicate) + throws IOException + { + ImmutableList.Builder builder = ImmutableList.builder(); + try (Stream dir = Files.list(Paths.get(directory.getPath()))) { + dir.filter(predicate).forEach(builder::add); + } + return builder.build(); + } + + private static class LocalExchangeStorageWriter + implements ExchangeStorageWriter + { + private static final int INSTANCE_SIZE = ClassLayout.parseClass(LocalExchangeStorageWriter.class).instanceSize(); + + private final OutputStream outputStream; + + public LocalExchangeStorageWriter(URI file, Optional secretKey) + throws FileNotFoundException + { + if (secretKey.isPresent()) { + try { + Cipher cipher = Cipher.getInstance("AES"); + cipher.init(Cipher.ENCRYPT_MODE, secretKey.get()); + this.outputStream = new CipherOutputStream(new FileOutputStream(Paths.get(file.getPath()).toFile()), cipher); + } + catch (NoSuchAlgorithmException | NoSuchPaddingException | InvalidKeyException e) { + throw new TrinoException(GENERIC_INTERNAL_ERROR, "Failed to create CipherOutputStream: " + e.getMessage(), e); + } + } + else { + this.outputStream = new FileOutputStream(Paths.get(file.getPath()).toFile()); + } + } + + @Override + public ListenableFuture write(Slice slice) + { + try { + outputStream.write(slice.getBytes()); + } + catch (IOException | RuntimeException e) { + return immediateFailedFuture(e); + } + return immediateVoidFuture(); + } + + @Override + public ListenableFuture finish() + { + try { + outputStream.close(); + } + catch (IOException | RuntimeException e) { + return immediateFailedFuture(e); + } + return immediateVoidFuture(); + } + + @Override + public ListenableFuture abort() + { + try { + outputStream.close(); + } + catch (IOException | RuntimeException e) { + return immediateFailedFuture(e); + } + return immediateVoidFuture(); + } + + @Override + public long getRetainedSize() + { + return INSTANCE_SIZE; + } + } +} diff --git a/plugin/trino-exchange/src/main/java/io/trino/plugin/exchange/s3/ByteBufferAsyncRequestBody.java b/plugin/trino-exchange/src/main/java/io/trino/plugin/exchange/s3/ByteBufferAsyncRequestBody.java new file mode 100644 index 000000000000..5d492e031590 --- /dev/null +++ b/plugin/trino-exchange/src/main/java/io/trino/plugin/exchange/s3/ByteBufferAsyncRequestBody.java @@ -0,0 +1,116 @@ +/* + * Licensed 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 io.trino.plugin.exchange.s3; + +import io.airlift.log.Logger; +import org.reactivestreams.Subscriber; +import org.reactivestreams.Subscription; +import software.amazon.awssdk.core.async.AsyncRequestBody; +import software.amazon.awssdk.core.internal.async.ByteArrayAsyncRequestBody; +import software.amazon.awssdk.core.internal.util.Mimetype; + +import java.nio.ByteBuffer; +import java.util.Optional; + +import static java.util.Objects.requireNonNull; + +/** + * This class mimics the implementation of {@link ByteArrayAsyncRequestBody} except for we use a ByteBuffer + * to avoid unnecessary memory copy + * + * An implementation of {@link AsyncRequestBody} for providing data from memory. This is created using static + * methods on {@link AsyncRequestBody} + * + * @see AsyncRequestBody#fromBytes(byte[]) + * @see AsyncRequestBody#fromByteBuffer(ByteBuffer) + * @see AsyncRequestBody#fromString(String) + */ +public final class ByteBufferAsyncRequestBody + implements AsyncRequestBody +{ + private static final Logger log = Logger.get(ByteBufferAsyncRequestBody.class); + + private final ByteBuffer byteBuffer; + + private final String mimetype; + + public ByteBufferAsyncRequestBody(ByteBuffer byteBuffer, String mimetype) + { + this.byteBuffer = requireNonNull(byteBuffer, "byteBuffer is null"); + this.mimetype = requireNonNull(mimetype, "mimetype is null"); + } + + @Override + public Optional contentLength() + { + return Optional.of((long) byteBuffer.remaining()); + } + + @Override + public String contentType() + { + return mimetype; + } + + @Override + public void subscribe(Subscriber s) + { + // As per rule 1.9 we must throw NullPointerException if the subscriber parameter is null + if (s == null) { + throw new NullPointerException("Subscription MUST NOT be null."); + } + + // As per 2.13, this method must return normally (i.e. not throw). + try { + s.onSubscribe( + new Subscription() { + private boolean done; + + @Override + public void request(long n) + { + if (done) { + return; + } + if (n > 0) { + done = true; + s.onNext(byteBuffer.asReadOnlyBuffer()); + s.onComplete(); + } + else { + s.onError(new IllegalArgumentException("§3.9: non-positive requests are not allowed!")); + } + } + + @Override + public void cancel() + { + synchronized (this) { + if (!done) { + done = true; + } + } + } + }); + } + catch (Throwable ex) { + log.error(ex, " violated the Reactive Streams rule 2.13 by throwing an exception from onSubscribe."); + } + } + + static AsyncRequestBody fromByteBuffer(ByteBuffer byteBuffer) + { + return new ByteBufferAsyncRequestBody(byteBuffer, Mimetype.MIMETYPE_OCTET_STREAM); + } +} diff --git a/plugin/trino-exchange/src/main/java/io/trino/plugin/exchange/s3/ExchangeS3Config.java b/plugin/trino-exchange/src/main/java/io/trino/plugin/exchange/s3/ExchangeS3Config.java new file mode 100644 index 000000000000..4d9186a7e557 --- /dev/null +++ b/plugin/trino-exchange/src/main/java/io/trino/plugin/exchange/s3/ExchangeS3Config.java @@ -0,0 +1,122 @@ +/* + * Licensed 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 io.trino.plugin.exchange.s3; + +import io.airlift.configuration.Config; +import io.airlift.configuration.ConfigDescription; +import io.airlift.configuration.ConfigSecuritySensitive; +import io.airlift.units.DataSize; +import io.airlift.units.MaxDataSize; +import io.airlift.units.MinDataSize; +import software.amazon.awssdk.regions.Region; + +import javax.validation.constraints.Min; +import javax.validation.constraints.NotNull; + +import java.util.Optional; + +import static io.airlift.units.DataSize.Unit.MEGABYTE; +import static java.util.Locale.ENGLISH; + +public class ExchangeS3Config +{ + private String s3AwsAccessKey; + private String s3AwsSecretKey; + private Optional s3Region = Optional.empty(); + private Optional s3Endpoint = Optional.empty(); + private int s3MaxErrorRetries = 3; + // Default to S3 multi-part upload minimum size to avoid excessive memory consumption from buffering + private DataSize s3UploadPartSize = DataSize.of(5, MEGABYTE); + + public String getS3AwsAccessKey() + { + return s3AwsAccessKey; + } + + @Config("exchange.s3.aws-access-key") + public ExchangeS3Config setS3AwsAccessKey(String s3AwsAccessKey) + { + this.s3AwsAccessKey = s3AwsAccessKey; + return this; + } + + public String getS3AwsSecretKey() + { + return s3AwsSecretKey; + } + + @Config("exchange.s3.aws-secret-key") + @ConfigSecuritySensitive + public ExchangeS3Config setS3AwsSecretKey(String s3AwsSecretKey) + { + this.s3AwsSecretKey = s3AwsSecretKey; + return this; + } + + public Optional getS3Region() + { + return s3Region; + } + + @Config("exchange.s3.region") + public ExchangeS3Config setS3Region(String s3Region) + { + if (s3Region != null) { + this.s3Region = Optional.of(Region.of(s3Region.toLowerCase(ENGLISH))); + } + + return this; + } + + public Optional getS3Endpoint() + { + return s3Endpoint; + } + + @Config("exchange.s3.endpoint") + public ExchangeS3Config setS3Endpoint(String s3Endpoint) + { + this.s3Endpoint = Optional.ofNullable(s3Endpoint); + return this; + } + + @Min(0) + public int getS3MaxErrorRetries() + { + return s3MaxErrorRetries; + } + + @Config("exchange.s3.max-error-retries") + public ExchangeS3Config setS3MaxErrorRetries(int s3MaxErrorRetries) + { + this.s3MaxErrorRetries = s3MaxErrorRetries; + return this; + } + + @NotNull + @MinDataSize("5MB") + @MaxDataSize("256MB") + public DataSize getS3UploadPartSize() + { + return s3UploadPartSize; + } + + @Config("exchange.s3.upload.part-size") + @ConfigDescription("Part size for S3 multi-part upload") + public ExchangeS3Config setS3UploadPartSize(DataSize s3UploadPartSize) + { + this.s3UploadPartSize = s3UploadPartSize; + return this; + } +} diff --git a/plugin/trino-exchange/src/main/java/io/trino/plugin/exchange/s3/S3FileSystemExchangeStorage.java b/plugin/trino-exchange/src/main/java/io/trino/plugin/exchange/s3/S3FileSystemExchangeStorage.java new file mode 100644 index 000000000000..cfd8cbcad4d9 --- /dev/null +++ b/plugin/trino-exchange/src/main/java/io/trino/plugin/exchange/s3/S3FileSystemExchangeStorage.java @@ -0,0 +1,558 @@ +/* + * Licensed 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 io.trino.plugin.exchange.s3; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Lists; +import com.google.common.io.Closer; +import com.google.common.util.concurrent.FutureCallback; +import com.google.common.util.concurrent.Futures; +import com.google.common.util.concurrent.ListenableFuture; +import io.airlift.slice.InputStreamSliceInput; +import io.airlift.slice.Slice; +import io.airlift.slice.SliceInput; +import io.trino.plugin.exchange.ExchangeStorageWriter; +import io.trino.plugin.exchange.FileStatus; +import io.trino.plugin.exchange.FileSystemExchangeStorage; +import org.openjdk.jol.info.ClassLayout; +import software.amazon.awssdk.auth.credentials.AwsBasicCredentials; +import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider; +import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider; +import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider; +import software.amazon.awssdk.core.async.AsyncRequestBody; +import software.amazon.awssdk.core.client.config.ClientOverrideConfiguration; +import software.amazon.awssdk.core.retry.RetryPolicy; +import software.amazon.awssdk.core.sync.ResponseTransformer; +import software.amazon.awssdk.regions.Region; +import software.amazon.awssdk.services.s3.S3AsyncClient; +import software.amazon.awssdk.services.s3.S3AsyncClientBuilder; +import software.amazon.awssdk.services.s3.S3Client; +import software.amazon.awssdk.services.s3.S3ClientBuilder; +import software.amazon.awssdk.services.s3.model.AbortMultipartUploadRequest; +import software.amazon.awssdk.services.s3.model.AbortMultipartUploadResponse; +import software.amazon.awssdk.services.s3.model.CommonPrefix; +import software.amazon.awssdk.services.s3.model.CompleteMultipartUploadRequest; +import software.amazon.awssdk.services.s3.model.CompleteMultipartUploadResponse; +import software.amazon.awssdk.services.s3.model.CompletedMultipartUpload; +import software.amazon.awssdk.services.s3.model.CompletedPart; +import software.amazon.awssdk.services.s3.model.CreateMultipartUploadRequest; +import software.amazon.awssdk.services.s3.model.CreateMultipartUploadResponse; +import software.amazon.awssdk.services.s3.model.Delete; +import software.amazon.awssdk.services.s3.model.DeleteObjectsRequest; +import software.amazon.awssdk.services.s3.model.DeleteObjectsResponse; +import software.amazon.awssdk.services.s3.model.GetObjectRequest; +import software.amazon.awssdk.services.s3.model.HeadObjectRequest; +import software.amazon.awssdk.services.s3.model.HeadObjectResponse; +import software.amazon.awssdk.services.s3.model.ListObjectsV2Request; +import software.amazon.awssdk.services.s3.model.NoSuchKeyException; +import software.amazon.awssdk.services.s3.model.ObjectIdentifier; +import software.amazon.awssdk.services.s3.model.PutObjectRequest; +import software.amazon.awssdk.services.s3.model.S3Object; +import software.amazon.awssdk.services.s3.model.UploadPartRequest; +import software.amazon.awssdk.services.s3.paginators.ListObjectsV2Iterable; +import software.amazon.awssdk.services.s3.paginators.ListObjectsV2Publisher; + +import javax.annotation.PreDestroy; +import javax.annotation.concurrent.NotThreadSafe; +import javax.crypto.SecretKey; +import javax.inject.Inject; + +import java.io.IOException; +import java.net.URI; +import java.net.URISyntaxException; +import java.util.ArrayList; +import java.util.List; +import java.util.Optional; + +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.base.Preconditions.checkState; +import static com.google.common.base.Strings.nullToEmpty; +import static com.google.common.base.Verify.verify; +import static com.google.common.collect.ImmutableList.toImmutableList; +import static com.google.common.util.concurrent.Futures.immediateFailedFuture; +import static com.google.common.util.concurrent.Futures.immediateVoidFuture; +import static com.google.common.util.concurrent.MoreExecutors.directExecutor; +import static io.airlift.concurrent.MoreFutures.asVoid; +import static io.airlift.concurrent.MoreFutures.getFutureValue; +import static io.airlift.concurrent.MoreFutures.toListenableFuture; +import static io.trino.plugin.exchange.FileSystemExchangeManager.PATH_SEPARATOR; +import static io.trino.plugin.exchange.s3.S3RequestUtil.configureEncryption; +import static java.lang.Math.toIntExact; +import static java.util.Objects.requireNonNull; +import static java.util.Objects.requireNonNullElseGet; +import static software.amazon.awssdk.core.client.config.SdkAdvancedClientOption.USER_AGENT_PREFIX; +import static software.amazon.awssdk.core.client.config.SdkAdvancedClientOption.USER_AGENT_SUFFIX; + +public class S3FileSystemExchangeStorage + implements FileSystemExchangeStorage +{ + private static final String DIRECTORY_SUFFIX = "_$folder$"; + + private final Optional region; + private final Optional endpoint; + private final int multiUploadPartSize; + private final S3Client s3Client; + private final S3AsyncClient s3AsyncClient; + + @Inject + public S3FileSystemExchangeStorage(ExchangeS3Config config) + { + requireNonNull(config, "config is null"); + this.region = config.getS3Region(); + this.endpoint = config.getS3Endpoint(); + this.multiUploadPartSize = toIntExact(config.getS3UploadPartSize().toBytes()); + + AwsCredentialsProvider credentialsProvider = createAwsCredentialsProvider(config); + RetryPolicy retryPolicy = RetryPolicy.builder() + .numRetries(config.getS3MaxErrorRetries()) + .build(); + ClientOverrideConfiguration overrideConfig = ClientOverrideConfiguration.builder() + .retryPolicy(retryPolicy) + .putAdvancedOption(USER_AGENT_PREFIX, "") + .putAdvancedOption(USER_AGENT_SUFFIX, "Trino-exchange") + .build(); + + this.s3Client = createS3Client(credentialsProvider, overrideConfig); + this.s3AsyncClient = createS3AsyncClient(credentialsProvider, overrideConfig); + } + + @Override + public void createDirectories(URI dir) + throws IOException + { + // Nothing to do for S3 + } + + @Override + public SliceInput getSliceInput(URI file, Optional secretKey) + throws IOException + { + GetObjectRequest.Builder getObjectRequestBuilder = GetObjectRequest.builder() + .bucket(getBucketName(file)) + .key(keyFromUri(file)); + configureEncryption(secretKey, getObjectRequestBuilder); + + try { + return new InputStreamSliceInput(s3Client.getObject(getObjectRequestBuilder.build(), ResponseTransformer.toInputStream())); + } + catch (RuntimeException e) { + throw new IOException(e); + } + } + + @Override + public ExchangeStorageWriter createExchangeStorageWriter(URI file, Optional secretKey) + { + String bucketName = getBucketName(file); + String key = keyFromUri(file); + + return new S3ExchangeStorageWriter(s3AsyncClient, bucketName, key, multiUploadPartSize, secretKey); + } + + @Override + public boolean exists(URI file) + throws IOException + { + // Only used for commit marker files and doesn't need secretKey + return headObject(file, Optional.empty()) != null; + } + + @Override + public ListenableFuture createEmptyFile(URI file) + { + PutObjectRequest request = PutObjectRequest.builder() + .bucket(getBucketName(file)) + .key(keyFromUri(file)) + .build(); + + return transformFuture(toListenableFuture(s3AsyncClient.putObject(request, AsyncRequestBody.empty()))); + } + + @Override + public ListenableFuture deleteRecursively(URI uri) + { + checkArgument(isDirectory(uri), "deleteRecursively called on file uri"); + + ImmutableList.Builder keys = ImmutableList.builder(); + return transformFuture(Futures.transformAsync( + toListenableFuture((listObjectsRecursively(uri).subscribe(listObjectsV2Response -> + listObjectsV2Response.contents().stream().map(S3Object::key).forEach(keys::add)))), + ignored -> { + keys.add(keyFromUri(uri) + DIRECTORY_SUFFIX); + return deleteObjects(getBucketName(uri), keys.build()); + }, + directExecutor())); + } + + @Override + public List listFiles(URI dir) + throws IOException + { + ImmutableList.Builder builder = ImmutableList.builder(); + try { + for (S3Object object : listObjects(dir).contents()) { + builder.add(new FileStatus( + new URI(dir.getScheme(), dir.getHost(), PATH_SEPARATOR + object.key(), dir.getFragment()).toString(), + object.size())); + } + } + catch (RuntimeException e) { + throw new IOException(e); + } + catch (URISyntaxException e) { + throw new IllegalArgumentException(e); + } + return builder.build(); + } + + @Override + public List listDirectories(URI dir) + throws IOException + { + ImmutableList.Builder builder = ImmutableList.builder(); + try { + for (CommonPrefix prefix : listObjects(dir).commonPrefixes()) { + builder.add(new URI(dir.getScheme(), dir.getHost(), PATH_SEPARATOR + prefix.prefix(), dir.getFragment())); + } + } + catch (RuntimeException e) { + throw new IOException(e); + } + catch (URISyntaxException e) { + throw new IllegalArgumentException(e); + } + return builder.build(); + } + + @Override + public int getWriteBufferSize() + { + return multiUploadPartSize; + } + + @PreDestroy + @Override + public void close() + throws IOException + { + try (Closer closer = Closer.create()) { + closer.register(s3Client::close); + closer.register(s3AsyncClient::close); + } + } + + private HeadObjectResponse headObject(URI uri, Optional secretKey) + throws IOException + { + HeadObjectRequest.Builder headObjectRequestBuilder = HeadObjectRequest.builder() + .bucket(getBucketName(uri)) + .key(keyFromUri(uri)); + configureEncryption(secretKey, headObjectRequestBuilder); + + try { + return s3Client.headObject(headObjectRequestBuilder.build()); + } + catch (RuntimeException e) { + if (e instanceof NoSuchKeyException) { + return null; + } + throw new IOException(e); + } + } + + private ListObjectsV2Iterable listObjects(URI dir) + { + String key = keyFromUri(dir); + if (!key.isEmpty()) { + key += PATH_SEPARATOR; + } + + ListObjectsV2Request request = ListObjectsV2Request.builder() + .bucket(getBucketName(dir)) + .prefix(key) + .delimiter(PATH_SEPARATOR) + .build(); + + return s3Client.listObjectsV2Paginator(request); + } + + private ListObjectsV2Publisher listObjectsRecursively(URI dir) + { + ListObjectsV2Request request = ListObjectsV2Request.builder() + .bucket(getBucketName(dir)) + .prefix(keyFromUri(dir)) + .build(); + + return s3AsyncClient.listObjectsV2Paginator(request); + } + + private ListenableFuture> deleteObjects(String bucketName, List keys) + { + List> subList = Lists.partition(keys, 1000); // deleteObjects has a limit of 1000 + return Futures.allAsList(subList.stream().map(list -> { + DeleteObjectsRequest request = DeleteObjectsRequest.builder() + .bucket(bucketName) + .delete(Delete.builder().objects(list.stream().map(key -> ObjectIdentifier.builder().key(key).build()).collect(toImmutableList())).build()) + .build(); + return toListenableFuture(s3AsyncClient.deleteObjects(request)); + }).collect(toImmutableList())); + } + + /** + * Helper function used to work around the fact that if you use an S3 bucket with an '_' that java.net.URI + * behaves differently and sets the host value to null whereas S3 buckets without '_' have a properly + * set host field. '_' is only allowed in S3 bucket names in us-east-1. + * + * @param uri The URI from which to extract a host value. + * @return The host value where uri.getAuthority() is used when uri.getHost() returns null as long as no UserInfo is present. + * @throws IllegalArgumentException If the bucket cannot be determined from the URI. + */ + private static String getBucketName(URI uri) + { + if (uri.getHost() != null) { + return uri.getHost(); + } + + if (uri.getUserInfo() == null) { + return uri.getAuthority(); + } + + throw new IllegalArgumentException("Unable to determine S3 bucket from URI."); + } + + private static String keyFromUri(URI uri) + { + checkArgument(uri.isAbsolute(), "Uri is not absolute: %s", uri); + String key = nullToEmpty(uri.getPath()); + if (key.startsWith(PATH_SEPARATOR)) { + key = key.substring(PATH_SEPARATOR.length()); + } + if (key.endsWith(PATH_SEPARATOR)) { + key = key.substring(0, key.length() - PATH_SEPARATOR.length()); + } + return key; + } + + // Helper function that translates exception and transform future type to avoid abstraction leak + private static ListenableFuture transformFuture(ListenableFuture listenableFuture) + { + return asVoid(Futures.catchingAsync(listenableFuture, Throwable.class, throwable -> { + if (throwable instanceof Error || throwable instanceof IOException) { + return immediateFailedFuture(throwable); + } + else { + return immediateFailedFuture(new IOException(throwable)); + } + }, directExecutor())); + } + + private static boolean isDirectory(URI uri) + { + return uri.toString().endsWith(PATH_SEPARATOR); + } + + private static AwsCredentialsProvider createAwsCredentialsProvider(ExchangeS3Config config) + { + if (config.getS3AwsAccessKey() != null && config.getS3AwsSecretKey() != null) { + return StaticCredentialsProvider.create(AwsBasicCredentials.create(config.getS3AwsAccessKey(), config.getS3AwsSecretKey())); + } + return DefaultCredentialsProvider.create(); + } + + private S3Client createS3Client(AwsCredentialsProvider credentialsProvider, ClientOverrideConfiguration overrideConfig) + { + S3ClientBuilder clientBuilder = S3Client.builder() + .credentialsProvider(credentialsProvider) + .overrideConfiguration(overrideConfig); + + region.ifPresent(clientBuilder::region); + endpoint.ifPresent(s3Endpoint -> clientBuilder.endpointOverride(URI.create(s3Endpoint))); + + return clientBuilder.build(); + } + + private S3AsyncClient createS3AsyncClient(AwsCredentialsProvider credentialsProvider, ClientOverrideConfiguration overrideConfig) + { + S3AsyncClientBuilder clientBuilder = S3AsyncClient.builder() + .credentialsProvider(credentialsProvider) + .overrideConfiguration(overrideConfig); + + region.ifPresent(clientBuilder::region); + endpoint.ifPresent(s3Endpoint -> clientBuilder.endpointOverride(URI.create(s3Endpoint))); + + return clientBuilder.build(); + } + + @NotThreadSafe + private static class S3ExchangeStorageWriter + implements ExchangeStorageWriter + { + private static final int INSTANCE_SIZE = ClassLayout.parseClass(S3ExchangeStorageWriter.class).instanceSize(); + + private final S3AsyncClient s3AsyncClient; + private final String bucketName; + private final String key; + private final int partSize; + private final Optional secretKey; + + private int currentPartNumber; + private ListenableFuture directUploadFuture; + private ListenableFuture multiPartUploadIdFuture; + private final List> multiPartUploadFutures = new ArrayList<>(); + private volatile boolean closed; + + public S3ExchangeStorageWriter(S3AsyncClient s3AsyncClient, String bucketName, String key, int partSize, Optional secretKey) + { + this.s3AsyncClient = requireNonNull(s3AsyncClient, "s3AsyncClient is null"); + this.bucketName = requireNonNull(bucketName, "bucketName is null"); + this.key = requireNonNull(key, "key is null"); + this.partSize = partSize; + this.secretKey = requireNonNull(secretKey, "secretKey is null"); + } + + @Override + public ListenableFuture write(Slice slice) + { + checkState(directUploadFuture == null, "Direct upload already started"); + if (closed) { + // Ignore writes after writer is closed + return immediateVoidFuture(); + } + + // Skip multipart upload if there would only be one part + if (slice.length() < partSize && multiPartUploadIdFuture == null) { + PutObjectRequest.Builder putObjectRequestBuilder = PutObjectRequest.builder() + .bucket(bucketName) + .key(key); + configureEncryption(secretKey, putObjectRequestBuilder); + directUploadFuture = transformFuture(toListenableFuture(s3AsyncClient.putObject(putObjectRequestBuilder.build(), + ByteBufferAsyncRequestBody.fromByteBuffer(slice.toByteBuffer())))); + return directUploadFuture; + } + + if (multiPartUploadIdFuture == null) { + multiPartUploadIdFuture = Futures.transform(createMultipartUpload(), CreateMultipartUploadResponse::uploadId, directExecutor()); + } + + int partNum = ++currentPartNumber; + ListenableFuture uploadFuture = Futures.transformAsync(multiPartUploadIdFuture, uploadId -> uploadPart(uploadId, slice, partNum), directExecutor()); + multiPartUploadFutures.add(uploadFuture); + + return transformFuture(uploadFuture); + } + + @Override + public ListenableFuture finish() + { + if (closed) { + return immediateVoidFuture(); + } + + if (multiPartUploadIdFuture == null) { + return requireNonNullElseGet(directUploadFuture, Futures::immediateVoidFuture); + } + + ListenableFuture finishFuture = transformFuture(Futures.transformAsync( + Futures.allAsList(multiPartUploadFutures), + completedParts -> completeMultipartUpload(getFutureValue(multiPartUploadIdFuture), completedParts), + directExecutor())); + Futures.addCallback(finishFuture, new FutureCallback<>() { + @Override + public void onSuccess(Void result) + { + closed = true; + } + + @Override + public void onFailure(Throwable ignored) + { + // Rely on caller to abort in case of exceptions during finish + } + }, directExecutor()); + return finishFuture; + } + + @Override + public ListenableFuture abort() + { + if (closed) { + return immediateVoidFuture(); + } + closed = true; + + if (multiPartUploadIdFuture == null) { + if (directUploadFuture != null) { + directUploadFuture.cancel(true); + } + return immediateVoidFuture(); + } + + verify(directUploadFuture == null); + multiPartUploadFutures.forEach(future -> future.cancel(true)); + return transformFuture(Futures.transformAsync(multiPartUploadIdFuture, this::abortMultipartUpload, directExecutor())); + } + + @Override + public long getRetainedSize() + { + return INSTANCE_SIZE; + } + + private ListenableFuture createMultipartUpload() + { + CreateMultipartUploadRequest.Builder createMultipartUploadRequestBuilder = CreateMultipartUploadRequest.builder() + .bucket(bucketName) + .key(key); + configureEncryption(secretKey, createMultipartUploadRequestBuilder); + return toListenableFuture(s3AsyncClient.createMultipartUpload(createMultipartUploadRequestBuilder.build())); + } + + private ListenableFuture uploadPart(String uploadId, Slice slice, int partNumber) + { + UploadPartRequest.Builder uploadPartRequestBuilder = UploadPartRequest.builder() + .bucket(bucketName) + .key(key) + .uploadId(uploadId) + .partNumber(partNumber); + configureEncryption(secretKey, uploadPartRequestBuilder); + UploadPartRequest uploadPartRequest = uploadPartRequestBuilder.build(); + return Futures.transform(toListenableFuture(s3AsyncClient.uploadPart(uploadPartRequest, ByteBufferAsyncRequestBody.fromByteBuffer(slice.toByteBuffer()))), + uploadPartResponse -> CompletedPart.builder().eTag(uploadPartResponse.eTag()).partNumber(partNumber).build(), directExecutor()); + } + + private ListenableFuture completeMultipartUpload(String uploadId, List completedParts) + { + CompletedMultipartUpload completedMultipartUpload = CompletedMultipartUpload.builder() + .parts(completedParts) + .build(); + CompleteMultipartUploadRequest completeMultipartUploadRequest = CompleteMultipartUploadRequest.builder() + .bucket(bucketName) + .key(key) + .uploadId(uploadId) + .multipartUpload(completedMultipartUpload) + .build(); + return toListenableFuture(s3AsyncClient.completeMultipartUpload(completeMultipartUploadRequest)); + } + + private ListenableFuture abortMultipartUpload(String uploadId) + { + AbortMultipartUploadRequest abortMultipartUploadRequest = AbortMultipartUploadRequest.builder() + .bucket(bucketName) + .key(key) + .uploadId(uploadId) + .build(); + return toListenableFuture(s3AsyncClient.abortMultipartUpload(abortMultipartUploadRequest)); + } + } +} diff --git a/plugin/trino-exchange/src/main/java/io/trino/plugin/exchange/s3/S3RequestUtil.java b/plugin/trino-exchange/src/main/java/io/trino/plugin/exchange/s3/S3RequestUtil.java new file mode 100644 index 000000000000..7ab840a9952e --- /dev/null +++ b/plugin/trino-exchange/src/main/java/io/trino/plugin/exchange/s3/S3RequestUtil.java @@ -0,0 +1,71 @@ +/* + * Licensed 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 io.trino.plugin.exchange.s3; + +import software.amazon.awssdk.services.s3.model.CreateMultipartUploadRequest; +import software.amazon.awssdk.services.s3.model.GetObjectRequest; +import software.amazon.awssdk.services.s3.model.HeadObjectRequest; +import software.amazon.awssdk.services.s3.model.PutObjectRequest; +import software.amazon.awssdk.services.s3.model.ServerSideEncryption; +import software.amazon.awssdk.services.s3.model.UploadPartRequest; +import software.amazon.awssdk.utils.Md5Utils; + +import javax.crypto.SecretKey; + +import java.util.Base64; +import java.util.Optional; +import java.util.function.Consumer; + +public final class S3RequestUtil +{ + private S3RequestUtil() {} + + static void configureEncryption(Optional secretKey, PutObjectRequest.Builder requestBuilder) + { + configureEncryption(secretKey, requestBuilder::sseCustomerAlgorithm, requestBuilder::sseCustomerKey, requestBuilder::sseCustomerKeyMD5); + } + + static void configureEncryption(Optional secretKey, CreateMultipartUploadRequest.Builder requestBuilder) + { + configureEncryption(secretKey, requestBuilder::sseCustomerAlgorithm, requestBuilder::sseCustomerKey, requestBuilder::sseCustomerKeyMD5); + } + + static void configureEncryption(Optional secretKey, UploadPartRequest.Builder requestBuilder) + { + configureEncryption(secretKey, requestBuilder::sseCustomerAlgorithm, requestBuilder::sseCustomerKey, requestBuilder::sseCustomerKeyMD5); + } + + static void configureEncryption(Optional secretKey, GetObjectRequest.Builder requestBuilder) + { + configureEncryption(secretKey, requestBuilder::sseCustomerAlgorithm, requestBuilder::sseCustomerKey, requestBuilder::sseCustomerKeyMD5); + } + + static void configureEncryption(Optional secretKey, HeadObjectRequest.Builder requestBuilder) + { + configureEncryption(secretKey, requestBuilder::sseCustomerAlgorithm, requestBuilder::sseCustomerKey, requestBuilder::sseCustomerKeyMD5); + } + + private static void configureEncryption( + Optional secretKey, + Consumer customAlgorithmSetter, + Consumer customKeySetter, + Consumer customMd5Setter) + { + secretKey.ifPresent(key -> { + customAlgorithmSetter.accept(ServerSideEncryption.AES256.name()); + customKeySetter.accept(Base64.getEncoder().encodeToString(key.getEncoded())); + customMd5Setter.accept(Md5Utils.md5AsBase64(key.getEncoded())); + }); + } +} diff --git a/testing/trino-testing/src/main/java/io/trino/testing/AbstractTestExchangeManager.java b/plugin/trino-exchange/src/test/java/io/trino/plugin/exchange/AbstractTestExchangeManager.java similarity index 94% rename from testing/trino-testing/src/main/java/io/trino/testing/AbstractTestExchangeManager.java rename to plugin/trino-exchange/src/test/java/io/trino/plugin/exchange/AbstractTestExchangeManager.java index fd922af0688c..7b24237bef3f 100644 --- a/testing/trino-testing/src/main/java/io/trino/testing/AbstractTestExchangeManager.java +++ b/plugin/trino-exchange/src/test/java/io/trino/plugin/exchange/AbstractTestExchangeManager.java @@ -11,7 +11,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.testing; +package io.trino.plugin.exchange; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableListMultimap; @@ -33,14 +33,12 @@ import java.util.List; import java.util.Map; -import java.util.concurrent.CompletableFuture; import java.util.function.Function; import static com.google.common.collect.ImmutableMap.toImmutableMap; import static io.airlift.concurrent.MoreFutures.getFutureValue; import static io.trino.spi.exchange.ExchangeId.createRandomExchangeId; import static org.assertj.core.api.Assertions.assertThat; -import static org.testng.Assert.assertTrue; public abstract class AbstractTestExchangeManager { @@ -141,10 +139,7 @@ public void testHappyPath() true); exchange.sinkFinished(sinkInstanceHandle); - CompletableFuture> inputPartitionHandlesFuture = exchange.getSourceHandles(); - assertTrue(inputPartitionHandlesFuture.isDone()); - - List partitionHandles = inputPartitionHandlesFuture.get(); + List partitionHandles = exchange.getSourceHandles().get(); assertThat(partitionHandles).hasSize(2); Map partitions = partitionHandles.stream() diff --git a/plugin/trino-exchange/src/test/java/io/trino/plugin/exchange/TestFileSystemExchangeConfig.java b/plugin/trino-exchange/src/test/java/io/trino/plugin/exchange/TestFileSystemExchangeConfig.java new file mode 100644 index 000000000000..3ce8a3a51029 --- /dev/null +++ b/plugin/trino-exchange/src/test/java/io/trino/plugin/exchange/TestFileSystemExchangeConfig.java @@ -0,0 +1,52 @@ +/* + * Licensed 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 io.trino.plugin.exchange; + +import com.google.common.collect.ImmutableMap; +import org.testng.annotations.Test; + +import java.util.Map; + +import static io.airlift.configuration.testing.ConfigAssertions.assertFullMapping; +import static io.airlift.configuration.testing.ConfigAssertions.assertRecordedDefaults; +import static io.airlift.configuration.testing.ConfigAssertions.recordDefaults; + +public class TestFileSystemExchangeConfig +{ + @Test + public void testDefaults() + { + assertRecordedDefaults(recordDefaults(FileSystemExchangeConfig.class) + .setBaseDirectory(null) + .setExchangeEncryptionEnabled(false) + .setExchangeSinkBufferPoolMinSize(0)); + } + + @Test + public void testExplicitPropertyMappings() + { + Map properties = new ImmutableMap.Builder() + .put("exchange.base-directory", "s3n://exchange-spooling-test/") + .put("exchange.encryption-enabled", "true") + .put("exchange.sink-buffer-pool-min-size", "10") + .build(); + + FileSystemExchangeConfig expected = new FileSystemExchangeConfig() + .setBaseDirectory("s3n://exchange-spooling-test/") + .setExchangeEncryptionEnabled(true) + .setExchangeSinkBufferPoolMinSize(10); + + assertFullMapping(properties, expected); + } +} diff --git a/plugin/trino-exchange/src/test/java/io/trino/plugin/exchange/containers/MinioStorage.java b/plugin/trino-exchange/src/test/java/io/trino/plugin/exchange/containers/MinioStorage.java new file mode 100644 index 000000000000..8b8a31597e07 --- /dev/null +++ b/plugin/trino-exchange/src/test/java/io/trino/plugin/exchange/containers/MinioStorage.java @@ -0,0 +1,97 @@ +/* + * Licensed 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 io.trino.plugin.exchange.containers; + +import com.google.common.collect.ImmutableMap; +import io.trino.testing.containers.Minio; +import org.testcontainers.containers.Network; +import software.amazon.awssdk.auth.credentials.AwsBasicCredentials; +import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider; +import software.amazon.awssdk.services.s3.S3Client; +import software.amazon.awssdk.services.s3.model.CreateBucketRequest; + +import java.net.URI; +import java.util.Map; + +import static java.util.Objects.requireNonNull; +import static org.testcontainers.containers.Network.newNetwork; +import static software.amazon.awssdk.regions.Region.US_EAST_1; + +public class MinioStorage + implements AutoCloseable +{ + public static final String ACCESS_KEY = "accesskey"; + public static final String SECRET_KEY = "secretkey"; + + private final String bucketName; + private final Network network; + private final Minio minio; + + public MinioStorage(String bucketName) + { + this.bucketName = requireNonNull(bucketName, "bucketName is null"); + this.network = newNetwork(); + this.minio = Minio.builder() + .withNetwork(network) + .withEnvVars(ImmutableMap.builder() + .put("MINIO_ACCESS_KEY", ACCESS_KEY) + .put("MINIO_SECRET_KEY", SECRET_KEY) + .build()) + .build(); + } + + public void start() + { + minio.start(); + S3Client s3Client = S3Client.builder() + .endpointOverride(URI.create("http://localhost:" + minio.getMinioApiEndpoint().getPort())) + .credentialsProvider(StaticCredentialsProvider.create(AwsBasicCredentials.create(ACCESS_KEY, SECRET_KEY))) + .region(US_EAST_1) + .build(); + CreateBucketRequest createBucketRequest = CreateBucketRequest.builder() + .bucket(bucketName) + .build(); + s3Client.createBucket(createBucketRequest); + } + + public Minio getMinio() + { + return minio; + } + + public String getBucketName() + { + return bucketName; + } + + @Override + public void close() + throws Exception + { + network.close(); + minio.close(); + } + + public static Map getExchangeManagerProperties(MinioStorage minioStorage) + { + return new ImmutableMap.Builder() + .put("exchange.base-directory", "s3n://" + minioStorage.getBucketName()) + .put("exchange.s3.aws-access-key", MinioStorage.ACCESS_KEY) + .put("exchange.s3.aws-secret-key", MinioStorage.SECRET_KEY) + .put("exchange.s3.region", "us-east-1") + // TODO: enable exchange encryption after https is supported for Trino MinIO + .put("exchange.s3.endpoint", "http://" + minioStorage.getMinio().getMinioApiEndpoint()) + .build(); + } +} diff --git a/testing/trino-testing/src/test/java/io/trino/testing/TestLocalFileSystemExchangeManager.java b/plugin/trino-exchange/src/test/java/io/trino/plugin/exchange/local/TestLocalFileSystemExchangeManager.java similarity index 66% rename from testing/trino-testing/src/test/java/io/trino/testing/TestLocalFileSystemExchangeManager.java rename to plugin/trino-exchange/src/test/java/io/trino/plugin/exchange/local/TestLocalFileSystemExchangeManager.java index ca4d642f919b..6fc0884d0cd6 100644 --- a/testing/trino-testing/src/test/java/io/trino/testing/TestLocalFileSystemExchangeManager.java +++ b/plugin/trino-exchange/src/test/java/io/trino/plugin/exchange/local/TestLocalFileSystemExchangeManager.java @@ -11,10 +11,11 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.testing; +package io.trino.plugin.exchange.local; import com.google.common.collect.ImmutableMap; -import io.trino.server.testing.exchange.LocalFileSystemExchangeManagerFactory; +import io.trino.plugin.exchange.AbstractTestExchangeManager; +import io.trino.plugin.exchange.FileSystemExchangeManagerFactory; import io.trino.spi.exchange.ExchangeManager; public class TestLocalFileSystemExchangeManager @@ -23,6 +24,8 @@ public class TestLocalFileSystemExchangeManager @Override protected ExchangeManager createExchangeManager() { - return new LocalFileSystemExchangeManagerFactory().create(ImmutableMap.of()); + return new FileSystemExchangeManagerFactory().create(ImmutableMap.of( + "exchange.base-directory", System.getProperty("java.io.tmpdir") + "/trino-local-file-system-exchange-manager", + "exchange.encryption-enabled", "true")); } } diff --git a/plugin/trino-exchange/src/test/java/io/trino/plugin/exchange/s3/TestExchangeS3Config.java b/plugin/trino-exchange/src/test/java/io/trino/plugin/exchange/s3/TestExchangeS3Config.java new file mode 100644 index 000000000000..58464a4c42af --- /dev/null +++ b/plugin/trino-exchange/src/test/java/io/trino/plugin/exchange/s3/TestExchangeS3Config.java @@ -0,0 +1,63 @@ +/* + * Licensed 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 io.trino.plugin.exchange.s3; + +import com.google.common.collect.ImmutableMap; +import io.airlift.units.DataSize; +import org.testng.annotations.Test; + +import java.util.Map; + +import static io.airlift.configuration.testing.ConfigAssertions.assertFullMapping; +import static io.airlift.configuration.testing.ConfigAssertions.assertRecordedDefaults; +import static io.airlift.configuration.testing.ConfigAssertions.recordDefaults; +import static io.airlift.units.DataSize.Unit.MEGABYTE; + +public class TestExchangeS3Config +{ + @Test + public void testDefaults() + { + assertRecordedDefaults(recordDefaults(ExchangeS3Config.class) + .setS3AwsAccessKey(null) + .setS3AwsSecretKey(null) + .setS3Region(null) + .setS3Endpoint(null) + .setS3MaxErrorRetries(3) + .setS3UploadPartSize(DataSize.of(5, MEGABYTE))); + } + + @Test + public void testExplicitPropertyMappings() + { + Map properties = new ImmutableMap.Builder() + .put("exchange.s3.aws-access-key", "access") + .put("exchange.s3.aws-secret-key", "secret") + .put("exchange.s3.region", "us-west-1") + .put("exchange.s3.endpoint", "https://s3.us-east-1.amazonaws.com") + .put("exchange.s3.max-error-retries", "8") + .put("exchange.s3.upload.part-size", "10MB") + .build(); + + ExchangeS3Config expected = new ExchangeS3Config() + .setS3AwsAccessKey("access") + .setS3AwsSecretKey("secret") + .setS3Region("us-west-1") + .setS3Endpoint("https://s3.us-east-1.amazonaws.com") + .setS3MaxErrorRetries(8) + .setS3UploadPartSize(DataSize.of(10, MEGABYTE)); + + assertFullMapping(properties, expected); + } +} diff --git a/plugin/trino-exchange/src/test/java/io/trino/plugin/exchange/s3/TestS3FileSystemExchangeManager.java b/plugin/trino-exchange/src/test/java/io/trino/plugin/exchange/s3/TestS3FileSystemExchangeManager.java new file mode 100644 index 000000000000..824f42f78d90 --- /dev/null +++ b/plugin/trino-exchange/src/test/java/io/trino/plugin/exchange/s3/TestS3FileSystemExchangeManager.java @@ -0,0 +1,50 @@ +/* + * Licensed 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 io.trino.plugin.exchange.s3; + +import io.trino.plugin.exchange.AbstractTestExchangeManager; +import io.trino.plugin.exchange.FileSystemExchangeManagerFactory; +import io.trino.plugin.exchange.containers.MinioStorage; +import io.trino.spi.exchange.ExchangeManager; +import org.testng.annotations.AfterClass; + +import static io.trino.plugin.exchange.containers.MinioStorage.getExchangeManagerProperties; +import static java.util.UUID.randomUUID; + +public class TestS3FileSystemExchangeManager + extends AbstractTestExchangeManager +{ + private MinioStorage minioStorage; + + @Override + protected ExchangeManager createExchangeManager() + { + this.minioStorage = new MinioStorage("test-exchange-spooling-" + randomUUID()); + minioStorage.start(); + + return new FileSystemExchangeManagerFactory().create(getExchangeManagerProperties(minioStorage)); + } + + @Override + @AfterClass(alwaysRun = true) + public void destroy() + throws Exception + { + super.destroy(); + if (minioStorage != null) { + minioStorage.close(); + minioStorage = null; + } + } +} diff --git a/plugin/trino-hive/pom.xml b/plugin/trino-hive/pom.xml index 0d5d386ac797..23215700bb06 100644 --- a/plugin/trino-hive/pom.xml +++ b/plugin/trino-hive/pom.xml @@ -314,6 +314,19 @@ test + + io.trino + trino-exchange + test + + + + io.trino + trino-exchange + test-jar + test + + io.trino trino-main @@ -427,6 +440,17 @@ + + org.basepom.maven + duplicate-finder-maven-plugin + + + + mime.types + about.html + + + diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/BaseHiveConnectorTest.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/BaseHiveConnectorTest.java index 1bfb5ee39071..d77d078e5309 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/BaseHiveConnectorTest.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/BaseHiveConnectorTest.java @@ -183,7 +183,7 @@ protected BaseHiveConnectorTest() this.bucketedSession = createBucketedSession(Optional.of(new SelectedRole(ROLE, Optional.of("admin")))); } - protected static QueryRunner createHiveQueryRunner(Map extraProperties) + protected static QueryRunner createHiveQueryRunner(Map extraProperties, Map exchangeManagerProperties) throws Exception { DistributedQueryRunner queryRunner = HiveQueryRunner.builder() @@ -195,6 +195,7 @@ protected static QueryRunner createHiveQueryRunner(Map extraProp // Make weighted split scheduling more conservative to avoid OOMs in test "hive.minimum-assigned-split-weight", "0.5")) .addExtraProperty("legacy.allow-set-view-authorization", "true") + .setExchangeManagerProperties(exchangeManagerProperties) .setInitialTables(REQUIRED_TPCH_TABLES) .build(); diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/HiveQueryRunner.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/HiveQueryRunner.java index 68dbe43da20c..59cbe6d1e78c 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/HiveQueryRunner.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/HiveQueryRunner.java @@ -20,6 +20,7 @@ import io.airlift.log.Logging; import io.trino.Session; import io.trino.metadata.QualifiedObjectName; +import io.trino.plugin.exchange.FileSystemExchangePlugin; import io.trino.plugin.hive.metastore.Database; import io.trino.plugin.hive.metastore.HiveMetastore; import io.trino.plugin.hive.metastore.MetastoreConfig; @@ -93,6 +94,7 @@ public static class Builder> { private boolean skipTimezoneSetup; private ImmutableMap.Builder hiveProperties = ImmutableMap.builder(); + private Map exchangeManagerProperties = ImmutableMap.of(); private List> initialTables = ImmutableList.of(); private Optional initialSchemasLocationBase = Optional.empty(); private Function initialTablesSessionMutator = Function.identity(); @@ -137,6 +139,12 @@ public SELF addHiveProperty(String key, String value) return self(); } + public SELF setExchangeManagerProperties(Map exchangeManagerProperties) + { + this.exchangeManagerProperties = ImmutableMap.copyOf(requireNonNull(exchangeManagerProperties, "exchangeManagerProperties is null")); + return self(); + } + public SELF setInitialTables(Iterable> initialTables) { this.initialTables = ImmutableList.copyOf(requireNonNull(initialTables, "initialTables is null")); @@ -182,6 +190,11 @@ public DistributedQueryRunner build() HiveMetastore metastore = this.metastore.apply(queryRunner); queryRunner.installPlugin(new TestingHivePlugin(metastore, module)); + if (!exchangeManagerProperties.isEmpty()) { + queryRunner.installPlugin(new FileSystemExchangePlugin()); + queryRunner.loadExchangeManager("filesystem", exchangeManagerProperties); + } + Map hiveProperties = new HashMap<>(); if (!skipTimezoneSetup) { assertEquals(DateTimeZone.getDefault(), TIME_ZONE, "Timezone not configured correctly. Add -Duser.timezone=America/Bahia_Banderas to your JVM arguments"); diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveConnectorTest.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveConnectorTest.java index fc9fafbfaea6..4477a56d0056 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveConnectorTest.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveConnectorTest.java @@ -23,6 +23,6 @@ public class TestHiveConnectorTest protected QueryRunner createQueryRunner() throws Exception { - return BaseHiveConnectorTest.createHiveQueryRunner(ImmutableMap.of()); + return BaseHiveConnectorTest.createHiveQueryRunner(ImmutableMap.of(), ImmutableMap.of()); } } diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveFaultTolerantExecutionAggregations.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveFaultTolerantExecutionAggregations.java index 25d065bba3dc..11c4b5ec1287 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveFaultTolerantExecutionAggregations.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveFaultTolerantExecutionAggregations.java @@ -13,23 +13,43 @@ */ package io.trino.plugin.hive; +import io.trino.plugin.exchange.containers.MinioStorage; import io.trino.testing.AbstractTestFaultTolerantExecutionAggregations; import io.trino.testing.QueryRunner; +import org.testng.annotations.AfterClass; import java.util.Map; +import static io.trino.plugin.exchange.containers.MinioStorage.getExchangeManagerProperties; +import static io.trino.testing.sql.TestTable.randomTableSuffix; import static io.trino.tpch.TpchTable.getTables; public class TestHiveFaultTolerantExecutionAggregations extends AbstractTestFaultTolerantExecutionAggregations { + private MinioStorage minioStorage; + @Override protected QueryRunner createQueryRunner(Map extraProperties) throws Exception { + this.minioStorage = new MinioStorage("test-exchange-spooling-" + randomTableSuffix()); + minioStorage.start(); + return HiveQueryRunner.builder() .setExtraProperties(extraProperties) + .setExchangeManagerProperties(getExchangeManagerProperties(minioStorage)) .setInitialTables(getTables()) .build(); } + + @AfterClass(alwaysRun = true) + public void destroy() + throws Exception + { + if (minioStorage != null) { + minioStorage.close(); + minioStorage = null; + } + } } diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveFaultTolerantExecutionConnectorTest.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveFaultTolerantExecutionConnectorTest.java index 0468f3b0d7eb..ca13930b52e8 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveFaultTolerantExecutionConnectorTest.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveFaultTolerantExecutionConnectorTest.java @@ -13,19 +13,28 @@ */ package io.trino.plugin.hive; -import io.trino.testing.FaultTolerantExecutionConnectorTestHelper; +import io.trino.plugin.exchange.containers.MinioStorage; import io.trino.testing.QueryRunner; +import org.testng.annotations.AfterClass; +import static io.trino.plugin.exchange.containers.MinioStorage.getExchangeManagerProperties; +import static io.trino.testing.FaultTolerantExecutionConnectorTestHelper.getExtraProperties; +import static io.trino.testing.sql.TestTable.randomTableSuffix; import static org.assertj.core.api.Assertions.assertThatThrownBy; public class TestHiveFaultTolerantExecutionConnectorTest extends BaseHiveConnectorTest { + private MinioStorage minioStorage; + @Override protected QueryRunner createQueryRunner() throws Exception { - return BaseHiveConnectorTest.createHiveQueryRunner(FaultTolerantExecutionConnectorTestHelper.getExtraProperties()); + this.minioStorage = new MinioStorage("test-exchange-spooling-" + randomTableSuffix()); + minioStorage.start(); + + return BaseHiveConnectorTest.createHiveQueryRunner(getExtraProperties(), getExchangeManagerProperties(minioStorage)); } @Override @@ -93,4 +102,14 @@ public void testOptimizeHiveSystemTable() assertThatThrownBy(super::testOptimizeHiveSystemTable) .hasMessageContaining("This connector does not support query retries"); } + + @AfterClass(alwaysRun = true) + public void destroy() + throws Exception + { + if (minioStorage != null) { + minioStorage.close(); + minioStorage = null; + } + } } diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveFaultTolerantExecutionJoinQueries.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveFaultTolerantExecutionJoinQueries.java index 246df628dedf..04bcc2ada62e 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveFaultTolerantExecutionJoinQueries.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveFaultTolerantExecutionJoinQueries.java @@ -13,31 +13,43 @@ */ package io.trino.plugin.hive; +import io.trino.plugin.exchange.containers.MinioStorage; import io.trino.testing.AbstractTestFaultTolerantExecutionJoinQueries; import io.trino.testing.QueryRunner; -import org.testng.annotations.Test; +import org.testng.annotations.AfterClass; import java.util.Map; +import static io.trino.plugin.exchange.containers.MinioStorage.getExchangeManagerProperties; +import static io.trino.testing.sql.TestTable.randomTableSuffix; import static io.trino.tpch.TpchTable.getTables; public class TestHiveFaultTolerantExecutionJoinQueries extends AbstractTestFaultTolerantExecutionJoinQueries { + private MinioStorage minioStorage; + @Override protected QueryRunner createQueryRunner(Map extraProperties) throws Exception { + this.minioStorage = new MinioStorage("test-exchange-spooling-" + randomTableSuffix()); + minioStorage.start(); + return HiveQueryRunner.builder() .setExtraProperties(extraProperties) + .setExchangeManagerProperties(getExchangeManagerProperties(minioStorage)) .setInitialTables(getTables()) .build(); } - @Override - @Test(enabled = false) - public void testOutputDuplicatesInsensitiveJoin() + @AfterClass(alwaysRun = true) + public void destroy() + throws Exception { - // flaky + if (minioStorage != null) { + minioStorage.close(); + minioStorage = null; + } } } diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveFaultTolerantExecutionOrderByQueries.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveFaultTolerantExecutionOrderByQueries.java index 30f18d230ae2..5d080a603711 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveFaultTolerantExecutionOrderByQueries.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveFaultTolerantExecutionOrderByQueries.java @@ -13,23 +13,43 @@ */ package io.trino.plugin.hive; +import io.trino.plugin.exchange.containers.MinioStorage; import io.trino.testing.AbstractTestFaultTolerantExecutionOrderByQueries; import io.trino.testing.QueryRunner; +import org.testng.annotations.AfterClass; import java.util.Map; +import static io.trino.plugin.exchange.containers.MinioStorage.getExchangeManagerProperties; +import static io.trino.testing.sql.TestTable.randomTableSuffix; import static io.trino.tpch.TpchTable.getTables; public class TestHiveFaultTolerantExecutionOrderByQueries extends AbstractTestFaultTolerantExecutionOrderByQueries { + private MinioStorage minioStorage; + @Override protected QueryRunner createQueryRunner(Map extraProperties) throws Exception { + this.minioStorage = new MinioStorage("test-exchange-spooling-" + randomTableSuffix()); + minioStorage.start(); + return HiveQueryRunner.builder() .setExtraProperties(extraProperties) + .setExchangeManagerProperties(getExchangeManagerProperties(minioStorage)) .setInitialTables(getTables()) .build(); } + + @AfterClass(alwaysRun = true) + public void destroy() + throws Exception + { + if (minioStorage != null) { + minioStorage.close(); + minioStorage = null; + } + } } diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveFaultTolerantExecutionWindowQueries.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveFaultTolerantExecutionWindowQueries.java index af31ffe857d9..b9751a295fd1 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveFaultTolerantExecutionWindowQueries.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveFaultTolerantExecutionWindowQueries.java @@ -13,23 +13,43 @@ */ package io.trino.plugin.hive; +import io.trino.plugin.exchange.containers.MinioStorage; import io.trino.testing.AbstractTestFaultTolerantExecutionWindowQueries; import io.trino.testing.QueryRunner; +import org.testng.annotations.AfterClass; import java.util.Map; +import static io.trino.plugin.exchange.containers.MinioStorage.getExchangeManagerProperties; +import static io.trino.testing.sql.TestTable.randomTableSuffix; import static io.trino.tpch.TpchTable.getTables; public class TestHiveFaultTolerantExecutionWindowQueries extends AbstractTestFaultTolerantExecutionWindowQueries { + private MinioStorage minioStorage; + @Override protected QueryRunner createQueryRunner(Map extraProperties) throws Exception { + this.minioStorage = new MinioStorage("test-exchange-spooling-" + randomTableSuffix()); + minioStorage.start(); + return HiveQueryRunner.builder() .setExtraProperties(extraProperties) + .setExchangeManagerProperties(getExchangeManagerProperties(minioStorage)) .setInitialTables(getTables()) .build(); } + + @AfterClass(alwaysRun = true) + public void destroy() + throws Exception + { + if (minioStorage != null) { + minioStorage.close(); + minioStorage = null; + } + } } diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveFileQueryFailureRecoveryTest.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveFileQueryFailureRecoveryTest.java deleted file mode 100644 index bf2b7246eb3d..000000000000 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveFileQueryFailureRecoveryTest.java +++ /dev/null @@ -1,41 +0,0 @@ -/* - * Licensed 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 io.trino.plugin.hive; - -import io.trino.operator.RetryPolicy; -import io.trino.testing.QueryRunner; -import io.trino.tpch.TpchTable; - -import java.util.List; -import java.util.Map; - -public class TestHiveFileQueryFailureRecoveryTest - extends BaseHiveFailureRecoveryTest -{ - public TestHiveFileQueryFailureRecoveryTest() - { - super(RetryPolicy.QUERY); - } - - @Override - protected QueryRunner createQueryRunner(List> requiredTpchTables, Map configProperties, Map coordinatorProperties) - throws Exception - { - return HiveQueryRunner.builder() - .setInitialTables(requiredTpchTables) - .setCoordinatorProperties(coordinatorProperties) - .setExtraProperties(configProperties) - .build(); - } -} diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveFileTaskFailureRecoveryTest.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveFileTaskFailureRecoveryTest.java deleted file mode 100644 index 94171661015e..000000000000 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveFileTaskFailureRecoveryTest.java +++ /dev/null @@ -1,55 +0,0 @@ -/* - * Licensed 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 io.trino.plugin.hive; - -import com.google.common.collect.ImmutableMap; -import io.trino.operator.RetryPolicy; -import io.trino.testing.QueryRunner; -import io.trino.tpch.TpchTable; - -import java.util.List; -import java.util.Map; - -import static org.assertj.core.api.Assertions.assertThatThrownBy; - -public class TestHiveFileTaskFailureRecoveryTest - extends BaseHiveFailureRecoveryTest -{ - protected TestHiveFileTaskFailureRecoveryTest() - { - super(RetryPolicy.TASK); - } - - @Override - protected QueryRunner createQueryRunner(List> requiredTpchTables, Map configProperties, Map coordinatorProperties) - throws Exception - { - return HiveQueryRunner.builder() - .setInitialTables(requiredTpchTables) - .setCoordinatorProperties(coordinatorProperties) - .setExtraProperties(ImmutableMap.builder() - .putAll(configProperties) - // currently not supported for fault tolerant execution mode - .put("enable-dynamic-filtering", "false") - .build()) - .build(); - } - - @Override - public void testJoinDynamicFilteringEnabled() - { - assertThatThrownBy(super::testJoinDynamicFilteringEnabled) - .hasMessageContaining("Dynamic filtering is not supported with automatic task retries enabled"); - } -} diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveMinioQueryFailureRecoveryTest.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveQueryFailureRecoveryTest.java similarity index 70% rename from plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveMinioQueryFailureRecoveryTest.java rename to plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveQueryFailureRecoveryTest.java index f93f27bfd8d2..dbd6541af030 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveMinioQueryFailureRecoveryTest.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveQueryFailureRecoveryTest.java @@ -15,6 +15,7 @@ import com.google.common.collect.ImmutableMap; import io.trino.operator.RetryPolicy; +import io.trino.plugin.exchange.containers.MinioStorage; import io.trino.plugin.hive.containers.HiveHadoop; import io.trino.plugin.hive.containers.HiveMinioDataLake; import io.trino.plugin.hive.s3.S3HiveQueryRunner; @@ -25,26 +26,33 @@ import java.util.List; import java.util.Map; +import static io.trino.plugin.exchange.containers.MinioStorage.getExchangeManagerProperties; import static io.trino.testing.sql.TestTable.randomTableSuffix; -public class TestHiveMinioQueryFailureRecoveryTest +public class TestHiveQueryFailureRecoveryTest extends BaseHiveFailureRecoveryTest { - public TestHiveMinioQueryFailureRecoveryTest() + public TestHiveQueryFailureRecoveryTest() { super(RetryPolicy.QUERY); } - private String bucketName; private HiveMinioDataLake dockerizedS3DataLake; + private MinioStorage minioStorage; @Override - protected QueryRunner createQueryRunner(List> requiredTpchTables, Map configProperties, Map coordinatorProperties) + protected QueryRunner createQueryRunner( + List> requiredTpchTables, + Map configProperties, + Map coordinatorProperties) throws Exception { - this.bucketName = "test-hive-insert-overwrite-" + randomTableSuffix(); // randomizing bucket name to ensure cached TrinoS3FileSystem objects are not reused + String bucketName = "test-hive-insert-overwrite-" + randomTableSuffix(); // randomizing bucket name to ensure cached TrinoS3FileSystem objects are not reused this.dockerizedS3DataLake = new HiveMinioDataLake(bucketName, ImmutableMap.of(), HiveHadoop.DEFAULT_IMAGE); - this.dockerizedS3DataLake.start(); + dockerizedS3DataLake.start(); + + this.minioStorage = new MinioStorage("test-exchange-spooling-" + randomTableSuffix()); + minioStorage.start(); return S3HiveQueryRunner.builder(dockerizedS3DataLake) .setInitialTables(requiredTpchTables) @@ -56,6 +64,7 @@ protected QueryRunner createQueryRunner(List> requiredTpchTables, M // the tests to run out of memory as the buffer space is eagerly allocated for each output file. .put("hive.s3.streaming.enabled", "false") .build()) + .setExchangeManagerProperties(getExchangeManagerProperties(minioStorage)) .build(); } @@ -67,5 +76,9 @@ public void destroy() dockerizedS3DataLake.close(); dockerizedS3DataLake = null; } + if (minioStorage != null) { + minioStorage.close(); + minioStorage = null; + } } } diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveTaskFailureRecoveryTest.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveTaskFailureRecoveryTest.java new file mode 100644 index 000000000000..22c34db647c3 --- /dev/null +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveTaskFailureRecoveryTest.java @@ -0,0 +1,96 @@ +/* + * Licensed 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 io.trino.plugin.hive; + +import com.google.common.collect.ImmutableMap; +import io.trino.operator.RetryPolicy; +import io.trino.plugin.exchange.containers.MinioStorage; +import io.trino.plugin.hive.containers.HiveHadoop; +import io.trino.plugin.hive.containers.HiveMinioDataLake; +import io.trino.plugin.hive.s3.S3HiveQueryRunner; +import io.trino.testing.QueryRunner; +import io.trino.tpch.TpchTable; +import org.testng.annotations.AfterClass; + +import java.util.List; +import java.util.Map; + +import static io.trino.plugin.exchange.containers.MinioStorage.getExchangeManagerProperties; +import static io.trino.testing.sql.TestTable.randomTableSuffix; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +public class TestHiveTaskFailureRecoveryTest + extends BaseHiveFailureRecoveryTest +{ + public TestHiveTaskFailureRecoveryTest() + { + super(RetryPolicy.TASK); + } + + private HiveMinioDataLake dockerizedS3DataLake; + private MinioStorage minioStorage; + + @Override + protected QueryRunner createQueryRunner( + List> requiredTpchTables, + Map configProperties, + Map coordinatorProperties) + throws Exception + { + String bucketName = "test-hive-insert-overwrite-" + randomTableSuffix(); // randomizing bucket name to ensure cached TrinoS3FileSystem objects are not reused + this.dockerizedS3DataLake = new HiveMinioDataLake(bucketName, ImmutableMap.of(), HiveHadoop.DEFAULT_IMAGE); + dockerizedS3DataLake.start(); + + this.minioStorage = new MinioStorage("test-exchange-spooling-" + randomTableSuffix()); + minioStorage.start(); + + return S3HiveQueryRunner.builder(dockerizedS3DataLake) + .setInitialTables(requiredTpchTables) + .setExtraProperties(ImmutableMap.builder() + .putAll(configProperties) + // currently not supported for fault tolerant execution mode + .put("enable-dynamic-filtering", "false") + .build()) + .setCoordinatorProperties(coordinatorProperties) + .setHiveProperties(ImmutableMap.builder() + // Streaming upload allocates non trivial amount of memory for buffering (16MB per output file by default). + // When streaming upload is enabled insert into a table with high number of buckets / partitions may cause + // the tests to run out of memory as the buffer space is eagerly allocated for each output file. + .put("hive.s3.streaming.enabled", "false") + .build()) + .setExchangeManagerProperties(getExchangeManagerProperties(minioStorage)) + .build(); + } + + @Override + public void testJoinDynamicFilteringEnabled() + { + assertThatThrownBy(super::testJoinDynamicFilteringEnabled) + .hasMessageContaining("Dynamic filtering is not supported with automatic task retries enabled"); + } + + @AfterClass(alwaysRun = true) + public void destroy() + throws Exception + { + if (dockerizedS3DataLake != null) { + dockerizedS3DataLake.close(); + dockerizedS3DataLake = null; + } + if (minioStorage != null) { + minioStorage.close(); + minioStorage = null; + } + } +} diff --git a/plugin/trino-iceberg/pom.xml b/plugin/trino-iceberg/pom.xml index bff6cf13350c..0f42c51dce23 100644 --- a/plugin/trino-iceberg/pom.xml +++ b/plugin/trino-iceberg/pom.xml @@ -200,6 +200,19 @@ + + io.trino + trino-exchange + test-jar + test + + + + io.trino + trino-exchange + test + + io.trino trino-hive @@ -239,6 +252,12 @@ test + + io.trino + trino-testing-containers + test + + io.trino trino-testing-services @@ -298,6 +317,17 @@ + + org.basepom.maven + duplicate-finder-maven-plugin + + + + mime.types + about.html + + + diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/IcebergQueryRunner.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/IcebergQueryRunner.java index 631216d799f9..41804f4f3a78 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/IcebergQueryRunner.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/IcebergQueryRunner.java @@ -17,6 +17,7 @@ import com.google.common.collect.ImmutableMap; import io.airlift.log.Logger; import io.airlift.log.Logging; +import io.trino.plugin.exchange.FileSystemExchangePlugin; import io.trino.plugin.tpch.TpchPlugin; import io.trino.testing.DistributedQueryRunner; import io.trino.tpch.TpchTable; @@ -90,6 +91,7 @@ public static class Builder { private Optional metastoreDirectory = Optional.empty(); private ImmutableMap.Builder icebergProperties = ImmutableMap.builder(); + private Map exchangeManagerProperties = ImmutableMap.of(); private List> initialTables = ImmutableList.of(); protected Builder() @@ -119,6 +121,12 @@ public Builder addIcebergProperty(String key, String value) return self(); } + public Builder setExchangeManagerProperties(Map exchangeManagerProperties) + { + this.exchangeManagerProperties = ImmutableMap.copyOf(requireNonNull(exchangeManagerProperties, "exchangeManagerProperties is null")); + return self(); + } + public Builder setInitialTables(Iterable> initialTables) { this.initialTables = ImmutableList.copyOf(requireNonNull(initialTables, "initialTables is null")); @@ -134,6 +142,11 @@ public DistributedQueryRunner build() queryRunner.installPlugin(new TpchPlugin()); queryRunner.createCatalog("tpch", "tpch"); + if (!exchangeManagerProperties.isEmpty()) { + queryRunner.installPlugin(new FileSystemExchangePlugin()); + queryRunner.loadExchangeManager("filesystem", exchangeManagerProperties); + } + Path dataDir = metastoreDirectory.map(File::toPath).orElseGet(() -> queryRunner.getCoordinator().getBaseDataDir().resolve("iceberg_data")); queryRunner.installPlugin(new IcebergPlugin()); diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergQueryFailureRecoveryTest.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergQueryFailureRecoveryTest.java index c7cfa53df8dd..e916560e20a3 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergQueryFailureRecoveryTest.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergQueryFailureRecoveryTest.java @@ -14,28 +14,52 @@ package io.trino.plugin.iceberg; import io.trino.operator.RetryPolicy; +import io.trino.plugin.exchange.containers.MinioStorage; import io.trino.testing.QueryRunner; import io.trino.tpch.TpchTable; +import org.testng.annotations.AfterClass; import java.util.List; import java.util.Map; +import static io.trino.plugin.exchange.containers.MinioStorage.getExchangeManagerProperties; +import static io.trino.testing.sql.TestTable.randomTableSuffix; + public class TestIcebergQueryFailureRecoveryTest extends BaseIcebergFailureRecoveryTest { + private MinioStorage minioStorage; + protected TestIcebergQueryFailureRecoveryTest() { super(RetryPolicy.QUERY); } @Override - protected QueryRunner createQueryRunner(List> requiredTpchTables, Map configProperties, Map coordinatorProperties) + protected QueryRunner createQueryRunner( + List> requiredTpchTables, + Map configProperties, + Map coordinatorProperties) throws Exception { + this.minioStorage = new MinioStorage("test-exchange-spooling-" + randomTableSuffix()); + minioStorage.start(); + return IcebergQueryRunner.builder() .setInitialTables(requiredTpchTables) .setCoordinatorProperties(coordinatorProperties) .setExtraProperties(configProperties) + .setExchangeManagerProperties(getExchangeManagerProperties(minioStorage)) .build(); } + + @AfterClass(alwaysRun = true) + public void destroy() + throws Exception + { + if (minioStorage != null) { + minioStorage.close(); + minioStorage = null; + } + } } diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergTaskFailureRecoveryTest.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergTaskFailureRecoveryTest.java index d228e8387d66..797695e08bd4 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergTaskFailureRecoveryTest.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergTaskFailureRecoveryTest.java @@ -15,26 +15,38 @@ import com.google.common.collect.ImmutableMap; import io.trino.operator.RetryPolicy; +import io.trino.plugin.exchange.containers.MinioStorage; import io.trino.testing.QueryRunner; import io.trino.tpch.TpchTable; +import org.testng.annotations.AfterClass; import java.util.List; import java.util.Map; +import static io.trino.plugin.exchange.containers.MinioStorage.getExchangeManagerProperties; +import static io.trino.testing.sql.TestTable.randomTableSuffix; import static org.assertj.core.api.Assertions.assertThatThrownBy; public class TestIcebergTaskFailureRecoveryTest extends BaseIcebergFailureRecoveryTest { + private MinioStorage minioStorage; + protected TestIcebergTaskFailureRecoveryTest() { super(RetryPolicy.TASK); } @Override - protected QueryRunner createQueryRunner(List> requiredTpchTables, Map configProperties, Map coordinatorProperties) + protected QueryRunner createQueryRunner( + List> requiredTpchTables, + Map configProperties, + Map coordinatorProperties) throws Exception { + this.minioStorage = new MinioStorage("test-exchange-spooling-" + randomTableSuffix()); + minioStorage.start(); + return IcebergQueryRunner.builder() .setInitialTables(requiredTpchTables) .setCoordinatorProperties(coordinatorProperties) @@ -43,6 +55,7 @@ protected QueryRunner createQueryRunner(List> requiredTpchTables, M // currently not supported for fault tolerant execution mode .put("enable-dynamic-filtering", "false") .build()) + .setExchangeManagerProperties(getExchangeManagerProperties(minioStorage)) .build(); } @@ -52,4 +65,14 @@ public void testJoinDynamicFilteringEnabled() assertThatThrownBy(super::testJoinDynamicFilteringEnabled) .hasMessageContaining("Dynamic filtering is not supported with automatic task retries enabled"); } + + @AfterClass(alwaysRun = true) + public void destroy() + throws Exception + { + if (minioStorage != null) { + minioStorage.close(); + minioStorage = null; + } + } } diff --git a/plugin/trino-thrift/src/test/java/io/trino/plugin/thrift/integration/ThriftQueryRunner.java b/plugin/trino-thrift/src/test/java/io/trino/plugin/thrift/integration/ThriftQueryRunner.java index 749080f27c42..79dee8001ab9 100644 --- a/plugin/trino-thrift/src/test/java/io/trino/plugin/thrift/integration/ThriftQueryRunner.java +++ b/plugin/trino-thrift/src/test/java/io/trino/plugin/thrift/integration/ThriftQueryRunner.java @@ -333,5 +333,11 @@ public void injectTaskFailure( { source.injectTaskFailure(traceToken, stageId, partitionId, attemptId, injectionType, errorType); } + + @Override + public void loadExchangeManager(String name, Map properties) + { + source.loadExchangeManager(name, properties); + } } } diff --git a/pom.xml b/pom.xml index 617faea07bf2..744ad5aeca14 100644 --- a/pom.xml +++ b/pom.xml @@ -116,6 +116,7 @@ plugin/trino-druid plugin/trino-elasticsearch plugin/trino-example-http + plugin/trino-exchange plugin/trino-geospatial plugin/trino-google-sheets plugin/trino-hive @@ -255,6 +256,19 @@ ${project.version} + + io.trino + trino-exchange + ${project.version} + + + + io.trino + trino-exchange + test-jar + ${project.version} + + io.trino trino-geospatial diff --git a/testing/trino-testing/src/main/java/io/trino/testing/BaseFailureRecoveryTest.java b/testing/trino-testing/src/main/java/io/trino/testing/BaseFailureRecoveryTest.java index a3a41d3a3a92..3104a73fb4d0 100644 --- a/testing/trino-testing/src/main/java/io/trino/testing/BaseFailureRecoveryTest.java +++ b/testing/trino-testing/src/main/java/io/trino/testing/BaseFailureRecoveryTest.java @@ -116,7 +116,10 @@ protected final QueryRunner createQueryRunner() .buildOrThrow()); } - protected abstract QueryRunner createQueryRunner(List> requiredTpchTables, Map configProperties, Map coordinatorProperties) + protected abstract QueryRunner createQueryRunner( + List> requiredTpchTables, + Map configProperties, + Map coordinatorProperties) throws Exception; @BeforeClass diff --git a/testing/trino-testing/src/main/java/io/trino/testing/DistributedQueryRunner.java b/testing/trino-testing/src/main/java/io/trino/testing/DistributedQueryRunner.java index e8cb7f166fe9..7291ffe74ee0 100644 --- a/testing/trino-testing/src/main/java/io/trino/testing/DistributedQueryRunner.java +++ b/testing/trino-testing/src/main/java/io/trino/testing/DistributedQueryRunner.java @@ -571,6 +571,14 @@ public void injectTaskFailure( } } + @Override + public void loadExchangeManager(String name, Map properties) + { + for (TestingTrinoServer server : servers) { + server.loadExchangeManager(name, properties); + } + } + @Override public final void close() { diff --git a/testing/trino-testing/src/main/java/io/trino/testing/FaultTolerantExecutionConnectorTestHelper.java b/testing/trino-testing/src/main/java/io/trino/testing/FaultTolerantExecutionConnectorTestHelper.java index fd2381985515..b25f17996336 100644 --- a/testing/trino-testing/src/main/java/io/trino/testing/FaultTolerantExecutionConnectorTestHelper.java +++ b/testing/trino-testing/src/main/java/io/trino/testing/FaultTolerantExecutionConnectorTestHelper.java @@ -17,7 +17,7 @@ import java.util.Map; -public class FaultTolerantExecutionConnectorTestHelper +public final class FaultTolerantExecutionConnectorTestHelper { private FaultTolerantExecutionConnectorTestHelper() {} diff --git a/testing/trino-testing/src/main/java/io/trino/testing/StandaloneQueryRunner.java b/testing/trino-testing/src/main/java/io/trino/testing/StandaloneQueryRunner.java index 945d8a1968fc..930065725347 100644 --- a/testing/trino-testing/src/main/java/io/trino/testing/StandaloneQueryRunner.java +++ b/testing/trino-testing/src/main/java/io/trino/testing/StandaloneQueryRunner.java @@ -294,6 +294,12 @@ public void injectTaskFailure( errorType); } + @Override + public void loadExchangeManager(String name, Map properties) + { + server.loadExchangeManager(name, properties); + } + private static TestingTrinoServer createTestingTrinoServer() { return TestingTrinoServer.builder()