Skip to content

[SPARK-37369][SQL] Avoid redundant ColumnarToRow transistion on InMemoryTableScan#34642

Closed
viirya wants to merge 14 commits intoapache:masterfrom
viirya:SPARK-37369
Closed

[SPARK-37369][SQL] Avoid redundant ColumnarToRow transistion on InMemoryTableScan#34642
viirya wants to merge 14 commits intoapache:masterfrom
viirya:SPARK-37369

Conversation

@viirya
Copy link
Member

@viirya viirya commented Nov 18, 2021

What changes were proposed in this pull request?

This patch proposes to let InMemoryTableScanExec produces row output directly, if its parent query plan only accepts rows instead of columnar output. In particular, this change adds a new method in SparkPlan called supportsRowBased, alongside with the existing supportsColumnar.

Why are the changes needed?

We currently have supportsColumnar indicating if a physical node can produce columnar output. The current columnar transition rule seems taking an assumption that one node can only produce columnar output but not row-based one if supportsColumnar returns true. But actually one node can possibly produce both format, i.e. columnar and row-based. For such node, if we require row-based output, the columnar transition rule will add additional columnar-to-row after it due to the wrong assumption.

So this change introduces supportsRowBased which is used to indicates if the node can produce row-based output. The rule can check this method when deciding if a columnar-to-row transition is necessary or not.

For example, InMemoryTableScanExec can produce columnar output. So if its parent plan isn't columnar, the rule adds a ColumnarToRow between them, e.g.,

+- Union
:- ColumnarToRow
: +- InMemoryTableScan i#8, j#9
: +- InMemoryRelation i#8, j#9, StorageLevel(disk, memory, deserialized, 1 replicas)

But InMemoryTableScanExec is capable of row-based output too. After this change, for such case, we can simply ask InMemoryTableScanExec to produce row output instead of a redundant conversion.

================================================================================================
Int In-memory
================================================================================================

OpenJDK 64-Bit Server VM 1.8.0_265-b01 on Mac OS X 10.16
Intel(R) Core(TM) i7-9750H CPU @ 2.60GHz
Int In-Memory scan:                         Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
--------------------------------------------------------------------------------------------------------------------------
columnar deserialization + columnar-to-row            228            245          15          4.4         227.7       1.0X
row-based deserialization                             179            187          10          5.6         179.4       1.3X

Does this PR introduce any user-facing change?

No

How was this patch tested?

Existing tests.

@github-actions github-actions bot added the SQL label Nov 18, 2021
@SparkQA
Copy link

SparkQA commented Nov 18, 2021

Test build #145370 has finished for PR 34642 at commit b73941c.

  • This patch fails to build.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Nov 18, 2021

Kubernetes integration test unable to build dist.

exiting with code: 1
URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/49843/

@SparkQA
Copy link

SparkQA commented Nov 18, 2021

Kubernetes integration test starting
URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/49846/

@SparkQA
Copy link

SparkQA commented Nov 18, 2021

Kubernetes integration test status failure
URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/49846/

@SparkQA
Copy link

SparkQA commented Nov 18, 2021

Test build #145373 has finished for PR 34642 at commit 6c379f4.

  • This patch fails Spark unit tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@viirya viirya marked this pull request as draft November 18, 2021 08:57
Copy link
Contributor

@attilapiros attilapiros left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think we need an explicit unittest to validate the to-row transition is missing from the plan in this case.

conf.cacheVectorizedReaderEnabled &&
!WholeStageCodegenExec.isTooManyFields(conf, relation.schema) &&
relation.cacheBuilder.serializer.supportsColumnarOutput(relation.schema)
relation.cacheBuilder.serializer.supportsColumnarOutput(relation.schema) && outputColumnar
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

As evaluating the outputColumnar flag is one of the fastest in this expression (where only && operators are used) I would move it before the isTooManyFields call (which uses a recursive function).

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yea, moved.

@viirya
Copy link
Member Author

viirya commented Nov 18, 2021

Thanks @attilapiros. Yea, I will add some tests later.

@SparkQA
Copy link

SparkQA commented Nov 19, 2021

Kubernetes integration test starting
URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/49912/

@SparkQA
Copy link

SparkQA commented Nov 19, 2021

