Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[GLUTEN-3378][CORE] DeltaScanTransformer to support delta table #3982

Merged
merged 3 commits into from
Dec 12, 2023
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
@@ -0,0 +1,59 @@
/*
* 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 io.glutenproject.execution

import org.apache.spark.sql.catalyst.expressions.Expression
import org.apache.spark.sql.execution.FileSourceScanExec
import org.apache.spark.sql.execution.datasources.v2.BatchScanExec

/**
* Data sources transformer should implement this trait so that they can register an alias to their
* data source transformer. This allows users to give the data source transformer alias as the
* format type over the fully qualified class name.
*/
trait DataSourceScanTransformerRegister {

/**
* The class name that used to identify what kind of datasource this is。
*
* For DataSource V1, it should be the child class name of
* [[org.apache.spark.sql.execution.datasources.FileIndex]].
*
* For DataSource V2, it should be the child class name of
* [[org.apache.spark.sql.connector.read.Scan]].
*
* For example:
* {{{
* override val scanClassName: String = "org.apache.iceberg.spark.source.SparkBatchQueryScan"
* }}}
*/
val scanClassName: String

def createDataSourceTransformer(
batchScan: FileSourceScanExec,
newPartitionFilters: Seq[Expression]): FileSourceScanExecTransformer = {
throw new UnsupportedOperationException(
"This should not be called, please implement this method in child class.");
}

def createDataSourceV2Transformer(
batchScan: BatchScanExec,
newPartitionFilters: Seq[Expression]): BatchScanExecTransformer = {
throw new UnsupportedOperationException(
"This should not be called, please implement this method in child class.");
}
}

This file was deleted.

