Skip to content
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
Expand Up @@ -697,6 +697,18 @@ public ListenableFuture<Void> setBytes(long bytes)
return blocked;
}

@Override
public ListenableFuture<Void> addBytes(long delta)
{
if (delta == 0) {
return NOT_BLOCKED;
}
ListenableFuture<Void> blocked = delegate.addBytes(delta);
updateMemoryFuture(blocked, memoryFuture);
allocationListener.run();
return blocked;
}

@Override
public boolean trySetBytes(long bytes)
{
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -415,6 +415,12 @@ public ListenableFuture<Void> setBytes(long bytes)
return immediateVoidFuture();
}

@Override
public ListenableFuture<Void> addBytes(long delta)
{
return immediateVoidFuture();
}

@Override
public boolean trySetBytes(long bytes)
{
Expand Down
5 changes: 5 additions & 0 deletions lib/trino-filesystem-azure/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -74,6 +74,11 @@
<classifier>classes</classifier>
</dependency>

<dependency>
<groupId>io.airlift</groupId>
<artifactId>concurrent</artifactId>
</dependency>

<dependency>
<groupId>io.airlift</groupId>
<artifactId>configuration</artifactId>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -60,6 +60,7 @@
import java.util.Optional;
import java.util.OptionalLong;
import java.util.Set;
import java.util.concurrent.ExecutorService;

import static com.azure.storage.common.implementation.Constants.HeaderConstants.ETAG_WILDCARD;
import static com.google.common.base.Preconditions.checkArgument;
Expand All @@ -80,25 +81,30 @@ public class AzureFileSystem
implements TrinoFileSystem
{
private final HttpClient httpClient;
private final ExecutorService uploadExecutor;
private final TracingOptions tracingOptions;
private final AzureAuth azureAuth;
private final String endpoint;
private final int readBlockSizeBytes;
private final long writeBlockSizeBytes;
private final int maxWriteConcurrency;
private final long maxSingleUploadSizeBytes;
private final boolean multipartWriteEnabled;

public AzureFileSystem(
HttpClient httpClient,
ExecutorService uploadExecutor,
TracingOptions tracingOptions,
AzureAuth azureAuth,
String endpoint,
DataSize readBlockSize,
DataSize writeBlockSize,
int maxWriteConcurrency,
DataSize maxSingleUploadSize)
DataSize maxSingleUploadSize,
boolean multipartWriteEnabled)
{
this.httpClient = requireNonNull(httpClient, "httpClient is null");
this.uploadExecutor = requireNonNull(uploadExecutor, "uploadExecutor is null");
this.tracingOptions = requireNonNull(tracingOptions, "tracingOptions is null");
this.azureAuth = requireNonNull(azureAuth, "azureAuth is null");
this.endpoint = requireNonNull(endpoint, "endpoint is null");
Expand All @@ -107,6 +113,7 @@ public AzureFileSystem(
checkArgument(maxWriteConcurrency >= 0, "maxWriteConcurrency is negative");
this.maxWriteConcurrency = maxWriteConcurrency;
this.maxSingleUploadSizeBytes = maxSingleUploadSize.toBytes();
this.multipartWriteEnabled = multipartWriteEnabled;
}

@Override
Expand Down Expand Up @@ -162,15 +169,15 @@ public TrinoOutputFile newOutputFile(Location location)
{
AzureLocation azureLocation = new AzureLocation(location);
BlobClient client = createBlobClient(azureLocation, Optional.empty());
return new AzureOutputFile(azureLocation, client, writeBlockSizeBytes, maxWriteConcurrency, maxSingleUploadSizeBytes);
return new AzureOutputFile(azureLocation, client, uploadExecutor, writeBlockSizeBytes, maxWriteConcurrency, maxSingleUploadSizeBytes, multipartWriteEnabled);
}

@Override
public TrinoOutputFile newEncryptedOutputFile(Location location, EncryptionKey key)
{
AzureLocation azureLocation = new AzureLocation(location);
BlobClient client = createBlobClient(azureLocation, Optional.of(key));
return new AzureOutputFile(azureLocation, client, writeBlockSizeBytes, maxWriteConcurrency, maxSingleUploadSizeBytes);
return new AzureOutputFile(azureLocation, client, uploadExecutor, writeBlockSizeBytes, maxWriteConcurrency, maxSingleUploadSizeBytes, multipartWriteEnabled);
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -39,6 +39,7 @@ public enum AuthType
private DataSize maxSingleUploadSize = DataSize.of(4, Unit.MEGABYTE);
private Integer maxHttpRequests = 2 * Runtime.getRuntime().availableProcessors();
private String applicationId = "Trino";
private boolean multipartWriteEnabled;

@NotNull
public AuthType getAuthType()
Expand Down Expand Up @@ -145,4 +146,17 @@ public AzureFileSystemConfig setApplicationId(String applicationId)
this.applicationId = applicationId;
return this;
}

@Config("azure.multipart-write-enabled")
@ConfigDescription("Enable multipart writes for large files")
public AzureFileSystemConfig setMultipartWriteEnabled(boolean multipartWriteEnabled)
{
this.multipartWriteEnabled = multipartWriteEnabled;
return this;
}

public boolean isMultipartWriteEnabled()
{
return multipartWriteEnabled;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -31,13 +31,18 @@
import reactor.netty.resources.ConnectionProvider;

import java.util.concurrent.ExecutionException;
import java.util.concurrent.ExecutorService;

import static com.google.common.base.Preconditions.checkArgument;
import static io.airlift.concurrent.Threads.daemonThreadsNamed;
import static java.util.Objects.requireNonNull;
import static java.util.concurrent.Executors.newCachedThreadPool;

public class AzureFileSystemFactory
implements TrinoFileSystemFactory
{
private final ExecutorService uploadExecutor = newCachedThreadPool(daemonThreadsNamed("azure-upload-%s"));

private final AzureAuth auth;
private final String endpoint;
private final DataSize readBlockSize;
Expand All @@ -48,6 +53,7 @@ public class AzureFileSystemFactory
private final HttpClient httpClient;
private final ConnectionProvider connectionProvider;
private final EventLoopGroup eventLoopGroup;
private final boolean multipart;

@Inject
public AzureFileSystemFactory(OpenTelemetry openTelemetry, AzureAuth azureAuth, AzureFileSystemConfig config)
Expand All @@ -60,7 +66,8 @@ public AzureFileSystemFactory(OpenTelemetry openTelemetry, AzureAuth azureAuth,
config.getMaxWriteConcurrency(),
config.getMaxSingleUploadSize(),
config.getMaxHttpRequests(),
config.getApplicationId());
config.getApplicationId(),
config.isMultipartWriteEnabled());
}

public AzureFileSystemFactory(
Expand All @@ -72,7 +79,8 @@ public AzureFileSystemFactory(
int maxWriteConcurrency,
DataSize maxSingleUploadSize,
int maxHttpRequests,
String applicationId)
String applicationId,
boolean multipart)
{
this.auth = requireNonNull(azureAuth, "azureAuth is null");
this.endpoint = requireNonNull(endpoint, "endpoint is null");
Expand All @@ -88,11 +96,14 @@ public AzureFileSystemFactory(
clientOptions.setTracingOptions(tracingOptions);
clientOptions.setApplicationId(applicationId);
httpClient = createAzureHttpClient(connectionProvider, eventLoopGroup, clientOptions);
this.multipart = multipart;
}

@PreDestroy
public void destroy()
{
uploadExecutor.shutdown();

if (connectionProvider != null) {
connectionProvider.dispose();
}
Expand All @@ -113,7 +124,7 @@ public void destroy()
@Override
public TrinoFileSystem create(ConnectorIdentity identity)
{
return new AzureFileSystem(httpClient, tracingOptions, auth, endpoint, readBlockSize, writeBlockSize, maxWriteConcurrency, maxSingleUploadSize);
return new AzureFileSystem(httpClient, uploadExecutor, tracingOptions, auth, endpoint, readBlockSize, writeBlockSize, maxWriteConcurrency, maxSingleUploadSize, multipart);
}

public static HttpClient createAzureHttpClient(ConnectionProvider connectionProvider, EventLoopGroup eventLoopGroup, HttpClientOptions clientOptions)
Expand Down
Loading
Loading