Skip to content
Closed
Show file tree
Hide file tree
Changes from 6 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 @@ -144,7 +144,7 @@ private[spark] class SortShuffleManager(conf: SparkConf) extends ShuffleManager
metrics: ShuffleWriteMetricsReporter): ShuffleWriter[K, V] = {
val mapTaskIds = taskIdMapsForShuffle.computeIfAbsent(
handle.shuffleId, _ => new OpenHashSet[Long](16))
mapTaskIds.synchronized { mapTaskIds.add(context.taskAttemptId()) }
mapTaskIds.synchronized { mapTaskIds.add(mapId) }
val env = SparkEnv.get
handle match {
case unsafeShuffleHandle: SerializedShuffleHandle[K @unchecked, V @unchecked] =>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,19 +17,27 @@

package org.apache.spark.shuffle.sort

import java.io.File

import scala.collection.JavaConverters._

import org.apache.commons.io.FileUtils
import org.apache.commons.io.filefilter.TrueFileFilter
import org.mockito.Mockito.{mock, when}
import org.mockito.invocation.InvocationOnMock
import org.mockito.stubbing.Answer
import org.scalatest.matchers.must.Matchers
import org.scalatest.matchers.should.Matchers._

import org.apache.spark._
import org.apache.spark.rdd.ShuffledRDD
import org.apache.spark.serializer.{JavaSerializer, KryoSerializer, Serializer}

/**
* Tests for the fallback logic in UnsafeShuffleManager. Actual tests of shuffling data are
* performed in other suites.
*/
class SortShuffleManagerSuite extends SparkFunSuite with Matchers {
class SortShuffleManagerSuite extends SparkFunSuite with Matchers with LocalSparkContext {

private def doReturn(value: Any) = org.mockito.Mockito.doReturn(value, Seq.empty: _*)

Expand Down Expand Up @@ -131,4 +139,29 @@ class SortShuffleManagerSuite extends SparkFunSuite with Matchers {
)))
}

Copy link
Member

Choose a reason for hiding this comment

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

nit: revert the unrelated change.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

line-133 is empty line, we add test in SortShuffleManagerSuite before, after #31664 (comment), move the test to ShuffleSuite, So i change the empty line together.

Copy link
Member

Choose a reason for hiding this comment

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

I know the reason, but it's still not a necessary change, especially when there's no other changes in the file.

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 will revert it.

test("Data could not be cleaned up when unregisterShuffle") {
Copy link
Member

Choose a reason for hiding this comment

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

nit: SPARK-34541: Data ...

Copy link
Member

Choose a reason for hiding this comment

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

BTW, this test requires spark.shuffle.useOldFetchProtocol=true?

And it's better to test both true and false.

Copy link
Contributor Author

@yikf yikf Mar 1, 2021

Choose a reason for hiding this comment

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

It seems that context.taskAttemptId and partitionId are the same, both increasing from 0. I don't understand why the protocol should be differentiated on the write side.

I met this problem before, but now the scene has not been recovered

Copy link
Member

Choose a reason for hiding this comment

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

We can run a simple job before our target job to make the taskAttemptId starts from 1. e.g.,

sc.parallelize(1 to 10, 1).count().

I tried this way and the issue can be reproduced.

Copy link
Contributor Author

@yikf yikf Mar 1, 2021

Choose a reason for hiding this comment

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

OK, thank you. I'll add test later, And I don't understand why the protocol should be differentiated on the WriteSide, As follow:
ShuffleMapTask#runTask
// While we use the old shuffle fetch protocol, we use partitionId as mapId in the ShuffleBlockId construction. val mapId = if (SparkEnv.get.conf.get(config.SHUFFLE_USE_OLD_FETCH_PROTOCOL)) { partitionId } else context.taskAttemptId()

In readSide, we need to use protoco to distinguish messages, But in writeSide, register to ExternalShuffleService by RegisterExecutor , It paas the localDir to shuffleService, So shuffleService know the middle file by shuffle, But seems unrelated to mapId.

Copy link
Member

Choose a reason for hiding this comment

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

I roughly remember that's because we want to ensure the unique file name at write size. cc @xuanyuanking

Copy link
Contributor Author

Choose a reason for hiding this comment

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

updated

Copy link
Member

Choose a reason for hiding this comment

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

at write size -> at write side? :)
Yes, you can check the description in #25620. TL;DR: we need a unique file name to resolve the indeterminate shuffle issue.

withTempDir { tmpDir =>
val conf = new SparkConf(loadDefaults = false)
conf.set("spark.local.dir", tmpDir.getAbsolutePath)
sc = new SparkContext("local", "SPARK-34541", conf)
val rdd = sc.parallelize(1 to 10, 1).map(x => (x, x))
// Create a shuffleRdd
val shuffledRdd = new ShuffledRDD[Int, Int, Int](rdd, new HashPartitioner(4))
.setSerializer(new JavaSerializer(conf))
def getAllFiles: Set[File] =
FileUtils.listFiles(tmpDir, TrueFileFilter.INSTANCE, TrueFileFilter.INSTANCE).asScala.toSet
val filesBeforeShuffle = getAllFiles
// Force the shuffle to be performed
shuffledRdd.count()
// Ensure that the shuffle actually created files that will need to be cleaned up
val filesCreatedByShuffle = getAllFiles -- filesBeforeShuffle
filesCreatedByShuffle.map(_.getName) should be
Set("shuffle_0_0_0.data", "shuffle_0_0_0.index")
// Check that the cleanup actually removes the files
sc.env.blockManager.master.removeShuffle(0, blocking = true)
for (file <- filesCreatedByShuffle) {
assert (!file.exists(), s"Shuffle file $file was not cleaned up")
}
}
}
}