abhishekagarwal87 commented on code in PR #14329: URL: https://github.com/apache/druid/pull/14329#discussion_r1243457526
########## docs/development/extensions-contrib/iceberg.md: ########## @@ -0,0 +1,121 @@ +--- +id: iceberg +title: "Iceberg" +--- + +<!-- + ~ 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. + --> + +## Iceberg Ingest Extension + +This extension provides [IcebergInputSource](../../ingestion/input-sources.md#iceberg-input-source) which enables ingestion of data stored in the Iceberg table format into Druid. + +Apache Iceberg is an open table format for huge analytic datasets. Even though iceberg manages most of its metadata in metadata files in the object storage, it is still dependent on a metastore for managing a certain amount of metadata. +These metastores are defined as Iceberg catalogs and this extension supports connecting to the following catalog types: +* Hive metastore catalog +* Local catalog + +Support for AWS Glue and REST based catalogs are not available yet. + +For a given catalog, iceberg table name and filters, The IcebergInputSource works by reading the table from the catalog, applying the filters and extracting all the underlying live data files up to the latest snapshot. Review Comment: ```suggestion For a given catalog, iceberg table name, and filters, the IcebergInputSource works by reading the table from the catalog, applying the filters, and extracting all the underlying live data files up to the latest snapshot. ``` ########## docs/development/extensions-contrib/iceberg.md: ########## @@ -0,0 +1,121 @@ +--- +id: iceberg +title: "Iceberg" +--- + +<!-- + ~ 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. + --> + +## Iceberg Ingest Extension + +This extension provides [IcebergInputSource](../../ingestion/input-sources.md#iceberg-input-source) which enables ingestion of data stored in the Iceberg table format into Druid. + +Apache Iceberg is an open table format for huge analytic datasets. Even though iceberg manages most of its metadata in metadata files in the object storage, it is still dependent on a metastore for managing a certain amount of metadata. +These metastores are defined as Iceberg catalogs and this extension supports connecting to the following catalog types: +* Hive metastore catalog +* Local catalog + +Support for AWS Glue and REST based catalogs are not available yet. + +For a given catalog, iceberg table name and filters, The IcebergInputSource works by reading the table from the catalog, applying the filters and extracting all the underlying live data files up to the latest snapshot. +The data files are in either Parquet, ORC or Avro formats and all of these have InputFormat support in Druid. The data files typically reside in a warehouse location which could be in HDFS, S3 or the local filesystem. +This extension relies on the existing InputSource connectors in Druid to read the data files from the warehouse. Therefore, the IcebergInputSource can be considered as an intermediate InputSource which provides the file paths for other InputSource implementations. + +### Load the Iceberg Ingest extension + +To use the iceberg extension, add the `druid-iceberg-extensions` to the list of loaded extensions. See [Loading extensions](../../configuration/extensions.md#loading-extensions) for more information. + + +### Hive Metastore catalog + +For Druid to seamlessly talk to the Hive Metastore, ensure that the Hive specific configuration files such as `hive-site.xml` and `core-site.xml` are available in the Druid classpath for peon processes. +Hive specific properties can also be specified under the `catalogProperties` object in the ingestion spec. + +Hive metastore catalogs can be associated with different types of warehouses, but this extension presently only supports HDFS and S3 warehouse directories. + +#### Reading from HDFS warehouse + +Ensure that the extension `druid-hdfs-storage` is loaded. The data file paths are extracted from the Hive metastore catalog and [HDFS input source](../../ingestion/input-sources.md#hdfs-input-source) is used to ingest these files. +The `warehouseSource` type in the ingestion spec should be `hdfs`. + +If the Hive metastore supports Kerberos authentication, the following properties will be required in the `catalogProperties`: + +```json +"catalogProperties": { + "principal": "krb_principal", + "keytab": "/path/to/keytab" +} +``` +Only Kerberos based authentication is supported as of now. + +#### Reading from S3 warehouse + +Ensure that the extension `druid-s3-extensions` is loaded. The data file paths are extracted from the Hive metastore catalog and `S3InputSource` is used to ingest these files. +The `warehouseSource` type in the ingestion spec should be `s3`. If the S3 endpoint for the warehouse is different from the endpoint configured as the deep storage, the following properties are required in the `warehouseSource` section to define the S3 endpoint settings: + +```json +"warehouseSource": { + "type": "s3", + "endpointConfig": { + "url": "S3_ENDPOINT_URL", + "signingRegion": "us-east-1" + }, + "clientConfig": { + "protocol": "http", + "disableChunkedEncoding": true, + "enablePathStyleAccess": true, + "forceGlobalBucketAccessEnabled": false + }, + "properties": { + "accessKeyId": { + "type": "default", + "password": "<ACCESS_KEY_ID" + }, + "secretAccessKey": { + "type": "default", + "password": "<SECRET_ACCESS_KEY>" + } + } +} +``` + +This extension uses the [Hadoop AWS module](https://hadoop.apache.org/docs/stable/hadoop-aws/tools/hadoop-aws/) to connect to S3 and retrieve the metadata and data file paths. +The following properties will be required in the `catalogProperties`: + +```json +"catalogProperties": { + "fs.s3a.access.key" : "S3_ACCESS_KEY", + "fs.s3a.secret.key" : "S3_SECRET_KEY", + "fs.s3a.endpoint" : "S3_API_ENDPOINT" +} +``` +Since the AWS connector uses the `s3a` filesystem based client, the warehouse path should be specified with the `s3a://` protocol instead of `s3://`. Review Comment: ```suggestion Since the Hadoop AWS connector uses the `s3a` filesystem based client, the warehouse path should be specified with the `s3a://` protocol instead of `s3://`. ``` ########## docs/ingestion/input-sources.md: ########## @@ -794,6 +794,194 @@ The following is an example of a Combining input source spec: ... ``` +## Iceberg input source + +> You need to include the `druid-iceberg-extensions` as an extension to use the Iceberg input source. + +The Iceberg input source is used to read data stored in the Iceberg table format. For a given table, this input source scans up to the latest iceberg snapshot from the configured Hive catalog and the underlying live data files will be ingested using the existing input source formats available in Druid. + +The Iceberg input source cannot be independent as it relies on the existing input sources to perform the actual read from the Data files. +For example, if the warehouse associated with an iceberg catalog is on `S3`, please ensure that the [`druid-s3-extensions`](../development/extensions-core/s3.md) extension is also loaded. Review Comment: does it require S3 extension though because behind the scenes, its using hadoop-aws module? ########## extensions-contrib/druid-iceberg-extensions/src/main/java/org/apache/druid/iceberg/common/IcebergDruidModule.java: ########## @@ -0,0 +1,75 @@ +/* + * 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.common; + +import com.fasterxml.jackson.databind.Module; +import com.fasterxml.jackson.databind.jsontype.NamedType; +import com.fasterxml.jackson.databind.module.SimpleModule; +import com.google.inject.Binder; +import org.apache.druid.iceberg.guice.HiveConf; +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.initialization.DruidModule; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; + +import java.util.Collections; +import java.util.List; + +public class IcebergDruidModule implements DruidModule +{ + @Override + public List<? extends Module> getJacksonModules() + { + return Collections.singletonList( + new SimpleModule("IcebergDruidModule") + .registerSubtypes( + new NamedType(HiveIcebergCatalog.class, HiveIcebergCatalog.TYPE_KEY), + new NamedType(LocalCatalog.class, LocalCatalog.TYPE_KEY), + new NamedType(IcebergInputSource.class, IcebergInputSource.TYPE_KEY) + + ) + ); + } + + @Override + public void configure(Binder binder) + { + final Configuration conf = new Configuration(); + conf.setClassLoader(getClass().getClassLoader()); + + ClassLoader currCtxCl = Thread.currentThread().getContextClassLoader(); + try { + Thread.currentThread().setContextClassLoader(getClass().getClassLoader()); + FileSystem.get(conf); + } + catch (Exception ex) { + throw new RuntimeException(ex); Review Comment: can you use newly introduced `DruidException`? We are standardizing on that exception to encourage better user-facing error messages and in general more context about errors. ########## extensions-contrib/druid-iceberg-extensions/src/main/java/org/apache/druid/iceberg/filter/IcebergIntervalFilter.java: ########## @@ -0,0 +1,90 @@ +/* + * 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.filter; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Preconditions; +import org.apache.iceberg.TableScan; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.expressions.Literal; +import org.apache.iceberg.types.Types; +import org.joda.time.Interval; + +import java.util.ArrayList; +import java.util.List; + +public class IcebergIntervalFilter implements IcebergFilter +{ + @JsonProperty + private final String filterColumn; + + @JsonProperty + private final List<Interval> intervals; + + @JsonCreator + public IcebergIntervalFilter( + @JsonProperty("filterColumn") String filterColumn, + @JsonProperty("intervals") List<Interval> intervals + ) + { + Preconditions.checkNotNull(filterColumn, "filterColumn can not be null"); + Preconditions.checkNotNull(intervals, "intervals can not be null"); Review Comment: we should throw the same error for intervals being empty. Also since its a user-facing error we should make the message a bit more clear. E.g. "You must specify intervals on the interval iceberg filter" ########## extensions-contrib/druid-iceberg-extensions/src/main/java/org/apache/druid/iceberg/filter/IcebergIntervalFilter.java: ########## @@ -0,0 +1,90 @@ +/* + * 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.filter; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Preconditions; +import org.apache.iceberg.TableScan; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.expressions.Literal; +import org.apache.iceberg.types.Types; +import org.joda.time.Interval; + +import java.util.ArrayList; +import java.util.List; + +public class IcebergIntervalFilter implements IcebergFilter +{ + @JsonProperty + private final String filterColumn; + + @JsonProperty + private final List<Interval> intervals; + + @JsonCreator + public IcebergIntervalFilter( + @JsonProperty("filterColumn") String filterColumn, + @JsonProperty("intervals") List<Interval> intervals + ) + { + Preconditions.checkNotNull(filterColumn, "filterColumn can not be null"); + Preconditions.checkNotNull(intervals, "intervals can not be null"); + this.filterColumn = filterColumn; + this.intervals = intervals; + } + + @Override + public TableScan filter(TableScan tableScan) + { + return tableScan.filter(getFilterExpression()); + } + + @Override + public Expression getFilterExpression() + { + List<Expression> expressions = new ArrayList<>(); + for (Interval filterInterval : intervals) { + Long dateStart = (long) Literal.of(filterInterval.getStart().toString()) + .to(Types.TimestampType.withZone()) + .value(); + Long dateEnd = (long) Literal.of(filterInterval.getEnd().toString()) + .to(Types.TimestampType.withZone()) + .value(); + + expressions.add(Expressions.and( + Expressions.greaterThanOrEqual( + filterColumn, + dateStart + ), + Expressions.lessThan( + filterColumn, + dateEnd + ) + )); + } + Expression finalExpr = Expressions.alwaysFalse(); Review Comment: is this extra expression required? ########## processing/src/main/java/org/apache/druid/data/input/AbstractInputSourceAdapter.java: ########## @@ -0,0 +1,130 @@ +/* + * 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.data.input; + +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; +import org.apache.druid.data.input.impl.LocalInputSourceAdapter; +import org.apache.druid.data.input.impl.SplittableInputSource; +import org.apache.druid.java.util.common.CloseableIterators; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.parsers.CloseableIterator; + +import javax.annotation.Nullable; +import java.io.File; +import java.io.IOException; +import java.util.Collections; +import java.util.List; +import java.util.stream.Stream; + +/** + * A wrapper on top of {@link SplittableInputSource} that handles input source creation. Review Comment: I think this class could be called LazyInputSourceBuilder or something to that effect. since it doesn't seem like an adapter. its primary responsibility is lazy on-demand instantiation of input sources. ########## extensions-contrib/druid-iceberg-extensions/pom.xml: ########## @@ -0,0 +1,313 @@ +<?xml version="1.0" encoding="UTF-8"?> +<!-- + ~ 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. + --> +<project xmlns="http://maven.apache.org/POM/4.0.0" + xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" + xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"> + + <groupId>org.apache.druid.extensions</groupId> + <artifactId>druid-iceberg-extensions</artifactId> + <name>druid-iceberg-extensions</name> + <description>druid-iceberg-extensions</description> + + <parent> + <artifactId>druid</artifactId> + <groupId>org.apache.druid</groupId> + <version>27.0.0-SNAPSHOT</version> + <relativePath>../../pom.xml</relativePath> + </parent> + <modelVersion>4.0.0</modelVersion> + + <properties> + </properties> + <dependencies> + <dependency> + <groupId>org.apache.hadoop</groupId> + <artifactId>hadoop-common</artifactId> + <version>${hadoop.compile.version}</version> + <exclusions> + <exclusion> + <groupId>io.netty</groupId> + <artifactId>netty-buffer</artifactId> + </exclusion> + <exclusion> + <groupId>commons-cli</groupId> + <artifactId>commons-cli</artifactId> + </exclusion> + <exclusion> + <groupId>log4j</groupId> + <artifactId>log4j</artifactId> + </exclusion> + <exclusion> + <groupId>commons-codec</groupId> + <artifactId>commons-codec</artifactId> + </exclusion> + <exclusion> + <groupId>commons-logging</groupId> + <artifactId>commons-logging</artifactId> + </exclusion> + <exclusion> + <groupId>commons-io</groupId> + <artifactId>commons-io</artifactId> + </exclusion> + <exclusion> + <groupId>commons-lang</groupId> + <artifactId>commons-lang</artifactId> + </exclusion> + <exclusion> + <groupId>org.apache.httpcomponents</groupId> + <artifactId>httpclient</artifactId> + </exclusion> + <exclusion> + <groupId>org.apache.httpcomponents</groupId> + <artifactId>httpcore</artifactId> + </exclusion> + <exclusion> + <groupId>org.apache.zookeeper</groupId> + <artifactId>zookeeper</artifactId> + </exclusion> + <exclusion> + <groupId>org.slf4j</groupId> + <artifactId>slf4j-api</artifactId> + </exclusion> + <exclusion> + <groupId>org.slf4j</groupId> + <artifactId>slf4j-log4j12</artifactId> + </exclusion> + <exclusion> + <groupId>javax.ws.rs</groupId> + <artifactId>jsr311-api</artifactId> + </exclusion> + <exclusion> + <groupId>com.google.code.findbugs</groupId> + <artifactId>jsr305</artifactId> + </exclusion> + <exclusion> + <groupId>org.mortbay.jetty</groupId> + <artifactId>jetty-util</artifactId> + </exclusion> + <exclusion> + <groupId>com.google.protobuf</groupId> + <artifactId>protobuf-java</artifactId> + </exclusion> + <exclusion> + <groupId>com.sun.jersey</groupId> + <artifactId>jersey-core</artifactId> + </exclusion> + <exclusion> + <groupId>org.apache.curator</groupId> + <artifactId>curator-client</artifactId> + </exclusion> + <exclusion> + <groupId>org.apache.commons</groupId> + <artifactId>commons-math3</artifactId> + </exclusion> + <exclusion> + <groupId>com.google.guava</groupId> + <artifactId>guava</artifactId> + </exclusion> + <exclusion> + <groupId>org.apache.avro</groupId> + <artifactId>avro</artifactId> + </exclusion> + <exclusion> + <groupId>net.java.dev.jets3t</groupId> + <artifactId>jets3t</artifactId> + </exclusion> + <exclusion> + <groupId>com.sun.jersey</groupId> + <artifactId>jersey-json</artifactId> + </exclusion> + <exclusion> + <groupId>com.jcraft</groupId> + <artifactId>jsch</artifactId> + </exclusion> + <exclusion> + <groupId>org.mortbay.jetty</groupId> + <artifactId>jetty</artifactId> + </exclusion> + <exclusion> + <groupId>com.sun.jersey</groupId> + <artifactId>jersey-server</artifactId> + </exclusion> + <!-- Following are excluded to remove security vulnerabilities: --> + <exclusion> + <groupId>commons-beanutils</groupId> + <artifactId>commons-beanutils-core</artifactId> + </exclusion> + </exclusions> + </dependency> + + <dependency> + <groupId>org.apache.iceberg</groupId> + <artifactId>iceberg-spark-runtime-3.3_2.12</artifactId> + <version>1.0.0</version> + <exclusions> + <exclusion> + <groupId>com.fasterxml.jackson.core</groupId> + <artifactId>jackson-databind</artifactId> + </exclusion> + <exclusion> + <groupId>com.fasterxml.jackson.core</groupId> + <artifactId>jackson-core</artifactId> + </exclusion> + <exclusion> + <groupId>com.fasterxml.jackson.core</groupId> + <artifactId>jackson-annotations</artifactId> + </exclusion> + </exclusions> + </dependency> + + <dependency> + <groupId>org.apache.hive</groupId> + <artifactId>hive-metastore</artifactId> + <version>3.1.3</version> + <exclusions> + <exclusion> + <groupId>com.fasterxml.jackson.core</groupId> + <artifactId>jackson-databind</artifactId> + </exclusion> + <exclusion> + <groupId>com.fasterxml.jackson.core</groupId> + <artifactId>jackson-core</artifactId> + </exclusion> + <exclusion> + <groupId>com.fasterxml.jackson.core</groupId> + <artifactId>jackson-annotations</artifactId> + </exclusion> + <exclusion> + <groupId>org.apache.hadoop</groupId> + <artifactId>hadoop-hdfs</artifactId> + </exclusion> + <exclusion> + <groupId>org.apache.hbase</groupId> + <artifactId>hbase-client</artifactId> + </exclusion> + </exclusions> + </dependency> + + <dependency> + <groupId>org.apache.druid</groupId> + <artifactId>druid-processing</artifactId> + <version>${project.parent.version}</version> + <scope>provided</scope> + <exclusions> + <exclusion> + <groupId>org.slf4j</groupId> + <artifactId>slf4j-api</artifactId> + </exclusion> + </exclusions> + </dependency> + <dependency> + <groupId>com.fasterxml.jackson.core</groupId> + <artifactId>jackson-annotations</artifactId> + <scope>provided</scope> + </dependency> + <dependency> + <groupId>com.fasterxml.jackson.core</groupId> + <artifactId>jackson-databind</artifactId> + <scope>provided</scope> + </dependency> + + <dependency> + <groupId>org.apache.hadoop</groupId> + <artifactId>hadoop-hdfs-client</artifactId> + <scope>runtime</scope> + </dependency> + + <dependency> + <groupId>org.apache.hadoop</groupId> + <artifactId>hadoop-mapreduce-client-core</artifactId> Review Comment: my understanding is that any hadoop client requires hadoop-client-api and hadoop-client-runtime which are like uber shaded jars. would that not work here? ########## processing/src/main/java/org/apache/druid/data/input/AbstractInputSourceAdapter.java: ########## @@ -0,0 +1,130 @@ +/* + * 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.data.input; + +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; +import org.apache.druid.data.input.impl.LocalInputSourceAdapter; +import org.apache.druid.data.input.impl.SplittableInputSource; +import org.apache.druid.java.util.common.CloseableIterators; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.parsers.CloseableIterator; + +import javax.annotation.Nullable; +import java.io.File; +import java.io.IOException; +import java.util.Collections; +import java.util.List; +import java.util.stream.Stream; + +/** + * A wrapper on top of {@link SplittableInputSource} that handles input source creation. Review Comment: thinking a bit more about it, memoization doesn't require a class of its own at all. That's something IcebergInputSource can do itself. So all we require is the ability to generate an input source dynamically. and a single-method interface is good enough to achieve that. We can call it `FileInputSourceBuilder` or `FileInputSourceGenerator`. ########## extensions-contrib/druid-iceberg-extensions/src/main/java/org/apache/druid/iceberg/filter/IcebergAndFilter.java: ########## @@ -0,0 +1,95 @@ +/* + * 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.filter; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Preconditions; +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.iceberg.TableScan; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.expressions.Expressions; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.LinkedHashSet; +import java.util.List; + +public class IcebergAndFilter implements IcebergFilter +{ + + private final List<IcebergFilter> filters; + + private static final Logger log = new Logger(IcebergAndFilter.class); + + @JsonCreator + public IcebergAndFilter( + @JsonProperty("filters") List<IcebergFilter> filters + ) + { + Preconditions.checkArgument(filters != null && filters.size() > 0, "filter requires atleast one field"); + this.filters = filters; + } + + @JsonProperty + public List<IcebergFilter> getFilters() + { + return filters; + } + + @Override + public TableScan filter(TableScan tableScan) + { + return tableScan.filter(getFilterExpression()); + } + + @Override + public Expression getFilterExpression() + { + List<Expression> expressions = new ArrayList<>(); + LinkedHashSet<IcebergFilter> flatFilters = flattenAndChildren(filters); + if (!flatFilters.isEmpty()) { + for (IcebergFilter filter : flatFilters) { + expressions.add(filter.getFilterExpression()); + } + } else { + log.error("Empty filter set, running iceberg table scan without filters"); Review Comment: that's not an error. In fact, we shouldn't ever get here at all so might as well remove this else block entirely. ########## extensions-contrib/druid-iceberg-extensions/src/main/java/org/apache/druid/iceberg/filter/IcebergIntervalFilter.java: ########## @@ -0,0 +1,90 @@ +/* + * 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.filter; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Preconditions; +import org.apache.iceberg.TableScan; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.expressions.Literal; +import org.apache.iceberg.types.Types; +import org.joda.time.Interval; + +import java.util.ArrayList; +import java.util.List; + +public class IcebergIntervalFilter implements IcebergFilter +{ + @JsonProperty + private final String filterColumn; + + @JsonProperty + private final List<Interval> intervals; + + @JsonCreator + public IcebergIntervalFilter( + @JsonProperty("filterColumn") String filterColumn, + @JsonProperty("intervals") List<Interval> intervals + ) + { + Preconditions.checkNotNull(filterColumn, "filterColumn can not be null"); + Preconditions.checkNotNull(intervals, "intervals can not be null"); + this.filterColumn = filterColumn; + this.intervals = intervals; + } + + @Override + public TableScan filter(TableScan tableScan) + { + return tableScan.filter(getFilterExpression()); + } + + @Override + public Expression getFilterExpression() + { + List<Expression> expressions = new ArrayList<>(); + for (Interval filterInterval : intervals) { + Long dateStart = (long) Literal.of(filterInterval.getStart().toString()) + .to(Types.TimestampType.withZone()) + .value(); + Long dateEnd = (long) Literal.of(filterInterval.getEnd().toString()) + .to(Types.TimestampType.withZone()) + .value(); + + expressions.add(Expressions.and( Review Comment: we should make it clear in the docs that start is inclusive but end is not. ########## docs/development/extensions-contrib/iceberg.md: ########## @@ -0,0 +1,121 @@ +--- +id: iceberg +title: "Iceberg" +--- + +<!-- + ~ 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. + --> + +## Iceberg Ingest Extension + +This extension provides [IcebergInputSource](../../ingestion/input-sources.md#iceberg-input-source) which enables ingestion of data stored in the Iceberg table format into Druid. + +Apache Iceberg is an open table format for huge analytic datasets. Even though iceberg manages most of its metadata in metadata files in the object storage, it is still dependent on a metastore for managing a certain amount of metadata. +These metastores are defined as Iceberg catalogs and this extension supports connecting to the following catalog types: +* Hive metastore catalog +* Local catalog + +Support for AWS Glue and REST based catalogs are not available yet. + +For a given catalog, iceberg table name and filters, The IcebergInputSource works by reading the table from the catalog, applying the filters and extracting all the underlying live data files up to the latest snapshot. +The data files are in either Parquet, ORC or Avro formats and all of these have InputFormat support in Druid. The data files typically reside in a warehouse location which could be in HDFS, S3 or the local filesystem. +This extension relies on the existing InputSource connectors in Druid to read the data files from the warehouse. Therefore, the IcebergInputSource can be considered as an intermediate InputSource which provides the file paths for other InputSource implementations. + +### Load the Iceberg Ingest extension + +To use the iceberg extension, add the `druid-iceberg-extensions` to the list of loaded extensions. See [Loading extensions](../../configuration/extensions.md#loading-extensions) for more information. + + +### Hive Metastore catalog + +For Druid to seamlessly talk to the Hive Metastore, ensure that the Hive specific configuration files such as `hive-site.xml` and `core-site.xml` are available in the Druid classpath for peon processes. +Hive specific properties can also be specified under the `catalogProperties` object in the ingestion spec. + +Hive metastore catalogs can be associated with different types of warehouses, but this extension presently only supports HDFS and S3 warehouse directories. + +#### Reading from HDFS warehouse + +Ensure that the extension `druid-hdfs-storage` is loaded. The data file paths are extracted from the Hive metastore catalog and [HDFS input source](../../ingestion/input-sources.md#hdfs-input-source) is used to ingest these files. +The `warehouseSource` type in the ingestion spec should be `hdfs`. + +If the Hive metastore supports Kerberos authentication, the following properties will be required in the `catalogProperties`: + +```json +"catalogProperties": { + "principal": "krb_principal", + "keytab": "/path/to/keytab" +} +``` +Only Kerberos based authentication is supported as of now. + +#### Reading from S3 warehouse + +Ensure that the extension `druid-s3-extensions` is loaded. The data file paths are extracted from the Hive metastore catalog and `S3InputSource` is used to ingest these files. +The `warehouseSource` type in the ingestion spec should be `s3`. If the S3 endpoint for the warehouse is different from the endpoint configured as the deep storage, the following properties are required in the `warehouseSource` section to define the S3 endpoint settings: + +```json +"warehouseSource": { + "type": "s3", + "endpointConfig": { + "url": "S3_ENDPOINT_URL", + "signingRegion": "us-east-1" + }, + "clientConfig": { + "protocol": "http", + "disableChunkedEncoding": true, + "enablePathStyleAccess": true, + "forceGlobalBucketAccessEnabled": false + }, + "properties": { + "accessKeyId": { + "type": "default", + "password": "<ACCESS_KEY_ID" + }, + "secretAccessKey": { + "type": "default", + "password": "<SECRET_ACCESS_KEY>" + } + } +} +``` + +This extension uses the [Hadoop AWS module](https://hadoop.apache.org/docs/stable/hadoop-aws/tools/hadoop-aws/) to connect to S3 and retrieve the metadata and data file paths. +The following properties will be required in the `catalogProperties`: + +```json +"catalogProperties": { + "fs.s3a.access.key" : "S3_ACCESS_KEY", Review Comment: can you confirm that these get masked when we log these properties or when someone looks at the ingestion spec? ########## extensions-contrib/druid-iceberg-extensions/src/main/java/org/apache/druid/iceberg/filter/IcebergNotFilter.java: ########## @@ -0,0 +1,60 @@ +/* + * 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.filter; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Preconditions; +import org.apache.iceberg.TableScan; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.expressions.Expressions; + + +public class IcebergNotFilter implements IcebergFilter +{ + private final IcebergFilter filter; + + @JsonCreator + public IcebergNotFilter( + @JsonProperty("filter") IcebergFilter filter + ) + { + Preconditions.checkNotNull(filter, "filter cannot be null"); Review Comment: same comment about the error message. ########## extensions-contrib/druid-iceberg-extensions/src/main/java/org/apache/druid/iceberg/input/IcebergCatalog.java: ########## @@ -0,0 +1,104 @@ +/* + * 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.JsonTypeInfo; +import org.apache.druid.data.input.InputFormat; +import org.apache.druid.iceberg.filter.IcebergFilter; +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; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.io.CloseableIterable; + +import java.util.ArrayList; +import java.util.List; + +/* + * Druid wrapper for an iceberg catalog. + * The configured catalog is used to load the specified iceberg table and retrieve the underlying live data files upto the latest snapshot. + * This does not perform any projections on the table yet, therefore all the underlying columns will be retrieved from the data files. + */ + +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = InputFormat.TYPE_PROPERTY) +public abstract class IcebergCatalog +{ + private static final Logger log = new Logger(IcebergCatalog.class); + + public abstract BaseMetastoreCatalog retrieveCatalog(); + + /** + * Extract the iceberg data files upto the latest snapshot associated with the table + * + * @param tableNamespace The catalog namespace under which the table is defined + * @param tableName The iceberg table name + * @return a list of data file paths + */ + public List<String> extractSnapshotDataFiles( + String tableNamespace, + String tableName, + IcebergFilter icebergFilter + ) + { + Catalog catalog = retrieveCatalog(); + Namespace namespace = Namespace.of(tableNamespace); + String tableIdentifier = tableNamespace + "." + tableName; + + List<String> dataFilePaths = new ArrayList<>(); + + ClassLoader currCtxClassloader = Thread.currentThread().getContextClassLoader(); + try { + Thread.currentThread().setContextClassLoader(getClass().getClassLoader()); + TableIdentifier icebergTableIdentifier = catalog.listTables(namespace).stream() + .filter(tableId -> tableId.toString().equals(tableIdentifier)) + .findFirst() + .orElseThrow(() -> new IAE( + " Couldn't retrieve table identifier for '%s'", + tableIdentifier + )); + + long start = System.currentTimeMillis(); + TableScan tableScan = catalog.loadTable(icebergTableIdentifier).newScan(); + + if (icebergFilter != null) { + tableScan = icebergFilter.filter(tableScan); + } + + CloseableIterable<FileScanTask> tasks = tableScan.planFiles(); + CloseableIterable.transform(tasks, FileScanTask::file) + .forEach(dataFile -> dataFilePaths.add(dataFile.path().toString())); + + long duration = System.currentTimeMillis() - start; + log.info("Data file scan and fetch took %d ms for %d paths", duration, dataFilePaths.size()); Review Comment: ```suggestion log.info("Data file scan and fetch took [%d ms] time for [%d] paths", duration, dataFilePaths.size()); ``` ########## extensions-contrib/druid-iceberg-extensions/src/main/java/org/apache/druid/iceberg/filter/IcebergIntervalFilter.java: ########## @@ -0,0 +1,90 @@ +/* + * 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.filter; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Preconditions; +import org.apache.iceberg.TableScan; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.expressions.Literal; +import org.apache.iceberg.types.Types; +import org.joda.time.Interval; + +import java.util.ArrayList; +import java.util.List; + +public class IcebergIntervalFilter implements IcebergFilter +{ + @JsonProperty + private final String filterColumn; + + @JsonProperty + private final List<Interval> intervals; + + @JsonCreator + public IcebergIntervalFilter( + @JsonProperty("filterColumn") String filterColumn, + @JsonProperty("intervals") List<Interval> intervals + ) + { + Preconditions.checkNotNull(filterColumn, "filterColumn can not be null"); + Preconditions.checkNotNull(intervals, "intervals can not be null"); + this.filterColumn = filterColumn; + this.intervals = intervals; + } + + @Override + public TableScan filter(TableScan tableScan) + { + return tableScan.filter(getFilterExpression()); + } + + @Override + public Expression getFilterExpression() + { + List<Expression> expressions = new ArrayList<>(); + for (Interval filterInterval : intervals) { + Long dateStart = (long) Literal.of(filterInterval.getStart().toString()) + .to(Types.TimestampType.withZone()) Review Comment: what does this call do? ########## extensions-contrib/druid-iceberg-extensions/src/main/java/org/apache/druid/iceberg/input/IcebergCatalog.java: ########## @@ -0,0 +1,104 @@ +/* + * 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.JsonTypeInfo; +import org.apache.druid.data.input.InputFormat; +import org.apache.druid.iceberg.filter.IcebergFilter; +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; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.io.CloseableIterable; + +import java.util.ArrayList; +import java.util.List; + +/* + * Druid wrapper for an iceberg catalog. + * The configured catalog is used to load the specified iceberg table and retrieve the underlying live data files upto the latest snapshot. + * This does not perform any projections on the table yet, therefore all the underlying columns will be retrieved from the data files. + */ + +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = InputFormat.TYPE_PROPERTY) +public abstract class IcebergCatalog +{ + private static final Logger log = new Logger(IcebergCatalog.class); + + public abstract BaseMetastoreCatalog retrieveCatalog(); + + /** + * Extract the iceberg data files upto the latest snapshot associated with the table + * + * @param tableNamespace The catalog namespace under which the table is defined + * @param tableName The iceberg table name + * @return a list of data file paths + */ + public List<String> extractSnapshotDataFiles( + String tableNamespace, + String tableName, + IcebergFilter icebergFilter + ) + { + Catalog catalog = retrieveCatalog(); + Namespace namespace = Namespace.of(tableNamespace); + String tableIdentifier = tableNamespace + "." + tableName; + + List<String> dataFilePaths = new ArrayList<>(); + + ClassLoader currCtxClassloader = Thread.currentThread().getContextClassLoader(); + try { + Thread.currentThread().setContextClassLoader(getClass().getClassLoader()); + TableIdentifier icebergTableIdentifier = catalog.listTables(namespace).stream() + .filter(tableId -> tableId.toString().equals(tableIdentifier)) + .findFirst() + .orElseThrow(() -> new IAE( + " Couldn't retrieve table identifier for '%s'", Review Comment: is there an action that the user can take to remedy this error? Like verifying that the table indeed exists in the catalog. ########## extensions-contrib/druid-iceberg-extensions/src/main/java/org/apache/druid/iceberg/filter/IcebergOrFilter.java: ########## @@ -0,0 +1,96 @@ +/* + * 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.filter; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Preconditions; +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.iceberg.TableScan; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.expressions.Expressions; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.LinkedHashSet; +import java.util.List; + +public class IcebergOrFilter implements IcebergFilter +{ + private final List<IcebergFilter> filters; + + private static final Logger log = new Logger(IcebergAndFilter.class); + + @JsonCreator + public IcebergOrFilter( + @JsonProperty("filters") List<IcebergFilter> filters + ) + { + Preconditions.checkArgument(filters != null && filters.size() > 0, "filter requires atleast one field"); + this.filters = filters; + } + + @JsonProperty + public List<IcebergFilter> getFilters() + { + return filters; + } + + @Override + public TableScan filter(TableScan tableScan) + { + return tableScan.filter(getFilterExpression()); + } + + @Override + public Expression getFilterExpression() + { + List<Expression> expressions = new ArrayList<>(); + LinkedHashSet<IcebergFilter> flatFilters = flattenOrChildren(filters); + if (!flatFilters.isEmpty()) { + for (IcebergFilter filter : flatFilters) { + expressions.add(filter.getFilterExpression()); + } + } else { + log.error("Empty filter set, running iceberg table scan without filters"); Review Comment: same comment about the error message. ########## processing/src/main/java/org/apache/druid/data/input/AbstractInputSourceAdapter.java: ########## @@ -0,0 +1,133 @@ +/* + * 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.data.input; + +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; +import org.apache.druid.data.input.impl.LocalInputSourceAdapter; +import org.apache.druid.data.input.impl.SplittableInputSource; +import org.apache.druid.java.util.common.CloseableIterators; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.parsers.CloseableIterator; + +import javax.annotation.Nullable; +import java.io.File; +import java.io.IOException; +import java.util.Collections; +import java.util.List; +import java.util.stream.Stream; + +/** + * A wrapper on top of {@link SplittableInputSource} that handles input source creation. + * For composing input sources such as IcebergInputSource, the delegate input source instantiation might fail upon deserialization since the input file paths + * are not available yet and this might fail the input source precondition checks. + * This adapter helps create the delegate input source once the input file paths are fully determined. + */ +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type") +@JsonSubTypes(value = { + @JsonSubTypes.Type(name = LocalInputSourceAdapter.TYPE_KEY, value = LocalInputSourceAdapter.class) +}) +public abstract class AbstractInputSourceAdapter +{ + private SplittableInputSource inputSource; + + public abstract SplittableInputSource generateInputSource(List<String> inputFilePaths); + + public void setupInputSource(List<String> inputFilePaths) + { + if (inputSource != null) { + throw new ISE("Inputsource is already initialized!"); + } + if (inputFilePaths.isEmpty()) { + inputSource = new EmptyInputSource(); + } else { + inputSource = generateInputSource(inputFilePaths); + } + } + + public SplittableInputSource getInputSource() + { + if (inputSource == null) { + throw new ISE("Inputsource is not initialized yet!"); + } + return inputSource; + } + + private static class EmptyInputSource implements SplittableInputSource Review Comment: you should add a note here that this class exists because some underlying input sources might not accept an empty list of input sources. While an empty list is possible when working with iceberg. ########## extensions-contrib/druid-iceberg-extensions/src/main/java/org/apache/druid/iceberg/input/IcebergCatalog.java: ########## @@ -0,0 +1,104 @@ +/* + * 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.JsonTypeInfo; +import org.apache.druid.data.input.InputFormat; +import org.apache.druid.iceberg.filter.IcebergFilter; +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; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.io.CloseableIterable; + +import java.util.ArrayList; +import java.util.List; + +/* + * Druid wrapper for an iceberg catalog. + * The configured catalog is used to load the specified iceberg table and retrieve the underlying live data files upto the latest snapshot. + * This does not perform any projections on the table yet, therefore all the underlying columns will be retrieved from the data files. + */ + +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = InputFormat.TYPE_PROPERTY) +public abstract class IcebergCatalog +{ + private static final Logger log = new Logger(IcebergCatalog.class); + + public abstract BaseMetastoreCatalog retrieveCatalog(); + + /** + * Extract the iceberg data files upto the latest snapshot associated with the table Review Comment: what does extracting mean exactly? It returns a list of remote file paths? ########## extensions-contrib/druid-iceberg-extensions/pom.xml: ########## @@ -0,0 +1,314 @@ +<?xml version="1.0" encoding="UTF-8"?> +<!-- + ~ 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. + --> +<project xmlns="http://maven.apache.org/POM/4.0.0" + xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" + xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"> + + <groupId>org.apache.druid.extensions</groupId> + <artifactId>druid-iceberg-extensions</artifactId> + <name>druid-iceberg-extensions</name> + <description>druid-iceberg-extensions</description> + + + <parent> + <artifactId>druid</artifactId> + <groupId>org.apache.druid</groupId> + <version>27.0.0-SNAPSHOT</version> + <relativePath>../../pom.xml</relativePath> + </parent> + <modelVersion>4.0.0</modelVersion> + + <properties> + </properties> + <dependencies> + <dependency> Review Comment: not required. though can we not even add this module if hadoop2 profile is activated? Assuming such a thing is possible. ########## processing/src/main/java/org/apache/druid/data/input/AbstractInputSourceAdapter.java: ########## @@ -0,0 +1,130 @@ +/* + * 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.data.input; + +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; +import org.apache.druid.data.input.impl.LocalInputSourceAdapter; +import org.apache.druid.data.input.impl.SplittableInputSource; +import org.apache.druid.java.util.common.CloseableIterators; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.parsers.CloseableIterator; + +import javax.annotation.Nullable; +import java.io.File; +import java.io.IOException; +import java.util.Collections; +import java.util.List; +import java.util.stream.Stream; + +/** + * A wrapper on top of {@link SplittableInputSource} that handles input source creation. Review Comment: In fact, this class could be split into one concrete class that does memoization and one interface that has a `build` method. The extensions can just implement the interface. -- 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]
