-
Notifications
You must be signed in to change notification settings - Fork 28.5k
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
[SPARK-36559][SQL][PYTHON] Create plans dedicated to distributed-sequence index for optimization #33807
[SPARK-36559][SQL][PYTHON] Create plans dedicated to distributed-sequence index for optimization #33807
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,62 @@ | ||
/* | ||
* 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.execution.python | ||
|
||
import org.apache.spark.rdd.RDD | ||
import org.apache.spark.sql.catalyst.InternalRow | ||
import org.apache.spark.sql.catalyst.expressions._ | ||
import org.apache.spark.sql.catalyst.plans.physical._ | ||
import org.apache.spark.sql.execution.{SparkPlan, UnaryExecNode} | ||
|
||
/** | ||
* A physical plan that adds a new long column with `sequenceAttr` that | ||
* increases one by one. This is for 'distributed-sequence' default index | ||
* in pandas API on Spark. | ||
*/ | ||
case class AttachDistributedSequenceExec( | ||
sequenceAttr: Attribute, | ||
child: SparkPlan) | ||
extends UnaryExecNode { | ||
|
||
override def producedAttributes: AttributeSet = AttributeSet(sequenceAttr) | ||
|
||
override val output: Seq[Attribute] = sequenceAttr +: child.output | ||
|
||
override def outputPartitioning: Partitioning = child.outputPartitioning | ||
|
||
override protected def doExecute(): RDD[InternalRow] = { | ||
child.execute().map(_.copy()) | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. why do we need to copy the unsafe rows before calling There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Oh, i forgot to describe it. |
||
.localCheckpoint() // to avoid execute multiple jobs. zipWithIndex launches a Spark job. | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I am still not sure if we need to There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. e.g.) if the child RDD has a shuffle, the shuffle will be triggered twice, and this checkpoint is to avoid that. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. The shuffle will be reused. I think |
||
.zipWithIndex().mapPartitions { iter => | ||
val unsafeProj = UnsafeProjection.create(output, output) | ||
val joinedRow = new JoinedRow | ||
val unsafeRowWriter = | ||
new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(1) | ||
|
||
iter.map { case (row, id) => | ||
// Writes to an UnsafeRow directly | ||
unsafeRowWriter.reset() | ||
unsafeRowWriter.write(0, id) | ||
joinedRow(unsafeRowWriter.getRow, row) | ||
}.map(unsafeProj) | ||
} | ||
} | ||
|
||
override protected def withNewChildInternal(newChild: SparkPlan): AttachDistributedSequenceExec = | ||
copy(child = newChild) | ||
} |
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.
We could think about implementing this with an expression (like Python UDF or Window) .. but just decided to do this with plans to avoid making it too much complicated.