diff --git a/docs/development/extensions-contrib/iceberg.md b/docs/development/extensions-contrib/iceberg.md index 041453930334..e2a5a06cb9ec 100644 --- a/docs/development/extensions-contrib/iceberg.md +++ b/docs/development/extensions-contrib/iceberg.md @@ -31,12 +31,12 @@ 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: + +* Glue catalog * REST-based catalog * Hive metastore catalog * Local catalog -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. The `druid-iceberg-extensions` extension relies on the existing input source connectors in Druid to read the data files from the warehouse. Therefore, the Iceberg input source can be considered as an intermediate input source, which provides the file paths for other input source implementations. @@ -116,6 +116,12 @@ The `warehouseSource` is set to `local` because this catalog only supports readi 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. +## Glue catalog + +Configure the `icebergCatalog` type as `glue`.`warehousePath` and properties must be provided in `catalogProperties` object. +Refer [Iceberg Glue Catalog documentation](https://iceberg.apache.org/docs/1.6.0/aws/#glue-catalog) for setting properties. + + ## 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 4583ff40f595..d739a5fa8bb9 100644 --- a/docs/ingestion/input-sources.md +++ b/docs/ingestion/input-sources.md @@ -1065,7 +1065,7 @@ The following is a sample spec for a S3 warehouse source: ### Catalog Object -The catalog object supports `rest`, `hive` and `local` catalog types. +The catalog object supports `rest`, `hive`, `glue` and `local` catalog types. The following table lists the properties of a `local` catalog: @@ -1094,6 +1094,30 @@ The following table lists the properties of a `rest` catalog: |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| +The following table lists the properties of a `glue` catalog: + +|Property| Description |Default| Required | +|--------|------------------------------------------------------------------------------------------------------------------------------------------------------|-------|----------| +|type| Set this value to `glue`. |None| yes | +|catalogProperties| Map of any additional properties that needs to be attached to the catalog. This expects all the config as per [Iceberg Catalog configuration docs](https://iceberg.apache.org/docs/latest/configuration/#catalog-properties) |None| Yes | + +Sample: + +```angular2html +... +"icebergCatalog": +{ + "type" : "glue", + "catalogProperties" : + { + "warehouse": "s3a://bucket/warehouse", + "io-impl": "org.apache.iceberg.aws.s3.S3FileIO" + } +} +.. +``` + + ### 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. diff --git a/extensions-contrib/druid-iceberg-extensions/pom.xml b/extensions-contrib/druid-iceberg-extensions/pom.xml index 8d40e974e156..a36561cff055 100644 --- a/extensions-contrib/druid-iceberg-extensions/pom.xml +++ b/extensions-contrib/druid-iceberg-extensions/pom.xml @@ -36,6 +36,7 @@ 1.6.1 + 2.28.28 3.1.3 @@ -153,10 +154,6 @@ com.google.guava guava - - org.apache.avro - avro - net.java.dev.jets3t jets3t @@ -271,7 +268,18 @@ iceberg-hive-metastore ${iceberg.core.version} - + + + org.apache.iceberg + iceberg-aws + ${iceberg.core.version} + + + + software.amazon.awssdk + glue + ${awssdk.version} + org.apache.hive hive-metastore 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 418b63850b81..d238cccc248c 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 @@ -25,6 +25,7 @@ import com.google.inject.Binder; import org.apache.druid.error.DruidException; import org.apache.druid.iceberg.guice.HiveConf; +import org.apache.druid.iceberg.input.GlueIcebergCatalog; import org.apache.druid.iceberg.input.HiveIcebergCatalog; import org.apache.druid.iceberg.input.IcebergInputSource; import org.apache.druid.iceberg.input.LocalCatalog; @@ -47,8 +48,8 @@ public List getJacksonModules() 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) - + new NamedType(IcebergInputSource.class, IcebergInputSource.TYPE_KEY), + new NamedType(GlueIcebergCatalog.class, GlueIcebergCatalog.TYPE_KEY) ) ); } diff --git a/extensions-contrib/druid-iceberg-extensions/src/main/java/org/apache/druid/iceberg/input/GlueIcebergCatalog.java b/extensions-contrib/druid-iceberg-extensions/src/main/java/org/apache/druid/iceberg/input/GlueIcebergCatalog.java new file mode 100644 index 000000000000..9919ac7f8475 --- /dev/null +++ b/extensions-contrib/druid-iceberg-extensions/src/main/java/org/apache/druid/iceberg/input/GlueIcebergCatalog.java @@ -0,0 +1,107 @@ +/* + * 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 org.apache.druid.guice.annotations.Json; +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.utils.DynamicConfigProviderUtils; +import org.apache.iceberg.CatalogUtil; +import org.apache.iceberg.catalog.Catalog; + +import javax.annotation.Nullable; +import java.util.Map; + +/** + * Glue specific implementation of iceberg catalog. + * It expects Catalog Properties key:value pair, which is iceberg compatible: + * https://iceberg.apache.org/docs/latest/configuration/#catalog-properties + */ +public class GlueIcebergCatalog extends IcebergCatalog +{ + private static final String CATALOG_NAME = "glue"; + private Catalog catalog; + + public static final String TYPE_KEY = "glue"; + + @JsonProperty + private Map catalogProperties; + + @JsonProperty + private final Boolean caseSensitive; + private static final Logger log = new Logger(GlueIcebergCatalog.class); + + /** + * catalogProperties must have all the config that iceberg glue catalog expect. + * Ref: https://iceberg.apache.org/docs/nightly/kafka-connect/?h=kafka#glue-example + * and https://iceberg.apache.org/concepts/catalog/?h=catalog + * e.g. + * "catalogProperties" : + { + "type" : "glue", + "io-impl": "org.apache.iceberg.aws.s3.S3FileIO", + "warehouse": "s3://bucket/iceberg_catalog/druid/warehouse" + } + * + * */ + @JsonCreator + public GlueIcebergCatalog( + @JsonProperty("catalogProperties") @Nullable Map catalogProperties, + @JsonProperty("caseSensitive") Boolean caseSensitive, + @JacksonInject @Json ObjectMapper mapper + ) + { + this.catalogProperties = DynamicConfigProviderUtils.extraConfigAndSetStringMap( + catalogProperties, + DRUID_DYNAMIC_CONFIG_PROVIDER_KEY, + mapper + ); + this.caseSensitive = caseSensitive == null ? true : caseSensitive; + this.catalog = retrieveCatalog(); + } + + @Override + public Catalog retrieveCatalog() + { + if (catalog == null) { + log.info("catalog is null, setting up default glue catalog."); + catalog = setupGlueCatalog(); + } + log.info("Glue catalog set [%s].", catalog.toString()); + return catalog; + } + + private Catalog setupGlueCatalog() + { + // We are not passing any hadoop config, third parameter is null + catalogProperties.put("type", TYPE_KEY); + catalog = CatalogUtil.buildIcebergCatalog(CATALOG_NAME, catalogProperties, null); + return catalog; + } + + @Override + public boolean isCaseSensitive() + { + return caseSensitive; + } +} diff --git a/extensions-contrib/druid-iceberg-extensions/src/test/java/org/apache/druid/iceberg/input/GlueIcebergCatalogTest.java b/extensions-contrib/druid-iceberg-extensions/src/test/java/org/apache/druid/iceberg/input/GlueIcebergCatalogTest.java new file mode 100644 index 000000000000..d748911181ef --- /dev/null +++ b/extensions-contrib/druid-iceberg-extensions/src/test/java/org/apache/druid/iceberg/input/GlueIcebergCatalogTest.java @@ -0,0 +1,58 @@ +/* + * 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 org.apache.druid.jackson.DefaultObjectMapper; +import org.junit.Assert; +import org.junit.Test; + +import java.util.HashMap; + +/** + * Test cases for GlueCatalog Iceberg extension. + * */ +public class GlueIcebergCatalogTest +{ + private final ObjectMapper mapper = new DefaultObjectMapper(); + + @Test + public void testCatalogCreate() + { + GlueIcebergCatalog glueCatalog = new GlueIcebergCatalog( + new HashMap<>(), + true, + mapper + ); + Assert.assertEquals("glue", glueCatalog.retrieveCatalog().name()); + } + + @Test + public void testIsCaseSensitive() + { + GlueIcebergCatalog glueCatalog = new GlueIcebergCatalog( + new HashMap<>(), + true, + mapper + ); + Assert.assertEquals(true, glueCatalog.isCaseSensitive()); + } +} diff --git a/pom.xml b/pom.xml index e70542cdea64..a4462e6a8673 100644 --- a/pom.xml +++ b/pom.xml @@ -1826,6 +1826,22 @@ org.apache.maven.plugins maven-dependency-plugin 3.1.1 + + + analyze + + analyze-only + + + true + + + org.apache.iceberg:iceberg-aws + software.amazon.awssdk:glue + + + + org.apache.maven.plugins