This repository has been archived by the owner on Sep 18, 2023. It is now read-only.
-
Notifications
You must be signed in to change notification settings - Fork 75
[NSE-574]implement columnar limit #575
Merged
Merged
Changes from 4 commits
Commits
Show all changes
5 commits
Select commit
Hold shift + click to select a range
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
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -23,6 +23,7 @@ import com.intel.oap.vectorized._ | |
import org.apache.spark.sql.catalyst.InternalRow | ||
import org.apache.spark.sql.catalyst.expressions._ | ||
import org.apache.spark.sql.catalyst.expressions.codegen._ | ||
import org.apache.spark.sql.catalyst.plans.physical._ | ||
import org.apache.spark.internal.Logging | ||
import org.apache.spark.rdd.RDD | ||
import org.apache.spark.sql.execution._ | ||
|
@@ -308,3 +309,144 @@ case class ColumnarUnionExec(children: Seq[SparkPlan]) extends SparkPlan { | |
throw new UnsupportedOperationException(s"This operator doesn't support doExecute().") | ||
} | ||
} | ||
|
||
//TODO(): consolidate locallimit and globallimit | ||
case class ColumnarLocalLimitExec(limit: Int, child: SparkPlan) extends LimitExec { | ||
// updating nullability to make all the children consistent | ||
|
||
buildCheck() | ||
|
||
def buildCheck(): Unit = { | ||
for (child <- children) { | ||
for (schema <- child.schema) { | ||
try { | ||
ConverterUtils.checkIfTypeSupported(schema.dataType) | ||
} catch { | ||
case e: UnsupportedOperationException => | ||
throw new UnsupportedOperationException( | ||
s"${schema.dataType} is not supported in ColumnarLocalLimitExec") | ||
} | ||
} | ||
} | ||
} | ||
|
||
|
||
override def outputOrdering: Seq[SortOrder] = child.outputOrdering | ||
|
||
override def outputPartitioning: Partitioning = child.outputPartitioning | ||
|
||
override def supportsColumnar = true | ||
override def output: Seq[Attribute] = child.output | ||
protected override def doExecuteColumnar(): RDD[ColumnarBatch] = { | ||
child.executeColumnar().mapPartitions { iter => | ||
val hasInput = iter.hasNext | ||
val res = if (hasInput) { | ||
new Iterator[ColumnarBatch] { | ||
var rowCount = 0 | ||
override def hasNext: Boolean = { | ||
val hasNext = iter.hasNext | ||
hasNext && (rowCount <= limit) | ||
} | ||
|
||
override def next(): ColumnarBatch = { | ||
|
||
if (!hasNext) { | ||
throw new NoSuchElementException("End of ColumnarBatch iterator") | ||
} | ||
|
||
if (rowCount < limit) { | ||
val delta = iter.next() | ||
rowCount += delta.numRows | ||
if (rowCount > limit) { | ||
val newSize = rowCount - limit | ||
delta.setNumRows(newSize) | ||
} | ||
delta | ||
} else { | ||
throw new NoSuchElementException("End of ColumnarBatch iterator") | ||
} | ||
} | ||
} | ||
} else { | ||
Iterator.empty | ||
} | ||
new CloseableColumnBatchIterator(res) | ||
} | ||
} | ||
|
||
protected override def doExecute() | ||
: org.apache.spark.rdd.RDD[org.apache.spark.sql.catalyst.InternalRow] = { | ||
throw new UnsupportedOperationException(s"This operator doesn't support doExecute().") | ||
} | ||
|
||
} | ||
|
||
case class ColumnarGlobalLimitExec(limit: Int, child: SparkPlan) extends LimitExec { | ||
// updating nullability to make all the children consistent | ||
|
||
buildCheck() | ||
|
||
def buildCheck(): Unit = { | ||
for (child <- children) { | ||
for (schema <- child.schema) { | ||
try { | ||
ConverterUtils.checkIfTypeSupported(schema.dataType) | ||
} catch { | ||
case e: UnsupportedOperationException => | ||
throw new UnsupportedOperationException( | ||
s"${schema.dataType} is not supported in ColumnarGlobalLimitExec") | ||
} | ||
} | ||
} | ||
} | ||
|
||
|
||
override def outputOrdering: Seq[SortOrder] = child.outputOrdering | ||
|
||
override def outputPartitioning: Partitioning = child.outputPartitioning | ||
override def requiredChildDistribution: List[Distribution] = AllTuples :: Nil | ||
|
||
override def supportsColumnar = true | ||
override def output: Seq[Attribute] = child.output | ||
protected override def doExecuteColumnar(): RDD[ColumnarBatch] = { | ||
child.executeColumnar().mapPartitions { iter => | ||
val hasInput = iter.hasNext | ||
val res = if (hasInput) { | ||
new Iterator[ColumnarBatch] { | ||
var rowCount = 0 | ||
override def hasNext: Boolean = { | ||
val hasNext = iter.hasNext | ||
hasNext && (rowCount <= limit) | ||
} | ||
|
||
override def next(): ColumnarBatch = { | ||
|
||
if (!hasNext) { | ||
throw new NoSuchElementException("End of ColumnarBatch iterator") | ||
} | ||
|
||
if (rowCount < limit) { | ||
val delta = iter.next() | ||
rowCount += delta.numRows | ||
if (rowCount > limit) { | ||
val newSize = rowCount - limit | ||
delta.setNumRows(newSize) | ||
} | ||
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. hi, delta should be returned |
||
delta | ||
} else { | ||
throw new NoSuchElementException("End of ColumnarBatch iterator") | ||
} | ||
} | ||
} | ||
} else { | ||
Iterator.empty | ||
} | ||
new CloseableColumnBatchIterator(res) | ||
} | ||
} | ||
|
||
protected override def doExecute() | ||
: org.apache.spark.rdd.RDD[org.apache.spark.sql.catalyst.InternalRow] = { | ||
throw new UnsupportedOperationException(s"This operator doesn't support doExecute().") | ||
} | ||
} |
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
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.
The newSize calculation logic can be changed like below codes:
if (rowCount < limit) {
val delta = iter.next()
val preRowCount = rowCount
rowCount += delta.numRows
if (rowCount > limit) {
delta.setNumRows(limit - preRowCount)
}
delta
} else {
throw new NoSuchElementException("End of ColumnarBatch iterator")
}
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.
cool, thanks for the check