|
| 1 | +/* |
| 2 | + * Licensed to the Apache Software Foundation (ASF) under one |
| 3 | + * or more contributor license agreements. See the NOTICE file |
| 4 | + * distributed with this work for additional information |
| 5 | + * regarding copyright ownership. The ASF licenses this file |
| 6 | + * to you under the Apache License, Version 2.0 (the |
| 7 | + * "License"); you may not use this file except in compliance |
| 8 | + * with the License. You may obtain a copy of the License at |
| 9 | + * |
| 10 | + * http://www.apache.org/licenses/LICENSE-2.0 |
| 11 | + * |
| 12 | + * Unless required by applicable law or agreed to in writing, software |
| 13 | + * distributed under the License is distributed on an "AS IS" BASIS, |
| 14 | + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
| 15 | + * See the License for the specific language governing permissions and |
| 16 | + * limitations under the License. |
| 17 | + */ |
| 18 | +package org.apache.beam.sdk.io.iceberg; |
| 19 | + |
| 20 | +import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull; |
| 21 | + |
| 22 | +import java.io.IOException; |
| 23 | +import java.util.List; |
| 24 | +import java.util.concurrent.ExecutionException; |
| 25 | +import org.apache.beam.sdk.metrics.Counter; |
| 26 | +import org.apache.beam.sdk.metrics.Metrics; |
| 27 | +import org.apache.beam.sdk.transforms.DoFn; |
| 28 | +import org.apache.beam.sdk.values.KV; |
| 29 | +import org.apache.iceberg.CombinedScanTask; |
| 30 | +import org.apache.iceberg.DataOperations; |
| 31 | +import org.apache.iceberg.IncrementalAppendScan; |
| 32 | +import org.apache.iceberg.Table; |
| 33 | +import org.apache.iceberg.io.CloseableIterable; |
| 34 | +import org.checkerframework.checker.nullness.qual.Nullable; |
| 35 | +import org.joda.time.Instant; |
| 36 | +import org.slf4j.Logger; |
| 37 | +import org.slf4j.LoggerFactory; |
| 38 | + |
| 39 | +/** |
| 40 | + * Scans the given snapshot and creates multiple {@link ReadTask}s. Each task represents a portion |
| 41 | + * of a data file that was appended within the snapshot range. |
| 42 | + */ |
| 43 | +class CreateReadTasksDoFn |
| 44 | + extends DoFn<KV<String, List<SnapshotInfo>>, KV<ReadTaskDescriptor, ReadTask>> { |
| 45 | + private static final Logger LOG = LoggerFactory.getLogger(CreateReadTasksDoFn.class); |
| 46 | + private static final Counter totalScanTasks = |
| 47 | + Metrics.counter(CreateReadTasksDoFn.class, "totalScanTasks"); |
| 48 | + private final IcebergScanConfig scanConfig; |
| 49 | + |
| 50 | + CreateReadTasksDoFn(IcebergScanConfig scanConfig) { |
| 51 | + this.scanConfig = scanConfig; |
| 52 | + } |
| 53 | + |
| 54 | + @Setup |
| 55 | + public void setup() { |
| 56 | + TableCache.setup(scanConfig); |
| 57 | + } |
| 58 | + |
| 59 | + @ProcessElement |
| 60 | + public void process( |
| 61 | + @Element KV<String, List<SnapshotInfo>> element, |
| 62 | + OutputReceiver<KV<ReadTaskDescriptor, ReadTask>> out) |
| 63 | + throws IOException, ExecutionException { |
| 64 | + // force refresh because the table must be updated before scanning snapshots |
| 65 | + Table table = TableCache.getRefreshed(element.getKey()); |
| 66 | + |
| 67 | + // scan snapshots individually and assign commit timestamp to files |
| 68 | + for (SnapshotInfo snapshot : element.getValue()) { |
| 69 | + @Nullable Long fromSnapshot = snapshot.getParentId(); |
| 70 | + long toSnapshot = snapshot.getSnapshotId(); |
| 71 | + |
| 72 | + if (!DataOperations.APPEND.equals(snapshot.getOperation())) { |
| 73 | + LOG.info( |
| 74 | + "Skipping non-append snapshot of operation '{}'. Sequence number: {}, id: {}", |
| 75 | + snapshot.getOperation(), |
| 76 | + snapshot.getSequenceNumber(), |
| 77 | + snapshot.getSnapshotId()); |
| 78 | + } |
| 79 | + |
| 80 | + LOG.info("Planning to scan snapshot {}", toSnapshot); |
| 81 | + IncrementalAppendScan scan = table.newIncrementalAppendScan().toSnapshot(toSnapshot); |
| 82 | + if (fromSnapshot != null) { |
| 83 | + scan = scan.fromSnapshotExclusive(fromSnapshot); |
| 84 | + } |
| 85 | + |
| 86 | + createAndOutputReadTasks(scan, snapshot, out); |
| 87 | + } |
| 88 | + } |
| 89 | + |
| 90 | + private void createAndOutputReadTasks( |
| 91 | + IncrementalAppendScan scan, |
| 92 | + SnapshotInfo snapshot, |
| 93 | + OutputReceiver<KV<ReadTaskDescriptor, ReadTask>> out) |
| 94 | + throws IOException { |
| 95 | + int numTasks = 0; |
| 96 | + try (CloseableIterable<CombinedScanTask> combinedScanTasks = scan.planTasks()) { |
| 97 | + for (CombinedScanTask combinedScanTask : combinedScanTasks) { |
| 98 | + ReadTask task = ReadTask.builder().setCombinedScanTask(combinedScanTask).build(); |
| 99 | + ReadTaskDescriptor descriptor = |
| 100 | + ReadTaskDescriptor.builder() |
| 101 | + .setTableIdentifierString(checkStateNotNull(snapshot.getTableIdentifierString())) |
| 102 | + .build(); |
| 103 | + |
| 104 | + out.outputWithTimestamp( |
| 105 | + KV.of(descriptor, task), Instant.ofEpochMilli(snapshot.getTimestampMillis())); |
| 106 | + numTasks += combinedScanTask.tasks().size(); |
| 107 | + } |
| 108 | + } |
| 109 | + totalScanTasks.inc(numTasks); |
| 110 | + LOG.info("Snapshot {} produced {} read tasks.", snapshot.getSnapshotId(), numTasks); |
| 111 | + } |
| 112 | +} |
0 commit comments