YannByron commented on code in PR #6704: URL: https://github.com/apache/paimon/pull/6704#discussion_r2617126564
########## paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/PaimonSparkCopyOnWriteOperation.scala: ########## @@ -0,0 +1,109 @@ +/* + * 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.paimon.spark + +import org.apache.paimon.CoreOptions +import org.apache.paimon.CoreOptions.BucketFunctionType +import org.apache.paimon.options.Options +import org.apache.paimon.spark.catalog.functions.BucketFunction +import org.apache.paimon.spark.schema.PaimonMetadataColumn.FILE_PATH_COLUMN +import org.apache.paimon.spark.util.OptionUtils +import org.apache.paimon.spark.write.PaimonV2WriteBuilder +import org.apache.paimon.table.{FileStoreTable, InnerTable, Table} +import org.apache.paimon.table.BucketMode.{BUCKET_UNAWARE, HASH_FIXED, POSTPONE_MODE} + +import org.apache.spark.sql.connector.expressions.{Expressions, NamedReference} +import org.apache.spark.sql.connector.read.{Scan, ScanBuilder} +import org.apache.spark.sql.connector.write.{LogicalWriteInfo, RowLevelOperation, RowLevelOperationInfo, WriteBuilder} +import org.apache.spark.sql.util.CaseInsensitiveStringMap + +class PaimonSparkCopyOnWriteOperation(table: Table, info: RowLevelOperationInfo) + extends RowLevelOperation { + + private lazy val coreOptions = new CoreOptions(table.options()) + + private var copyOnWriteScan: Option[PaimonCopyOnWriteScan] = None + + private lazy val useV2Write: Boolean = { + val v2WriteConfigured = OptionUtils.useV2Write() + v2WriteConfigured && supportsV2Write + } + + private def supportsV2Write: Boolean = { + coreOptions.bucketFunctionType() == BucketFunctionType.DEFAULT && { + table match { + case storeTable: FileStoreTable => + storeTable.bucketMode() match { + case HASH_FIXED => BucketFunction.supportsTable(storeTable) + case BUCKET_UNAWARE | POSTPONE_MODE => true + case _ => false + } + + case _ => false + } + } && coreOptions.clusteringColumns().isEmpty + } + + override def command(): RowLevelOperation.Command = info.command() + + override def newScanBuilder(options: CaseInsensitiveStringMap): ScanBuilder = { + table match { + case t: InnerTable => + new PaimonScanBuilder(t.copy(options.asCaseSensitiveMap).asInstanceOf[InnerTable]) { + override def build(): Scan = { + val scan = PaimonCopyOnWriteScan( + t, + requiredSchema, + pushedPaimonPredicates, + reservedFilters, + pushDownLimit, + pushDownTopN) + PaimonSparkCopyOnWriteOperation.this.copyOnWriteScan = Option(scan) + scan + } + } + case _ => + throw new UnsupportedOperationException( Review Comment: It should not run here. Shall we shift to v1 write directly when table is not `InnerTable`(even `FileStoreTable`) in `PaimonDeleteTable.apply`, so that we can guarantee it always works inside of it. ########## paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/PaimonCopyOnWriteScan.scala: ########## @@ -0,0 +1,107 @@ +/* + * 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.paimon.spark + +import org.apache.paimon.predicate.{Predicate, TopN} +import org.apache.paimon.spark.schema.PaimonMetadataColumn.FILE_PATH_COLUMN +import org.apache.paimon.table.{FileStoreTable, InnerTable} +import org.apache.paimon.table.source.{DataSplit, Split} + +import org.apache.spark.sql.connector.expressions.{Expressions, NamedReference} +import org.apache.spark.sql.connector.read.{Batch, SupportsRuntimeFiltering} +import org.apache.spark.sql.sources.{Filter, In} +import org.apache.spark.sql.types.StructType + +import java.nio.file.Paths + +import scala.collection.JavaConverters._ +import scala.collection.mutable + +case class PaimonCopyOnWriteScan( + table: InnerTable, + requiredSchema: StructType, + filters: Seq[Predicate], + reservedFilters: Seq[Filter], + override val pushDownLimit: Option[Int], + override val pushDownTopN: Option[TopN], + bucketedScanDisabled: Boolean = false) + extends PaimonScanCommon( + table, + requiredSchema, + filters, + reservedFilters, + pushDownLimit, + pushDownTopN) + with SupportsRuntimeFiltering { + + var filteredLocations: mutable.Set[String] = mutable.Set[String]() + + var filteredFileNames: mutable.Set[String] = mutable.Set[String]() + + var dataSplits: Array[DataSplit] = Array() + + def disableBucketedScan(): PaimonCopyOnWriteScan = { + copy(bucketedScanDisabled = true) + } + + // Since Spark 3.2 + override def filterAttributes(): Array[NamedReference] = { + Array(Expressions.column(FILE_PATH_COLUMN)) + } + + override def filter(runtimefilters: Array[Filter]): Unit = { + for (filter <- runtimefilters) { + filter match { + case in: In if in.attribute.equalsIgnoreCase(FILE_PATH_COLUMN) => + for (value <- in.values) { + val location = value.asInstanceOf[String] + filteredLocations.add(location) + filteredFileNames.add(Paths.get(location).getFileName.toString) + } + case _ => logWarning("Unsupported runtime filter") + } + } + + table match { + case fileStoreTable: FileStoreTable => + val snapshotReader = fileStoreTable.newSnapshotReader() + if (fileStoreTable.coreOptions().manifestDeleteFileDropStats()) { + snapshotReader.dropStats() + } + + filters.foreach(snapshotReader.withFilter) + + snapshotReader.withDataFileNameFilter(fileName => filteredFileNames.contains(fileName)) + + dataSplits = + snapshotReader.read().splits().asScala.collect { case s: DataSplit => s }.toArray + + case _ => throw new RuntimeException("Only FileStoreTable support.") Review Comment: Make sure only `FileStoreTable` can create `PaimonCopyOnWriteScan` ? ########## paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/PaimonSparkCopyOnWriteOperation.scala: ########## @@ -0,0 +1,109 @@ +/* + * 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.paimon.spark + +import org.apache.paimon.CoreOptions +import org.apache.paimon.CoreOptions.BucketFunctionType +import org.apache.paimon.options.Options +import org.apache.paimon.spark.catalog.functions.BucketFunction +import org.apache.paimon.spark.schema.PaimonMetadataColumn.FILE_PATH_COLUMN +import org.apache.paimon.spark.util.OptionUtils +import org.apache.paimon.spark.write.PaimonV2WriteBuilder +import org.apache.paimon.table.{FileStoreTable, InnerTable, Table} +import org.apache.paimon.table.BucketMode.{BUCKET_UNAWARE, HASH_FIXED, POSTPONE_MODE} + +import org.apache.spark.sql.connector.expressions.{Expressions, NamedReference} +import org.apache.spark.sql.connector.read.{Scan, ScanBuilder} +import org.apache.spark.sql.connector.write.{LogicalWriteInfo, RowLevelOperation, RowLevelOperationInfo, WriteBuilder} +import org.apache.spark.sql.util.CaseInsensitiveStringMap + +class PaimonSparkCopyOnWriteOperation(table: Table, info: RowLevelOperationInfo) + extends RowLevelOperation { + + private lazy val coreOptions = new CoreOptions(table.options()) + + private var copyOnWriteScan: Option[PaimonCopyOnWriteScan] = None + + private lazy val useV2Write: Boolean = { + val v2WriteConfigured = OptionUtils.useV2Write() + v2WriteConfigured && supportsV2Write + } + + private def supportsV2Write: Boolean = { + coreOptions.bucketFunctionType() == BucketFunctionType.DEFAULT && { + table match { + case storeTable: FileStoreTable => + storeTable.bucketMode() match { + case HASH_FIXED => BucketFunction.supportsTable(storeTable) + case BUCKET_UNAWARE | POSTPONE_MODE => true + case _ => false + } + + case _ => false + } + } && coreOptions.clusteringColumns().isEmpty + } + + override def command(): RowLevelOperation.Command = info.command() + + override def newScanBuilder(options: CaseInsensitiveStringMap): ScanBuilder = { + table match { + case t: InnerTable => + new PaimonScanBuilder(t.copy(options.asCaseSensitiveMap).asInstanceOf[InnerTable]) { + override def build(): Scan = { + val scan = PaimonCopyOnWriteScan( + t, + requiredSchema, + pushedPaimonPredicates, + reservedFilters, + pushDownLimit, + pushDownTopN) + PaimonSparkCopyOnWriteOperation.this.copyOnWriteScan = Option(scan) + scan + } + } + case _ => + throw new UnsupportedOperationException( + s"Scan is only supported for InnerTable. " + + s"Actual table type: ${Option(table).map(_.getClass.getSimpleName).getOrElse("null")}" + ) + } + } + + override def newWriteBuilder(info: LogicalWriteInfo): WriteBuilder = { + table match { + case fileStoreTable: FileStoreTable if useV2Write => + val options = Options.fromMap(info.options) + val builder = new PaimonV2WriteBuilder(fileStoreTable, info.schema(), options) + builder.overwriteFiles(copyOnWriteScan) + case _ => Review Comment: ditto ########## paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/PaimonCopyOnWriteScan.scala: ########## @@ -0,0 +1,107 @@ +/* + * 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.paimon.spark + +import org.apache.paimon.predicate.{Predicate, TopN} +import org.apache.paimon.spark.schema.PaimonMetadataColumn.FILE_PATH_COLUMN +import org.apache.paimon.table.{FileStoreTable, InnerTable} +import org.apache.paimon.table.source.{DataSplit, Split} + +import org.apache.spark.sql.connector.expressions.{Expressions, NamedReference} +import org.apache.spark.sql.connector.read.{Batch, SupportsRuntimeFiltering} +import org.apache.spark.sql.sources.{Filter, In} +import org.apache.spark.sql.types.StructType + +import java.nio.file.Paths + +import scala.collection.JavaConverters._ +import scala.collection.mutable + +case class PaimonCopyOnWriteScan( + table: InnerTable, + requiredSchema: StructType, + filters: Seq[Predicate], + reservedFilters: Seq[Filter], + override val pushDownLimit: Option[Int], + override val pushDownTopN: Option[TopN], + bucketedScanDisabled: Boolean = false) + extends PaimonScanCommon( + table, + requiredSchema, + filters, + reservedFilters, + pushDownLimit, + pushDownTopN) + with SupportsRuntimeFiltering { + + var filteredLocations: mutable.Set[String] = mutable.Set[String]() + + var filteredFileNames: mutable.Set[String] = mutable.Set[String]() + + var dataSplits: Array[DataSplit] = Array() + + def disableBucketedScan(): PaimonCopyOnWriteScan = { + copy(bucketedScanDisabled = true) + } + + // Since Spark 3.2 Review Comment: Remove this line. ########## paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/write/BaseV2WriteBuilder.scala: ########## @@ -32,6 +33,15 @@ abstract class BaseV2WriteBuilder(table: Table) protected var overwriteDynamic = false protected var overwritePartitions: Option[Map[String, String]] = None + protected var isOverwriteFiles = false + protected var copyOnWriteScan: Option[PaimonCopyOnWriteScan] = None + + def overwriteFiles(scan: Option[PaimonCopyOnWriteScan]): WriteBuilder = { + this.isOverwriteFiles = true Review Comment: Shall we assert scan is not empty? Or only scan is not empty, then set `isOverwriteFiles` and `copyOnWriteScan`. ########## paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/PaimonCopyOnWriteScan.scala: ########## @@ -0,0 +1,107 @@ +/* + * 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.paimon.spark + +import org.apache.paimon.predicate.{Predicate, TopN} +import org.apache.paimon.spark.schema.PaimonMetadataColumn.FILE_PATH_COLUMN +import org.apache.paimon.table.{FileStoreTable, InnerTable} +import org.apache.paimon.table.source.{DataSplit, Split} + +import org.apache.spark.sql.connector.expressions.{Expressions, NamedReference} +import org.apache.spark.sql.connector.read.{Batch, SupportsRuntimeFiltering} +import org.apache.spark.sql.sources.{Filter, In} +import org.apache.spark.sql.types.StructType + +import java.nio.file.Paths + +import scala.collection.JavaConverters._ +import scala.collection.mutable + +case class PaimonCopyOnWriteScan( + table: InnerTable, + requiredSchema: StructType, + filters: Seq[Predicate], + reservedFilters: Seq[Filter], + override val pushDownLimit: Option[Int], + override val pushDownTopN: Option[TopN], + bucketedScanDisabled: Boolean = false) + extends PaimonScanCommon( + table, + requiredSchema, + filters, + reservedFilters, + pushDownLimit, + pushDownTopN) + with SupportsRuntimeFiltering { Review Comment: to extend `SupportsRuntimeV2Filtering` ? -- 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]
