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 semaphoreWaitTime and gpuOpTime for GpuRowToColumnarExec #2823

Merged
merged 8 commits into from
Jun 28, 2021

Conversation

andygrove
Copy link
Contributor

Signed-off-by: Andy Grove andygrove@nvidia.com

Closes #2820

This PR changes the metrics reported for GpuRowToColumnarExec to include semaphoreWaitTime and gpuOpTime (previously reported as totalTime) as well as totalTime.

Before

gpu-r2c-before

After

There are two code paths for row-to-column.

GeneratedUnsafeRowToCudfRowIterator

gpu-r2c-after-gpu

GpuRowToColumnConverter

gpu-r2c-after-cpu

@andygrove andygrove marked this pull request as ready for review June 25, 2021 21:54
Copy link
Member

@jlowe jlowe left a comment

Choose a reason for hiding this comment

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

I was going to say we need to update the tuning guide to discuss these new metrics, but #2720 wasn't merged?

Signed-off-by: Andy Grove <andygrove@nvidia.com>
@sameerz sameerz added this to the June 21 - July 2 milestone Jun 28, 2021
@sameerz sameerz added the bug Something isn't working label Jun 28, 2021
revans2
revans2 previously approved these changes Jun 28, 2021
Copy link
Collaborator

@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.

Overall it looks good. Just a question and a small nit.

@@ -806,6 +817,8 @@ case class GpuRowToColumnarExec(child: SparkPlan, goal: CoalesceSizeGoal)
}

override lazy val additionalMetrics: Map[String, GpuMetric] = Map(
SEMAPHORE_WAIT_TIME -> createNanoTimingMetric(DEBUG_LEVEL, DESCRIPTION_SEMAPHORE_WAIT_TIME),
GPU_OP_TIME -> createNanoTimingMetric(MODERATE_LEVEL, DESCRIPTION_GPU_OP_TIME),
TOTAL_TIME -> createNanoTimingMetric(MODERATE_LEVEL, DESCRIPTION_TOTAL_TIME),
Copy link
Collaborator

Choose a reason for hiding this comment

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

I am tempted to drop TOTAL_TIME to DEBUG_LEVEL, but I don't know how you want to use it with benchmarks/etc so I don't know if that is a good idea or not.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

What I would really like is OP_TIME. I should be able to calculate that as TOTAL_TIME - SEMAPHORE_WAIT_TIME - GPU_OPTIME. I'll look at that next.

Copy link
Member

Choose a reason for hiding this comment

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

Wouldn't that just be measuring mostly the time spent in earlier child execs within the stage? TOTAL_TIME includes time spent fetching inputs from child iterators. It wouldn't be very op-specific, and thus OP_TIME would be an odd name for it.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Yes, that's a good point. I would need to subtract the cost of the fetches too, but that might be expensive because this is a row-based iterator. I'll take a look and see what the options are.

Copy link
Collaborator

Choose a reason for hiding this comment

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

This is converting rows on the CPU to columns on the GPU. There is close to no processing on the CPU beyond fetching data from upstream and putting it into a buffer. If we try to measure the amount of time it takes to convert from UnsafeRow to CudfUnsafeRow, or to just put it into the arrow format in a buffer (depending on the code path we take), we are likely going to spend more time measuring than actually doing the conversion. Unless knowing that number is critically important we would propose that we just lump it all together.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Ok, I went ahead and changed the level of TOTAL_TIME to DEBUG_LEVEL.

val nvtxRange = new NvtxRange("Acquire GPU", NvtxColor.RED)
def acquireIfNecessary(context: TaskContext, waitMetric: Option[GpuMetric] = None): Unit = {
val nvtxRange = waitMetric match {
case Some(m) => new NvtxWithMetrics("Acquire GPU", NvtxColor.RED, m)
Copy link
Collaborator

Choose a reason for hiding this comment

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

We have done this in a few places. Would it be better to just have NvtxWithMetrics have a constructor that takes a Option[GpuMetric] and hide it internally?

@revans2
Copy link
Collaborator

revans2 commented Jun 28, 2021

On a side not I noticed that data bricks photon has a "cumulative time" metric that fills the same role we have been using for "total time" but feels a bit more accurate in the naming. Just curious if it is worth refactoring/renaming to match?

@andygrove
Copy link
Contributor Author

On a side not I noticed that data bricks photon has a "cumulative time" metric that fills the same role we have been using for "total time" but feels a bit more accurate in the naming. Just curious if it is worth refactoring/renaming to match?

I like the cumulative name better. I'll create a separate PR to propose renaming this metric.

Signed-off-by: Andy Grove <andygrove@nvidia.com>
Signed-off-by: Andy Grove <andygrove@nvidia.com>
revans2
revans2 previously approved these changes Jun 28, 2021
@andygrove
Copy link
Contributor Author

build

Signed-off-by: Andy Grove <andygrove@nvidia.com>
@jlowe
Copy link
Member

jlowe commented Jun 28, 2021

build

1 similar comment
@andygrove
Copy link
Contributor Author

build

@andygrove andygrove merged commit e04e34e into NVIDIA:branch-21.08 Jun 28, 2021
@andygrove andygrove deleted the improve-row-to-col-metrics branch June 28, 2021 23:50
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
bug Something isn't working
Projects
None yet
Development

Successfully merging this pull request may close these issues.

[BUG] Metrics are inconsistent for GpuRowToColumnarToExec
4 participants