flyrain commented on code in PR #5740:
URL: https://github.com/apache/iceberg/pull/5740#discussion_r968965129


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkCatalog.java:
##########
@@ -605,19 +636,27 @@ private Pair<Table, Long> load(Identifier ident) {
       }
 
       // loading the namespace as a table worked, check the name to see if it 
is a valid selector
+      // or if the name points to the changelog
+

Review Comment:
   Can we remove the empty line?



##########
core/src/main/java/org/apache/iceberg/MetadataColumns.java:
##########
@@ -75,6 +75,21 @@ private MetadataColumns() {}
   public static final String DELETE_FILE_ROW_FIELD_NAME = "row";
   public static final int DELETE_FILE_ROW_FIELD_ID = Integer.MAX_VALUE - 103;
   public static final String DELETE_FILE_ROW_DOC = "Deleted row values";
+  public static final NestedField CHANGELOG_OPERATION =
+      NestedField.required(
+          Integer.MAX_VALUE - 104,
+          "changelog_operation",
+          Types.StringType.get(),
+          "Record type in changelog");
+  public static final NestedField CHANGE_ORDINAL =
+      NestedField.optional(
+          Integer.MAX_VALUE - 105, "change_ordinal", Types.IntegerType.get(), 
"Change ordinal");
+  public static final NestedField COMMIT_SNAPSHOT_ID =
+      NestedField.optional(
+          Integer.MAX_VALUE - 106,
+          "commit_snapshot_id",

Review Comment:
   To `_commit_snapshot_id`?



##########
core/src/main/java/org/apache/iceberg/MetadataColumns.java:
##########
@@ -75,6 +75,21 @@ private MetadataColumns() {}
   public static final String DELETE_FILE_ROW_FIELD_NAME = "row";
   public static final int DELETE_FILE_ROW_FIELD_ID = Integer.MAX_VALUE - 103;
   public static final String DELETE_FILE_ROW_DOC = "Deleted row values";
+  public static final NestedField CHANGELOG_OPERATION =

Review Comment:
   I'm OK to keep them here. I'd consider change log columns as metadata 
columns as well.



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkChangelogBatch.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.iceberg.spark.source;
+
+import java.util.List;
+import org.apache.iceberg.ChangelogScanTask;
+import org.apache.iceberg.ScanTaskGroup;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SchemaParser;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.spark.SparkReadConf;
+import org.apache.iceberg.util.Tasks;
+import org.apache.iceberg.util.ThreadPools;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.broadcast.Broadcast;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.connector.read.Batch;
+import org.apache.spark.sql.connector.read.InputPartition;
+import org.apache.spark.sql.connector.read.PartitionReader;
+import org.apache.spark.sql.connector.read.PartitionReaderFactory;
+
+public class SparkChangelogBatch implements Batch {
+
+  private final JavaSparkContext sparkContext;
+  private final Table table;
+  private final Schema expectedSchema;
+  private final boolean caseSensitive;
+  private final boolean localityEnabled;
+  private final SparkChangelogScan scan;
+
+  SparkChangelogBatch(
+      SparkSession spark,
+      Table table,
+      SparkReadConf readConf,
+      Schema expectedSchema,
+      SparkChangelogScan scan) {
+    this.sparkContext = 
JavaSparkContext.fromSparkContext(spark.sparkContext());
+    this.table = table;
+    this.expectedSchema = expectedSchema;
+    this.caseSensitive = readConf.caseSensitive();
+    this.localityEnabled = readConf.localityEnabled();
+    this.scan = scan;
+  }
+
+  @Override
+  public InputPartition[] planInputPartitions() {
+    Table serializableTable = SerializableTableWithSize.copyOf(table);
+    Broadcast<Table> tableBroadcast = 
sparkContext.broadcast(serializableTable);
+    String expectedSchemaString = SchemaParser.toJson(expectedSchema);
+    List<ScanTaskGroup<ChangelogScanTask>> taskGroups = scan.taskGroups();
+
+    InputPartition[] partitions = new InputPartition[taskGroups.size()];
+
+    Tasks.range(partitions.length)
+        .stopOnFailure()
+        .executeWith(localityEnabled ? ThreadPools.getWorkerPool() : null)
+        .run(
+            index ->
+                partitions[index] =
+                    new SparkInputPartition(
+                        taskGroups.get(index),
+                        tableBroadcast,
+                        expectedSchemaString,
+                        caseSensitive,
+                        localityEnabled));
+
+    return partitions;
+  }
+
+  @Override
+  public PartitionReaderFactory createReaderFactory() {
+    return new ReaderFactory();
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+
+    SparkChangelogBatch that = (SparkChangelogBatch) o;
+    return scan.equals(that.scan);
+  }
+
+  @Override
+  public int hashCode() {
+    return scan.hashCode();
+  }
+
+  private static class ReaderFactory implements PartitionReaderFactory {
+    @Override
+    public PartitionReader<InternalRow> createReader(InputPartition partition) 
{
+      if (partition instanceof SparkInputPartition) {
+        return new RowReader((SparkInputPartition) partition);
+      } else {
+        throw new UnsupportedOperationException("Incorrect input partition 
type: " + partition);

Review Comment:
   I think we normally use Preconditions like this
   ```
   Preconditions.checkArgument(partition instanceof SparkInputPartition, 
message);
   ```



##########
core/src/main/java/org/apache/iceberg/MetadataColumns.java:
##########
@@ -75,6 +75,21 @@ private MetadataColumns() {}
   public static final String DELETE_FILE_ROW_FIELD_NAME = "row";
   public static final int DELETE_FILE_ROW_FIELD_ID = Integer.MAX_VALUE - 103;
   public static final String DELETE_FILE_ROW_DOC = "Deleted row values";
+  public static final NestedField CHANGELOG_OPERATION =
+      NestedField.required(
+          Integer.MAX_VALUE - 104,
+          "changelog_operation",

Review Comment:
   To `_change_type` or `_changelog_type`?



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkInputPartition.java:
##########
@@ -0,0 +1,83 @@
+/*
+ * 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.iceberg.spark.source;
+
+import java.io.Serializable;
+import org.apache.iceberg.ScanTask;
+import org.apache.iceberg.ScanTaskGroup;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SchemaParser;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.hadoop.HadoopInputFile;
+import org.apache.iceberg.hadoop.Util;
+import org.apache.spark.broadcast.Broadcast;
+import org.apache.spark.sql.connector.read.InputPartition;
+
+class SparkInputPartition implements InputPartition, Serializable {

Review Comment:
   +1 for this refactor.



##########
core/src/main/java/org/apache/iceberg/MetadataColumns.java:
##########
@@ -75,6 +75,21 @@ private MetadataColumns() {}
   public static final String DELETE_FILE_ROW_FIELD_NAME = "row";
   public static final int DELETE_FILE_ROW_FIELD_ID = Integer.MAX_VALUE - 103;
   public static final String DELETE_FILE_ROW_DOC = "Deleted row values";
+  public static final NestedField CHANGELOG_OPERATION =
+      NestedField.required(
+          Integer.MAX_VALUE - 104,
+          "changelog_operation",
+          Types.StringType.get(),
+          "Record type in changelog");
+  public static final NestedField CHANGE_ORDINAL =
+      NestedField.optional(
+          Integer.MAX_VALUE - 105, "change_ordinal", Types.IntegerType.get(), 
"Change ordinal");

Review Comment:
   To `_change_ordinal`? I'm not a native speaker, but does `ordinal` imply 
that the value would be `1st, 2nd, 3rd, ...`. We actually use `0, 1, 2, ...`. A 
name like `_change_seq_num` is more suitable?



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkCatalog.java:
##########
@@ -662,15 +709,22 @@ private Pair<Table, Long> 
loadFromPathIdentifier(PathIdentifier ident) {
         "Cannot specify both snapshot-id and as-of-timestamp: %s",
         ident.location());
 
-    Table table =
+    Preconditions.checkArgument(
+        !isChangelog || (snapshotId == null && asOfTimestamp == null),
+        "Cannot specify snapshot-id and as-of-timestamp for changelogs");
+
+    org.apache.iceberg.Table table =
         tables.load(parsed.first() + (metadataTableName != null ? "#" + 
metadataTableName : ""));
 
-    if (snapshotId != null) {
-      return Pair.of(table, snapshotId);
+    if (isChangelog) {
+      return new SparkChangelogTable(table, !cacheEnabled);
+    } else if (snapshotId != null) {
+      return new SparkTable(table, snapshotId, !cacheEnabled);
     } else if (asOfTimestamp != null) {
-      return Pair.of(table, SnapshotUtil.snapshotIdAsOfTime(table, 
asOfTimestamp));
+      return new SparkTable(
+          table, SnapshotUtil.snapshotIdAsOfTime(table, asOfTimestamp), 
!cacheEnabled);
     } else {
-      return Pair.of(table, null);
+      return new SparkTable(table, null, !cacheEnabled);

Review Comment:
   A refactor suggestion: we may use a builder here.



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

Reply via email to