luoyuxia commented on code in PR #1684:
URL: https://github.com/apache/fluss/pull/1684#discussion_r2347761625


##########
fluss-lake/fluss-lake-iceberg/src/main/java/org/apache/fluss/lake/iceberg/source/IcebergSplitPlanner.java:
##########
@@ -0,0 +1,94 @@
+/*
+ * 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.fluss.lake.iceberg.source;
+
+import org.apache.fluss.config.Configuration;
+import org.apache.fluss.lake.iceberg.utils.IcebergCatalogUtils;
+import org.apache.fluss.lake.source.Planner;
+import org.apache.fluss.metadata.TablePath;
+
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.catalog.Catalog;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.transforms.TransformUtils;
+import org.apache.iceberg.types.Types;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+import static org.apache.fluss.lake.iceberg.utils.IcebergConversions.toIceberg;
+
+/** Iceberg split planner. */
+public class IcebergSplitPlanner implements Planner<IcebergSplit> {
+
+    private final Configuration icebergConfig;
+    private final TablePath tablePath;
+    private final long snapshotId;
+
+    public IcebergSplitPlanner(Configuration icebergConfig, TablePath 
tablePath, long snapshotId) {
+        this.icebergConfig = icebergConfig;
+        this.tablePath = tablePath;
+        this.snapshotId = snapshotId;
+    }
+
+    @Override
+    public List<IcebergSplit> plan() throws IOException {
+        List<IcebergSplit> splits = new ArrayList<>();
+        Catalog catalog = 
IcebergCatalogUtils.createIcebergCatalog(icebergConfig);
+        Table table = catalog.loadTable(toIceberg(tablePath));
+        Function<FileScanTask, List<String>> partitionExtract = 
createPartitionExtractor(table);
+        try (CloseableIterable<FileScanTask> tasks =
+                table.newScan()
+                        .useSnapshot(snapshotId)
+                        .includeColumnStats()
+                        .ignoreResiduals()
+                        .planFiles()) {
+            tasks.forEach(
+                    task -> splits.add(new IcebergSplit(task, -1, 
partitionExtract.apply(task))));
+        }
+        return splits;
+    }
+
+    private Function<FileScanTask, List<String>> 
createPartitionExtractor(Table table) {
+        PartitionSpec partitionSpec = table.spec();
+        List<PartitionField> partitionFields = partitionSpec.fields();
+        Types.StructType partitionType = partitionSpec.partitionType();
+
+        List<Integer> nonBucketFieldIndices =
+                partitionFields.stream()
+                        .filter(field -> 
!TransformUtils.isBucketTransform(field.transform()))

Review Comment:
   We don't always use `bucket` partition for log table.
   See code 
   
https://github.com/apache/fluss/blob/d6a01f58ce9f3c24ca89556d97de9fcd76e9cafb/fluss-lake/fluss-lake-iceberg/src/main/java/org/apache/fluss/lake/iceberg/IcebergLakeCatalog.java#L225
   



##########
fluss-lake/fluss-lake-iceberg/src/main/java/org/apache/fluss/lake/iceberg/source/IcebergRecordReader.java:
##########
@@ -0,0 +1,137 @@
+/*
+ * 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.fluss.lake.iceberg.source;
+
+import org.apache.fluss.lake.source.RecordReader;
+import org.apache.fluss.record.ChangeType;
+import org.apache.fluss.record.GenericRecord;
+import org.apache.fluss.record.LogRecord;
+import org.apache.fluss.row.ProjectedRow;
+import org.apache.fluss.utils.CloseableIterator;
+
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableScan;
+import org.apache.iceberg.data.IcebergGenericReader;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.types.Types;
+
+import javax.annotation.Nullable;
+
+import java.io.IOException;
+import java.time.OffsetDateTime;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.stream.IntStream;
+
+import static org.apache.fluss.metadata.TableDescriptor.OFFSET_COLUMN_NAME;
+import static org.apache.fluss.metadata.TableDescriptor.TIMESTAMP_COLUMN_NAME;
+
+/** Iceberg record reader. */
+public class IcebergRecordReader implements RecordReader {
+    protected IcebergRecordAsFlussRecordIterator iterator;
+    protected @Nullable int[][] project;
+    protected Types.StructType struct;
+
+    public IcebergRecordReader(FileScanTask fileScanTask, Table table, 
@Nullable int[][] project) {
+        TableScan tableScan = table.newScan();
+        if (project != null) {
+            tableScan = applyProject(tableScan, project);
+        }
+        IcebergGenericReader reader = new IcebergGenericReader(tableScan, 
true);
+        struct = tableScan.schema().asStruct();
+        this.iterator = new 
IcebergRecordAsFlussRecordIterator(reader.open(fileScanTask), struct);
+    }
+
+    @Override
+    public CloseableIterator<LogRecord> read() throws IOException {
+        return iterator;
+    }
+
+    private TableScan applyProject(TableScan tableScan, int[][] projects) {
+        Types.StructType structType = tableScan.schema().asStruct();
+        List<Types.NestedField> cols = new ArrayList<>(projects.length + 2);
+
+        for (int[] project : projects) {
+            // iceberg field index starts from 1
+            cols.add(structType.field(project[0] + 1));

Review Comment:
   ```suggestion
               cols.add(structType.fields().get(project[0]));
   ```
   So that no need to add comment `iceberg field index starts from 1`



##########
fluss-lake/fluss-lake-iceberg/src/main/java/org/apache/fluss/lake/iceberg/source/IcebergSplitPlanner.java:
##########
@@ -0,0 +1,94 @@
+/*
+ * 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.fluss.lake.iceberg.source;
+
+import org.apache.fluss.config.Configuration;
+import org.apache.fluss.lake.iceberg.utils.IcebergCatalogUtils;
+import org.apache.fluss.lake.source.Planner;
+import org.apache.fluss.metadata.TablePath;
+
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.catalog.Catalog;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.transforms.TransformUtils;
+import org.apache.iceberg.types.Types;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+import static org.apache.fluss.lake.iceberg.utils.IcebergConversions.toIceberg;
+
+/** Iceberg split planner. */
+public class IcebergSplitPlanner implements Planner<IcebergSplit> {
+
+    private final Configuration icebergConfig;
+    private final TablePath tablePath;
+    private final long snapshotId;
+
+    public IcebergSplitPlanner(Configuration icebergConfig, TablePath 
tablePath, long snapshotId) {
+        this.icebergConfig = icebergConfig;
+        this.tablePath = tablePath;
+        this.snapshotId = snapshotId;
+    }
+
+    @Override
+    public List<IcebergSplit> plan() throws IOException {
+        List<IcebergSplit> splits = new ArrayList<>();
+        Catalog catalog = 
IcebergCatalogUtils.createIcebergCatalog(icebergConfig);
+        Table table = catalog.loadTable(toIceberg(tablePath));
+        Function<FileScanTask, List<String>> partitionExtract = 
createPartitionExtractor(table);
+        try (CloseableIterable<FileScanTask> tasks =
+                table.newScan()
+                        .useSnapshot(snapshotId)
+                        .includeColumnStats()
+                        .ignoreResiduals()
+                        .planFiles()) {
+            tasks.forEach(
+                    task -> splits.add(new IcebergSplit(task, -1, 
partitionExtract.apply(task))));

Review Comment:
   For log table:
   For log table, we still can't just set `-1` directly. 
   If the log table is defined without bucket key, it's bucket unwared. we can 
set it to `-1`, that's fine.
   But If the log table is defined with bucket key,it's bucket wared. we can't 
set it to `-1`.



-- 
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]

Reply via email to