chamikaramj commented on code in PR #30805: URL: https://github.com/apache/beam/pull/30805#discussion_r1546669357
########## sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/IcebergIO.java: ########## @@ -0,0 +1,105 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.io.iceberg; + +import java.util.Arrays; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.transforms.SerializableBiFunction; +import org.apache.beam.sdk.transforms.SerializableFunctions; +import org.apache.beam.sdk.values.PCollection; +import org.apache.iceberg.Table; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.data.Record; +import org.checkerframework.checker.nullness.qual.NonNull; + +public class IcebergIO { + + public enum ScanType { + TABLE, + BATCH + } + + public static <ElementT, DestinationT> Write<ElementT, DestinationT> writeToDestinations( + IcebergCatalog catalog, + DynamicDestinations<ElementT, DestinationT> dynamicDestinations, + SerializableBiFunction<Record, ElementT, Record> toRecord) { + return new Write<>(catalog, dynamicDestinations, toRecord); + } + + public static TableFactory<String> forCatalog(final IcebergCatalog catalog) { + return new TableFactory<String>() { + @Override + public Table getTable(String id) { + TableIdentifier tableId = TableIdentifier.parse(id); + // If the first element in the namespace is our catalog, remove that. + if (tableId.hasNamespace()) { + Namespace ns = tableId.namespace(); + if (catalog.catalog().name().equals(ns.level(0))) { + String[] levels = ns.levels(); + @SuppressWarnings("nullness") // we know that copyOfRange will not do any padding + @NonNull + String[] levelsMinusFirst = Arrays.copyOfRange(levels, 1, levels.length); + tableId = TableIdentifier.of(Namespace.of(levelsMinusFirst), tableId.name()); + } + } + return catalog.catalog().loadTable(tableId); + } + }; + } + + public static class Write<ElementT, DestinationT> Review Comment: Noting that this also has Write stuff but I'm ignoring those for this review. ########## sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/IcebergBoundedSource.java: ########## @@ -0,0 +1,129 @@ +/* + * 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.beam.io.iceberg; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.RowCoder; +import org.apache.beam.sdk.io.BoundedSource; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.transforms.display.DisplayData; +import org.apache.beam.sdk.values.Row; +import org.apache.iceberg.CombinedScanTask; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.TableScan; +import org.apache.iceberg.catalog.Catalog; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.io.CloseableIterable; +import org.checkerframework.checker.nullness.qual.Nullable; + +public class IcebergBoundedSource extends BoundedSource<Row> { Review Comment: Ideally we should use SDF but I think it would suffice to add a TODO to convert this to an SDF in the future. ########## sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/CombinedScanReader.java: ########## @@ -0,0 +1,214 @@ +/* + * 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.beam.io.iceberg; + +import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull; + +import java.io.IOException; +import java.util.ArrayDeque; +import java.util.NoSuchElementException; +import java.util.Queue; +import javax.annotation.Nullable; +import org.apache.beam.sdk.io.BoundedSource; +import org.apache.beam.sdk.schemas.Schema; +import org.apache.beam.sdk.values.Row; +import org.apache.iceberg.CombinedScanTask; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.Table; +import org.apache.iceberg.avro.Avro; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.data.avro.DataReader; +import org.apache.iceberg.data.orc.GenericOrcReader; +import org.apache.iceberg.data.parquet.GenericParquetReaders; +import org.apache.iceberg.encryption.EncryptionManager; +import org.apache.iceberg.encryption.InputFilesDecryptor; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.io.CloseableIterator; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.io.InputFile; +import org.apache.iceberg.orc.ORC; +import org.apache.iceberg.parquet.Parquet; +import org.checkerframework.checker.nullness.qual.NonNull; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class CombinedScanReader extends BoundedSource.BoundedReader<Row> { + private static final Logger LOG = LoggerFactory.getLogger(CombinedScanReader.class); + + private final IcebergBoundedSource source; + private final Table table; + + private final @Nullable CombinedScanTask task; + + private final Schema schema; + + transient @Nullable org.apache.iceberg.Schema project; + transient @Nullable FileIO io; + transient @Nullable InputFilesDecryptor decryptor; + transient @Nullable Queue<FileScanTask> files; + transient @Nullable CloseableIterator<Record> baseIter; + transient @Nullable Record current; + + public CombinedScanReader( + IcebergBoundedSource source, @Nullable CombinedScanTask task, Schema schema) { + this.source = source; + this.table = + checkStateNotNull( + source.table(), "CombinedScanReader requires an IcebergBoundedSource with a table"); + this.task = task; + this.schema = schema; + if (this.schema != null) { + project = SchemaHelper.convert(this.schema); + } + } + + @Override + public boolean start() throws IOException { + if (task == null) { + return false; + } + + EncryptionManager encryptionManager = table.encryption(); + io = table.io(); + decryptor = new InputFilesDecryptor(task, io, encryptionManager); + + files = new ArrayDeque<>(); + files.addAll(task.files()); + + return advance(); + } + + @Override + public boolean advance() throws IOException { + Queue<FileScanTask> files = + checkStateNotNull(this.files, "files null in advance() - did you call start()?"); + InputFilesDecryptor decryptor = + checkStateNotNull(this.decryptor, "decryptor null in adance() - did you call start()?"); + + // This is a lie, but the most expedient way to work with IcebergIO's + // which are not null-safe. + @SuppressWarnings("nullness") + org.apache.iceberg.@NonNull Schema project = this.project; + + do { + // If our current iterator is working... do that. + if (baseIter != null && baseIter.hasNext()) { + current = baseIter.next(); + return true; + } + + // Close out the current iterator and try to open a new one + if (baseIter != null) { + baseIter.close(); + baseIter = null; + } + + LOG.info("Trying to open new file."); + @Nullable FileScanTask fileTask = null; + while (!files.isEmpty() && fileTask == null) { + fileTask = files.remove(); + if (fileTask.isDataTask()) { Review Comment: Why are we skipping data tasks ? Was this supposed to be `if (!fileTask.isDataTask())` ? Seems like DataTasks contain actual data: https://iceberg.apache.org/javadoc/0.11.0/org/apache/iceberg/DataTask.html ########## sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/CombinedScanReader.java: ########## @@ -0,0 +1,214 @@ +/* + * 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.beam.io.iceberg; + +import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull; + +import java.io.IOException; +import java.util.ArrayDeque; +import java.util.NoSuchElementException; +import java.util.Queue; +import javax.annotation.Nullable; +import org.apache.beam.sdk.io.BoundedSource; +import org.apache.beam.sdk.schemas.Schema; +import org.apache.beam.sdk.values.Row; +import org.apache.iceberg.CombinedScanTask; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.Table; +import org.apache.iceberg.avro.Avro; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.data.avro.DataReader; +import org.apache.iceberg.data.orc.GenericOrcReader; +import org.apache.iceberg.data.parquet.GenericParquetReaders; +import org.apache.iceberg.encryption.EncryptionManager; +import org.apache.iceberg.encryption.InputFilesDecryptor; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.io.CloseableIterator; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.io.InputFile; +import org.apache.iceberg.orc.ORC; +import org.apache.iceberg.parquet.Parquet; +import org.checkerframework.checker.nullness.qual.NonNull; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class CombinedScanReader extends BoundedSource.BoundedReader<Row> { + private static final Logger LOG = LoggerFactory.getLogger(CombinedScanReader.class); + + private final IcebergBoundedSource source; + private final Table table; + + private final @Nullable CombinedScanTask task; + + private final Schema schema; + + transient @Nullable org.apache.iceberg.Schema project; + transient @Nullable FileIO io; + transient @Nullable InputFilesDecryptor decryptor; + transient @Nullable Queue<FileScanTask> files; + transient @Nullable CloseableIterator<Record> baseIter; + transient @Nullable Record current; + + public CombinedScanReader( + IcebergBoundedSource source, @Nullable CombinedScanTask task, Schema schema) { + this.source = source; + this.table = + checkStateNotNull( + source.table(), "CombinedScanReader requires an IcebergBoundedSource with a table"); + this.task = task; + this.schema = schema; + if (this.schema != null) { + project = SchemaHelper.convert(this.schema); + } + } + + @Override + public boolean start() throws IOException { + if (task == null) { + return false; + } + + EncryptionManager encryptionManager = table.encryption(); + io = table.io(); + decryptor = new InputFilesDecryptor(task, io, encryptionManager); + + files = new ArrayDeque<>(); + files.addAll(task.files()); + + return advance(); + } + + @Override + public boolean advance() throws IOException { + Queue<FileScanTask> files = + checkStateNotNull(this.files, "files null in advance() - did you call start()?"); + InputFilesDecryptor decryptor = + checkStateNotNull(this.decryptor, "decryptor null in adance() - did you call start()?"); + + // This is a lie, but the most expedient way to work with IcebergIO's + // which are not null-safe. + @SuppressWarnings("nullness") + org.apache.iceberg.@NonNull Schema project = this.project; + + do { + // If our current iterator is working... do that. + if (baseIter != null && baseIter.hasNext()) { + current = baseIter.next(); + return true; + } + + // Close out the current iterator and try to open a new one + if (baseIter != null) { + baseIter.close(); + baseIter = null; + } + + LOG.info("Trying to open new file."); + @Nullable FileScanTask fileTask = null; + while (!files.isEmpty() && fileTask == null) { + fileTask = files.remove(); + if (fileTask.isDataTask()) { + LOG.error("{} is a DataTask. Skipping.", fileTask.toString()); + fileTask = null; + } + } + + // We have a new file to start reading + if (fileTask != null) { + DataFile file = fileTask.file(); + InputFile input = decryptor.getInputFile(fileTask); + + @Nullable CloseableIterable<Record> iterable = null; + switch (file.format()) { + case ORC: + LOG.info("Preparing ORC input"); + iterable = + ORC.read(input) + .project(project) + .createReaderFunc( + fileSchema -> GenericOrcReader.buildReader(project, fileSchema)) + .filter(fileTask.residual()) + .build(); + break; + case PARQUET: + LOG.info("Preparing Parquet input."); + iterable = + Parquet.read(input) + .project(project) + .createReaderFunc( + fileSchema -> GenericParquetReaders.buildReader(project, fileSchema)) + .filter(fileTask.residual()) + .build(); + break; + case AVRO: + LOG.info("Preparing Avro input."); + iterable = + Avro.read(input).project(project).createReaderFunc(DataReader::create).build(); + break; + default: + throw new UnsupportedOperationException("Cannot read format: " + file.format()); + } + + if (iterable != null) { + baseIter = iterable.iterator(); + } + } else { + LOG.info("We have exhausted all available files in this CombinedScanTask"); + } + + } while (baseIter != null); + return false; + } + + private Row convert(Record record) { + Row.Builder b = Row.withSchema(schema); + for (int i = 0; i < schema.getFieldCount(); i++) { + // TODO: A lot obviously Review Comment: What does this mean ? ########## sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/IcebergBoundedSource.java: ########## @@ -0,0 +1,129 @@ +/* + * 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.beam.io.iceberg; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.RowCoder; +import org.apache.beam.sdk.io.BoundedSource; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.transforms.display.DisplayData; +import org.apache.beam.sdk.values.Row; +import org.apache.iceberg.CombinedScanTask; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.TableScan; +import org.apache.iceberg.catalog.Catalog; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.io.CloseableIterable; +import org.checkerframework.checker.nullness.qual.Nullable; + +public class IcebergBoundedSource extends BoundedSource<Row> { + + private @Nullable CombinedScanTask task; + private IcebergScan scan; + + public IcebergBoundedSource(IcebergScan scan, @Nullable CombinedScanTask task) { + this.task = task; + this.scan = scan; + } + + public IcebergBoundedSource(IcebergScan scan) { + this(scan, null); + } + + public Catalog catalog() { + return scan.getCatalog().catalog(); + } + + public Table table() { + return catalog() + .loadTable(TableIdentifier.of(scan.getTable().toArray(new String[scan.getTable().size()]))); + } + + @Override + public List<? extends BoundedSource<Row>> split( + long desiredBundleSizeBytes, PipelineOptions options) throws Exception { + ArrayList<IcebergBoundedSource> tasks = new ArrayList<>(); + switch (scan.getType()) { + case TABLE: + // Override the split size with our desired size + TableScan tableScan = table().newScan(); + + if (desiredBundleSizeBytes > 0) { + tableScan = tableScan.option(TableProperties.SPLIT_SIZE, "" + desiredBundleSizeBytes); + } + + // Always project to our destination schema + tableScan = tableScan.project(SchemaHelper.convert(scan.getSchema())); + + if (scan.getFilter() != null) { + tableScan = tableScan.filter(scan.getFilter()); + } + if (scan.getCaseSensitive() != null) { + tableScan = tableScan.caseSensitive(scan.getCaseSensitive()); + } + if (scan.getSnapshot() != null) { + tableScan = tableScan.useSnapshot(scan.getSnapshot()); + } + if (scan.getBranch() != null) { + tableScan = tableScan.useRef(scan.getBranch()); + } else if (scan.getTag() != null) { + tableScan = tableScan.useRef(scan.getTag()); + } + try (CloseableIterable<CombinedScanTask> t = tableScan.planTasks()) { + for (CombinedScanTask c : t) { + tasks.add(new IcebergBoundedSource(scan, c)); + } + } catch (IOException e) { + throw new RuntimeException(e); + } + break; + case BATCH: + // TODO: Add batch scan Review Comment: We should fail here to prevent data loss. ########## sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/IcebergIO.java: ########## @@ -0,0 +1,105 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.io.iceberg; + +import java.util.Arrays; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.transforms.SerializableBiFunction; +import org.apache.beam.sdk.transforms.SerializableFunctions; +import org.apache.beam.sdk.values.PCollection; +import org.apache.iceberg.Table; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.data.Record; +import org.checkerframework.checker.nullness.qual.NonNull; + +public class IcebergIO { Review Comment: Seems like we are missing the `Read` transform ? ########## sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/CombinedScanReader.java: ########## @@ -0,0 +1,214 @@ +/* + * 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.beam.io.iceberg; + +import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull; + +import java.io.IOException; +import java.util.ArrayDeque; +import java.util.NoSuchElementException; +import java.util.Queue; +import javax.annotation.Nullable; +import org.apache.beam.sdk.io.BoundedSource; +import org.apache.beam.sdk.schemas.Schema; +import org.apache.beam.sdk.values.Row; +import org.apache.iceberg.CombinedScanTask; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.Table; +import org.apache.iceberg.avro.Avro; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.data.avro.DataReader; +import org.apache.iceberg.data.orc.GenericOrcReader; +import org.apache.iceberg.data.parquet.GenericParquetReaders; +import org.apache.iceberg.encryption.EncryptionManager; +import org.apache.iceberg.encryption.InputFilesDecryptor; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.io.CloseableIterator; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.io.InputFile; +import org.apache.iceberg.orc.ORC; +import org.apache.iceberg.parquet.Parquet; +import org.checkerframework.checker.nullness.qual.NonNull; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class CombinedScanReader extends BoundedSource.BoundedReader<Row> { + private static final Logger LOG = LoggerFactory.getLogger(CombinedScanReader.class); + + private final IcebergBoundedSource source; + private final Table table; + + private final @Nullable CombinedScanTask task; + + private final Schema schema; + + transient @Nullable org.apache.iceberg.Schema project; + transient @Nullable FileIO io; + transient @Nullable InputFilesDecryptor decryptor; + transient @Nullable Queue<FileScanTask> files; + transient @Nullable CloseableIterator<Record> baseIter; + transient @Nullable Record current; + + public CombinedScanReader( + IcebergBoundedSource source, @Nullable CombinedScanTask task, Schema schema) { + this.source = source; + this.table = + checkStateNotNull( + source.table(), "CombinedScanReader requires an IcebergBoundedSource with a table"); + this.task = task; + this.schema = schema; + if (this.schema != null) { + project = SchemaHelper.convert(this.schema); + } + } + + @Override + public boolean start() throws IOException { + if (task == null) { + return false; + } + + EncryptionManager encryptionManager = table.encryption(); + io = table.io(); + decryptor = new InputFilesDecryptor(task, io, encryptionManager); + + files = new ArrayDeque<>(); + files.addAll(task.files()); + + return advance(); + } + + @Override + public boolean advance() throws IOException { + Queue<FileScanTask> files = + checkStateNotNull(this.files, "files null in advance() - did you call start()?"); + InputFilesDecryptor decryptor = + checkStateNotNull(this.decryptor, "decryptor null in adance() - did you call start()?"); + + // This is a lie, but the most expedient way to work with IcebergIO's Review Comment: Update/remove comment ? ########## sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/IcebergBoundedSource.java: ########## @@ -0,0 +1,129 @@ +/* + * 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.beam.io.iceberg; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.RowCoder; +import org.apache.beam.sdk.io.BoundedSource; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.transforms.display.DisplayData; +import org.apache.beam.sdk.values.Row; +import org.apache.iceberg.CombinedScanTask; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.TableScan; +import org.apache.iceberg.catalog.Catalog; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.io.CloseableIterable; +import org.checkerframework.checker.nullness.qual.Nullable; + +public class IcebergBoundedSource extends BoundedSource<Row> { + + private @Nullable CombinedScanTask task; + private IcebergScan scan; + + public IcebergBoundedSource(IcebergScan scan, @Nullable CombinedScanTask task) { + this.task = task; + this.scan = scan; + } + + public IcebergBoundedSource(IcebergScan scan) { + this(scan, null); + } + + public Catalog catalog() { + return scan.getCatalog().catalog(); + } + + public Table table() { + return catalog() + .loadTable(TableIdentifier.of(scan.getTable().toArray(new String[scan.getTable().size()]))); + } + + @Override + public List<? extends BoundedSource<Row>> split( + long desiredBundleSizeBytes, PipelineOptions options) throws Exception { + ArrayList<IcebergBoundedSource> tasks = new ArrayList<>(); + switch (scan.getType()) { + case TABLE: + // Override the split size with our desired size + TableScan tableScan = table().newScan(); + + if (desiredBundleSizeBytes > 0) { + tableScan = tableScan.option(TableProperties.SPLIT_SIZE, "" + desiredBundleSizeBytes); Review Comment: String.valueOf(desiredBundleSizeBytes) ########## sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/CombinedScanReader.java: ########## @@ -0,0 +1,214 @@ +/* + * 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.beam.io.iceberg; + +import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull; + +import java.io.IOException; +import java.util.ArrayDeque; +import java.util.NoSuchElementException; +import java.util.Queue; +import javax.annotation.Nullable; +import org.apache.beam.sdk.io.BoundedSource; +import org.apache.beam.sdk.schemas.Schema; +import org.apache.beam.sdk.values.Row; +import org.apache.iceberg.CombinedScanTask; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.Table; +import org.apache.iceberg.avro.Avro; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.data.avro.DataReader; +import org.apache.iceberg.data.orc.GenericOrcReader; +import org.apache.iceberg.data.parquet.GenericParquetReaders; +import org.apache.iceberg.encryption.EncryptionManager; +import org.apache.iceberg.encryption.InputFilesDecryptor; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.io.CloseableIterator; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.io.InputFile; +import org.apache.iceberg.orc.ORC; +import org.apache.iceberg.parquet.Parquet; +import org.checkerframework.checker.nullness.qual.NonNull; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class CombinedScanReader extends BoundedSource.BoundedReader<Row> { + private static final Logger LOG = LoggerFactory.getLogger(CombinedScanReader.class); + + private final IcebergBoundedSource source; + private final Table table; + + private final @Nullable CombinedScanTask task; + + private final Schema schema; + + transient @Nullable org.apache.iceberg.Schema project; + transient @Nullable FileIO io; + transient @Nullable InputFilesDecryptor decryptor; + transient @Nullable Queue<FileScanTask> files; + transient @Nullable CloseableIterator<Record> baseIter; + transient @Nullable Record current; + + public CombinedScanReader( + IcebergBoundedSource source, @Nullable CombinedScanTask task, Schema schema) { + this.source = source; + this.table = + checkStateNotNull( + source.table(), "CombinedScanReader requires an IcebergBoundedSource with a table"); + this.task = task; + this.schema = schema; + if (this.schema != null) { + project = SchemaHelper.convert(this.schema); + } + } + + @Override + public boolean start() throws IOException { + if (task == null) { + return false; + } + + EncryptionManager encryptionManager = table.encryption(); + io = table.io(); + decryptor = new InputFilesDecryptor(task, io, encryptionManager); + + files = new ArrayDeque<>(); + files.addAll(task.files()); + + return advance(); + } + + @Override + public boolean advance() throws IOException { + Queue<FileScanTask> files = + checkStateNotNull(this.files, "files null in advance() - did you call start()?"); + InputFilesDecryptor decryptor = + checkStateNotNull(this.decryptor, "decryptor null in adance() - did you call start()?"); + + // This is a lie, but the most expedient way to work with IcebergIO's + // which are not null-safe. + @SuppressWarnings("nullness") + org.apache.iceberg.@NonNull Schema project = this.project; + + do { + // If our current iterator is working... do that. + if (baseIter != null && baseIter.hasNext()) { + current = baseIter.next(); + return true; + } + + // Close out the current iterator and try to open a new one + if (baseIter != null) { + baseIter.close(); + baseIter = null; + } + + LOG.info("Trying to open new file."); + @Nullable FileScanTask fileTask = null; + while (!files.isEmpty() && fileTask == null) { + fileTask = files.remove(); + if (fileTask.isDataTask()) { + LOG.error("{} is a DataTask. Skipping.", fileTask.toString()); + fileTask = null; + } + } + + // We have a new file to start reading + if (fileTask != null) { + DataFile file = fileTask.file(); + InputFile input = decryptor.getInputFile(fileTask); + + @Nullable CloseableIterable<Record> iterable = null; + switch (file.format()) { + case ORC: + LOG.info("Preparing ORC input"); + iterable = + ORC.read(input) + .project(project) + .createReaderFunc( + fileSchema -> GenericOrcReader.buildReader(project, fileSchema)) + .filter(fileTask.residual()) + .build(); + break; + case PARQUET: + LOG.info("Preparing Parquet input."); + iterable = + Parquet.read(input) + .project(project) + .createReaderFunc( + fileSchema -> GenericParquetReaders.buildReader(project, fileSchema)) + .filter(fileTask.residual()) + .build(); + break; + case AVRO: + LOG.info("Preparing Avro input."); + iterable = + Avro.read(input).project(project).createReaderFunc(DataReader::create).build(); + break; + default: + throw new UnsupportedOperationException("Cannot read format: " + file.format()); + } + + if (iterable != null) { + baseIter = iterable.iterator(); + } + } else { + LOG.info("We have exhausted all available files in this CombinedScanTask"); + } + + } while (baseIter != null); + return false; Review Comment: I think the protocol expect the `advance()` call to return true while there are elements. So we should set`current` and return `true` when a new iterator is set. -- 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]
