mirror of https://github.com/apache/druid.git
AWS Glue Catalog for Iceberg ingest extension (#17392)
* iceberg glue catalog dependencies added * GlueIcebergCatalog added in druid module * default version of iceberg glue catalog implementation - basics * basic tests added * removed dependecy iceberg-aws-bundle * glue catalog support - docs update for iceberg * Update IcebergDruidModule.java * Update IcebergDruidModule.java * updates in dependencies and warehousePath must be under catalogProp * removed some dependencies - which not required * only glue sdk added * update license * avro exclusion removed * doc update * doc update * set the type to glue * minor change * minor change * fixing codestyle * checkstyle fixes * checkstyle fixes * checkstyle fixes * dependency check fixes * update pom for ignore warning for glue catalog * compile scope needed - iceberg-aws and awssdk * updates pom with comment * minor change * mvn dependency check in iceberg extension * revert pom.xml changes * aws sdk sts and s3 for gluecatalog initialize * dependency check - ignore aws sdk s3 and sts --------- Co-authored-by: SHEKHAR PRASAD RAJAK <shekhar_rajak@apple.com>
This commit is contained in:
parent
f906d0d446
commit
ae049a4bab
|
@ -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 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:
|
Iceberg refers to these metastores as catalogs. The Iceberg extension lets you connect to the following Iceberg catalog types:
|
||||||
|
|
||||||
|
* Glue catalog
|
||||||
* REST-based catalog
|
* REST-based catalog
|
||||||
* Hive metastore catalog
|
* Hive metastore catalog
|
||||||
* Local 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.
|
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 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.
|
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.
|
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.
|
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
|
## Downloading Iceberg extension
|
||||||
|
|
||||||
To download `druid-iceberg-extensions`, run the following command after replacing `<VERSION>` with the desired
|
To download `druid-iceberg-extensions`, run the following command after replacing `<VERSION>` with the desired
|
||||||
|
|
|
@ -1065,7 +1065,7 @@ The following is a sample spec for a S3 warehouse source:
|
||||||
|
|
||||||
### Catalog Object
|
### 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:
|
The following table lists the properties of a `local` catalog:
|
||||||
|
|
||||||
|
@ -1094,6 +1094,29 @@ The following table lists the properties of a `rest` catalog:
|
||||||
|catalogUri|The URI associated with the catalog's HTTP endpoint.|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|
|
|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
|
### 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.
|
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.
|
||||||
|
|
|
@ -36,6 +36,7 @@
|
||||||
|
|
||||||
<properties>
|
<properties>
|
||||||
<iceberg.core.version>1.6.1</iceberg.core.version>
|
<iceberg.core.version>1.6.1</iceberg.core.version>
|
||||||
|
<awssdk.version>2.28.28</awssdk.version>
|
||||||
<hive.version>3.1.3</hive.version>
|
<hive.version>3.1.3</hive.version>
|
||||||
</properties>
|
</properties>
|
||||||
<dependencies>
|
<dependencies>
|
||||||
|
@ -153,10 +154,6 @@
|
||||||
<groupId>com.google.guava</groupId>
|
<groupId>com.google.guava</groupId>
|
||||||
<artifactId>guava</artifactId>
|
<artifactId>guava</artifactId>
|
||||||
</exclusion>
|
</exclusion>
|
||||||
<exclusion>
|
|
||||||
<groupId>org.apache.avro</groupId>
|
|
||||||
<artifactId>avro</artifactId>
|
|
||||||
</exclusion>
|
|
||||||
<exclusion>
|
<exclusion>
|
||||||
<groupId>net.java.dev.jets3t</groupId>
|
<groupId>net.java.dev.jets3t</groupId>
|
||||||
<artifactId>jets3t</artifactId>
|
<artifactId>jets3t</artifactId>
|
||||||
|
@ -271,6 +268,32 @@
|
||||||
<artifactId>iceberg-hive-metastore</artifactId>
|
<artifactId>iceberg-hive-metastore</artifactId>
|
||||||
<version>${iceberg.core.version}</version>
|
<version>${iceberg.core.version}</version>
|
||||||
</dependency>
|
</dependency>
|
||||||
|
<!-- Iceberg GlueCatalog & CatalogUtil -->
|
||||||
|
<dependency>
|
||||||
|
<groupId>org.apache.iceberg</groupId>
|
||||||
|
<artifactId>iceberg-aws</artifactId>
|
||||||
|
<version>${iceberg.core.version}</version>
|
||||||
|
</dependency>
|
||||||
|
<!-- GlueCatalog needs AWS SDK STS module compile time -->
|
||||||
|
<dependency>
|
||||||
|
<groupId>software.amazon.awssdk</groupId>
|
||||||
|
<artifactId>glue</artifactId>
|
||||||
|
<version>${awssdk.version}</version>
|
||||||
|
</dependency>
|
||||||
|
<!-- software/amazon/awssdk/services/sts/model/Tag needed for GlueCatalog initialize -->
|
||||||
|
<dependency>
|
||||||
|
<groupId>software.amazon.awssdk</groupId>
|
||||||
|
<artifactId>sts</artifactId>
|
||||||
|
<version>${awssdk.version}</version>
|
||||||
|
</dependency>
|
||||||
|
|
||||||
|
<!-- software/amazon/awssdk/services/s3/model/ObjectCannedACL needed for GlueCatalog initialize -->
|
||||||
|
<dependency>
|
||||||
|
<groupId>software.amazon.awssdk</groupId>
|
||||||
|
<artifactId>s3</artifactId>
|
||||||
|
<version>${awssdk.version}</version>
|
||||||
|
</dependency>
|
||||||
|
|
||||||
|
|
||||||
<dependency>
|
<dependency>
|
||||||
<groupId>org.apache.hive</groupId>
|
<groupId>org.apache.hive</groupId>
|
||||||
|
@ -742,5 +765,23 @@
|
||||||
</configuration>
|
</configuration>
|
||||||
</plugin>
|
</plugin>
|
||||||
</plugins>
|
</plugins>
|
||||||
|
|
||||||
|
<pluginManagement>
|
||||||
|
<plugins>
|
||||||
|
<plugin>
|
||||||
|
<groupId>org.apache.maven.plugins</groupId>
|
||||||
|
<artifactId>maven-dependency-plugin</artifactId>
|
||||||
|
<configuration>
|
||||||
|
<!-- ignore annotations for "unused but declared" warnings -->
|
||||||
|
<ignoredUnusedDeclaredDependencies>
|
||||||
|
<ignoredUnusedDeclaredDependency>org.apache.iceberg:iceberg-aws</ignoredUnusedDeclaredDependency>
|
||||||
|
<ignoredUnusedDeclaredDependency>software.amazon.awssdk:glue</ignoredUnusedDeclaredDependency>
|
||||||
|
<ignoredUnusedDeclaredDependency>software.amazon.awssdk:s3</ignoredUnusedDeclaredDependency>
|
||||||
|
<ignoredUnusedDeclaredDependency>software.amazon.awssdk:sts</ignoredUnusedDeclaredDependency>
|
||||||
|
</ignoredUnusedDeclaredDependencies>
|
||||||
|
</configuration>
|
||||||
|
</plugin>
|
||||||
|
</plugins>
|
||||||
|
</pluginManagement>
|
||||||
</build>
|
</build>
|
||||||
</project>
|
</project>
|
||||||
|
|
|
@ -25,6 +25,7 @@ import com.fasterxml.jackson.databind.module.SimpleModule;
|
||||||
import com.google.inject.Binder;
|
import com.google.inject.Binder;
|
||||||
import org.apache.druid.error.DruidException;
|
import org.apache.druid.error.DruidException;
|
||||||
import org.apache.druid.iceberg.guice.HiveConf;
|
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.HiveIcebergCatalog;
|
||||||
import org.apache.druid.iceberg.input.IcebergInputSource;
|
import org.apache.druid.iceberg.input.IcebergInputSource;
|
||||||
import org.apache.druid.iceberg.input.LocalCatalog;
|
import org.apache.druid.iceberg.input.LocalCatalog;
|
||||||
|
@ -47,8 +48,8 @@ public class IcebergDruidModule implements DruidModule
|
||||||
new NamedType(HiveIcebergCatalog.class, HiveIcebergCatalog.TYPE_KEY),
|
new NamedType(HiveIcebergCatalog.class, HiveIcebergCatalog.TYPE_KEY),
|
||||||
new NamedType(LocalCatalog.class, LocalCatalog.TYPE_KEY),
|
new NamedType(LocalCatalog.class, LocalCatalog.TYPE_KEY),
|
||||||
new NamedType(RestIcebergCatalog.class, RestIcebergCatalog.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)
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
|
@ -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<String, String> 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<String, Object> 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;
|
||||||
|
}
|
||||||
|
}
|
|
@ -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());
|
||||||
|
}
|
||||||
|
}
|
Loading…
Reference in New Issue