Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Implement parallel read from S3 for exchange storage #11174

Merged
merged 5 commits into from
Mar 16, 2022
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
@@ -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;
linzebing marked this conversation as resolved.
Show resolved Hide resolved

import javax.annotation.concurrent.Immutable;
import javax.crypto.SecretKey;

import java.net.URI;
import java.util.Optional;

import static java.util.Objects.requireNonNull;

@Immutable
public class ExchangeSourceFile
{
private final URI fileUri;
private final Optional<SecretKey> secretKey;
private final long fileSize;

public ExchangeSourceFile(URI fileUri, Optional<SecretKey> secretKey, long fileSize)
{
this.fileUri = requireNonNull(fileUri, "fileUri is null");
this.secretKey = requireNonNull(secretKey, "secretKey is null");
this.fileSize = fileSize;
}

public URI getFileUri()
{
return fileUri;
}

public Optional<SecretKey> getSecretKey()
{
return secretKey;
}

public long getFileSize()
{
return fileSize;
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,38 @@
/*
* 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.ListenableFuture;
import io.airlift.slice.Slice;

import javax.annotation.concurrent.ThreadSafe;

import java.io.Closeable;
import java.io.IOException;

@ThreadSafe
public interface ExchangeStorageReader
extends Closeable
{
Slice read() throws IOException;

ListenableFuture<Void> isBlocked();

long getRetainedSize();

boolean isFinished();

@Override
void close();
}
Original file line number Diff line number Diff line change
@@ -1,4 +1,3 @@
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.
Expand All @@ -12,6 +11,7 @@
* 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.ListenableFuture;
import io.airlift.slice.Slice;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -14,15 +14,24 @@
package io.trino.plugin.exchange;

import io.airlift.configuration.Config;
import io.airlift.configuration.ConfigDescription;
import io.airlift.units.DataSize;

import javax.validation.constraints.Min;
import javax.validation.constraints.NotNull;

import static io.airlift.units.DataSize.Unit.MEGABYTE;

public class FileSystemExchangeConfig
{
private String baseDirectory;
private boolean exchangeEncryptionEnabled;
private int exchangeSinkBufferPoolMinSize;
private boolean exchangeEncryptionEnabled = true;
// For S3, we make read requests aligned with part boundaries. Incomplete slice at the end of the buffer is
// possible and will be copied to the beginning of the new buffer, and we need to make room for that.
// Therefore, it's recommended to set `maxPageStorageSize` to be slightly larger than a multiple of part size.
private DataSize maxPageStorageSize = DataSize.of(16, MEGABYTE);
linzebing marked this conversation as resolved.
Show resolved Hide resolved
private int exchangeSinkBufferPoolMinSize = 10;
private int exchangeSourceConcurrentReaders = 4;

@NotNull
public String getBaseDirectory()
Expand All @@ -49,6 +58,20 @@ public FileSystemExchangeConfig setExchangeEncryptionEnabled(boolean exchangeEnc
return this;
}

@NotNull
public DataSize getMaxPageStorageSize()
{
return maxPageStorageSize;
}

@Config("exchange.max-page-storage-size")
@ConfigDescription("Max storage size of a page written to a sink, including the page itself and its size represented as an int")
public FileSystemExchangeConfig setMaxPageStorageSize(DataSize maxPageStorageSize)
{
this.maxPageStorageSize = maxPageStorageSize;
return this;
}

@Min(0)
public int getExchangeSinkBufferPoolMinSize()
{
Expand All @@ -61,4 +84,17 @@ public FileSystemExchangeConfig setExchangeSinkBufferPoolMinSize(int exchangeSin
this.exchangeSinkBufferPoolMinSize = exchangeSinkBufferPoolMinSize;
return this;
}

@Min(1)
public int getExchangeSourceConcurrentReaders()
{
return exchangeSourceConcurrentReaders;
}

@Config("exchange.source-concurrent-readers")
public FileSystemExchangeConfig setExchangeSourceConcurrentReaders(int exchangeSourceConcurrentReaders)
linzebing marked this conversation as resolved.
Show resolved Hide resolved
{
this.exchangeSourceConcurrentReaders = exchangeSourceConcurrentReaders;
return this;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -13,7 +13,6 @@
*/
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;
Expand All @@ -30,14 +29,15 @@

import java.net.URI;
import java.security.NoSuchAlgorithmException;
import java.util.AbstractMap;
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.lang.Math.toIntExact;
import static java.util.Objects.requireNonNull;
import static java.util.concurrent.Executors.newCachedThreadPool;

Expand All @@ -51,7 +51,9 @@ public class FileSystemExchangeManager
private final FileSystemExchangeStorage exchangeStorage;
private final URI baseDirectory;
private final boolean exchangeEncryptionEnabled;
private final int maxPageStorageSize;
linzebing marked this conversation as resolved.
Show resolved Hide resolved
private final int exchangeSinkBufferPoolMinSize;
private final int exchangeSourceConcurrentReaders;
private final ExecutorService executor;

@Inject
Expand All @@ -67,7 +69,9 @@ public FileSystemExchangeManager(FileSystemExchangeStorage exchangeStorage, File
}
this.baseDirectory = URI.create(baseDirectory);
this.exchangeEncryptionEnabled = fileSystemExchangeConfig.isExchangeEncryptionEnabled();
this.maxPageStorageSize = toIntExact(fileSystemExchangeConfig.getMaxPageStorageSize().toBytes());
this.exchangeSinkBufferPoolMinSize = fileSystemExchangeConfig.getExchangeSinkBufferPoolMinSize();
this.exchangeSourceConcurrentReaders = fileSystemExchangeConfig.getExchangeSourceConcurrentReaders();
this.executor = newCachedThreadPool(daemonThreadsNamed("exchange-source-handles-creation-%s"));
}

Expand Down Expand Up @@ -99,22 +103,25 @@ public ExchangeSink createSink(ExchangeSinkInstanceHandle handle, boolean preser
instanceHandle.getOutputDirectory(),
instanceHandle.getOutputPartitionCount(),
instanceHandle.getSinkHandle().getSecretKey().map(key -> new SecretKeySpec(key, 0, key.length, "AES")),
maxPageStorageSize,
exchangeSinkBufferPoolMinSize);
}

