Skip to content

Commit

Permalink
review comments
Browse files Browse the repository at this point in the history
  • Loading branch information
twuebi committed Feb 26, 2025
1 parent fec2e69 commit d377ecc
Show file tree
Hide file tree
Showing 2 changed files with 49 additions and 33 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -40,8 +40,11 @@

import static io.trino.plugin.iceberg.IcebergTestUtils.*;
import static io.trino.testing.TestingNames.randomNameSuffix;
import static java.lang.String.format;
import static io.trino.testing.containers.Minio.MINIO_SECRET_KEY;
import static io.trino.testing.containers.Minio.MINIO_ACCESS_KEY;
import static org.apache.iceberg.FileFormat.PARQUET;

import static java.lang.String.format;
import static org.assertj.core.api.Assertions.assertThatThrownBy;
import static org.assertj.core.api.AssertionsForInterfaceTypes.assertThat;
import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS;
Expand Down Expand Up @@ -80,21 +83,21 @@ protected QueryRunner createQueryRunner()
.addIcebergProperty("iceberg.rest-catalog.nested-namespace-enabled", "true")
.addIcebergProperty("iceberg.rest-catalog.uri", lakekeeperCatalog.restUri() + "/catalog")
.addIcebergProperty("iceberg.rest-catalog.warehouse", TestingLakekeeperCatalog.DEFAULT_WAREHOUSE)
.addIcebergProperty("s3.endpoint", "http://localhost:" + lakekeeperCatalog.minioPort())
.addIcebergProperty("s3.endpoint", lakekeeperCatalog.externalMinioAddress())
.addIcebergProperty("fs.native-s3.enabled", "true")
.addIcebergProperty("s3.region", "dummy")
.addIcebergProperty("s3.path-style-access", "true")
// we need to hard-code this here since vended-credentials-enabled cannot be true while register-table-procedure is true
// and we need register-table-procedure to be true since the tests.
.addIcebergProperty("s3.aws-access-key", TestingLakekeeperCatalog.MINIO_ROOT_USER)
.addIcebergProperty("s3.aws-secret-key", TestingLakekeeperCatalog.MINIO_ROOT_PASSWORD)
.addIcebergProperty("s3.aws-access-key", MINIO_ACCESS_KEY)
.addIcebergProperty("s3.aws-secret-key", MINIO_SECRET_KEY)
.setInitialTables(REQUIRED_TPCH_TABLES)
.build();
}

