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

[SPARK-4691][shuffle] Restructure a few lines in shuffle code #3553

Closed
wants to merge 3 commits into from
Closed
Show file tree
Hide file tree
Changes from 1 commit
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 @@ -45,7 +45,7 @@ private[spark] class HashShuffleReader[K, C](
} else {
new InterruptibleIterator(context, dep.aggregator.get.combineValuesByKey(iter, context))
}
} else if (dep.aggregator.isEmpty && dep.mapSideCombine) {
} else if (dep.mapSideCombine) {
Copy link
Contributor

Choose a reason for hiding this comment

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

I think the previous way is much more clear and obvious from my understanding :-).

Copy link
Contributor Author

Choose a reason for hiding this comment

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

"if(dep.aggregator.isDefined) else if (dep.aggregator.isEmpty)" seems duplicate. isEmpty == !isDefined
We need to do another one more judgement for "dep.aggregator.isEmpty".
Also, in SortShuffleWriter.scala, i think "dep.aggregator.isEmpty" is better than "!dep.aggregator.isDefined".

Copy link
Contributor

Choose a reason for hiding this comment

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

Could also write this as

if (dep.aggregator.isDefined) {
  ...
} else {
  require(!dep.mapSideCombine, "Map-side combine requested without Aggregator specified!")

  // Convert the Product2s to pairs since this is what downstream RDDs currently expect
  iter.asInstanceOf[Iterator[Product2[K, C]]].map(pair => (pair._1, pair._2))
}

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, this seems simple and elegant

throw new IllegalStateException("Aggregator is empty for map-side combine")
} else {
// Convert the Product2s to pairs since this is what downstream RDDs currently expect
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -56,7 +56,7 @@ private[spark] class HashShuffleWriter[K, V](
} else {
records
}
} else if (dep.aggregator.isEmpty && dep.mapSideCombine) {
} else if (dep.mapSideCombine) {
throw new IllegalStateException("Aggregator is empty for map-side combine")
} else {
records
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -50,7 +50,7 @@ private[spark] class SortShuffleWriter[K, V, C](
/** Write a bunch of records to this task's output */
override def write(records: Iterator[_ <: Product2[K, V]]): Unit = {
if (dep.mapSideCombine) {
if (!dep.aggregator.isDefined) {
if (dep.aggregator.isEmpty) {
throw new IllegalStateException("Aggregator is empty for map-side combine")
Copy link
Contributor

Choose a reason for hiding this comment

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

could change this guy to a require as well, and maybe change the message too

}
sorter = new ExternalSorter[K, V, C](
Expand Down