@Override
public ExchangeSource createSource(List<ExchangeSourceHandle> handles)
{
List<URI> files = handles.stream()
List<ExchangeSourceFile> sourceFiles = handles.stream()
.map(FileSystemExchangeSourceHandle.class::cast)
.flatMap(handle -> handle.getFiles().stream())
.map(fileStatus -> URI.create(fileStatus.getFilePath()))
.map(handle -> {
Optional<SecretKey> secretKey = handle.getSecretKey().map(key -> new SecretKeySpec(key, 0, key.length, "AES"));
return new AbstractMap.SimpleEntry<>(handle, secretKey);
})
.flatMap(entry -> entry.getKey().getFiles().stream().map(fileStatus ->
new ExchangeSourceFile(
URI.create(fileStatus.getFilePath()),
entry.getValue(),
fileStatus.getFileSize())))
.collect(toImmutableList());
ImmutableList.Builder<Optional<SecretKey>> 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());
return new FileSystemExchangeSource(exchangeStorage, sourceFiles, maxPageStorageSize, exchangeSourceConcurrentReaders);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@
import io.airlift.slice.Slice;
import io.airlift.slice.SliceOutput;
import io.airlift.slice.Slices;
import io.trino.spi.TrinoException;
import io.trino.spi.exchange.ExchangeSink;
import org.openjdk.jol.info.ClassLayout;

Expand Down Expand Up @@ -47,8 +48,11 @@
import static io.airlift.concurrent.MoreFutures.asVoid;
import static io.airlift.concurrent.MoreFutures.toCompletableFuture;
import static io.airlift.slice.SizeOf.estimatedSizeOf;
import static io.airlift.units.DataSize.succinctBytes;
import static io.trino.spi.StandardErrorCode.NOT_SUPPORTED;
import static java.lang.Math.max;
import static java.lang.Math.min;
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.failedFuture;
Expand All @@ -66,6 +70,7 @@ public class FileSystemExchangeSink
private final URI outputDirectory;
private final int outputPartitionCount;
private final Optional<SecretKey> secretKey;
private final int maxPageStorageSize;
private final BufferPool bufferPool;

private final Map<Integer, BufferedStorageWriter> writersMap = new ConcurrentHashMap<>();
Expand All @@ -77,12 +82,14 @@ public FileSystemExchangeSink(
URI outputDirectory,
int outputPartitionCount,
Optional<SecretKey> secretKey,
int maxPageStorageSize,
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");
this.maxPageStorageSize = maxPageStorageSize;
// double buffering to overlap computation and I/O
this.bufferPool = new BufferPool(max(outputPartitionCount * 2, exchangeSinkBufferPoolMinSize), exchangeStorage.getWriteBufferSize());
}
Expand All @@ -101,6 +108,11 @@ public void add(int partitionId, Slice data)

checkArgument(partitionId < outputPartitionCount, "partition id is expected to be less than %s: %s", outputPartitionCount, partitionId);

int requiredPageStorageSize = data.length() + Integer.BYTES;
if (requiredPageStorageSize > maxPageStorageSize) {
throw new TrinoException(NOT_SUPPORTED, format("Max row size of %s exceeded: %s", succinctBytes(maxPageStorageSize), succinctBytes(requiredPageStorageSize)));
}

// Ensure no new writers can be created after `closed` is set to true
BufferedStorageWriter writer;
synchronized (this) {
Expand Down
Loading