-
Notifications
You must be signed in to change notification settings - Fork 919
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
[KYUUBI #6024] Insert crc checksum observer after all project nodes #6025
Open
wForget
wants to merge
1
commit into
apache:master
Choose a base branch
from
wForget:KYUUBI-6024
base: master
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.
Open
Changes from all commits
Commits
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 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 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
73 changes: 73 additions & 0 deletions
73
...3-5/src/main/scala/org/apache/kyuubi/sql/observe/InsertChecksumObserverAfterProject.scala
This file contains 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,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 org.apache.kyuubi.sql.observe | ||
|
||
import java.util.concurrent.atomic.AtomicLong | ||
|
||
import org.apache.spark.sql.SparkSession | ||
import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, Cast, Crc32, Expression, Literal, NamedExpression} | ||
import org.apache.spark.sql.catalyst.expressions.aggregate.{Count, Sum} | ||
import org.apache.spark.sql.catalyst.plans.logical.{CollectMetrics, LogicalPlan, Project} | ||
import org.apache.spark.sql.catalyst.rules.Rule | ||
import org.apache.spark.sql.catalyst.trees.TreeNodeTag | ||
import org.apache.spark.sql.types.{BinaryType, ByteType, DecimalType, IntegerType, LongType, ShortType, StringType} | ||
|
||
import org.apache.kyuubi.sql.KyuubiSQLConf.INSERT_CHECKSUM_OBSERVER_AFTER_PROJECT_ENABLED | ||
import org.apache.kyuubi.sql.observe.InsertChecksumObserverAfterProject._ | ||
|
||
case class InsertChecksumObserverAfterProject(session: SparkSession) extends Rule[LogicalPlan] { | ||
|
||
private val INSERT_COLLECT_METRICS_TAG = TreeNodeTag[Unit]("__INSERT_COLLECT_METRICS_TAG") | ||
|
||
override def apply(plan: LogicalPlan): LogicalPlan = { | ||
if (conf.getConf(INSERT_CHECKSUM_OBSERVER_AFTER_PROJECT_ENABLED)) { | ||
plan resolveOperatorsUp { | ||
case p: Project if p.resolved && p.getTagValue(INSERT_COLLECT_METRICS_TAG).isEmpty => | ||
val metricExprs = p.output.map(toChecksumExpr) :+ countExpr | ||
p.setTagValue(INSERT_COLLECT_METRICS_TAG, ()) | ||
CollectMetrics(nextObserverName, metricExprs, p) | ||
} | ||
} else { | ||
plan | ||
} | ||
} | ||
|
||
private def toChecksumExpr(attr: Attribute): NamedExpression = { | ||
// sum(cast(crc32(cast(attr as binary)) as decimal(20, 0))) as attr_crc_sum | ||
Alias( | ||
Sum(Cast(Crc32(toBinaryExpr(attr)), LongDecimal)).toAggregateExpression(), | ||
attr.name + "_crc_sum")() | ||
} | ||
|
||
private def toBinaryExpr(attr: Attribute): Expression = { | ||
attr.dataType match { | ||
case BinaryType => attr | ||
case StringType | ByteType | ShortType | IntegerType | LongType => Cast(attr, BinaryType) | ||
case _ => Cast(Cast(attr, StringType), BinaryType) | ||
} | ||
} | ||
|
||
private def countExpr: NamedExpression = { | ||
Alias(Count(Literal(1)).toAggregateExpression(), "cnt")() | ||
} | ||
} | ||
|
||
object InsertChecksumObserverAfterProject { | ||
private val id = new AtomicLong(0) | ||
private def nextObserverName: String = s"CHECKSUM_OBSERVER_${id.getAndIncrement()}" | ||
private val LongDecimal = DecimalType(20, 0) | ||
} |
41 changes: 41 additions & 0 deletions
41
...src/test/scala/org/apache/spark/sql/observe/InsertChecksumObserverAfterProjectSuite.scala
This file contains 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,41 @@ | ||
/* | ||
* 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.observe | ||
|
||
import org.apache.spark.sql.{KyuubiSparkSQLExtensionTest, QueryTest, Row} | ||
|
||
import org.apache.kyuubi.sql.KyuubiSQLConf.INSERT_CHECKSUM_OBSERVER_AFTER_PROJECT_ENABLED | ||
|
||
class InsertChecksumObserverAfterProjectSuite extends KyuubiSparkSQLExtensionTest { | ||
|
||
test("insert checksum observer after project") { | ||
withSQLConf(INSERT_CHECKSUM_OBSERVER_AFTER_PROJECT_ENABLED.key -> "true") { | ||
withTable("t") { | ||
sql("CREATE TABLE t(i int)") | ||
sql("INSERT INTO t VALUES (1), (2), (3)") | ||
val df = sql("select a from (SELECT i as a FROM t) where a > 1") | ||
df.collect() | ||
val metrics = df.queryExecution.observedMetrics | ||
assert(metrics.size == 2) | ||
QueryTest.sameRows( | ||
Seq(Row(BigDecimal(6569872598L), 2), Row(BigDecimal(8017165408L), 3)), | ||
metrics.values.toSeq) | ||
} | ||
} | ||
} | ||
|
||
} |
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.
What happens if the plan is for writing ?
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.
Just add observer after all project nodes, I think there will also be project node before writing.
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.
why we only add collect metrics for project ?
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.
Most data inconsistencies are caused by project, such as using udf/udaf or type conversion.
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.
How about we remove AfterProject from the rule name and support more nodes in the future?