Jiayi-Liao commented on a change in pull request #1083:
URL: https://github.com/apache/iceberg/pull/1083#discussion_r433820767



##########
File path: 
spark/src/main/java/org/apache/iceberg/actions/RewriteDataFilesAction.java
##########
@@ -0,0 +1,186 @@
+/*
+ * 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.actions;
+
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.RewriteFiles;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.encryption.EncryptionManager;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.hadoop.HadoopFileIO;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.spark.source.RowDataRewriter;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.TableScanUtil;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.broadcast.Broadcast;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.util.SerializableConfiguration;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class RewriteDataFilesAction
+    extends BaseSnapshotUpdateAction<RewriteDataFilesAction, 
RewriteDataFilesActionResult> {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(RewriteDataFilesAction.class);
+
+  private final SparkSession spark;
+  private final JavaSparkContext sparkContext;
+  private final Table table;
+  private final FileIO fileIO;
+  private final EncryptionManager encryptionManager;
+  private final long targetDataFileSizeBytes;
+
+  private PartitionSpec spec = null;
+  private Expression filter;
+
+  RewriteDataFilesAction(SparkSession spark, Table table) {
+    this.spark = spark;
+    this.sparkContext = new JavaSparkContext(spark.sparkContext());
+    this.table = table;
+    this.spec = table.spec();
+    this.filter = Expressions.alwaysTrue();
+    this.targetDataFileSizeBytes = PropertyUtil.propertyAsLong(
+        table.properties(),
+        TableProperties.WRITE_TARGET_FILE_SIZE_BYTES,
+        TableProperties.WRITE_TARGET_FILE_SIZE_BYTES_DEFAULT);
+
+    if (table.io() instanceof HadoopFileIO) {
+      // we need to use Spark's SerializableConfiguration to avoid issues with 
Kryo serialization
+      SerializableConfiguration conf = new 
SerializableConfiguration(((HadoopFileIO) table.io()).conf());
+      this.fileIO = new HadoopFileIO(conf::value);
+    } else {
+      this.fileIO = table.io();
+    }
+    this.encryptionManager = table.encryption();
+  }
+
+  @Override
+  protected RewriteDataFilesAction self() {
+    return this;
+  }
+
+  @Override
+  protected Table table() {
+    return table;
+  }
+
+  public RewriteDataFilesAction specId(int specId) {
+    Preconditions.checkArgument(table.specs().containsKey(specId), "Invalid 
spec id %d", specId);
+    this.spec = table.specs().get(specId);
+    return this;
+  }
+
+  public RewriteDataFilesAction filter(Expression expr) {
+    this.filter = Expressions.and(filter, expr);
+    return this;
+  }
+
+  @Override
+  public RewriteDataFilesActionResult execute() {
+    CloseableIterable<FileScanTask> fileScanTasks = table.newScan()
+        .filter(filter)
+        .planFiles();
+    List<DataFile> currentDataFiles =
+        Lists.newArrayList(CloseableIterable.transform(fileScanTasks, 
FileScanTask::file));
+    if (currentDataFiles.isEmpty()) {
+      return RewriteDataFilesActionResult.empty();
+    }
+
+    List<CloseableIterable<FileScanTask>> groupedTasks = 
groupTasksByPartition(fileScanTasks);
+
+    long splitSize = PropertyUtil.propertyAsLong(
+        table.properties(),
+        TableProperties.SPLIT_SIZE,
+        TableProperties.SPLIT_SIZE_DEFAULT);
+    int lookbak = PropertyUtil.propertyAsInt(

Review comment:
       typo

##########
File path: 
spark/src/main/java/org/apache/iceberg/actions/RewriteDataFilesAction.java
##########
@@ -0,0 +1,186 @@
+/*
+ * 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.actions;
+
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.RewriteFiles;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.encryption.EncryptionManager;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.hadoop.HadoopFileIO;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.spark.source.RowDataRewriter;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.TableScanUtil;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.broadcast.Broadcast;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.util.SerializableConfiguration;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class RewriteDataFilesAction
+    extends BaseSnapshotUpdateAction<RewriteDataFilesAction, 
RewriteDataFilesActionResult> {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(RewriteDataFilesAction.class);
+
+  private final SparkSession spark;
+  private final JavaSparkContext sparkContext;
+  private final Table table;
+  private final FileIO fileIO;
+  private final EncryptionManager encryptionManager;
+  private final long targetDataFileSizeBytes;
+
+  private PartitionSpec spec = null;
+  private Expression filter;
+
+  RewriteDataFilesAction(SparkSession spark, Table table) {
+    this.spark = spark;
+    this.sparkContext = new JavaSparkContext(spark.sparkContext());
+    this.table = table;
+    this.spec = table.spec();
+    this.filter = Expressions.alwaysTrue();
+    this.targetDataFileSizeBytes = PropertyUtil.propertyAsLong(
+        table.properties(),
+        TableProperties.WRITE_TARGET_FILE_SIZE_BYTES,
+        TableProperties.WRITE_TARGET_FILE_SIZE_BYTES_DEFAULT);
+
+    if (table.io() instanceof HadoopFileIO) {
+      // we need to use Spark's SerializableConfiguration to avoid issues with 
Kryo serialization
+      SerializableConfiguration conf = new 
SerializableConfiguration(((HadoopFileIO) table.io()).conf());
+      this.fileIO = new HadoopFileIO(conf::value);
+    } else {
+      this.fileIO = table.io();
+    }
+    this.encryptionManager = table.encryption();
+  }
+
+  @Override
+  protected RewriteDataFilesAction self() {
+    return this;
+  }
+
+  @Override
+  protected Table table() {
+    return table;
+  }
+
+  public RewriteDataFilesAction specId(int specId) {
+    Preconditions.checkArgument(table.specs().containsKey(specId), "Invalid 
spec id %d", specId);
+    this.spec = table.specs().get(specId);
+    return this;
+  }
+
+  public RewriteDataFilesAction filter(Expression expr) {
+    this.filter = Expressions.and(filter, expr);
+    return this;
+  }
+
+  @Override
+  public RewriteDataFilesActionResult execute() {
+    CloseableIterable<FileScanTask> fileScanTasks = table.newScan()
+        .filter(filter)
+        .planFiles();
+    List<DataFile> currentDataFiles =
+        Lists.newArrayList(CloseableIterable.transform(fileScanTasks, 
FileScanTask::file));
+    if (currentDataFiles.isEmpty()) {
+      return RewriteDataFilesActionResult.empty();
+    }
+
+    List<CloseableIterable<FileScanTask>> groupedTasks = 
groupTasksByPartition(fileScanTasks);
+
+    long splitSize = PropertyUtil.propertyAsLong(
+        table.properties(),
+        TableProperties.SPLIT_SIZE,
+        TableProperties.SPLIT_SIZE_DEFAULT);
+    int lookbak = PropertyUtil.propertyAsInt(
+        table.properties(),
+        TableProperties.SPLIT_LOOKBACK,
+        TableProperties.SPLIT_LOOKBACK_DEFAULT);
+    long openFileCost = PropertyUtil.propertyAsLong(
+        table.properties(),
+        TableProperties.SPLIT_OPEN_FILE_COST,
+        TableProperties.SPLIT_LOOKBACK_DEFAULT);

Review comment:
       shoudn't be SPLIT_OPEN_FILE_COST_DEFAULT?

##########
File path: 
spark/src/main/java/org/apache/iceberg/spark/source/RowDataRewriter.java
##########
@@ -0,0 +1,138 @@
+/*
+ * 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 java.util.Arrays;
+import java.util.List;
+import java.util.Locale;
+import java.util.Optional;
+import java.util.stream.Collectors;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.SchemaParser;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.encryption.EncryptionManager;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.spark.TaskContext;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.broadcast.Broadcast;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.execution.streaming.MicroBatchExecution;
+import org.apache.spark.sql.sources.v2.writer.DataWriter;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class RowDataRewriter implements Serializable {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(RowDataRewriter.class);
+
+  private final transient JavaSparkContext sparkContext;
+  private final Broadcast<FileIO> fileIO;
+  private final Broadcast<EncryptionManager> encryptionManager;
+  private final String tableSchema;
+  private final Writer.WriterFactory writerFactory;
+  private final boolean caseSensitive;
+
+  public RowDataRewriter(Table table, JavaSparkContext sparkContext, 
PartitionSpec spec, boolean caseSensitive,
+                         Broadcast<FileIO> fileIO, 
Broadcast<EncryptionManager> encryptionManager,
+                         long targetDataFileSizeInBytes) {
+    this.sparkContext = sparkContext;
+    this.fileIO = fileIO;
+    this.encryptionManager = encryptionManager;
+
+    this.caseSensitive = caseSensitive;
+    this.tableSchema = SchemaParser.toJson(table.schema());
+
+    String formatString = table.properties().getOrDefault(
+        TableProperties.DEFAULT_FILE_FORMAT, 
TableProperties.DEFAULT_FILE_FORMAT_DEFAULT);
+    FileFormat fileFormat = 
FileFormat.valueOf(formatString.toUpperCase(Locale.ENGLISH));
+    this.writerFactory = new Writer.WriterFactory(spec, fileFormat, 
table.locationProvider(), table.properties(),
+        fileIO, encryptionManager, targetDataFileSizeInBytes, table.schema(), 
SparkSchemaUtil.convert(table.schema()));
+  }
+
+  public List<DataFile> 
rewriteDataForTasks(CloseableIterable<CombinedScanTask> tasks) {
+    List<CombinedScanTask> taskList = Lists.newArrayList(tasks);
+    int parallelism = taskList.size();
+
+    JavaRDD<CombinedScanTask> taskRDD = sparkContext.parallelize(taskList, 
parallelism);
+    JavaRDD<Writer.TaskCommit> taskCommitRDD = taskRDD.map(task -> 
rewriteDataForTask(task));
+
+    return taskCommitRDD.collect().stream()
+        .flatMap(taskCommit -> Arrays.stream(taskCommit.files()))
+        .collect(Collectors.toList());
+  }
+
+  private Writer.TaskCommit rewriteDataForTask(CombinedScanTask task) throws 
Exception {
+    TaskContext context = TaskContext.get();
+
+    RowDataReader dataReader = new RowDataReader(task, 
SchemaParser.fromJson(tableSchema),
+        SchemaParser.fromJson(tableSchema), fileIO.value(), 
encryptionManager.value(), caseSensitive);
+
+    int partitionId = context.partitionId();
+    long taskId = context.taskAttemptId();
+    long epochId = Long.parseLong(
+        
Optional.ofNullable(context.getLocalProperty(MicroBatchExecution.BATCH_ID_KEY())).orElse("0"));
+    DataWriter<InternalRow> dataWriter = 
writerFactory.createDataWriter(partitionId, taskId, epochId);
+
+    Throwable originalThrowable = null;
+    try {
+      while (dataReader.next()) {
+        InternalRow row = dataReader.get();
+        dataWriter.write(row);
+      }
+
+      dataReader.close();
+      return (Writer.TaskCommit) dataWriter.commit();
+
+    } catch (Throwable t) {
+      originalThrowable = t;
+      try {
+        LOG.error("Aborting task", originalThrowable);
+        context.markTaskFailed(originalThrowable);
+
+        LOG.error("Aborting commit for partition {} (task {}, attempt {}, 
stage {}.{})",
+            partitionId, taskId, context.attemptNumber(), context.stageId(), 
context.stageAttemptNumber());
+        dataReader.close();
+        dataWriter.abort();
+        LOG.error("Aborted commit for partition {} (task {}, attempt {}, stage 
{}.{})",

Review comment:
       Same as Line 118?




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

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