kfaraz commented on code in PR #19143: URL: https://github.com/apache/druid/pull/19143#discussion_r2925973480
########## embedded-tests/src/test/java/org/apache/druid/testing/embedded/iceberg/ITIcebergRestCatalogIngestionTest.java: ########## @@ -0,0 +1,225 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 org.apache.druid.testing.embedded.iceberg; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import org.apache.druid.data.input.parquet.ParquetExtensionsModule; +import org.apache.druid.java.util.common.FileUtils; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.query.http.SqlTaskStatus; +import org.apache.druid.testing.embedded.EmbeddedBroker; +import org.apache.druid.testing.embedded.EmbeddedCoordinator; +import org.apache.druid.testing.embedded.EmbeddedDruidCluster; +import org.apache.druid.testing.embedded.EmbeddedHistorical; +import org.apache.druid.testing.embedded.EmbeddedIndexer; +import org.apache.druid.testing.embedded.EmbeddedOverlord; +import org.apache.druid.testing.embedded.junit5.EmbeddedClusterTestBase; +import org.apache.druid.testing.embedded.msq.EmbeddedMSQApis; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.data.parquet.GenericParquetWriter; +import org.apache.iceberg.io.DataWriter; +import org.apache.iceberg.io.OutputFile; +import org.apache.iceberg.parquet.Parquet; +import org.apache.iceberg.rest.RESTCatalog; +import org.apache.iceberg.types.Types; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; + +import java.io.File; +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; +import java.util.UUID; + +/** + * Ingestion test for Iceberg tables via a REST catalog. + * Exercises the classloader-sensitive {@code DynConstructors} path in + * {@code RestIcebergCatalog.setupCatalog()} (see #18015, #18017). + */ +public class ITIcebergRestCatalogIngestionTest extends EmbeddedClusterTestBase +{ + private static final String ICEBERG_NAMESPACE = "default"; + private static final String ICEBERG_TABLE_NAME = "test_events"; + + private static final Schema ICEBERG_SCHEMA = new Schema( + Types.NestedField.required(1, "event_time", Types.StringType.get()), + Types.NestedField.required(2, "name", Types.StringType.get()), + Types.NestedField.required(3, "value", Types.LongType.get()) + ); + + private final File warehouseDir = FileUtils.createTempDir(); + private final IcebergRestCatalogResource icebergCatalog = new IcebergRestCatalogResource(warehouseDir); + + private final EmbeddedOverlord overlord = new EmbeddedOverlord(); + private final EmbeddedCoordinator coordinator = new EmbeddedCoordinator(); + private final EmbeddedIndexer indexer = new EmbeddedIndexer() + .setServerMemory(300_000_000L) + .addProperty("druid.worker.capacity", "2"); + private final EmbeddedBroker broker = new EmbeddedBroker(); + + private EmbeddedMSQApis msqApis; + private RESTCatalog clientCatalog; Review Comment: The `clientCatalog` should probably be packaged inside the `IcebergRestCatalogResource` since the two will always be used in conjunction with each other. ########## embedded-tests/src/test/java/org/apache/druid/testing/embedded/iceberg/ITIcebergRestCatalogIngestionTest.java: ########## @@ -0,0 +1,225 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 org.apache.druid.testing.embedded.iceberg; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import org.apache.druid.data.input.parquet.ParquetExtensionsModule; +import org.apache.druid.java.util.common.FileUtils; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.query.http.SqlTaskStatus; +import org.apache.druid.testing.embedded.EmbeddedBroker; +import org.apache.druid.testing.embedded.EmbeddedCoordinator; +import org.apache.druid.testing.embedded.EmbeddedDruidCluster; +import org.apache.druid.testing.embedded.EmbeddedHistorical; +import org.apache.druid.testing.embedded.EmbeddedIndexer; +import org.apache.druid.testing.embedded.EmbeddedOverlord; +import org.apache.druid.testing.embedded.junit5.EmbeddedClusterTestBase; +import org.apache.druid.testing.embedded.msq.EmbeddedMSQApis; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.data.parquet.GenericParquetWriter; +import org.apache.iceberg.io.DataWriter; +import org.apache.iceberg.io.OutputFile; +import org.apache.iceberg.parquet.Parquet; +import org.apache.iceberg.rest.RESTCatalog; +import org.apache.iceberg.types.Types; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; + +import java.io.File; +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; +import java.util.UUID; + +/** + * Ingestion test for Iceberg tables via a REST catalog. + * Exercises the classloader-sensitive {@code DynConstructors} path in + * {@code RestIcebergCatalog.setupCatalog()} (see #18015, #18017). + */ +public class ITIcebergRestCatalogIngestionTest extends EmbeddedClusterTestBase +{ + private static final String ICEBERG_NAMESPACE = "default"; + private static final String ICEBERG_TABLE_NAME = "test_events"; + + private static final Schema ICEBERG_SCHEMA = new Schema( + Types.NestedField.required(1, "event_time", Types.StringType.get()), + Types.NestedField.required(2, "name", Types.StringType.get()), + Types.NestedField.required(3, "value", Types.LongType.get()) + ); + + private final File warehouseDir = FileUtils.createTempDir(); + private final IcebergRestCatalogResource icebergCatalog = new IcebergRestCatalogResource(warehouseDir); + + private final EmbeddedOverlord overlord = new EmbeddedOverlord(); + private final EmbeddedCoordinator coordinator = new EmbeddedCoordinator(); + private final EmbeddedIndexer indexer = new EmbeddedIndexer() + .setServerMemory(300_000_000L) + .addProperty("druid.worker.capacity", "2"); + private final EmbeddedBroker broker = new EmbeddedBroker(); + + private EmbeddedMSQApis msqApis; + private RESTCatalog clientCatalog; + + @Override + protected EmbeddedDruidCluster createCluster() + { + return EmbeddedDruidCluster + .withEmbeddedDerbyAndZookeeper() + .useLatchableEmitter() + .addResource(icebergCatalog) + .addExtension(ParquetExtensionsModule.class) + .addServer(overlord) + .addServer(coordinator) + .addServer(indexer) + .addServer(broker) + .addServer(new EmbeddedHistorical()); + } + + @BeforeAll + public void setupIcebergTable() throws IOException + { + msqApis = new EmbeddedMSQApis(cluster, overlord); + + clientCatalog = createClientCatalog(); + clientCatalog.createNamespace(Namespace.of(ICEBERG_NAMESPACE)); + + final TableIdentifier tableId = TableIdentifier.of(ICEBERG_NAMESPACE, ICEBERG_TABLE_NAME); + final Table table = clientCatalog.createTable(tableId, ICEBERG_SCHEMA); + writeTestData(table); + } + + @Test + public void testIngestFromIcebergRestCatalog() + { + final String catalogUri = icebergCatalog.getCatalogUri(); + + final String sql = StringUtils.format( + "INSERT INTO %s\n" + + "SELECT\n" + + " TIME_PARSE(\"event_time\") AS __time,\n" + + " \"name\",\n" + + " \"value\"\n" + + "FROM TABLE(\n" + + " EXTERN(\n" + + " '{\"type\":\"iceberg\"," + + "\"tableName\":\"%s\"," + + "\"namespace\":\"%s\"," + + "\"icebergCatalog\":{\"type\":\"rest\",\"catalogUri\":\"%s\"," + + "\"catalogProperties\":{\"io-impl\":\"org.apache.iceberg.hadoop.HadoopFileIO\"}}," + + "\"warehouseSource\":{\"type\":\"local\"}}',\n" + + " '{\"type\":\"parquet\"}',\n" + + " '[{\"type\":\"string\",\"name\":\"event_time\"}," + + "{\"type\":\"string\",\"name\":\"name\"}," + + "{\"type\":\"long\",\"name\":\"value\"}]'\n" + + " )\n" + + ")\n" + + "PARTITIONED BY ALL TIME", + dataSource, + ICEBERG_TABLE_NAME, + ICEBERG_NAMESPACE, + catalogUri + ); + + final SqlTaskStatus taskStatus = msqApis.submitTaskSql(sql); + cluster.callApi().waitForTaskToSucceed(taskStatus.getTaskId(), overlord); + cluster.callApi().waitForAllSegmentsToBeAvailable(dataSource, coordinator, broker); + + cluster.callApi().verifySqlQuery( + "SELECT __time, \"name\", \"value\" FROM %s ORDER BY __time", + dataSource, + "2024-01-01T00:00:00.000Z,alice,100\n" + + "2024-01-01T01:00:00.000Z,bob,200\n" + + "2024-01-01T02:00:00.000Z,charlie,300" + ); + } + + @AfterAll + public void tearDownIceberg() + { + if (clientCatalog != null) { + try { + clientCatalog.dropTable(TableIdentifier.of(ICEBERG_NAMESPACE, ICEBERG_TABLE_NAME)); + clientCatalog.dropNamespace(Namespace.of(ICEBERG_NAMESPACE)); Review Comment: The `IcebergRestCatalogResource` should expose methods to create and drop namespaces and tables. ########## embedded-tests/src/test/java/org/apache/druid/testing/embedded/iceberg/IcebergRestCatalogResource.java: ########## @@ -0,0 +1,87 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 org.apache.druid.testing.embedded.iceberg; + +import org.apache.druid.iceberg.common.IcebergDruidModule; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.testing.embedded.EmbeddedDruidCluster; +import org.apache.druid.testing.embedded.TestcontainerResource; +import org.testcontainers.containers.BindMode; +import org.testcontainers.containers.GenericContainer; +import org.testcontainers.containers.wait.strategy.Wait; + +import java.io.File; + +/** + * Testcontainer resource wrapping {@code tabulario/iceberg-rest} with a local + * filesystem warehouse shared via bind mount. + */ +public class IcebergRestCatalogResource extends TestcontainerResource<GenericContainer<?>> +{ + // 1.6.0 is the latest available tag; the REST protocol is compatible with the 1.6.1 client libraries + private static final String ICEBERG_REST_IMAGE = "tabulario/iceberg-rest:1.6.0"; + private static final int REST_CATALOG_PORT = 8181; + private static final String CONTAINER_WAREHOUSE_PATH = "/tmp/iceberg-warehouse"; + + private final File warehouseDir; + + public IcebergRestCatalogResource(File warehouseDir) Review Comment: Instead of passing a `warehouseDir` in the constructor, override the `beforeStart` method and use `cluster.getTestFolder().getOrCreateNewFolder("iceberg-warehouse")`. ########## embedded-tests/src/test/java/org/apache/druid/testing/embedded/iceberg/ITIcebergRestCatalogIngestionTest.java: ########## @@ -0,0 +1,225 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 org.apache.druid.testing.embedded.iceberg; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import org.apache.druid.data.input.parquet.ParquetExtensionsModule; +import org.apache.druid.java.util.common.FileUtils; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.query.http.SqlTaskStatus; +import org.apache.druid.testing.embedded.EmbeddedBroker; +import org.apache.druid.testing.embedded.EmbeddedCoordinator; +import org.apache.druid.testing.embedded.EmbeddedDruidCluster; +import org.apache.druid.testing.embedded.EmbeddedHistorical; +import org.apache.druid.testing.embedded.EmbeddedIndexer; +import org.apache.druid.testing.embedded.EmbeddedOverlord; +import org.apache.druid.testing.embedded.junit5.EmbeddedClusterTestBase; +import org.apache.druid.testing.embedded.msq.EmbeddedMSQApis; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.data.parquet.GenericParquetWriter; +import org.apache.iceberg.io.DataWriter; +import org.apache.iceberg.io.OutputFile; +import org.apache.iceberg.parquet.Parquet; +import org.apache.iceberg.rest.RESTCatalog; +import org.apache.iceberg.types.Types; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; + +import java.io.File; +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; +import java.util.UUID; + +/** + * Ingestion test for Iceberg tables via a REST catalog. + * Exercises the classloader-sensitive {@code DynConstructors} path in + * {@code RestIcebergCatalog.setupCatalog()} (see #18015, #18017). + */ +public class ITIcebergRestCatalogIngestionTest extends EmbeddedClusterTestBase +{ + private static final String ICEBERG_NAMESPACE = "default"; + private static final String ICEBERG_TABLE_NAME = "test_events"; + + private static final Schema ICEBERG_SCHEMA = new Schema( + Types.NestedField.required(1, "event_time", Types.StringType.get()), + Types.NestedField.required(2, "name", Types.StringType.get()), + Types.NestedField.required(3, "value", Types.LongType.get()) + ); + + private final File warehouseDir = FileUtils.createTempDir(); + private final IcebergRestCatalogResource icebergCatalog = new IcebergRestCatalogResource(warehouseDir); + + private final EmbeddedOverlord overlord = new EmbeddedOverlord(); + private final EmbeddedCoordinator coordinator = new EmbeddedCoordinator(); + private final EmbeddedIndexer indexer = new EmbeddedIndexer() + .setServerMemory(300_000_000L) + .addProperty("druid.worker.capacity", "2"); + private final EmbeddedBroker broker = new EmbeddedBroker(); + + private EmbeddedMSQApis msqApis; + private RESTCatalog clientCatalog; + + @Override + protected EmbeddedDruidCluster createCluster() + { + return EmbeddedDruidCluster + .withEmbeddedDerbyAndZookeeper() + .useLatchableEmitter() + .addResource(icebergCatalog) + .addExtension(ParquetExtensionsModule.class) + .addServer(overlord) + .addServer(coordinator) + .addServer(indexer) + .addServer(broker) + .addServer(new EmbeddedHistorical()); + } + + @BeforeAll + public void setupIcebergTable() throws IOException + { + msqApis = new EmbeddedMSQApis(cluster, overlord); + + clientCatalog = createClientCatalog(); + clientCatalog.createNamespace(Namespace.of(ICEBERG_NAMESPACE)); + + final TableIdentifier tableId = TableIdentifier.of(ICEBERG_NAMESPACE, ICEBERG_TABLE_NAME); + final Table table = clientCatalog.createTable(tableId, ICEBERG_SCHEMA); + writeTestData(table); + } + + @Test + public void testIngestFromIcebergRestCatalog() + { + final String catalogUri = icebergCatalog.getCatalogUri(); + + final String sql = StringUtils.format( + "INSERT INTO %s\n" + + "SELECT\n" + + " TIME_PARSE(\"event_time\") AS __time,\n" + + " \"name\",\n" + + " \"value\"\n" + + "FROM TABLE(\n" + + " EXTERN(\n" + + " '{\"type\":\"iceberg\"," + + "\"tableName\":\"%s\"," + + "\"namespace\":\"%s\"," + + "\"icebergCatalog\":{\"type\":\"rest\",\"catalogUri\":\"%s\"," + + "\"catalogProperties\":{\"io-impl\":\"org.apache.iceberg.hadoop.HadoopFileIO\"}}," + + "\"warehouseSource\":{\"type\":\"local\"}}',\n" + + " '{\"type\":\"parquet\"}',\n" + + " '[{\"type\":\"string\",\"name\":\"event_time\"}," + + "{\"type\":\"string\",\"name\":\"name\"}," + + "{\"type\":\"long\",\"name\":\"value\"}]'\n" + + " )\n" + + ")\n" + + "PARTITIONED BY ALL TIME", + dataSource, + ICEBERG_TABLE_NAME, + ICEBERG_NAMESPACE, + catalogUri + ); + + final SqlTaskStatus taskStatus = msqApis.submitTaskSql(sql); + cluster.callApi().waitForTaskToSucceed(taskStatus.getTaskId(), overlord); + cluster.callApi().waitForAllSegmentsToBeAvailable(dataSource, coordinator, broker); + + cluster.callApi().verifySqlQuery( + "SELECT __time, \"name\", \"value\" FROM %s ORDER BY __time", + dataSource, + "2024-01-01T00:00:00.000Z,alice,100\n" + + "2024-01-01T01:00:00.000Z,bob,200\n" + + "2024-01-01T02:00:00.000Z,charlie,300" + ); + } + + @AfterAll + public void tearDownIceberg() + { + if (clientCatalog != null) { + try { + clientCatalog.dropTable(TableIdentifier.of(ICEBERG_NAMESPACE, ICEBERG_TABLE_NAME)); + clientCatalog.dropNamespace(Namespace.of(ICEBERG_NAMESPACE)); + } + catch (Exception e) { + // Best-effort cleanup + } + try { + clientCatalog.close(); + } + catch (Exception e) { + // Best-effort cleanup + } + } + org.apache.commons.io.FileUtils.deleteQuietly(warehouseDir); + } + + private RESTCatalog createClientCatalog() Review Comment: This method would move inside the `IcebergRestCatalogResource`. ########## embedded-tests/src/test/java/org/apache/druid/testing/embedded/iceberg/ITIcebergRestCatalogIngestionTest.java: ########## @@ -0,0 +1,225 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 org.apache.druid.testing.embedded.iceberg; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import org.apache.druid.data.input.parquet.ParquetExtensionsModule; +import org.apache.druid.java.util.common.FileUtils; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.query.http.SqlTaskStatus; +import org.apache.druid.testing.embedded.EmbeddedBroker; +import org.apache.druid.testing.embedded.EmbeddedCoordinator; +import org.apache.druid.testing.embedded.EmbeddedDruidCluster; +import org.apache.druid.testing.embedded.EmbeddedHistorical; +import org.apache.druid.testing.embedded.EmbeddedIndexer; +import org.apache.druid.testing.embedded.EmbeddedOverlord; +import org.apache.druid.testing.embedded.junit5.EmbeddedClusterTestBase; +import org.apache.druid.testing.embedded.msq.EmbeddedMSQApis; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.data.parquet.GenericParquetWriter; +import org.apache.iceberg.io.DataWriter; +import org.apache.iceberg.io.OutputFile; +import org.apache.iceberg.parquet.Parquet; +import org.apache.iceberg.rest.RESTCatalog; +import org.apache.iceberg.types.Types; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; + +import java.io.File; +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; +import java.util.UUID; + +/** + * Ingestion test for Iceberg tables via a REST catalog. + * Exercises the classloader-sensitive {@code DynConstructors} path in + * {@code RestIcebergCatalog.setupCatalog()} (see #18015, #18017). + */ +public class ITIcebergRestCatalogIngestionTest extends EmbeddedClusterTestBase Review Comment: We can probably drop the `IT` prefix since these tests run like regular unit-tests. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected] --------------------------------------------------------------------- To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
