-
Notifications
You must be signed in to change notification settings - Fork 249
Feat: Add CometLocalTableScanExec operator #2735
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
Open
kazantsev-maksim
wants to merge
10
commits into
apache:main
Choose a base branch
from
kazantsev-maksim:local_table_scan_exec_impl
base: main
Could not load branches
Branch not found: {{ refName }}
Loading
Could not load tags
Nothing to show
Loading
Are you sure you want to change the base?
Some commits from the old base branch may be removed from the timeline,
and old review comments may become outdated.
+217
−4
Open
Changes from 8 commits
Commits
Show all changes
10 commits
Select commit
Hold shift + click to select a range
610afe7
WIP
1967a25
WIP
d699f52
Merge remote-tracking branch 'origin/main' into local_table_scan_exec…
0c6a721
WIP
1edc37e
Add native CometLocalTableScanExec operator
7797ac9
Some refactoring
55992c3
Some refactoring
87f7b40
Add some unit tests
e621100
Fix PR issues
d7dffe2
Fix PR issues
File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change | ||||||||||||||||||||||||||||||
|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|
|
|
@@ -534,6 +534,14 @@ case class CometExecRule(session: SparkSession) extends Rule[SparkPlan] { | |||||||||||||||||||||||||||||||
| s | ||||||||||||||||||||||||||||||||
| } | ||||||||||||||||||||||||||||||||
|
|
||||||||||||||||||||||||||||||||
| case op: LocalTableScanExec if CometConf.COMET_EXEC_LOCAL_TABLE_SCAN_ENABLED.get(conf) => | ||||||||||||||||||||||||||||||||
| val nativeOp = QueryPlanSerde.operator2Proto(op) | ||||||||||||||||||||||||||||||||
| val cometOp = CometLocalTableScanExec(op, op.rows, op.output) | ||||||||||||||||||||||||||||||||
| CometScanWrapper(nativeOp.get, cometOp) | ||||||||||||||||||||||||||||||||
|
|
||||||||||||||||||||||||||||||||
| case op: LocalTableScanExec if !CometConf.COMET_EXEC_LOCAL_TABLE_SCAN_ENABLED.get(conf) => | ||||||||||||||||||||||||||||||||
| withInfo(op, "LocalTableScan is not enabled") | ||||||||||||||||||||||||||||||||
|
||||||||||||||||||||||||||||||||
| case op: LocalTableScanExec if CometConf.COMET_EXEC_LOCAL_TABLE_SCAN_ENABLED.get(conf) => | |
| val nativeOp = QueryPlanSerde.operator2Proto(op) | |
| val cometOp = CometLocalTableScanExec(op, op.rows, op.output) | |
| CometScanWrapper(nativeOp.get, cometOp) | |
| case op: LocalTableScanExec if !CometConf.COMET_EXEC_LOCAL_TABLE_SCAN_ENABLED.get(conf) => | |
| withInfo(op, "LocalTableScan is not enabled") | |
| case op: LocalTableScanExec => | |
| if (CometConf.COMET_EXEC_LOCAL_TABLE_SCAN_ENABLED.get(conf)) { | |
| val nativeOp = QueryPlanSerde.operator2Proto(op) | |
| val cometOp = CometLocalTableScanExec(op, op.rows, op.output) | |
| CometScanWrapper(nativeOp.get, cometOp) | |
| } else { | |
| withInfo(op, "LocalTableScan is not enabled") | |
| } |
Contributor
Author
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Thanks, fixed.
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
48 changes: 48 additions & 0 deletions
48
spark/src/main/scala/org/apache/comet/serde/operator/CometLocalTableScan.scala
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,48 @@ | ||
| /* | ||
| * 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.comet.serde.operator | ||
|
|
||
| import scala.jdk.CollectionConverters.asJavaIterableConverter | ||
|
|
||
| import org.apache.spark.sql.execution.LocalTableScanExec | ||
|
|
||
| import org.apache.comet.{CometConf, ConfigEntry} | ||
| import org.apache.comet.serde.{CometOperatorSerde, OperatorOuterClass} | ||
| import org.apache.comet.serde.OperatorOuterClass.Operator | ||
| import org.apache.comet.serde.QueryPlanSerde.serializeDataType | ||
|
|
||
| object CometLocalTableScan extends CometOperatorSerde[LocalTableScanExec] { | ||
|
|
||
| override def enabledConfig: Option[ConfigEntry[Boolean]] = Some( | ||
| CometConf.COMET_EXEC_LOCAL_TABLE_SCAN_ENABLED) | ||
|
|
||
| override def convert( | ||
| op: LocalTableScanExec, | ||
| builder: Operator.Builder, | ||
| childOp: Operator*): Option[Operator] = { | ||
| val scanTypes = op.output.flatten(attr => serializeDataType(attr.dataType)) | ||
| val scanBuilder = OperatorOuterClass.Scan | ||
| .newBuilder() | ||
| .setSource(op.getClass.getSimpleName) | ||
| .addAllFields(scanTypes.asJava) | ||
| .setArrowFfiSafe(false) | ||
| Some(builder.setScan(scanBuilder).build()) | ||
| } | ||
| } |
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
119 changes: 119 additions & 0 deletions
119
spark/src/main/scala/org/apache/spark/sql/comet/CometLocalTableScanExec.scala
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -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.spark.sql.comet | ||
|
|
||
| import org.apache.spark.TaskContext | ||
| import org.apache.spark.rdd.RDD | ||
| import org.apache.spark.sql.catalyst.InternalRow | ||
| import org.apache.spark.sql.catalyst.expressions.{Attribute, UnsafeProjection} | ||
| import org.apache.spark.sql.comet.CometLocalTableScanExec.createMetricsIterator | ||
| import org.apache.spark.sql.comet.execution.arrow.CometArrowConverters | ||
| import org.apache.spark.sql.execution.{LeafExecNode, LocalTableScanExec} | ||
| import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} | ||
| import org.apache.spark.sql.vectorized.ColumnarBatch | ||
|
|
||
| import com.google.common.base.Objects | ||
|
|
||
| import org.apache.comet.CometConf | ||
|
|
||
| case class CometLocalTableScanExec( | ||
| originalPlan: LocalTableScanExec, | ||
| @transient rows: Seq[InternalRow], | ||
| override val output: Seq[Attribute]) | ||
| extends CometExec | ||
| with LeafExecNode { | ||
|
|
||
| override lazy val metrics: Map[String, SQLMetric] = Map( | ||
| "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows")) | ||
|
|
||
| @transient private lazy val unsafeRows: Array[InternalRow] = { | ||
| if (rows.isEmpty) { | ||
| Array.empty | ||
| } else { | ||
| val proj = UnsafeProjection.create(output, output) | ||
| rows.map(r => proj(r).copy()).toArray | ||
| } | ||
| } | ||
|
|
||
| @transient private lazy val rdd: RDD[InternalRow] = { | ||
| if (rows.isEmpty) { | ||
| sparkContext.emptyRDD | ||
| } else { | ||
| val numSlices = math.min(unsafeRows.length, session.leafNodeDefaultParallelism) | ||
| sparkContext.parallelize(unsafeRows, numSlices) | ||
| } | ||
| } | ||
|
|
||
| override def doExecuteColumnar(): RDD[ColumnarBatch] = { | ||
| val numInputRows = longMetric("numOutputRows") | ||
| val maxRecordsPerBatch = CometConf.COMET_BATCH_SIZE.get(conf) | ||
| val timeZoneId = conf.sessionLocalTimeZone | ||
| rdd.mapPartitionsInternal { sparkBatches => | ||
| val context = TaskContext.get() | ||
| val batches = CometArrowConverters.rowToArrowBatchIter( | ||
| sparkBatches, | ||
| originalPlan.schema, | ||
| maxRecordsPerBatch, | ||
| timeZoneId, | ||
| context) | ||
| createMetricsIterator(batches, numInputRows) | ||
| } | ||
| } | ||
|
|
||
| override protected def stringArgs: Iterator[Any] = { | ||
| if (rows.isEmpty) { | ||
| Iterator("<empty>", output) | ||
| } else { | ||
| Iterator(output) | ||
| } | ||
| } | ||
|
|
||
| override def supportsColumnar: Boolean = true | ||
|
|
||
| override def equals(obj: Any): Boolean = { | ||
| obj match { | ||
| case other: CometLocalTableScanExec => | ||
| this.originalPlan == other.originalPlan && | ||
| this.schema == other.schema && | ||
| this.output == other.output | ||
| case _ => | ||
| false | ||
| } | ||
| } | ||
|
|
||
| override def hashCode(): Int = Objects.hashCode(originalPlan, originalPlan.schema, output) | ||
| } | ||
|
|
||
| object CometLocalTableScanExec { | ||
|
|
||
| private def createMetricsIterator( | ||
| it: Iterator[ColumnarBatch], | ||
| numInputRows: SQLMetric): Iterator[ColumnarBatch] = { | ||
| new Iterator[ColumnarBatch] { | ||
| override def hasNext: Boolean = it.hasNext | ||
|
|
||
| override def next(): ColumnarBatch = { | ||
| val batch = it.next() | ||
| numInputRows.add(batch.numRows()) | ||
| batch | ||
| } | ||
| } | ||
| } | ||
| } |
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Thanks, fixed