|
| 1 | +/* |
| 2 | + * Licensed to the Apache Software Foundation (ASF) under one or more |
| 3 | + * contributor license agreements. See the NOTICE file distributed with |
| 4 | + * this work for additional information regarding copyright ownership. |
| 5 | + * The ASF licenses this file to You under the Apache License, Version 2.0 |
| 6 | + * (the "License"); you may not use this file except in compliance with |
| 7 | + * the License. You may obtain a copy of the License at |
| 8 | + * |
| 9 | + * http://www.apache.org/licenses/LICENSE-2.0 |
| 10 | + * |
| 11 | + * Unless required by applicable law or agreed to in writing, software |
| 12 | + * distributed under the License is distributed on an "AS IS" BASIS, |
| 13 | + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
| 14 | + * See the License for the specific language governing permissions and |
| 15 | + * limitations under the License. |
| 16 | + */ |
| 17 | + |
| 18 | +package org.apache.fluss.client.table.scanner.batch; |
| 19 | + |
| 20 | +import org.apache.fluss.client.admin.Admin; |
| 21 | +import org.apache.fluss.client.metadata.KvSnapshotMetadata; |
| 22 | +import org.apache.fluss.client.metadata.KvSnapshots; |
| 23 | +import org.apache.fluss.client.table.scanner.RemoteFileDownloader; |
| 24 | +import org.apache.fluss.config.ConfigOptions; |
| 25 | +import org.apache.fluss.exception.FlussRuntimeException; |
| 26 | +import org.apache.fluss.metadata.KvFormat; |
| 27 | +import org.apache.fluss.metadata.TableBucket; |
| 28 | +import org.apache.fluss.metadata.TableInfo; |
| 29 | +import org.apache.fluss.row.InternalRow; |
| 30 | +import org.apache.fluss.types.RowType; |
| 31 | +import org.apache.fluss.utils.CloseableIterator; |
| 32 | + |
| 33 | +import javax.annotation.Nullable; |
| 34 | + |
| 35 | +import java.io.IOException; |
| 36 | +import java.time.Duration; |
| 37 | +import java.util.ArrayList; |
| 38 | +import java.util.List; |
| 39 | +import java.util.concurrent.CompletableFuture; |
| 40 | +import java.util.concurrent.TimeUnit; |
| 41 | + |
| 42 | +/** |
| 43 | + * BatchScanner that performs a full-table (or single-partition) snapshot by reading |
| 44 | + * the latest KV snapshots per bucket and merging results client-side. Designed for small tables. |
| 45 | + */ |
| 46 | +public class DefaultBatchScanner implements BatchScanner { |
| 47 | + |
| 48 | + // Track underlying per-bucket scanners to ensure resources are released on early close/errors |
| 49 | + private final List<BatchScanner> activeBucketScanners = new ArrayList<>(); |
| 50 | + |
| 51 | + private final TableInfo tableInfo; |
| 52 | + private final Admin admin; |
| 53 | + private final RemoteFileDownloader downloader; |
| 54 | + @Nullable private final int[] projectedFields; |
| 55 | + @Nullable private final String partitionName; // when non-null, restrict to this partition |
| 56 | + |
| 57 | + private final String scannerTmpDir; |
| 58 | + private final KvFormat kvFormat; |
| 59 | + private final RowType rowType; |
| 60 | + |
| 61 | + private volatile boolean endOfInput = false; |
| 62 | + |
| 63 | + public DefaultBatchScanner( |
| 64 | + TableInfo tableInfo, |
| 65 | + Admin admin, |
| 66 | + RemoteFileDownloader downloader, |
| 67 | + @Nullable int[] projectedFields, |
| 68 | + @Nullable String partitionName, |
| 69 | + String scannerTmpDir) { |
| 70 | + this.tableInfo = tableInfo; |
| 71 | + this.admin = admin; |
| 72 | + this.downloader = downloader; |
| 73 | + this.projectedFields = projectedFields; |
| 74 | + this.partitionName = partitionName; |
| 75 | + this.scannerTmpDir = scannerTmpDir; |
| 76 | + this.kvFormat = tableInfo.getTableConfig().getKvFormat(); |
| 77 | + this.rowType = tableInfo.getRowType(); |
| 78 | + } |
| 79 | + |
| 80 | + public DefaultBatchScanner( |
| 81 | + TableInfo tableInfo, |
| 82 | + Admin admin, |
| 83 | + RemoteFileDownloader downloader, |
| 84 | + @Nullable int[] projectedFields, |
| 85 | + String scannerTmpDir) { |
| 86 | + this(tableInfo, admin, downloader, projectedFields, null, scannerTmpDir); |
| 87 | + } |
| 88 | + |
| 89 | + @Override |
| 90 | + public BatchScanner snapshotAll() { |
| 91 | + // no-op: scanning whole table is the default in this implementation |
| 92 | + return this; |
| 93 | + } |
| 94 | + |
| 95 | + @Override |
| 96 | + public BatchScanner snapshotAllPartition(String partitionName) { |
| 97 | + if (!tableInfo.isPartitioned()) { |
| 98 | + throw new UnsupportedOperationException( |
| 99 | + "Partition filter is only supported for partitioned tables"); |
| 100 | + } |
| 101 | + return new DefaultBatchScanner( |
| 102 | + tableInfo, admin, downloader, projectedFields, partitionName, scannerTmpDir); |
| 103 | + } |
| 104 | + |
| 105 | + @Nullable |
| 106 | + @Override |
| 107 | + public CloseableIterator<InternalRow> pollBatch(Duration timeout) throws IOException { |
| 108 | + if (endOfInput) { |
| 109 | + return null; |
| 110 | + } |
| 111 | + |
| 112 | + if (tableInfo.isPartitioned() && partitionName == null) { |
| 113 | + throw new IllegalArgumentException( |
| 114 | + "Full-table snapshot on a partitioned table requires a PartitionFilter with a partition name. " |
| 115 | + + "Use snapshotAllPartition(partitionName)."); |
| 116 | + } |
| 117 | + if (!tableInfo.hasPrimaryKey()) { |
| 118 | + throw new UnsupportedOperationException( |
| 119 | + "Full scan BatchScanner is only supported for primary key tables."); |
| 120 | + } |
| 121 | + |
| 122 | + List<BatchScanner> bucketScanners = new ArrayList<>(); |
| 123 | + try { |
| 124 | + // 1) Fetch latest KV snapshots for scope |
| 125 | + final KvSnapshots kvSnapshots; |
| 126 | + if (partitionName == null) { |
| 127 | + kvSnapshots = admin.getLatestKvSnapshots(tableInfo.getTablePath()).get(30, TimeUnit.SECONDS); |
| 128 | + } else { |
| 129 | + kvSnapshots = admin.getLatestKvSnapshots(tableInfo.getTablePath(), partitionName).get(30, TimeUnit.SECONDS); |
| 130 | + } |
| 131 | + |
| 132 | + // 2) Build per-bucket snapshot scanners |
| 133 | + for (int bucketId : kvSnapshots.getBucketIds()) { |
| 134 | + long snapshotId; |
| 135 | + if (kvSnapshots.getSnapshotId(bucketId).isPresent()) { |
| 136 | + snapshotId = kvSnapshots.getSnapshotId(bucketId).getAsLong(); |
| 137 | + } else { |
| 138 | + // No snapshot for this bucket yet; skip |
| 139 | + continue; |
| 140 | + } |
| 141 | + TableBucket tableBucket = new TableBucket( |
| 142 | + kvSnapshots.getTableId(), kvSnapshots.getPartitionId(), bucketId); |
| 143 | + |
| 144 | + KvSnapshotMetadata meta = getSnapshotMetadata(tableBucket, snapshotId); |
| 145 | + KvSnapshotBatchScanner scanner = |
| 146 | + new KvSnapshotBatchScanner( |
| 147 | + rowType, |
| 148 | + tableBucket, |
| 149 | + meta.getSnapshotFiles(), |
| 150 | + projectedFields, |
| 151 | + scannerTmpDir, |
| 152 | + kvFormat, |
| 153 | + downloader); |
| 154 | + bucketScanners.add(scanner); |
| 155 | + } |
| 156 | + |
| 157 | + // make them visible to close() |
| 158 | + synchronized (activeBucketScanners) { |
| 159 | + activeBucketScanners.clear(); |
| 160 | + activeBucketScanners.addAll(bucketScanners); |
| 161 | + } |
| 162 | + |
| 163 | + // 3) Collect all rows from bucket scanners in one shot |
| 164 | + List<InternalRow> rows = BatchScanUtils.collectAllRows(bucketScanners); |
| 165 | + |
| 166 | + // collection closes the scanners; clear the active set |
| 167 | + synchronized (activeBucketScanners) { |
| 168 | + activeBucketScanners.clear(); |
| 169 | + } |
| 170 | + |
| 171 | + endOfInput = true; |
| 172 | + return CloseableIterator.wrap(rows.iterator()); |
| 173 | + } catch (Exception e) { |
| 174 | + // close any scanners that may have been created |
| 175 | + for (BatchScanner scanner : bucketScanners) { |
| 176 | + try { scanner.close(); } catch (Exception ignore) {} |
| 177 | + } |
| 178 | + synchronized (activeBucketScanners) { |
| 179 | + activeBucketScanners.clear(); |
| 180 | + } |
| 181 | + throw new IOException("Failed to execute full snapshot scan", e); |
| 182 | + } |
| 183 | + } |
| 184 | + |
| 185 | + private KvSnapshotMetadata getSnapshotMetadata(TableBucket tableBucket, long snapshotId) { |
| 186 | + try { |
| 187 | + CompletableFuture<KvSnapshotMetadata> f = admin.getKvSnapshotMetadata(tableBucket, snapshotId); |
| 188 | + return f.get(30, TimeUnit.SECONDS); |
| 189 | + } catch (Exception e) { |
| 190 | + throw new FlussRuntimeException("Failed to get snapshot metadata for " + tableBucket |
| 191 | + + ", snapshotId=" + snapshotId, e); |
| 192 | + } |
| 193 | + } |
| 194 | + |
| 195 | + @Override |
| 196 | + public void close() throws IOException { |
| 197 | + endOfInput = true; |
| 198 | + // Ensure any active bucket-level scanners are closed to free resources |
| 199 | + synchronized (activeBucketScanners) { |
| 200 | + for (BatchScanner scanner : activeBucketScanners) { |
| 201 | + try { |
| 202 | + scanner.close(); |
| 203 | + } catch (Exception ignore) { |
| 204 | + // swallow to ensure best-effort close |
| 205 | + } |
| 206 | + } |
| 207 | + activeBucketScanners.clear(); |
| 208 | + } |
| 209 | + } |
| 210 | +} |
0 commit comments