zhztheplayer commented on code in PR #9397: URL: https://github.com/apache/incubator-gluten/pull/9397#discussion_r2068390354
########## backends-velox/src-iceberg/main/scala/org/apache/gluten/execution/OffloadIcebergWrite.scala: ########## @@ -0,0 +1,56 @@ +/* + * 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.gluten.execution + +import org.apache.gluten.extension.columnar.enumerated.RasOffload +import org.apache.gluten.extension.columnar.heuristic.HeuristicTransform +import org.apache.gluten.extension.columnar.offload.OffloadSingleNode +import org.apache.gluten.extension.columnar.validator.Validators +import org.apache.gluten.extension.injector.Injector +import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.execution.datasources.v2.AppendDataExec + +case class OffloadIcebergWrite() extends OffloadSingleNode { + override def offload(plan: SparkPlan): SparkPlan = plan match { + case a: AppendDataExec => + VeloxIcebergAppendDataExec(a) Review Comment: What happens if the `AppendDataExec` is for hudi or delta? ########## gluten-substrait/src/main/scala/org/apache/gluten/execution/ColumnarAppendDataExec.scala: ########## @@ -0,0 +1,119 @@ +/* + * 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.gluten.execution + +import org.apache.gluten.connector.write.ColumnarBatchDataWriterFactory +import org.apache.gluten.extension.columnar.transition.Convention +import org.apache.gluten.extension.columnar.transition.Convention.RowType + +import org.apache.spark.{SparkException, TaskContext} +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.connector.write.{BatchWrite, Write, WriterCommitMessage} +import org.apache.spark.sql.datasources.v2.{DataWritingColumnarBatchSparkTask, DataWritingColumnarBatchSparkTaskResult, StreamWriterCommitProgressUtil, WritingColumnarBatchSparkTask} +import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.execution.datasources.v2._ +import org.apache.spark.sql.execution.metric.SQLMetric +import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.vectorized.ColumnarBatch +import org.apache.spark.util.LongAccumulator + +abstract class ColumnarAppendDataExec(query: SparkPlan, refreshCache: () => Unit, write: Write) + extends V2ExistingTableWriteExec + with ValidatablePlan { + + def writingTaskBatch: WritingColumnarBatchSparkTask[_] = DataWritingColumnarBatchSparkTask + + override def doExecute(): RDD[InternalRow] = { + result + sparkContext.parallelize(Nil, 1) + } + + def createFactory(schema: StructType): ColumnarBatchDataWriterFactory + + protected def writeColumnarBatchWithV2(batchWrite: BatchWrite): Unit = { + val rdd: RDD[ColumnarBatch] = { + val tempRdd = query.executeColumnar() + // SPARK-23271 If we are attempting to write a zero partition rdd, create a dummy single + // partition rdd to make sure we at least set up one write task to write the metadata. + if (tempRdd.partitions.length == 0) { + sparkContext.parallelize(Array.empty[ColumnarBatch], 1) + } else { + tempRdd + } + } + // introduce a local var to avoid serializing the whole class + val task = writingTaskBatch + val messages = new Array[WriterCommitMessage](rdd.partitions.length) + val totalNumRowsAccumulator = new LongAccumulator() + + logInfo( + s"Start processing data source write support: $batchWrite. " + + s"The input RDD has ${messages.length} partitions.") + + // Avoid object not serializable issue. + val writeMetrics: Map[String, SQLMetric] = customMetrics + val factory = createFactory(query.schema) + try { + sparkContext.runJob( + rdd, + (context: TaskContext, iter: Iterator[ColumnarBatch]) => + task.run(factory, context, iter, writeMetrics), + rdd.partitions.indices, + (index, result: DataWritingColumnarBatchSparkTaskResult) => { + val commitMessage = result.writerCommitMessage + messages(index) = commitMessage + totalNumRowsAccumulator.add(result.numRows) + batchWrite.onDataWriterCommit(commitMessage) + } + ) + + logInfo(s"Data source write support $batchWrite is committing.") + batchWrite.commit(messages) + logInfo(s"Data source write support $batchWrite committed.") + commitProgress = Some( + StreamWriterCommitProgressUtil.getStreamWriterCommitProgress(totalNumRowsAccumulator.value)) + } catch { + case cause: Throwable => + logError(s"Data source write support $batchWrite is aborting.") + try { + batchWrite.abort(messages) + } catch { + case t: Throwable => Review Comment: same ########## gluten-substrait/src/main/scala/org/apache/gluten/execution/ColumnarAppendDataExec.scala: ########## @@ -0,0 +1,119 @@ +/* + * 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.gluten.execution + +import org.apache.gluten.connector.write.ColumnarBatchDataWriterFactory +import org.apache.gluten.extension.columnar.transition.Convention +import org.apache.gluten.extension.columnar.transition.Convention.RowType + +import org.apache.spark.{SparkException, TaskContext} +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.connector.write.{BatchWrite, Write, WriterCommitMessage} +import org.apache.spark.sql.datasources.v2.{DataWritingColumnarBatchSparkTask, DataWritingColumnarBatchSparkTaskResult, StreamWriterCommitProgressUtil, WritingColumnarBatchSparkTask} +import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.execution.datasources.v2._ +import org.apache.spark.sql.execution.metric.SQLMetric +import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.vectorized.ColumnarBatch +import org.apache.spark.util.LongAccumulator + +abstract class ColumnarAppendDataExec(query: SparkPlan, refreshCache: () => Unit, write: Write) + extends V2ExistingTableWriteExec + with ValidatablePlan { + + def writingTaskBatch: WritingColumnarBatchSparkTask[_] = DataWritingColumnarBatchSparkTask + + override def doExecute(): RDD[InternalRow] = { + result + sparkContext.parallelize(Nil, 1) + } + + def createFactory(schema: StructType): ColumnarBatchDataWriterFactory + + protected def writeColumnarBatchWithV2(batchWrite: BatchWrite): Unit = { + val rdd: RDD[ColumnarBatch] = { + val tempRdd = query.executeColumnar() + // SPARK-23271 If we are attempting to write a zero partition rdd, create a dummy single + // partition rdd to make sure we at least set up one write task to write the metadata. + if (tempRdd.partitions.length == 0) { + sparkContext.parallelize(Array.empty[ColumnarBatch], 1) + } else { + tempRdd + } + } + // introduce a local var to avoid serializing the whole class + val task = writingTaskBatch + val messages = new Array[WriterCommitMessage](rdd.partitions.length) + val totalNumRowsAccumulator = new LongAccumulator() + + logInfo( + s"Start processing data source write support: $batchWrite. " + + s"The input RDD has ${messages.length} partitions.") + + // Avoid object not serializable issue. + val writeMetrics: Map[String, SQLMetric] = customMetrics + val factory = createFactory(query.schema) + try { + sparkContext.runJob( + rdd, + (context: TaskContext, iter: Iterator[ColumnarBatch]) => + task.run(factory, context, iter, writeMetrics), + rdd.partitions.indices, + (index, result: DataWritingColumnarBatchSparkTaskResult) => { + val commitMessage = result.writerCommitMessage + messages(index) = commitMessage + totalNumRowsAccumulator.add(result.numRows) + batchWrite.onDataWriterCommit(commitMessage) + } + ) + + logInfo(s"Data source write support $batchWrite is committing.") + batchWrite.commit(messages) + logInfo(s"Data source write support $batchWrite committed.") + commitProgress = Some( + StreamWriterCommitProgressUtil.getStreamWriterCommitProgress(totalNumRowsAccumulator.value)) + } catch { + case cause: Throwable => Review Comment: Could cache `Exception` if enough ########## gluten-substrait/src/main/scala/org/apache/spark/sql/datasources/v2/AppendColumnarBatchDataExec.scala: ########## @@ -0,0 +1,103 @@ +/* + * 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.spark.sql.datasources.v2 + +import org.apache.gluten.connector.write.ColumnarBatchDataWriterFactory + +import org.apache.spark.TaskContext +import org.apache.spark.internal.Logging +import org.apache.spark.sql.connector.write._ +import org.apache.spark.sql.execution.datasources.v2.StreamWriterCommitProgress +import org.apache.spark.sql.execution.metric.{CustomMetrics, SQLMetric} +import org.apache.spark.sql.vectorized.ColumnarBatch +import org.apache.spark.util.Utils + +case class DataWritingColumnarBatchSparkTaskResult( + numRows: Long, + writerCommitMessage: WriterCommitMessage) + +trait WritingColumnarBatchSparkTask[W <: DataWriter[ColumnarBatch]] + extends Logging + with Serializable { + + protected def write(writer: W, row: ColumnarBatch): Unit + + def run( + factor: ColumnarBatchDataWriterFactory, Review Comment: nit: typo ########## backends-velox/src-iceberg/main/scala/org/apache/gluten/connector/write/IcebergDataWriteFactory.scala: ########## @@ -0,0 +1,63 @@ +/* + * 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.gluten.connector.write + +import org.apache.arrow.c.ArrowSchema +import org.apache.gluten.backendsapi.BackendsApiManager +import org.apache.gluten.execution.IcebergWriteJniWrapper +import org.apache.gluten.memory.arrow.alloc.ArrowBufferAllocators +import org.apache.gluten.runtime.Runtimes +import org.apache.gluten.utils.ArrowAbiUtil +import org.apache.spark.sql.connector.write.DataWriter +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.utils.SparkArrowUtil +import org.apache.spark.sql.vectorized.ColumnarBatch + +case class IcebergDataWriteFactory(schema: StructType, + format: Integer, directory: String, codec: String) + extends ColumnarBatchDataWriterFactory { + + /** + * Returns a data writer to do the actual writing work. Note that, Spark will reuse the same data + * object instance when sending data to the data writer, for better performance. Data writers + * are responsible for defensive copies if necessary, e.g. copy the data before buffer it in a + * list. + * <p> + * If this method fails (by throwing an exception), the corresponding Spark write task would fail + * and get retried until hitting the maximum retry times. + * + */ + override def createWriter(): DataWriter[ColumnarBatch] = { + val(writerHandle, jniWrapper) = getJniWrapper(schema, format, directory, codec) Review Comment: nit: A white space is needed between `val` and `(` But I see CI passed so the linter may have some trouble working with the iceberg source folders ########## cpp/CMakeLists.txt: ########## @@ -57,6 +57,7 @@ option(ENABLE_HDFS "Enable HDFS" OFF) option(ENABLE_ORC "Enable ORC" OFF) option(ENABLE_ABFS "Enable ABFS" OFF) option(ENABLE_GPU "Enable GPU" OFF) +option(ENABLE_ICEBERG_WRITE "Enable iceberg write" OFF) Review Comment: What happens if a user turns it on right now? Given the Velox PR is not landed -- 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]