Kubernetes integration test status failure
URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/49912/

@viirya viirya marked this pull request as ready for review November 19, 2021 08:26
@SparkQA
Copy link

SparkQA commented Nov 19, 2021

Kubernetes integration test starting
URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/49916/

@SparkQA
Copy link

SparkQA commented Nov 19, 2021

Test build #145440 has finished for PR 34642 at commit 7dc5b51.

  • This patch fails Spark unit tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Nov 19, 2021

Kubernetes integration test starting
URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/49919/

@SparkQA
Copy link

SparkQA commented Nov 19, 2021

Test build #145444 has finished for PR 34642 at commit 9c057b1.

  • This patch fails Spark unit tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Nov 19, 2021

Kubernetes integration test status failure
URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/49916/

@SparkQA
Copy link

SparkQA commented Nov 19, 2021

Kubernetes integration test status failure
URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/49919/

@SparkQA
Copy link

SparkQA commented Nov 19, 2021

Test build #145447 has finished for PR 34642 at commit eb85130.

  • This patch fails Spark unit tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@attilapiros
Copy link
Contributor

@viirya
Could it be at the root of this problem we have a missing flag?
I think we have 3 different states here and only 1 boolean flag:

supports columnar supports row based
true false
false true
true true

So we might abuse supportsColumnar == false and take it as supportsRowBased.

What about introducing a new flag right beside the old one:

Something like:

 def supportsRowBased: Boolean = !supportsColumnar 

And in InMemoryTableScanExec we can override it.

This would a more generic solution as any node which support both can avoid to use the unneeded to-row transition.
WDYT?

@viirya
Copy link
Member Author

viirya commented Nov 20, 2021

Ideally, yes, it is a more general one to have another flag. In practice, I suspect if there will be more such nodes that could choose to output row-based or columnar output for some conditions? For the in-memory relation scan here, adding a new flag supportsRowBased seems not bring too much benefit?

@SparkQA
Copy link

SparkQA commented Dec 9, 2021

Kubernetes integration test starting
URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/50497/

@SparkQA
Copy link

SparkQA commented Dec 9, 2021

Kubernetes integration test status failure
URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/50497/

@SparkQA
Copy link

SparkQA commented Dec 9, 2021

Test build #146017 has finished for PR 34642 at commit 2d078e1.

  • This patch fails Spark unit tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Dec 9, 2021

