Skip to content

Commit

Permalink
Add endpointOverride S3 support (deephaven#5087)
Browse files Browse the repository at this point in the history
  • Loading branch information
devinrsmith authored Feb 7, 2024
1 parent e2deaa8 commit f77e85a
Show file tree
Hide file tree
Showing 18 changed files with 647 additions and 114 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -27,7 +27,7 @@
import io.deephaven.engine.testutil.junit4.EngineCleanup;
import io.deephaven.engine.util.BigDecimalUtils;
import io.deephaven.engine.util.file.TrackedFileHandleFactory;
import io.deephaven.extensions.s3.AwsCredentials;
import io.deephaven.extensions.s3.Credentials;
import io.deephaven.parquet.base.NullStatistics;
import io.deephaven.parquet.base.InvalidParquetFileException;
import io.deephaven.parquet.table.location.ParquetTableLocationKey;
Expand Down Expand Up @@ -583,13 +583,13 @@ public void testArrayColumns() {
public void readSampleParquetFilesFromS3Test1() {
Assume.assumeTrue("Skipping test because s3 testing disabled.", ENABLE_S3_TESTING);
final S3Instructions s3Instructions = S3Instructions.builder()
.awsRegionName("us-east-1")
.regionName("us-east-1")
.readAheadCount(1)
.fragmentSize(5 * 1024 * 1024)
.maxConcurrentRequests(50)
.maxCacheSize(32)
.readTimeout(Duration.ofSeconds(60))
.credentials(AwsCredentials.defaultCredentials())
.credentials(Credentials.defaultCredentials())
.build();
final ParquetInstructions readInstructions = new ParquetInstructions.Builder()
.setSpecialInstructions(s3Instructions)
Expand Down Expand Up @@ -621,7 +621,7 @@ public void readSampleParquetFilesFromS3Test1() {
public void readSampleParquetFilesFromS3Test2() {
Assume.assumeTrue("Skipping test because s3 testing disabled.", ENABLE_S3_TESTING);
final S3Instructions s3Instructions = S3Instructions.builder()
.awsRegionName("us-east-2")
.regionName("us-east-2")
.readAheadCount(1)
.fragmentSize(5 * 1024 * 1024)
.maxConcurrentRequests(50)
Expand Down
32 changes: 32 additions & 0 deletions extensions/s3/build.gradle
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,38 @@ dependencies {

compileOnly depAnnotations

// For OSUtil
testImplementation project(':Stats')

Classpaths.inheritAutoService(project)
Classpaths.inheritImmutables(project)

Classpaths.inheritJUnitPlatform(project)
Classpaths.inheritAssertJ(project)
testImplementation 'org.junit.jupiter:junit-jupiter'
testRuntimeOnly 'org.junit.jupiter:junit-jupiter-engine'

testImplementation "org.testcontainers:testcontainers:1.19.4"
testImplementation "org.testcontainers:junit-jupiter:1.19.4"
testImplementation "org.testcontainers:localstack:1.19.4"
testImplementation "org.testcontainers:minio:1.19.4"

testRuntimeOnly project(':test-configs')
testRuntimeOnly project(':log-to-slf4j')
Classpaths.inheritSlf4j(project, 'slf4j-simple', 'testRuntimeOnly')
}

test {
useJUnitPlatform {
excludeTags("testcontainers")
}
}

tasks.register('testOutOfBand', Test) {
useJUnitPlatform {
includeTags("testcontainers")
}
systemProperty 'testcontainers.localstack.image', project.property('testcontainers.localstack.image')
systemProperty 'testcontainers.minio.image', project.property('testcontainers.minio.image')
}

4 changes: 4 additions & 0 deletions extensions/s3/gradle.properties
Original file line number Diff line number Diff line change
@@ -1 +1,5 @@
io.deephaven.project.ProjectType=JAVA_PUBLIC

# TODO(deephaven-core#5115): EPIC: Dependency management
testcontainers.localstack.image=localstack/localstack:3.1.0
testcontainers.minio.image=minio/minio:RELEASE.2024-02-04T22-36-13Z

This file was deleted.

Original file line number Diff line number Diff line change
Expand Up @@ -5,7 +5,7 @@

import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider;

interface AwsSdkV2Credentials extends AwsCredentials {
interface AwsSdkV2Credentials extends Credentials {

AwsCredentialsProvider awsCredentialsProvider();
AwsCredentialsProvider awsV2CredentialsProvider();
}
Original file line number Diff line number Diff line change
Expand Up @@ -11,25 +11,25 @@
import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;

/**
* AWS credentials provider that uses access key and secret key provided at construction.
* Basic credentials that uses access key id and secret access key provided at construction.
*/
@Immutable
@SimpleStyle
abstract class BasicCredentials implements AwsSdkV2Credentials {

static BasicCredentials of(final String awsAccessKeyId, final String awsSecretAccessKey) {
return ImmutableBasicCredentials.of(awsAccessKeyId, awsSecretAccessKey);
static BasicCredentials of(final String accessKeyId, final String secretAccessKey) {
return ImmutableBasicCredentials.of(accessKeyId, secretAccessKey);
}

@Value.Parameter
abstract String awsAccessKeyId();
abstract String accessKeyId();

@Value.Redacted
@Value.Parameter
abstract String awsSecretAccessKey();
abstract String secretAccessKey();

public AwsCredentialsProvider awsCredentialsProvider() {
final AwsBasicCredentials awsCreds = AwsBasicCredentials.create(awsAccessKeyId(), awsSecretAccessKey());
return StaticCredentialsProvider.create(awsCreds);
@Override
public final AwsCredentialsProvider awsV2CredentialsProvider() {
return StaticCredentialsProvider.create(AwsBasicCredentials.create(accessKeyId(), secretAccessKey()));
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,27 @@
/**
* Copyright (c) 2016-2022 Deephaven Data Labs and Patent Pending
*/
package io.deephaven.extensions.s3;

public interface Credentials {

/**
* The default credentials.
*
* @see <a href="https://docs.aws.amazon.com/sdk-for-java/latest/developer-guide/credentials-chain.html">Default
* credentials provider chain</a>
*/
static Credentials defaultCredentials() {
return DefaultCredentials.DEFAULT_CREDENTIALS;
}

/**
* Basic credentials with the specified access key id and secret access key.
*
* @param accessKeyId the access key id, used to identify the user
* @param secretAccessKey the secret access key, used to authenticate the user
*/
static Credentials basicCredentials(final String accessKeyId, final String secretAccessKey) {
return BasicCredentials.of(accessKeyId, secretAccessKey);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -14,7 +14,7 @@ enum DefaultCredentials implements AwsSdkV2Credentials {
DEFAULT_CREDENTIALS;

@Override
public AwsCredentialsProvider awsCredentialsProvider() {
public final AwsCredentialsProvider awsV2CredentialsProvider() {
return DefaultCredentialsProvider.create();
}
}
Loading

0 comments on commit f77e85a

Please sign in to comment.