Skip to content

Conversation

@a1k0n
Copy link

@a1k0n a1k0n commented Mar 4, 2016

What changes were proposed in this pull request?

If a job is being scheduled in one thread which has a dependency on an
RDD currently executing a shuffle in another thread, Spark would throw a
NullPointerException. This patch synchronizes access to mapStatuses and
skips null status entries (which are in-progress shuffle tasks).

How was this patch tested?

Our client code unit test suite, which was reliably reproducing the race
condition with 10 threads, shows that this fixes it. I have not found a minimal
test case to add to Spark, but I will attempt to do so if desired.

The same test case was tripping up on SPARK-4454, which was fixed by
making other DAGScheduler code thread-safe.

@shivaram @srowen

Copy link
Member

Choose a reason for hiding this comment

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

Could you clarify in which situation status will be null?

Copy link
Author

Choose a reason for hiding this comment

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

Sure. Let me add a comment.

@zsxwing
Copy link
Member

zsxwing commented Mar 4, 2016

ok to test

Copy link
Member

Choose a reason for hiding this comment

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

nice catch

@a1k0n
Copy link
Author

a1k0n commented Mar 4, 2016

I have to admit I don't completely understand the design, and that this might just be treating a symptom of something else -- when getPreferredLocations is walking the dependency graph, should those map output statuses really be null entries? Can they be populated sooner? How does this thing work?

@SparkQA
Copy link

SparkQA commented Mar 4, 2016

Test build #52427 has finished for PR 11505 at commit 8add38a.

  • This patch fails Spark unit tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Mar 4, 2016

Test build #52425 has finished for PR 11505 at commit 7fad8fa.

  • This patch fails Spark unit tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@a1k0n
Copy link
Author

a1k0n commented Mar 4, 2016

rebasing to pick up flaky test fix

Andy Sloane added 2 commits March 3, 2016 20:22
If a job is being scheduled in one thread which has a dependency on an
RDD currently executing a shuffle in another thread, Spark would throw a
NullPointerException.
@SparkQA
Copy link

SparkQA commented Mar 4, 2016

Test build #52444 has finished for PR 11505 at commit 4f78803.

  • This patch fails Spark unit tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@a1k0n
Copy link
Author

a1k0n commented Mar 4, 2016

Jenkins, retest this please

1 similar comment
@srowen
Copy link
Member

srowen commented Mar 4, 2016

Jenkins, retest this please

@SparkQA
Copy link

SparkQA commented Mar 4, 2016

Test build #52462 has finished for PR 11505 at commit 4f78803.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@srowen
Copy link
Member

srowen commented Mar 6, 2016

@zsxwing looks good to me, sounds like you're OK with it too

* @param fractionThreshold fraction of total map output size that a location must have
* for it to be considered large.
*
* This method is not thread-safe.
Copy link
Member

Choose a reason for hiding this comment

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

Are we sure this makes it thread-safe? I agree it seems to resolve a problem that could arise when calling this from multiple threads, but I am not as clear it solves all of them. I'm OK with removing this if we have no particular reason to believe it's not thread-safe at this point.

Copy link
Author

Choose a reason for hiding this comment

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

I don't see any reason to believe it's not thread-safe; it's kind of neither explicitly thread-safe nor explicitly unsafe IMO. It's possible that individual status entries might change their size from underneath this function, but I don't see any problem with that. I am not necessarily the best person to ask, though.

@srowen
Copy link
Member

srowen commented Mar 8, 2016

OK, well unless there's a moderately strong objection, I think we can go ahead and merge this, even for 1.6. It's a cheap defensive measure and I don't see a downside.

@zsxwing
Copy link
Member

zsxwing commented Mar 8, 2016

LGTM

@srowen
Copy link
Member

srowen commented Mar 9, 2016

Merged to master/1.6

asfgit pushed a commit that referenced this pull request Mar 9, 2016
## What changes were proposed in this pull request?

If a job is being scheduled in one thread which has a dependency on an
RDD currently executing a shuffle in another thread, Spark would throw a
NullPointerException. This patch synchronizes access to `mapStatuses` and
skips null status entries (which are in-progress shuffle tasks).

## How was this patch tested?

Our client code unit test suite, which was reliably reproducing the race
condition with 10 threads, shows that this fixes it. I have not found a minimal
test case to add to Spark, but I will attempt to do so if desired.

The same test case was tripping up on SPARK-4454, which was fixed by
making other DAGScheduler code thread-safe.

shivaram srowen

Author: Andy Sloane <[email protected]>

Closes #11505 from a1k0n/SPARK-13631.

(cherry picked from commit cbff280)
Signed-off-by: Sean Owen <[email protected]>
@asfgit asfgit closed this in cbff280 Mar 9, 2016
roygao94 pushed a commit to roygao94/spark that referenced this pull request Mar 22, 2016
## What changes were proposed in this pull request?

If a job is being scheduled in one thread which has a dependency on an
RDD currently executing a shuffle in another thread, Spark would throw a
NullPointerException. This patch synchronizes access to `mapStatuses` and
skips null status entries (which are in-progress shuffle tasks).

## How was this patch tested?

Our client code unit test suite, which was reliably reproducing the race
condition with 10 threads, shows that this fixes it. I have not found a minimal
test case to add to Spark, but I will attempt to do so if desired.

The same test case was tripping up on SPARK-4454, which was fixed by
making other DAGScheduler code thread-safe.

shivaram srowen

Author: Andy Sloane <[email protected]>

Closes apache#11505 from a1k0n/SPARK-13631.
@a1k0n a1k0n deleted the SPARK-13631 branch February 15, 2017 00:23
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

5 participants