Skip to content
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

Add retry and SplitAndRetry support to AcceleratedColumnarToRowIterator #9088

Merged
merged 4 commits into from
Aug 24, 2023
Merged
Show file tree
Hide file tree
Changes from 2 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -19,9 +19,10 @@ package com.nvidia.spark.rapids
import scala.annotation.tailrec
import scala.collection.mutable.Queue

import ai.rapids.cudf.{Cuda, HostColumnVector, NvtxColor, Table}
import com.nvidia.spark.rapids.Arm.withResource
import ai.rapids.cudf.{ColumnVector, Cuda, HostColumnVector, NvtxColor, Table}
import com.nvidia.spark.rapids.Arm.{closeOnExcept, withResource}
import com.nvidia.spark.rapids.RapidsPluginImplicits._
import com.nvidia.spark.rapids.RmmRapidsRetryIterator.splitSpillableInHalfByRows
import com.nvidia.spark.rapids.ScalableTaskCompletion.onTaskCompletion
import com.nvidia.spark.rapids.shims.ShimUnaryExecNode

Expand All @@ -44,10 +45,12 @@ class AcceleratedColumnarToRowIterator(
numInputBatches: GpuMetric,
numOutputRows: GpuMetric,
opTime: GpuMetric,
streamTime: GpuMetric) extends Iterator[InternalRow] with Serializable {
streamTime: GpuMetric,
inTestMode: Boolean = false) extends Iterator[InternalRow] with Serializable {
@transient private var pendingCvs: Queue[HostColumnVector] = Queue.empty
// GPU batches read in must be closed by the receiver (us)
@transient private var currentCv: Option[HostColumnVector] = None
@transient private var splitIter: Iterator[Array[ColumnVector]] = Iterator.empty
revans2 marked this conversation as resolved.
Show resolved Hide resolved
// This only works on fixedWidth types for now...
assert(schema.forall(attr => UnsafeRow.isFixedLength(attr.dataType)))
// We want to remap the rows to improve packing. This means that they should be sorted by
Expand All @@ -73,7 +76,10 @@ class AcceleratedColumnarToRowIterator(
private var at: Int = 0
private var total: Int = 0

onTaskCompletion(closeAllPendingBatches())
// There is no TaskContext during unit tests.
if (!inTestMode) {
onTaskCompletion(closeAllPendingBatches())
revans2 marked this conversation as resolved.
Show resolved Hide resolved
}

private def setCurrentBatch(wip: HostColumnVector): Unit = {
currentCv = Some(wip)
Expand All @@ -100,34 +106,50 @@ class AcceleratedColumnarToRowIterator(
new Table(rearrangedColumns : _*)
}

private[this] def setupBatch(cb: ColumnarBatch): Boolean = {
private def appendRowColumnsAndClose(rowCvs: Array[ColumnVector]): Unit = {
withResource(rowCvs) { _ =>
rowCvs.foreach { cv =>
pendingCvs += cv.copyToHost()
}
}
}

private[this] def setupBatch(scb: SpillableColumnarBatch): Boolean = {
numInputBatches += 1
// In order to match the numOutputRows metric in the generated code we update
// numOutputRows for each batch. This is less accurate than doing it at output
// because it will over count the number of rows output in the case of a limit,
// but it is more efficient.
numOutputRows += cb.numRows()
if (cb.numRows() > 0) {
numOutputRows += scb.numRows()
if (scb.numRows() > 0) {
Copy link
Collaborator

Choose a reason for hiding this comment

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

Sorry one more thing. If scb.numRows() <= 0 we probably should close it. Just to be on the safe side.

Copy link
Collaborator Author

@firestarman firestarman Aug 24, 2023

Choose a reason for hiding this comment

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

Nice catch. I will do it in a follow-up PR soon.

withResource(new NvtxWithMetrics("ColumnarToRow: batch", NvtxColor.RED, opTime)) { _ =>
withResource(rearrangeRows(cb)) { table =>
// The fixed-width optimized cudf kernel only supports up to 1.5 KB per row which means at
// most 184 double/long values. Spark by default limits codegen to 100 fields
// "spark.sql.codegen.maxFields". So, we are going to be cautious and start with that
// until we have tested it more. We branching over the size of the output to know which
// kernel to call. If schema.length < 100 we call the fixed-width optimized version,
// otherwise the generic one
withResource(if (schema.length < 100) {
table.convertToRowsFixedWidthOptimized()
} else {
table.convertToRows()
}) { rowsCvList =>
rowsCvList.foreach { rowsCv =>
pendingCvs += rowsCv.copyToHost()
val it = RmmRapidsRetryIterator.withRetry(scb, splitSpillableInHalfByRows) { attempt =>
withResource(attempt.getColumnarBatch()) { attemptCb =>
withResource(rearrangeRows(attemptCb)) { table =>
// The fixed-width optimized cudf kernel only supports up to 1.5 KB per row which
// means at most 184 double/long values. Spark by default limits codegen to 100
// fields "spark.sql.codegen.maxFields". So, we are going to be cautious and
// start with that until we have tested it more. We branching over the size of
// the output to know which kernel to call. If schema.length < 100 we call the
// fixed-width optimized version, otherwise the generic one
if (schema.length < 100) {
table.convertToRowsFixedWidthOptimized()
} else {
table.convertToRows()
}
}
setCurrentBatch(pendingCvs.dequeue())
return true
}
}
assert(it.hasNext, "Got an unexpected empty iterator after setting up batch with retry")
appendRowColumnsAndClose(it.next())
if(it.hasNext) {
// the previous cached iterator should be all consumed.
assert(splitIter.isEmpty)
// cache the iterator for next call
splitIter = it
}
setCurrentBatch(pendingCvs.dequeue())
return true
}
}
false
Expand All @@ -137,6 +159,9 @@ class AcceleratedColumnarToRowIterator(
closeCurrentBatch()
if (pendingCvs.nonEmpty) {
setCurrentBatch(pendingCvs.dequeue())
} else if(splitIter.hasNext) {
appendRowColumnsAndClose(splitIter.next())
setCurrentBatch(pendingCvs.dequeue())
} else {
populateBatch()
}
Expand All @@ -154,10 +179,14 @@ class AcceleratedColumnarToRowIterator(
}
}

private def fetchNextBatch(): Option[ColumnarBatch] = {
private def fetchNextBatch(): Option[SpillableColumnarBatch] = {
withResource(new NvtxWithMetrics("ColumnarToRow: fetch", NvtxColor.BLUE, streamTime)) { _ =>
if (batches.hasNext) {
Some(batches.next())
// Make it spillable once getting a columnar batch.
val spillBatch = closeOnExcept(batches.next()) { cb =>
SpillableColumnarBatch(cb, SpillPriorities.ACTIVE_ON_DECK_PRIORITY)
}
Some(spillBatch)
} else {
None
}
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,64 @@
/*
* Copyright (c) 2023, NVIDIA CORPORATION.
*
* Licensed 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 com.nvidia.spark.rapids

import ai.rapids.cudf.ColumnVector
import com.nvidia.spark.rapids.jni.RmmSpark

import org.apache.spark.sql.catalyst.expressions.{AttributeReference, ExprId}
import org.apache.spark.sql.types.IntegerType
import org.apache.spark.sql.vectorized.ColumnarBatch

class ColumnToRowIteratorRetrySuite extends RmmSparkRetrySuiteBase {

private val ref = GpuBoundReference(0, IntegerType, nullable = false)(ExprId(0), "a")
private val attrs = Seq(AttributeReference(ref.name, ref.dataType, ref.nullable)())
private val NUM_ROWS = 50

private def buildBatch: ColumnarBatch = {
val ints = 0 until NUM_ROWS
new ColumnarBatch(
Array(GpuColumnVector.from(ColumnVector.fromInts(ints: _*), IntegerType)), NUM_ROWS)
}

test("Accelerated columnar to row with retry OOM") {
val aCol2RowIter = new AcceleratedColumnarToRowIterator(
attrs,
Iterator(buildBatch),
NoopMetric, NoopMetric, NoopMetric, NoopMetric, inTestMode = true)
RmmSpark.forceRetryOOM(RmmSpark.getCurrentThreadId)
var numRows = 0
aCol2RowIter.foreach { _ =>
numRows += 1
}
assertResult(NUM_ROWS)(numRows)
}

test("Accelerated columnar_to_row with split and retry OOM") {
val aCol2RowIter = new AcceleratedColumnarToRowIterator(
attrs,
Iterator(buildBatch),
NoopMetric, NoopMetric, NoopMetric, NoopMetric, inTestMode = true)
RmmSpark.forceSplitAndRetryOOM(RmmSpark.getCurrentThreadId)
var numRows = 0
aCol2RowIter.foreach { _ =>
numRows += 1
}
assertResult(NUM_ROWS)(numRows)
}

}