a2l007 commented on code in PR #14329: URL: https://github.com/apache/druid/pull/14329#discussion_r1255047919
########## 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: Added docs, thanks. ########## 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: Yes, it gets the file paths from the scan tasks. ########## 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: Updated the error message. -- 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]
