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

Fix GpuShuffleCoalesce op time metric doesn't include concat batch time #5950

Merged
merged 2 commits into from
Jul 7, 2022
Merged
Show file tree
Hide file tree
Changes from all 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
7 changes: 5 additions & 2 deletions docs/tuning-guide.md
Original file line number Diff line number Diff line change
Expand Up @@ -303,7 +303,7 @@ Custom Spark SQL Metrics are available which can help identify performance bottl
| buildTime | build time | Time to load the build-side of a join. |
| collectTime | collect time | For a broadcast the amount of time it took to collect the broadcast data back to the driver before broadcasting it back out. |
| computeAggTime | aggregation time | Time computing an aggregation. |
| concatTime | concat batch time | Time to concatenate batches. |
| concatTime | concat batch time | Time to concatenate batches. Runs on CPU. |
| copyBufferTime | copy buffer time | Time spent on copying upstreaming data into Rapids buffers. |
| filterTime | filter time | Time spent applying filters within other operators, such as joins. |
| gpuDecodeTime | GPU decode time | Time spent on GPU decoding encrypted or compressed data. |
Expand All @@ -314,7 +314,7 @@ Custom Spark SQL Metrics are available which can help identify performance bottl
| numOutputBatches | output columnar batches | Number of columnar batches that the operator outputs. |
| numOutputRows | output rows | Number of rows that the operator outputs. |
| numPartitions | partitions | Number of output partitions from a file scan or shuffle exchange. |
| opTime | op time | Time that an operator takes, exclusive of the time for executing or fetching results from child operators, and typically outside of the time it takes to acquire the GPU semaphore. |
| opTime | op time | Time that an operator takes, exclusive of the time for executing or fetching results from child operators, and typically outside of the time it takes to acquire the GPU semaphore. <br/> Note: Sometimes contains CPU times, e.g.: concatTime |
| partitionSize | partition data size | Total size in bytes of output partitions. |
| peakDevMemory | peak device memory | Peak GPU memory used during execution of an operator. |
| semaphoreWaitTime | GPU semaphore wait time | Time spent waiting for the GPU semaphore. |
Expand All @@ -341,6 +341,9 @@ How long does operator X take on the GPU vs the CPU?

### Time taken on the GPU

`opTime` mainly convey the GPU time.
If GPU operators have some workload on CPU, the GPU time is: `opTime` - CPU time, e.g.:
`opTime` - `concatTime`.
Nearly all GPU operators will have an `op time` metric. This metric times how long a given
operation took to complete on the GPU separate from anything upstream or down stream of the
operator. By looking at the `op time` for each operator you should be able to get a feeling of
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -204,12 +204,20 @@ class GpuShuffleCoalesceIterator(iter: Iterator[HostConcatResult],
throw new NoSuchElementException("No more columnar batches")
}
withResource(new NvtxRange("Concat+Load Batch", NvtxColor.YELLOW)) { _ =>
withResource(iter.next()) { hostConcatResult =>
val hostConcatResult = withResource(new MetricRange(opTimeMetric)) { _ =>
// op time covers concat time performed in `iter.next()`.
// Note the concat runs on CPU.
// GPU time = opTime - concatTime
iter.next()
}

withResource(hostConcatResult) { _ =>
// We acquire the GPU regardless of whether `hostConcatResult`
// is an empty batch or not, because the downstream tasks expect
// the `GpuShuffleCoalesceIterator` to acquire the semaphore and may
// generate GPU data from batches that are empty.
GpuSemaphore.acquireIfNecessary(TaskContext.get(), semWaitTime)

withResource(new MetricRange(opTimeMetric)) { _ =>
val batch = cudf_utils.HostConcatResultUtil.getColumnarBatch(hostConcatResult, dataTypes)
outputBatchesMetric += 1
Expand Down