From a0fe4534fbaa78523e7137104ee65ddb9ebee234 Mon Sep 17 00:00:00 2001 From: Atul Mohan Date: Fri, 20 Sep 2024 11:20:00 -0700 Subject: [PATCH 1/3] Iceberg extension changes to support reads from REST catalogs --- .../development/extensions-contrib/iceberg.md | 7 +- docs/ingestion/input-sources.md | 11 +- .../druid-iceberg-extensions/pom.xml | 6 +- .../iceberg/common/IcebergDruidModule.java | 2 + .../iceberg/input/HiveIcebergCatalog.java | 7 +- .../druid/iceberg/input/IcebergCatalog.java | 4 +- .../druid/iceberg/input/LocalCatalog.java | 6 +- .../iceberg/input/RestIcebergCatalog.java | 105 ++++++++++++++++++ .../druid/iceberg/input/RestCatalogTest.java | 99 +++++++++++++++++ 9 files changed, 231 insertions(+), 16 deletions(-) create mode 100644 extensions-contrib/druid-iceberg-extensions/src/main/java/org/apache/druid/iceberg/input/RestIcebergCatalog.java create mode 100644 extensions-contrib/druid-iceberg-extensions/src/test/java/org/apache/druid/iceberg/input/RestCatalogTest.java diff --git a/docs/development/extensions-contrib/iceberg.md b/docs/development/extensions-contrib/iceberg.md index f4a1735ed3bb..989e64237958 100644 --- a/docs/development/extensions-contrib/iceberg.md +++ b/docs/development/extensions-contrib/iceberg.md @@ -31,7 +31,7 @@ Iceberg refers to these metastores as catalogs. The Iceberg extension lets you c * Hive metastore catalog * Local catalog -Druid does not support AWS Glue and REST based catalogs yet. +Druid does not support AWS Glue catalog yet. For a given catalog, Iceberg input source reads the table name from the catalog, applies the filters, and extracts all the underlying live data files up to the latest snapshot. The data files can be in Parquet, ORC, or Avro formats. The data files typically reside in a warehouse location, which can be in HDFS, S3, or the local filesystem. @@ -107,6 +107,11 @@ Since the Hadoop AWS connector uses the `s3a` filesystem client, specify the war The local catalog type can be used for catalogs configured on the local filesystem. Set the `icebergCatalog` type to `local`. You can use this catalog for demos or localized tests. It is not recommended for production use cases. The `warehouseSource` is set to `local` because this catalog only supports reading from a local filesystem. +## REST catalog + +To connect to an Iceberg REST Catalog server, configure the `icebergCatalog` type as `rest`. The Iceberg REST Open API spec gives catalogs greater control over the implementation and in most cases, the `warehousePath` does not have to be provided by the client. +Security credentials may be provided in the `catalogProperties` object. + ## Downloading Iceberg extension To download `druid-iceberg-extensions`, run the following command after replacing `` with the desired diff --git a/docs/ingestion/input-sources.md b/docs/ingestion/input-sources.md index fb8e1f98c91f..3aeecb20e07e 100644 --- a/docs/ingestion/input-sources.md +++ b/docs/ingestion/input-sources.md @@ -1063,7 +1063,7 @@ The following is a sample spec for a S3 warehouse source: ### Catalog Object -The catalog object supports `local` and `hive` catalog types. +The catalog object supports `local`,`hive` and `rest` catalog types. The following table lists the properties of a `local` catalog: @@ -1084,9 +1084,18 @@ The following table lists the properties of a `hive` catalog: |catalogProperties|Map of any additional properties that needs to be attached to the catalog.|None|no| |caseSensitive|Toggle case sensitivity for column names during Iceberg table reads.|true|no| +The following table lists the properties of a `rest` catalog: + +|Property|Description|Default|Required| +|--------|-----------|-------|---------| +|type|Set this value to `rest`.|None|yes| +|catalogUri|The URI associated with the catalog's HTTP endpoint.|None|yes| +|catalogProperties|Map of any additional properties that needs to be attached to the catalog.|None|no| + ### Iceberg filter object This input source provides the following filters: `and`, `equals`, `interval`, and `or`. You can use these filters to filter out data files from a snapshot, reducing the number of files Druid has to ingest. +If the filter column is not an Iceberg partition column, it is highly recommended to define an additional filter defined in the [`transformSpec`](./ingestion-spec.md#transformspec). This is because for non-partition columns, Iceberg filters may return rows that do not match the expression. `equals` Filter: diff --git a/extensions-contrib/druid-iceberg-extensions/pom.xml b/extensions-contrib/druid-iceberg-extensions/pom.xml index e3c996461e76..8ee359149a98 100644 --- a/extensions-contrib/druid-iceberg-extensions/pom.xml +++ b/extensions-contrib/druid-iceberg-extensions/pom.xml @@ -35,7 +35,7 @@ 4.0.0 - 1.4.1 + 1.6.1 3.1.3 @@ -258,10 +258,6 @@ io.airlift aircompressor - - org.apache.httpcomponents.client5 - httpclient5 - diff --git a/extensions-contrib/druid-iceberg-extensions/src/main/java/org/apache/druid/iceberg/common/IcebergDruidModule.java b/extensions-contrib/druid-iceberg-extensions/src/main/java/org/apache/druid/iceberg/common/IcebergDruidModule.java index 61ff8f878c44..418b63850b81 100644 --- a/extensions-contrib/druid-iceberg-extensions/src/main/java/org/apache/druid/iceberg/common/IcebergDruidModule.java +++ b/extensions-contrib/druid-iceberg-extensions/src/main/java/org/apache/druid/iceberg/common/IcebergDruidModule.java @@ -28,6 +28,7 @@ import org.apache.druid.iceberg.input.HiveIcebergCatalog; import org.apache.druid.iceberg.input.IcebergInputSource; import org.apache.druid.iceberg.input.LocalCatalog; +import org.apache.druid.iceberg.input.RestIcebergCatalog; import org.apache.druid.initialization.DruidModule; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; @@ -45,6 +46,7 @@ public List getJacksonModules() .registerSubtypes( new NamedType(HiveIcebergCatalog.class, HiveIcebergCatalog.TYPE_KEY), new NamedType(LocalCatalog.class, LocalCatalog.TYPE_KEY), + new NamedType(RestIcebergCatalog.class, RestIcebergCatalog.TYPE_KEY), new NamedType(IcebergInputSource.class, IcebergInputSource.TYPE_KEY) ) diff --git a/extensions-contrib/druid-iceberg-extensions/src/main/java/org/apache/druid/iceberg/input/HiveIcebergCatalog.java b/extensions-contrib/druid-iceberg-extensions/src/main/java/org/apache/druid/iceberg/input/HiveIcebergCatalog.java index 7b659d031296..f3e47ab32bdd 100644 --- a/extensions-contrib/druid-iceberg-extensions/src/main/java/org/apache/druid/iceberg/input/HiveIcebergCatalog.java +++ b/extensions-contrib/druid-iceberg-extensions/src/main/java/org/apache/druid/iceberg/input/HiveIcebergCatalog.java @@ -32,7 +32,7 @@ import org.apache.druid.utils.DynamicConfigProviderUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.security.UserGroupInformation; -import org.apache.iceberg.BaseMetastoreCatalog; +import org.apache.iceberg.catalog.Catalog; import org.apache.iceberg.hive.HiveCatalog; import javax.annotation.Nullable; @@ -45,7 +45,6 @@ */ public class HiveIcebergCatalog extends IcebergCatalog { - public static final String DRUID_DYNAMIC_CONFIG_PROVIDER_KEY = "druid.dynamic.config.provider"; public static final String TYPE_KEY = "hive"; @JsonProperty @@ -62,7 +61,7 @@ public class HiveIcebergCatalog extends IcebergCatalog private final Configuration configuration; - private BaseMetastoreCatalog hiveCatalog; + private Catalog hiveCatalog; private static final Logger log = new Logger(HiveIcebergCatalog.class); @@ -88,7 +87,7 @@ public HiveIcebergCatalog( } @Override - public BaseMetastoreCatalog retrieveCatalog() + public Catalog retrieveCatalog() { if (hiveCatalog == null) { hiveCatalog = setupCatalog(); diff --git a/extensions-contrib/druid-iceberg-extensions/src/main/java/org/apache/druid/iceberg/input/IcebergCatalog.java b/extensions-contrib/druid-iceberg-extensions/src/main/java/org/apache/druid/iceberg/input/IcebergCatalog.java index fe08dedef4d5..5dc5aa85a9a9 100644 --- a/extensions-contrib/druid-iceberg-extensions/src/main/java/org/apache/druid/iceberg/input/IcebergCatalog.java +++ b/extensions-contrib/druid-iceberg-extensions/src/main/java/org/apache/druid/iceberg/input/IcebergCatalog.java @@ -25,7 +25,6 @@ import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.RE; import org.apache.druid.java.util.common.logger.Logger; -import org.apache.iceberg.BaseMetastoreCatalog; import org.apache.iceberg.FileScanTask; import org.apache.iceberg.TableScan; import org.apache.iceberg.catalog.Catalog; @@ -46,9 +45,10 @@ @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = InputFormat.TYPE_PROPERTY) public abstract class IcebergCatalog { + public static final String DRUID_DYNAMIC_CONFIG_PROVIDER_KEY = "druid.dynamic.config.provider"; private static final Logger log = new Logger(IcebergCatalog.class); - public abstract BaseMetastoreCatalog retrieveCatalog(); + public abstract Catalog retrieveCatalog(); public boolean isCaseSensitive() { diff --git a/extensions-contrib/druid-iceberg-extensions/src/main/java/org/apache/druid/iceberg/input/LocalCatalog.java b/extensions-contrib/druid-iceberg-extensions/src/main/java/org/apache/druid/iceberg/input/LocalCatalog.java index 4539a582670f..471984724746 100644 --- a/extensions-contrib/druid-iceberg-extensions/src/main/java/org/apache/druid/iceberg/input/LocalCatalog.java +++ b/extensions-contrib/druid-iceberg-extensions/src/main/java/org/apache/druid/iceberg/input/LocalCatalog.java @@ -23,7 +23,7 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; import org.apache.hadoop.conf.Configuration; -import org.apache.iceberg.BaseMetastoreCatalog; +import org.apache.iceberg.catalog.Catalog; import org.apache.iceberg.hadoop.HadoopCatalog; import javax.annotation.Nullable; @@ -46,7 +46,7 @@ public class LocalCatalog extends IcebergCatalog @JsonProperty private final Boolean caseSensitive; - private BaseMetastoreCatalog catalog; + private Catalog catalog; @JsonCreator public LocalCatalog( @@ -83,7 +83,7 @@ public boolean isCaseSensitive() } @Override - public BaseMetastoreCatalog retrieveCatalog() + public Catalog retrieveCatalog() { if (catalog == null) { catalog = setupCatalog(); diff --git a/extensions-contrib/druid-iceberg-extensions/src/main/java/org/apache/druid/iceberg/input/RestIcebergCatalog.java b/extensions-contrib/druid-iceberg-extensions/src/main/java/org/apache/druid/iceberg/input/RestIcebergCatalog.java new file mode 100644 index 000000000000..7c518f45b0cd --- /dev/null +++ b/extensions-contrib/druid-iceberg-extensions/src/main/java/org/apache/druid/iceberg/input/RestIcebergCatalog.java @@ -0,0 +1,105 @@ +/* + * 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.iceberg.input; + +import com.fasterxml.jackson.annotation.JacksonInject; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Preconditions; +import org.apache.druid.guice.annotations.Json; +import org.apache.druid.iceberg.guice.HiveConf; +import org.apache.druid.utils.DynamicConfigProviderUtils; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.catalog.Catalog; +import org.apache.iceberg.catalog.SessionCatalog; +import org.apache.iceberg.rest.HTTPClient; +import org.apache.iceberg.rest.RESTCatalog; + +import javax.annotation.Nullable; +import java.util.Map; + +/** + * Catalog implementation for Iceberg REST catalogs. + */ +public class RestIcebergCatalog extends IcebergCatalog +{ + public static final String TYPE_KEY = "rest"; + + @JsonProperty + private final String catalogUri; + + @JsonProperty + private final Map catalogProperties; + + private final Configuration configuration; + + private Catalog restCatalog; + + @JsonCreator + public RestIcebergCatalog( + @JsonProperty("catalogUri") String catalogUri, + @JsonProperty("catalogProperties") @Nullable + Map catalogProperties, + @JacksonInject @Json ObjectMapper mapper, + @JacksonInject @HiveConf Configuration configuration + ) + { + this.catalogUri = Preconditions.checkNotNull(catalogUri, "catalogUri cannot be null"); + this.catalogProperties = DynamicConfigProviderUtils.extraConfigAndSetStringMap( + catalogProperties, + DRUID_DYNAMIC_CONFIG_PROVIDER_KEY, + mapper + ); + this.configuration = configuration; + } + + @Override + public Catalog retrieveCatalog() + { + if (restCatalog == null) { + restCatalog = setupCatalog(); + } + return restCatalog; + } + + public String getCatalogUri() + { + return catalogUri; + } + + public Map getCatalogProperties() + { + return catalogProperties; + } + + private RESTCatalog setupCatalog() + { + RESTCatalog restCatalog = new RESTCatalog( + SessionCatalog.SessionContext.createEmpty(), + config -> HTTPClient.builder(config).uri(config.get(CatalogProperties.URI)).build() + ); + restCatalog.setConf(configuration); + catalogProperties.put(CatalogProperties.URI, catalogUri); + restCatalog.initialize("rest", catalogProperties); + return restCatalog; + } +} diff --git a/extensions-contrib/druid-iceberg-extensions/src/test/java/org/apache/druid/iceberg/input/RestCatalogTest.java b/extensions-contrib/druid-iceberg-extensions/src/test/java/org/apache/druid/iceberg/input/RestCatalogTest.java new file mode 100644 index 000000000000..cc3b0d83e88a --- /dev/null +++ b/extensions-contrib/druid-iceberg-extensions/src/test/java/org/apache/druid/iceberg/input/RestCatalogTest.java @@ -0,0 +1,99 @@ +/* + * 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.iceberg.input; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.net.HttpHeaders; +import com.sun.net.httpserver.HttpServer; +import org.apache.commons.io.IOUtils; +import org.apache.druid.jackson.DefaultObjectMapper; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.rest.RESTCatalog; +import org.junit.Assert; +import org.junit.Test; + +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.net.InetSocketAddress; +import java.net.ServerSocket; +import java.nio.charset.StandardCharsets; +import java.util.HashMap; + +public class RestCatalogTest +{ + private final ObjectMapper mapper = new DefaultObjectMapper(); + + @Test + public void testCatalogCreate() throws IOException + { + + HttpServer server = null; + InputStream inputStream = null; + InputStream inputStreamPartial = null; + ServerSocket serverSocket = null; + try { + serverSocket = new ServerSocket(0); + int port = serverSocket.getLocalPort(); + serverSocket.close(); + server = HttpServer.create(new InetSocketAddress("localhost", port), 0); + server.createContext( + "/v1/config", // API for catalog fetchConfig which is invoked on catalog initialization + (httpExchange) -> { + String payload = "{}"; + byte[] outputBytes = payload.getBytes(StandardCharsets.UTF_8); + httpExchange.sendResponseHeaders(200, outputBytes.length); + OutputStream os = httpExchange.getResponseBody(); + httpExchange.getResponseHeaders().set(HttpHeaders.CONTENT_TYPE, "application/octet-stream"); + httpExchange.getResponseHeaders().set(HttpHeaders.CONTENT_LENGTH, String.valueOf(outputBytes.length)); + httpExchange.getResponseHeaders().set(HttpHeaders.CONTENT_RANGE, "bytes 0"); + os.write(outputBytes); + os.close(); + } + ); + server.start(); + + String catalogUri = "http://localhost:" + port; + + RestIcebergCatalog testRestCatalog = new RestIcebergCatalog( + catalogUri, + new HashMap<>(), + mapper, + new Configuration() + ); + RESTCatalog innerCatalog = (RESTCatalog) testRestCatalog.retrieveCatalog(); + + Assert.assertEquals("rest", innerCatalog.name()); + Assert.assertNotNull(innerCatalog.properties()); + Assert.assertNotNull(testRestCatalog.getCatalogProperties()); + Assert.assertEquals(testRestCatalog.getCatalogUri(), innerCatalog.properties().get("uri")); + } + finally { + IOUtils.closeQuietly(inputStream); + IOUtils.closeQuietly(inputStreamPartial); + if (server != null) { + server.stop(0); + } + if (serverSocket != null) { + serverSocket.close(); + } + } + } +} From fff8858b152414a137b989578fd14dac3b2cf51d Mon Sep 17 00:00:00 2001 From: Atul Mohan Date: Fri, 20 Sep 2024 15:53:23 -0700 Subject: [PATCH 2/3] Fix test --- .../org/apache/druid/iceberg/input/RestCatalogTest.java | 7 ------- 1 file changed, 7 deletions(-) diff --git a/extensions-contrib/druid-iceberg-extensions/src/test/java/org/apache/druid/iceberg/input/RestCatalogTest.java b/extensions-contrib/druid-iceberg-extensions/src/test/java/org/apache/druid/iceberg/input/RestCatalogTest.java index cc3b0d83e88a..9178bcf63b78 100644 --- a/extensions-contrib/druid-iceberg-extensions/src/test/java/org/apache/druid/iceberg/input/RestCatalogTest.java +++ b/extensions-contrib/druid-iceberg-extensions/src/test/java/org/apache/druid/iceberg/input/RestCatalogTest.java @@ -22,7 +22,6 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.net.HttpHeaders; import com.sun.net.httpserver.HttpServer; -import org.apache.commons.io.IOUtils; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.hadoop.conf.Configuration; import org.apache.iceberg.rest.RESTCatalog; @@ -30,7 +29,6 @@ import org.junit.Test; import java.io.IOException; -import java.io.InputStream; import java.io.OutputStream; import java.net.InetSocketAddress; import java.net.ServerSocket; @@ -44,10 +42,7 @@ public class RestCatalogTest @Test public void testCatalogCreate() throws IOException { - HttpServer server = null; - InputStream inputStream = null; - InputStream inputStreamPartial = null; ServerSocket serverSocket = null; try { serverSocket = new ServerSocket(0); @@ -86,8 +81,6 @@ public void testCatalogCreate() throws IOException Assert.assertEquals(testRestCatalog.getCatalogUri(), innerCatalog.properties().get("uri")); } finally { - IOUtils.closeQuietly(inputStream); - IOUtils.closeQuietly(inputStreamPartial); if (server != null) { server.stop(0); } From 6f8b9d731bc55c9b6ba5bc0fdddae47af71ba4b0 Mon Sep 17 00:00:00 2001 From: Atul Mohan Date: Mon, 23 Sep 2024 09:16:29 -0700 Subject: [PATCH 3/3] Fix up tests and docs --- .../development/extensions-contrib/iceberg.md | 1 + docs/ingestion/input-sources.md | 4 +- .../iceberg/input/RestIcebergCatalog.java | 7 +- .../druid/iceberg/input/RestCatalogTest.java | 93 ++++++++++--------- 4 files changed, 58 insertions(+), 47 deletions(-) diff --git a/docs/development/extensions-contrib/iceberg.md b/docs/development/extensions-contrib/iceberg.md index 989e64237958..9483f1b82ef2 100644 --- a/docs/development/extensions-contrib/iceberg.md +++ b/docs/development/extensions-contrib/iceberg.md @@ -28,6 +28,7 @@ Apache Iceberg is an open table format for huge analytic datasets. [IcebergInput Iceberg manages most of its metadata in metadata files in the object storage. However, it is still dependent on a metastore to manage a certain amount of metadata. Iceberg refers to these metastores as catalogs. The Iceberg extension lets you connect to the following Iceberg catalog types: +* REST-based catalog * Hive metastore catalog * Local catalog diff --git a/docs/ingestion/input-sources.md b/docs/ingestion/input-sources.md index 3aeecb20e07e..28bbf0ab5732 100644 --- a/docs/ingestion/input-sources.md +++ b/docs/ingestion/input-sources.md @@ -1063,7 +1063,7 @@ The following is a sample spec for a S3 warehouse source: ### Catalog Object -The catalog object supports `local`,`hive` and `rest` catalog types. +The catalog object supports `rest`, `hive` and `local` catalog types. The following table lists the properties of a `local` catalog: @@ -1095,7 +1095,7 @@ The following table lists the properties of a `rest` catalog: ### Iceberg filter object This input source provides the following filters: `and`, `equals`, `interval`, and `or`. You can use these filters to filter out data files from a snapshot, reducing the number of files Druid has to ingest. -If the filter column is not an Iceberg partition column, it is highly recommended to define an additional filter defined in the [`transformSpec`](./ingestion-spec.md#transformspec). This is because for non-partition columns, Iceberg filters may return rows that do not match the expression. +It is strongly recommended to apply filtering only on Iceberg partition columns. When filtering on non-partition columns, Iceberg filters may return rows that do not fully match the expression. To address this, it may help to define an additional filter in the [`transformSpec`](./ingestion-spec.md#transformspec) to remove residual rows. `equals` Filter: diff --git a/extensions-contrib/druid-iceberg-extensions/src/main/java/org/apache/druid/iceberg/input/RestIcebergCatalog.java b/extensions-contrib/druid-iceberg-extensions/src/main/java/org/apache/druid/iceberg/input/RestIcebergCatalog.java index 7c518f45b0cd..cc481c983fd2 100644 --- a/extensions-contrib/druid-iceberg-extensions/src/main/java/org/apache/druid/iceberg/input/RestIcebergCatalog.java +++ b/extensions-contrib/druid-iceberg-extensions/src/main/java/org/apache/druid/iceberg/input/RestIcebergCatalog.java @@ -23,7 +23,7 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.base.Preconditions; +import org.apache.druid.error.InvalidInput; import org.apache.druid.guice.annotations.Json; import org.apache.druid.iceberg.guice.HiveConf; import org.apache.druid.utils.DynamicConfigProviderUtils; @@ -63,7 +63,10 @@ public RestIcebergCatalog( @JacksonInject @HiveConf Configuration configuration ) { - this.catalogUri = Preconditions.checkNotNull(catalogUri, "catalogUri cannot be null"); + if (catalogUri == null) { + throw InvalidInput.exception("catalogUri cannot be null"); + } + this.catalogUri = catalogUri; this.catalogProperties = DynamicConfigProviderUtils.extraConfigAndSetStringMap( catalogProperties, DRUID_DYNAMIC_CONFIG_PROVIDER_KEY, diff --git a/extensions-contrib/druid-iceberg-extensions/src/test/java/org/apache/druid/iceberg/input/RestCatalogTest.java b/extensions-contrib/druid-iceberg-extensions/src/test/java/org/apache/druid/iceberg/input/RestCatalogTest.java index 9178bcf63b78..6597ade02ec3 100644 --- a/extensions-contrib/druid-iceberg-extensions/src/test/java/org/apache/druid/iceberg/input/RestCatalogTest.java +++ b/extensions-contrib/druid-iceberg-extensions/src/test/java/org/apache/druid/iceberg/input/RestCatalogTest.java @@ -25,7 +25,9 @@ import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.hadoop.conf.Configuration; import org.apache.iceberg.rest.RESTCatalog; +import org.junit.After; import org.junit.Assert; +import org.junit.Before; import org.junit.Test; import java.io.IOException; @@ -38,55 +40,60 @@ public class RestCatalogTest { private final ObjectMapper mapper = new DefaultObjectMapper(); + private int port = 0; + private HttpServer server = null; + private ServerSocket serverSocket = null; - @Test - public void testCatalogCreate() throws IOException + @Before + public void setup() throws Exception { - HttpServer server = null; - ServerSocket serverSocket = null; - try { - serverSocket = new ServerSocket(0); - int port = serverSocket.getLocalPort(); - serverSocket.close(); - server = HttpServer.create(new InetSocketAddress("localhost", port), 0); - server.createContext( - "/v1/config", // API for catalog fetchConfig which is invoked on catalog initialization - (httpExchange) -> { - String payload = "{}"; - byte[] outputBytes = payload.getBytes(StandardCharsets.UTF_8); - httpExchange.sendResponseHeaders(200, outputBytes.length); - OutputStream os = httpExchange.getResponseBody(); - httpExchange.getResponseHeaders().set(HttpHeaders.CONTENT_TYPE, "application/octet-stream"); - httpExchange.getResponseHeaders().set(HttpHeaders.CONTENT_LENGTH, String.valueOf(outputBytes.length)); - httpExchange.getResponseHeaders().set(HttpHeaders.CONTENT_RANGE, "bytes 0"); - os.write(outputBytes); - os.close(); - } - ); - server.start(); + serverSocket = new ServerSocket(0); + port = serverSocket.getLocalPort(); + serverSocket.close(); + server = HttpServer.create(new InetSocketAddress("localhost", port), 0); + server.createContext( + "/v1/config", // API for catalog fetchConfig which is invoked on catalog initialization + (httpExchange) -> { + String payload = "{}"; + byte[] outputBytes = payload.getBytes(StandardCharsets.UTF_8); + httpExchange.sendResponseHeaders(200, outputBytes.length); + OutputStream os = httpExchange.getResponseBody(); + httpExchange.getResponseHeaders().set(HttpHeaders.CONTENT_TYPE, "application/octet-stream"); + httpExchange.getResponseHeaders().set(HttpHeaders.CONTENT_LENGTH, String.valueOf(outputBytes.length)); + httpExchange.getResponseHeaders().set(HttpHeaders.CONTENT_RANGE, "bytes 0"); + os.write(outputBytes); + os.close(); + } + ); + server.start(); + } - String catalogUri = "http://localhost:" + port; + @Test + public void testCatalogCreate() + { + String catalogUri = "http://localhost:" + port; - RestIcebergCatalog testRestCatalog = new RestIcebergCatalog( - catalogUri, - new HashMap<>(), - mapper, - new Configuration() - ); - RESTCatalog innerCatalog = (RESTCatalog) testRestCatalog.retrieveCatalog(); + RestIcebergCatalog testRestCatalog = new RestIcebergCatalog( + catalogUri, + new HashMap<>(), + mapper, + new Configuration() + ); + RESTCatalog innerCatalog = (RESTCatalog) testRestCatalog.retrieveCatalog(); - Assert.assertEquals("rest", innerCatalog.name()); - Assert.assertNotNull(innerCatalog.properties()); - Assert.assertNotNull(testRestCatalog.getCatalogProperties()); - Assert.assertEquals(testRestCatalog.getCatalogUri(), innerCatalog.properties().get("uri")); + Assert.assertEquals("rest", innerCatalog.name()); + Assert.assertNotNull(innerCatalog.properties()); + Assert.assertNotNull(testRestCatalog.getCatalogProperties()); + Assert.assertEquals(testRestCatalog.getCatalogUri(), innerCatalog.properties().get("uri")); + } + @After + public void tearDown() throws IOException + { + if (server != null) { + server.stop(0); } - finally { - if (server != null) { - server.stop(0); - } - if (serverSocket != null) { - serverSocket.close(); - } + if (serverSocket != null) { + serverSocket.close(); } } }