Original file line number Diff line number Diff line change
Expand Up @@ -163,7 +163,6 @@ class FileSourceScanExecTransformer(
case "DwrfFileFormat" => ReadFileFormat.DwrfReadFormat
case "DeltaMergeTreeFileFormat" => ReadFileFormat.MergeTreeReadFormat
case "CSVFileFormat" => ReadFileFormat.TextReadFormat
case "DeltaParquetFileFormat" => ReadFileFormat.ParquetReadFormat
case _ => ReadFileFormat.UnknownFormat
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -31,7 +31,7 @@ import scala.collection.JavaConverters._

object ScanTransformerFactory {

private val dataSourceV2TransformerMap = new ConcurrentHashMap[String, Class[_]]()
private val scanTransformerMap = new ConcurrentHashMap[String, Class[_]]()

def createFileSourceScanTransformer(
scanExec: FileSourceScanExec,
Expand All @@ -44,17 +44,27 @@ object ScanTransformerFactory {
} else {
ExpressionConverter.transformDynamicPruningExpr(scanExec.partitionFilters, reuseSubquery)
}
new FileSourceScanExecTransformer(
scanExec.relation,
scanExec.output,
scanExec.requiredSchema,
newPartitionFilters,
scanExec.optionalBucketSet,
scanExec.optionalNumCoalescedBuckets,
scanExec.dataFilters ++ extraFilters,
scanExec.tableIdentifier,
scanExec.disableBucketedScan
)
val fileIndex = scanExec.relation.location
lookupDataSourceScanTransformer(fileIndex.getClass.getName) match {
case Some(clz) =>
clz
.getDeclaredConstructor()
.newInstance()
.asInstanceOf[DataSourceScanTransformerRegister]
.createDataSourceTransformer(scanExec, newPartitionFilters)
case _ =>
new FileSourceScanExecTransformer(
scanExec.relation,
scanExec.output,
scanExec.requiredSchema,
newPartitionFilters,
scanExec.optionalBucketSet,
scanExec.optionalNumCoalescedBuckets,
scanExec.dataFilters ++ extraFilters,
scanExec.tableIdentifier,
scanExec.disableBucketedScan
)
}
}

def createBatchScanTransformer(
Expand All @@ -67,13 +77,12 @@ object ScanTransformerFactory {
ExpressionConverter.transformDynamicPruningExpr(batchScanExec.runtimeFilters, reuseSubquery)
}
val scan = batchScanExec.scan
scan match {
case _ if dataSourceV2TransformerExists(scan.getClass.getName) =>
val cls = lookupDataSourceV2Transformer(scan.getClass.getName)
cls
lookupDataSourceScanTransformer(scan.getClass.getName) match {
case Some(clz) =>
clz
.getDeclaredConstructor()
.newInstance()
.asInstanceOf[DataSourceV2TransformerRegister]
.asInstanceOf[DataSourceScanTransformerRegister]
.createDataSourceV2Transformer(batchScanExec, newPartitionFilters)
case _ =>
new BatchScanExecTransformer(
Expand All @@ -86,19 +95,18 @@ object ScanTransformerFactory {

def supportedBatchScan(scan: Scan): Boolean = scan match {
case _: FileScan => true
case _ if dataSourceV2TransformerExists(scan.getClass.getName) => true
case _ => false
case _ => lookupDataSourceScanTransformer(scan.getClass.getName).nonEmpty
}

private def lookupDataSourceV2Transformer(scanClassName: String): Class[_] = {
dataSourceV2TransformerMap.computeIfAbsent(
private def lookupDataSourceScanTransformer(scanClassName: String): Option[Class[_]] = {
val clz = scanTransformerMap.computeIfAbsent(
scanClassName,
_ => {
val loader = Option(Thread.currentThread().getContextClassLoader)
.getOrElse(getClass.getClassLoader)
val serviceLoader = ServiceLoader.load(classOf[DataSourceV2TransformerRegister], loader)
val serviceLoader = ServiceLoader.load(classOf[DataSourceScanTransformerRegister], loader)
serviceLoader.asScala
.filter(_.scanClassName().equalsIgnoreCase(scanClassName))
.filter(_.scanClassName.equalsIgnoreCase(scanClassName))
.toList match {
case head :: Nil =>
// there is exactly one registered alias
Expand All @@ -107,9 +115,7 @@ object ScanTransformerFactory {
}
}
)
Option(clz)
}

private def dataSourceV2TransformerExists(scanClassName: String): Boolean = {
lookupDataSourceV2Transformer(scanClassName) != null
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1 @@
io.glutenproject.execution.DeltaScanTransformerProvider
Original file line number Diff line number Diff line change
@@ -0,0 +1,73 @@
/*
* 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 io.glutenproject.extension

import io.glutenproject.execution.FileSourceScanExecTransformer
import io.glutenproject.substrait.rel.LocalFilesNode.ReadFileFormat

import org.apache.spark.sql.catalyst.TableIdentifier
import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression}
import org.apache.spark.sql.execution.FileSourceScanExec
import org.apache.spark.sql.execution.datasources.HadoopFsRelation
import org.apache.spark.sql.types.StructType
import org.apache.spark.util.collection.BitSet

class DeltaScanTransformer(
@transient override val relation: HadoopFsRelation,
output: Seq[Attribute],
requiredSchema: StructType,
partitionFilters: Seq[Expression],
optionalBucketSet: Option[BitSet],
optionalNumCoalescedBuckets: Option[Int],
dataFilters: Seq[Expression],
override val tableIdentifier: Option[TableIdentifier],
disableBucketedScan: Boolean = false)
extends FileSourceScanExecTransformer(
relation,
output,
requiredSchema,
partitionFilters,
optionalBucketSet,
optionalNumCoalescedBuckets,
dataFilters,
tableIdentifier,
disableBucketedScan
) {

override lazy val fileFormat: ReadFileFormat = ReadFileFormat.ParquetReadFormat

}

object DeltaScanTransformer {

def apply(
scanExec: FileSourceScanExec,
newPartitionFilters: Seq[Expression]): DeltaScanTransformer = {
new DeltaScanTransformer(
scanExec.relation,
scanExec.output,
scanExec.requiredSchema,
newPartitionFilters,
scanExec.optionalBucketSet,
scanExec.optionalNumCoalescedBuckets,
scanExec.dataFilters,
scanExec.tableIdentifier,
scanExec.disableBucketedScan
)
}

}
Original file line number Diff line number Diff line change
@@ -0,0 +1,33 @@
/*
* 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 io.glutenproject.extension

import io.glutenproject.execution.{DataSourceScanTransformerRegister, FileSourceScanExecTransformer}

import org.apache.spark.sql.catalyst.expressions.Expression
import org.apache.spark.sql.execution.FileSourceScanExec

class DeltaScanTransformerProvider extends DataSourceScanTransformerRegister {

override val scanClassName: String = "org.apache.spark.sql.delta.files.TahoeLogFileIndex"

override def createDataSourceTransformer(
batchScan: FileSourceScanExec,
newPartitionFilters: Seq[Expression]): FileSourceScanExecTransformer = {
DeltaScanTransformer(batchScan, newPartitionFilters)
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -61,11 +61,13 @@ class IcebergScanTransformer(
}

object IcebergScanTransformer {
def apply(batchScan: BatchScanExec, partitionFilters: Seq[Expression]): IcebergScanTransformer = {
def apply(
batchScan: BatchScanExec,
newPartitionFilters: Seq[Expression]): IcebergScanTransformer = {
new IcebergScanTransformer(
batchScan.output,
batchScan.scan,
partitionFilters,
newPartitionFilters,
table = SparkShimLoader.getSparkShims.getBatchScanExecTable(batchScan))
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -19,13 +19,13 @@ package io.glutenproject.execution
import org.apache.spark.sql.catalyst.expressions.Expression
import org.apache.spark.sql.execution.datasources.v2.BatchScanExec

class IcebergTransformerProvider extends DataSourceV2TransformerRegister {
class IcebergTransformerProvider extends DataSourceScanTransformerRegister {

override def scanClassName(): String = "org.apache.iceberg.spark.source.SparkBatchQueryScan"
override val scanClassName: String = "org.apache.iceberg.spark.source.SparkBatchQueryScan"

override def createDataSourceV2Transformer(
batchScan: BatchScanExec,
partitionFilters: Seq[Expression]): BatchScanExecTransformer = {
IcebergScanTransformer.apply(batchScan, partitionFilters)
newPartitionFilters: Seq[Expression]): BatchScanExecTransformer = {
IcebergScanTransformer(batchScan, newPartitionFilters)
}
}
Loading