Test build #146021 has finished for PR 34642 at commit 788ae78.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds the following public classes (experimental):
  • class TimedeltaOps(DataTypeOps):
  • class TimedeltaIndex(Index):
  • class MissingPandasLikeTimedeltaIndex(MissingPandasLikeIndex):
  • class PandasSQLStringFormatter(string.Formatter):
  • class UDFBasicProfiler(BasicProfiler):
  • class CloudPickleSerializer(FramedSerializer):
  • class SQLStringFormatter(string.Formatter):
  • class ExecutorPodsPollingSnapshotSource(
  • class ExecutorPodsWatchSnapshotSource(
  • class ExecutorRollPlugin extends SparkPlugin with Logging
  • class AnsiCombinedTypeCoercionRule(rules: Seq[TypeCoercionRule]) extends
  • case class RelationTimeTravel(
  • case class AsOfTimestamp(timestamp: Long) extends TimeTravelSpec
  • case class AsOfVersion(version: String) extends TimeTravelSpec
  • class CombinedTypeCoercionRule(rules: Seq[TypeCoercionRule]) extends TypeCoercionRule
  • case class PrettyPythonUDF(
  • case class TryElementAt(left: Expression, right: Expression, child: Expression)
  • case class ConvertTimezone(
  • case class UnclosedCommentProcessor(
  • case class CreateTable(
  • case class TableSpec(
  • case class OptimizeSkewedJoin(ensureRequirements: EnsureRequirements)

@viirya
Copy link
Member Author

viirya commented Dec 9, 2021

cc @sunchao @dongjoon-hyun

Copy link
Member

@sunchao sunchao left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

So the motivation of this PR is to improve from the existing:

CachedBatch -> ColumnarBatch -> InternalRow

transition to:

CachedBatch -> InternalRow

Is that right? this makes sense to me.

val id: Int = SparkPlan.newPlanId()

/**
* Return true if this stage of the plan supports row-based execution.
Copy link
Member

@sunchao sunchao Dec 9, 2021

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Maybe add some explanation why we need both this and supportsColumnar? it's a bit confusing when reading this code.

Also I'm wondering if something like prefersColumnar is better, so that we have:

  • supportsColumnar: this plan can support columnar output, alongside the default row-based output which every plan supports.
  • prefersColumnar: this plan prefers to output columnar batches even if it is not explicitly requested (e.g., outputsColumnar is false).

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

supportsColumnar: this plan can support columnar output, alongside the default row-based output which every plan supports.

Hmm, seems not exactly? Not every plan supports row-based output.

prefersColumnar seems redundant? As I see, we usually prefer columnar output already.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Not every plan supports row-based output.

Hmm any example? every physical plan node has to implement doExecute which outputs rows, while doExecuteColumnar throws exception by default.

As I see, we usually prefer columnar output already.

I'm not sure about this part. To my understanding, at the moment it appears we prefer columnar output because 1) vectorized readers for OPC/Parquet yield much better performance so we always want to use that over the default row-based impls, and 2) supportsColumnar defaults to false as most operators don't support columnar execution yet, so we'll do the columnar-row conversion and switch back to whole-stage codegen.

However this may not hold true if we add columnar support for more operators like filter/project etc in future. Do we want to prefer columnar execution over the whole-stage codegen approach? I'm not sure yet and maybe some evaluation is required. prefersColumnar could give us a knob to control this.

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Hmm any example? every physical plan node has to implement doExecute which outputs rows, while doExecuteColumnar throws exception by default.

I think there is no guarantee that a physical node must implement a working doExecute. For a columnar node, it can just throw exception saying it is not implemented (like the default doExecuteColumnar) if it is not designed to be executed under row-based execution.

I also don't see a need to have implement both (working) row-based and columnar execution for a node in general. But in Spark, because we don't actually have official columnar execution nodes, so maybe I cannot get an example from Spark itself. Hopefully I convey the idea clearly.

prefersColumnar: this plan prefers to output columnar batches even if it is not explicitly requested (e.g., outputsColumnar is false).

BTW, outputsColumnar is not a preference option I think (at least for its usage now in the rule). It actually means that the output should be in columnar or not. Once outputsColumnar is false, the plan should output row-based output and it is why we add ColumnarToRowExec for the case.

Yea, the preference I mentioned is pretty limited so far. I agree that we maybe need to have a preference rule (or something) in the future. As we don't have real built-in columnar operators in Spark, so currently the situation seems that some columnar extensions/libraries replace row-based operators with columnar operators during planning. I'm not sure if we can estimate which one is preferred during planning.

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

BTW, IMHO, if we add columnar support for more operators in the future, I guess it already implicitly indicates we "prefer" it over current execution (whole-stage codegen or interpreted one)? Just like whole-stage codegen, seems we simply prefer it once we verify it having better performance generally. This is similar to the 3rd party extensions/libraries situation, I think.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I see, makes sense. I was referring to nodes in Spark itself but yea an extension could only implement doExecuteColumnar.

Although I'm still slightly in favor of prefersColumnar, but it's only a minor personal preference. Overall it looks OK.

Copy link
Contributor

@HeartSaVioR HeartSaVioR Dec 12, 2021

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

For now I guess columnar route seems to be considered as superior, otherwise there should be calculation for the cost of plan between row vs columnar.

This is just to cover the case that downstream doesn't support columnar but upstream can support both row and columnar, and has performance of producing output as columnar output > row output > columnar output + columnar-to-row conversion, so upstream wants to produce directly whatever downstream wants without conversion.

If upstream can produce columnar output faster enough to cover the overhead of columnar-to-row conversion (columnar output > columnar output + columnar-to-row conversion > row output), then it could just tactically say "it only cover columnar output" and Spark will take the conversion.

@@ -0,0 +1,12 @@
================================================================================================
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: ideally we should generate result using the GitHub workflow

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Updated with the result of GitHub Action.

private def buildBuffers(): RDD[CachedBatch] = {
val cb = if (cachedPlan.supportsColumnar) {
val cb = if (cachedPlan.supportsColumnar &&
serializer.supportsColumnarInput(cachedPlan.output)) {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

hmm why this is necessary? shouldn't cachedPlan.supportsColumnar already covers this? for instance in InMemoryTableScanExec

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This is actually a bug. cachedPlan.supportsColumnar only indicates the cached plan can output columnar format, but whether this cached rdd builder can take such input, is depending on its serializer.

There is one test which failed due to the proposed change. I remember that it happens for InMemoryRelation under InMemoryRelation.

Previously we always add additional ColumnarToRow transition between two InMemoryRelations, so we don't hit this.

@HeartSaVioR
Copy link
Contributor

cc. @revans2 @tgravescs @andygrove
I'd like to see reviews from them to ensure we don't break the origin intention of making the serializer pluggable.

@SparkQA
Copy link

SparkQA commented Dec 10, 2021

Kubernetes integration test starting
URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/50540/

@SparkQA
Copy link

SparkQA commented Dec 10, 2021

Kubernetes integration test status failure
URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/50540/

@SparkQA
Copy link

SparkQA commented Dec 10, 2021

Test build #146065 has finished for PR 34642 at commit a216a6d.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

Copy link
Contributor

@revans2 revans2 left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The change looks good to me. More comments on supportsRowBased and supportsColumnar might be good to make it clear how to use them, but it is fairly clear to me.

@SparkQA
Copy link

SparkQA commented Dec 10, 2021

Kubernetes integration test starting
URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/50554/

@SparkQA
Copy link

SparkQA commented Dec 10, 2021

Kubernetes integration test status failure
URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/50554/

@SparkQA
Copy link

SparkQA commented Dec 11, 2021

Test build #146079 has finished for PR 34642 at commit dc9ad83.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@viirya
Copy link
Member Author

viirya commented Dec 12, 2021

I think this change should be pretty clear. If no more comments or objection, I will merge this in next few days. Thanks.

Copy link
Member

@sunchao sunchao left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

LGTM

Copy link
Contributor

@HeartSaVioR HeartSaVioR left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The code change looks good, as long as this got approved by @revans2 who made recent change here and also depends on the change (I guess).

Would you mind if I ask to explain the addition of supportsRowBased in the PR description? It would help to track the change afterwards.

Copy link
Contributor

@attilapiros attilapiros left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

LGTM

@viirya
Copy link
Member Author

viirya commented Dec 12, 2021

@HeartSaVioR Updated the description. Thanks.

Copy link
Contributor

@HeartSaVioR HeartSaVioR left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

+1 thanks for the patience!

Copy link
Member

@dongjoon-hyun dongjoon-hyun left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

+1, LGTM. Thank you, @viirya and all.

@viirya
Copy link
Member Author

viirya commented Dec 13, 2021

Thank you all! Merging to master.

@viirya viirya closed this in a30bec1 Dec 13, 2021
@cloud-fan
Copy link
Contributor

we can simply ask InMemoryTableScanExec to produce row output instead of a redundant conversion.

Sorry for the late review. How do we "ask" for producing row output? I don't see any related change to the in-memory table scan in this PR.

@viirya
Copy link
Member Author

viirya commented Dec 15, 2021

Sorry for the late review. How do we "ask" for producing row output? I don't see any related change to the in-memory table scan in this PR.

Sorry for confusion. I should say we can let InMemoryTableScanExec to produce row output. We let the transition rule knows exactly the in-memory scan can do it via the newly added supportsRowBased flag.

cloud-fan pushed a commit that referenced this pull request Dec 30, 2021
### What changes were proposed in this pull request?
In PR #34642, we added a `supportsRowBased` in `SparkPlan` in order to avoid
redundant `ColumnarToRow` transition in `InMemoryTableScan `. But, this optimization
also applies to Union if its children both support row-based output.
So, this PR adds the `supportsRowBased` implementation for `UnionExec`.

### Why are the changes needed?
followup PR

### Does this PR introduce _any_ user-facing change?
No

### How was this patch tested?
Existing tests passed.

Closes #35061 from linhongliu-db/SPARK-37369-followup.

Authored-by: Linhong Liu <linhong.liu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
@viirya viirya deleted the SPARK-37369 branch December 27, 2023 18:25
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

Projects

None yet

Development

Successfully merging this pull request may close these issues.

8 participants