@Override
protected void dropTableFromMetastore(String tableName) {
lakekeeperCatalog.drop_without_purge(getSession().getSchema().orElseThrow(), tableName);
lakekeeperCatalog.dropWithoutPurge(getSession().getSchema().orElseThrow(), tableName);
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -32,6 +32,9 @@
import static io.airlift.http.client.StaticBodyGenerator.createStaticBodyGenerator;
import static io.airlift.http.client.StatusResponseHandler.createStatusResponseHandler;
import static io.airlift.http.client.JsonResponseHandler.createJsonResponseHandler;
import static io.trino.testing.containers.Minio.MINIO_SECRET_KEY;
import static io.trino.testing.containers.Minio.MINIO_ACCESS_KEY;
import static io.trino.testing.containers.Minio.DEFAULT_IMAGE;
import static java.nio.charset.StandardCharsets.UTF_8;
import static java.util.Objects.requireNonNull;

Expand All @@ -41,8 +44,14 @@ public final class TestingLakekeeperCatalog
public static final String DEFAULT_BUCKET = "examples";
public static final String DEFAULT_WAREHOUSE = "lakekeeper";
public static final String DEFAULT_PROJECT_ID = "00000000-0000-0000-0000-000000000000";
public static final String MINIO_ROOT_USER = "minio-root-user";
public static final String MINIO_ROOT_PASSWORD = "minio-root-password";

public static final String LAKEKEEPER_DB_URL_WRITE_KEY = "LAKEKEEPER__PG_DATABASE_URL_WRITE";
private static final String LAKEKEEPER_DB_URL_READ_KEY = "LAKEKEEPER__PG_DATABASE_URL_READ";
public static final String LAKEKEEPER_PG_ENCRYPTION_KEY = "LAKEKEEPER__PG_ENCRYPTION_KEY";
public static final String PG_ENCRYPTION_KEY = "This-is-NOT-Secure!";
public static final String POSTGRES_IMAGE = "postgres:16";

public static final String LAKEKEEPER_CATALOG_IMAGE = "quay.io/lakekeeper/catalog:v0.7.0";

private static final int MINIO_PORT = 9000;
private static final String MINIO_ALIAS = "minio-1";
Expand All @@ -54,36 +63,36 @@ public final class TestingLakekeeperCatalog
private final GenericContainer<?> migrator;
private final PostgreSQLContainer<?> postgresqlContainer;
private final GenericContainer<?> minio;
private final Network net;
private final Network network;


public TestingLakekeeperCatalog() {
this("1s");
}

public TestingLakekeeperCatalog(String task_queue_poll_interval) {
String lakekeeperDbUrlReadKey = "LAKEKEEPER__PG_DATABASE_URL_READ";
String lakekeeperDbUrlWriteKey = "LAKEKEEPER__PG_DATABASE_URL_WRITE";
String lakekeeperPGEncryptionKey = "LAKEKEEPER__PG_ENCRYPTION_KEY";
String lakekeeperCatalogImage = "quay.io/lakekeeper/catalog:v0.7.0";
String pgEncryptionKey = "This-is-NOT-Secure!";

net = NetworkImpl.SHARED;
network = NetworkImpl.SHARED;

postgresqlContainer = new PostgreSQLContainer<>("postgres:16");
postgresqlContainer.withNetwork(net).start();
postgresqlContainer = new PostgreSQLContainer<>(POSTGRES_IMAGE);
postgresqlContainer.withNetwork(network).start();

String pgConnectionString = "postgresql://" + postgresqlContainer.getUsername() + ":" + postgresqlContainer.getPassword() + "@" + postgresqlContainer.getContainerName().substring(1) + ":" + PostgreSQLContainer.POSTGRESQL_PORT + "/" + postgresqlContainer.getDatabaseName();
String pgConnectionString = String.format("postgresql://%s:%s@%s:%d/%s",
postgresqlContainer.getUsername(),
postgresqlContainer.getPassword(),
postgresqlContainer.getContainerName().substring(1),
PostgreSQLContainer.POSTGRESQL_PORT,
postgresqlContainer.getDatabaseName());

migrator = new GenericContainer<>(lakekeeperCatalogImage);
migrator.withEnv(lakekeeperPGEncryptionKey, pgEncryptionKey)
.withEnv(lakekeeperDbUrlReadKey, pgConnectionString)
.withEnv(lakekeeperDbUrlWriteKey, pgConnectionString).withNetwork(postgresqlContainer.getNetwork())
migrator = new GenericContainer<>(LAKEKEEPER_CATALOG_IMAGE);
migrator.withEnv(LAKEKEEPER_PG_ENCRYPTION_KEY, PG_ENCRYPTION_KEY)
.withEnv(LAKEKEEPER_DB_URL_READ_KEY, pgConnectionString)
.withEnv(LAKEKEEPER_DB_URL_WRITE_KEY, pgConnectionString).withNetwork(postgresqlContainer.getNetwork())
.withCommand("migrate").start();

minio = new GenericContainer<>("bitnami/minio:latest");
minio.withEnv(Map.of("MINIO_ROOT_USER", MINIO_ROOT_USER,
"MINIO_ROOT_PASSWORD", MINIO_ROOT_PASSWORD,
minio = new GenericContainer<>(DEFAULT_IMAGE);
minio.withEnv(Map.of("MINIO_ROOT_USER", MINIO_ACCESS_KEY,
"MINIO_ROOT_PASSWORD", MINIO_SECRET_KEY,
"MINIO_API_PORT_NUMBER", "%d".formatted(MINIO_PORT),
"MINIO_CONSOLE_PORT_NUMBER", "9001",
"MINIO_SCHEME", "http",
Expand All @@ -93,10 +102,10 @@ public TestingLakekeeperCatalog(String task_queue_poll_interval) {
.waitingFor(Wait.forSuccessfulCommand("mc ls local | grep examples"))
.withExposedPorts(MINIO_PORT, 9001).start();

lakekeeperCatalog = new GenericContainer<>(lakekeeperCatalogImage);
lakekeeperCatalog.withEnv(Map.of(lakekeeperPGEncryptionKey, pgEncryptionKey,
lakekeeperDbUrlReadKey, pgConnectionString,
lakekeeperDbUrlWriteKey, pgConnectionString,
lakekeeperCatalog = new GenericContainer<>(LAKEKEEPER_CATALOG_IMAGE);
lakekeeperCatalog.withEnv(Map.of(LAKEKEEPER_PG_ENCRYPTION_KEY, PG_ENCRYPTION_KEY,
LAKEKEEPER_DB_URL_READ_KEY, pgConnectionString,
LAKEKEEPER_DB_URL_WRITE_KEY, pgConnectionString,
"LAKEKEEPER__QUEUE_CONFIG__POLL_INTERVAL", String.format("\"%s\"", task_queue_poll_interval)))
.withNetwork(postgresqlContainer.getNetwork())
.withCommand("serve").withExposedPorts(LAKEKEEPER_PORT)
Expand All @@ -114,7 +123,7 @@ public TestingLakekeeperCatalog(String task_queue_poll_interval) {

}

public void drop_without_purge(String schema, String table) {
public void dropWithoutPurge(String schema, String table) {
Request request = Request.Builder.prepareDelete()
.setUri(URI.create(restUri() + "/catalog/v1/" + prefix + "/namespaces/" + schema + "/tables/" + table + "?purgeRequested=false"))
.setHeader("Content-Type", "application/json")
Expand All @@ -138,10 +147,14 @@ public void close() {
lakekeeperCatalog.close();
postgresqlContainer.close();
minio.close();
net.close();
network.close();
}

public String externalMinioAddress() {
return "http://localhost:%d".formatted(minioPort());
}

public int minioPort() {
private int minioPort() {
return this.minio.getMappedPort(MINIO_PORT);
}

Expand Down Expand Up @@ -207,8 +220,8 @@ private static Map<String, Object> storageCredential() {
return Map.of(
"type", "s3",
"credential-type", "access-key",
"aws-access-key-id", MINIO_ROOT_USER,
"aws-secret-access-key", MINIO_ROOT_PASSWORD
"aws-access-key-id", MINIO_ACCESS_KEY,
"aws-secret-access-key", MINIO_SECRET_KEY
);
}
}

0 comments on commit d377ecc

Please # to comment.