-
Notifications
You must be signed in to change notification settings - Fork 28.9k
[SPARK-15354] [CORE] Topology aware block replication strategies #13932
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
Changes from all commits
582488d
05e39c4
b639a2d
c8f4c54
105febe
29c9096
162bd55
45a4ddf
977e876
de26707
c0a75fc
e0729e6
0117dff
e32798c
463f754
6f0cffa
daa7cb4
fa55d5c
4cd86db
69cfc45
265a24e
30ba3e9
0abdb3d
0a892f6
485c9a1
88d60e2
f638f02
bdf69d6
d17f6e9
794a720
c465aaf
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -53,6 +53,46 @@ trait BlockReplicationPolicy { | |
| numReplicas: Int): List[BlockManagerId] | ||
| } | ||
|
|
||
| object BlockReplicationUtils { | ||
| // scalastyle:off line.size.limit | ||
| /** | ||
| * Uses sampling algorithm by Robert Floyd. Finds a random sample in O(n) while | ||
| * minimizing space usage. Please see <a href="http://math.stackexchange.com/questions/178690/whats-the-proof-of-correctness-for-robert-floyds-algorithm-for-selecting-a-sin"> | ||
| * here</a>. | ||
| * | ||
| * @param n total number of indices | ||
| * @param m number of samples needed | ||
| * @param r random number generator | ||
| * @return list of m random unique indices | ||
| */ | ||
| // scalastyle:on line.size.limit | ||
| private def getSampleIds(n: Int, m: Int, r: Random): List[Int] = { | ||
| val indices = (n - m + 1 to n).foldLeft(mutable.LinkedHashSet.empty[Int]) {case (set, i) => | ||
| val t = r.nextInt(i) + 1 | ||
| if (set.contains(t)) set + i else set + t | ||
| } | ||
| indices.map(_ - 1).toList | ||
| } | ||
|
|
||
| /** | ||
| * Get a random sample of size m from the elems | ||
| * | ||
| * @param elems | ||
| * @param m number of samples needed | ||
| * @param r random number generator | ||
| * @tparam T | ||
| * @return a random list of size m. If there are fewer than m elements in elems, we just | ||
| * randomly shuffle elems | ||
| */ | ||
| def getRandomSample[T](elems: Seq[T], m: Int, r: Random): List[T] = { | ||
| if (elems.size > m) { | ||
| getSampleIds(elems.size, m, r).map(elems(_)) | ||
| } else { | ||
| r.shuffle(elems).toList | ||
| } | ||
| } | ||
| } | ||
|
|
||
| @DeveloperApi | ||
| class RandomBlockReplicationPolicy | ||
| extends BlockReplicationPolicy | ||
|
|
@@ -67,6 +107,7 @@ class RandomBlockReplicationPolicy | |
| * @param peersReplicatedTo Set of peers already replicated to | ||
| * @param blockId BlockId of the block being replicated. This can be used as a source of | ||
| * randomness if needed. | ||
| * @param numReplicas Number of peers we need to replicate to | ||
| * @return A prioritized list of peers. Lower the index of a peer, higher its priority | ||
| */ | ||
| override def prioritize( | ||
|
|
@@ -78,7 +119,7 @@ class RandomBlockReplicationPolicy | |
| val random = new Random(blockId.hashCode) | ||
| logDebug(s"Input peers : ${peers.mkString(", ")}") | ||
| val prioritizedPeers = if (peers.size > numReplicas) { | ||
| getSampleIds(peers.size, numReplicas, random).map(peers(_)) | ||
| BlockReplicationUtils.getRandomSample(peers, numReplicas, random) | ||
| } else { | ||
| if (peers.size < numReplicas) { | ||
| logWarning(s"Expecting ${numReplicas} replicas with only ${peers.size} peer/s.") | ||
|
|
@@ -88,26 +129,96 @@ class RandomBlockReplicationPolicy | |
| logDebug(s"Prioritized peers : ${prioritizedPeers.mkString(", ")}") | ||
| prioritizedPeers | ||
| } | ||
| } | ||
|
|
||
| @DeveloperApi | ||
| class BasicBlockReplicationPolicy | ||
| extends BlockReplicationPolicy | ||
| with Logging { | ||
|
|
||
| // scalastyle:off line.size.limit | ||
| /** | ||
| * Uses sampling algorithm by Robert Floyd. Finds a random sample in O(n) while | ||
| * minimizing space usage. Please see <a href="http://math.stackexchange.com/questions/178690/whats-the-proof-of-correctness-for-robert-floyds-algorithm-for-selecting-a-sin"> | ||
| * here</a>. | ||
| * Method to prioritize a bunch of candidate peers of a block manager. This implementation | ||
| * replicates the behavior of block replication in HDFS. For a given number of replicas needed, | ||
| * we choose a peer within the rack, one outside and remaining blockmanagers are chosen at | ||
| * random, in that order till we meet the number of replicas needed. | ||
| * This works best with a total replication factor of 3, like HDFS. | ||
| * | ||
| * @param n total number of indices | ||
| * @param m number of samples needed | ||
| * @param r random number generator | ||
| * @return list of m random unique indices | ||
| * @param blockManagerId Id of the current BlockManager for self identification | ||
| * @param peers A list of peers of a BlockManager | ||
| * @param peersReplicatedTo Set of peers already replicated to | ||
| * @param blockId BlockId of the block being replicated. This can be used as a source of | ||
| * randomness if needed. | ||
| * @param numReplicas Number of peers we need to replicate to | ||
| * @return A prioritized list of peers. Lower the index of a peer, higher its priority | ||
| */ | ||
| // scalastyle:on line.size.limit | ||
| private def getSampleIds(n: Int, m: Int, r: Random): List[Int] = { | ||
| val indices = (n - m + 1 to n).foldLeft(Set.empty[Int]) {case (set, i) => | ||
| val t = r.nextInt(i) + 1 | ||
| if (set.contains(t)) set + i else set + t | ||
| override def prioritize( | ||
| blockManagerId: BlockManagerId, | ||
| peers: Seq[BlockManagerId], | ||
| peersReplicatedTo: mutable.HashSet[BlockManagerId], | ||
| blockId: BlockId, | ||
| numReplicas: Int): List[BlockManagerId] = { | ||
|
|
||
| logDebug(s"Input peers : $peers") | ||
| logDebug(s"BlockManagerId : $blockManagerId") | ||
|
|
||
| val random = new Random(blockId.hashCode) | ||
|
|
||
| // if block doesn't have topology info, we can't do much, so we randomly shuffle | ||
| // if there is, we see what's needed from peersReplicatedTo and based on numReplicas, | ||
| // we choose whats needed | ||
| if (blockManagerId.topologyInfo.isEmpty || numReplicas == 0) { | ||
| // no topology info for the block. The best we can do is randomly choose peers | ||
| BlockReplicationUtils.getRandomSample(peers, numReplicas, random) | ||
| } else { | ||
| // we have topology information, we see what is left to be done from peersReplicatedTo | ||
| val doneWithinRack = peersReplicatedTo.exists(_.topologyInfo == blockManagerId.topologyInfo) | ||
|
||
| val doneOutsideRack = peersReplicatedTo.exists { p => | ||
| p.topologyInfo.isDefined && p.topologyInfo != blockManagerId.topologyInfo | ||
| } | ||
|
|
||
| if (doneOutsideRack && doneWithinRack) { | ||
| // we are done, we just return a random sample | ||
| BlockReplicationUtils.getRandomSample(peers, numReplicas, random) | ||
| } else { | ||
| // we separate peers within and outside rack | ||
| val (inRackPeers, outOfRackPeers) = peers | ||
| .filter(_.host != blockManagerId.host) | ||
| .partition(_.topologyInfo == blockManagerId.topologyInfo) | ||
|
|
||
| val peerWithinRack = if (doneWithinRack) { | ||
| // we are done with in-rack replication, so don't need anymore peers | ||
| Seq.empty | ||
| } else { | ||
| if (inRackPeers.isEmpty) { | ||
| Seq.empty | ||
| } else { | ||
| Seq(inRackPeers(random.nextInt(inRackPeers.size))) | ||
| } | ||
| } | ||
|
|
||
| val peerOutsideRack = if (doneOutsideRack || numReplicas - peerWithinRack.size <= 0) { | ||
| Seq.empty | ||
| } else { | ||
| if (outOfRackPeers.isEmpty) { | ||
| Seq.empty | ||
| } else { | ||
| Seq(outOfRackPeers(random.nextInt(outOfRackPeers.size))) | ||
| } | ||
| } | ||
|
|
||
| val priorityPeers = peerWithinRack ++ peerOutsideRack | ||
| val numRemainingPeers = numReplicas - priorityPeers.size | ||
| val remainingPeers = if (numRemainingPeers > 0) { | ||
| val rPeers = peers.filter(p => !priorityPeers.contains(p)) | ||
| BlockReplicationUtils.getRandomSample(rPeers, numRemainingPeers, random) | ||
| } else { | ||
| Seq.empty | ||
| } | ||
|
|
||
| (priorityPeers ++ remainingPeers).toList | ||
| } | ||
|
|
||
| } | ||
| // we shuffle the result to ensure a random arrangement within the sample | ||
| // to avoid any bias from set implementations | ||
| r.shuffle(indices.map(_ - 1).toList) | ||
| } | ||
|
|
||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -18,34 +18,34 @@ | |
| package org.apache.spark.storage | ||
|
|
||
| import scala.collection.mutable | ||
| import scala.util.Random | ||
|
|
||
| import org.scalatest.{BeforeAndAfter, Matchers} | ||
|
|
||
| import org.apache.spark.{LocalSparkContext, SparkFunSuite} | ||
|
|
||
| class BlockReplicationPolicySuite extends SparkFunSuite | ||
| class RandomBlockReplicationPolicyBehavior extends SparkFunSuite | ||
| with Matchers | ||
| with BeforeAndAfter | ||
| with LocalSparkContext { | ||
|
|
||
| // Implicitly convert strings to BlockIds for test clarity. | ||
| private implicit def StringToBlockId(value: String): BlockId = new TestBlockId(value) | ||
| protected implicit def StringToBlockId(value: String): BlockId = new TestBlockId(value) | ||
|
|
||
| val replicationPolicy: BlockReplicationPolicy = new RandomBlockReplicationPolicy | ||
|
|
||
| val blockId = "test-block" | ||
| /** | ||
| * Test if we get the required number of peers when using random sampling from | ||
| * RandomBlockReplicationPolicy | ||
| * BlockReplicationPolicy | ||
| */ | ||
| test(s"block replication - random block replication policy") { | ||
| test("block replication - random block replication policy") { | ||
| val numBlockManagers = 10 | ||
| val storeSize = 1000 | ||
| val blockManagers = (1 to numBlockManagers).map { i => | ||
| BlockManagerId(s"store-$i", "localhost", 1000 + i, None) | ||
| } | ||
| val blockManagers = generateBlockManagerIds(numBlockManagers, Seq("/Rack-1")) | ||
| val candidateBlockManager = BlockManagerId("test-store", "localhost", 1000, None) | ||
| val replicationPolicy = new RandomBlockReplicationPolicy | ||
| val blockId = "test-block" | ||
|
|
||
| (1 to 10).foreach {numReplicas => | ||
| (1 to 10).foreach { numReplicas => | ||
| logDebug(s"Num replicas : $numReplicas") | ||
| val randomPeers = replicationPolicy.prioritize( | ||
| candidateBlockManager, | ||
|
|
@@ -68,7 +68,60 @@ class BlockReplicationPolicySuite extends SparkFunSuite | |
| logDebug(s"Random peers : ${secondPass.mkString(", ")}") | ||
| assert(secondPass.toSet.size === numReplicas) | ||
| } | ||
| } | ||
|
|
||
| protected def generateBlockManagerIds(count: Int, racks: Seq[String]): Seq[BlockManagerId] = { | ||
| (1 to count).map{i => | ||
| BlockManagerId(s"Exec-$i", s"Host-$i", 10000 + i, Some(racks(Random.nextInt(racks.size)))) | ||
| } | ||
| } | ||
| } | ||
|
|
||
| class TopologyAwareBlockReplicationPolicyBehavior extends RandomBlockReplicationPolicyBehavior { | ||
| override val replicationPolicy = new BasicBlockReplicationPolicy | ||
|
|
||
| test("All peers in the same rack") { | ||
| val racks = Seq("/default-rack") | ||
| val numBlockManager = 10 | ||
| (1 to 10).foreach {numReplicas => | ||
| val peers = generateBlockManagerIds(numBlockManager, racks) | ||
| val blockManager = BlockManagerId("Driver", "Host-driver", 10001, Some(racks.head)) | ||
|
|
||
| val prioritizedPeers = replicationPolicy.prioritize( | ||
| blockManager, | ||
| peers, | ||
| mutable.HashSet.empty, | ||
| blockId, | ||
| numReplicas | ||
| ) | ||
|
|
||
| assert(prioritizedPeers.toSet.size == numReplicas) | ||
| assert(prioritizedPeers.forall(p => p.host != blockManager.host)) | ||
| } | ||
| } | ||
|
|
||
| test("Peers in 2 racks") { | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. @shubhamchopra this test seems to be failing occasionally: https://spark-tests.appspot.com/test-details?suite_name=org.apache.spark.storage.TopologyAwareBlockReplicationPolicyBehavior&test_name=Peers+in+2+racks. Can you please take a look? Thanks!
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. fixed by #17624 |
||
| val racks = Seq("/Rack-1", "/Rack-2") | ||
| (1 to 10).foreach {numReplicas => | ||
| val peers = generateBlockManagerIds(10, racks) | ||
| val blockManager = BlockManagerId("Driver", "Host-driver", 9001, Some(racks.head)) | ||
|
|
||
| val prioritizedPeers = replicationPolicy.prioritize( | ||
| blockManager, | ||
| peers, | ||
| mutable.HashSet.empty, | ||
| blockId, | ||
| numReplicas | ||
| ) | ||
|
|
||
| assert(prioritizedPeers.toSet.size == numReplicas) | ||
| val priorityPeers = prioritizedPeers.take(2) | ||
| assert(priorityPeers.forall(p => p.host != blockManager.host)) | ||
| if(numReplicas > 1) { | ||
| // both these conditions should be satisfied when numReplicas > 1 | ||
| assert(priorityPeers.exists(p => p.topologyInfo == blockManager.topologyInfo)) | ||
| assert(priorityPeers.exists(p => p.topologyInfo != blockManager.topologyInfo)) | ||
|
||
| } | ||
| } | ||
| } | ||
| } | ||
Uh oh!
There was an error while loading. Please reload this page.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Given that you're already shuffling the sample here anyways, just out of curiosity is there any advantage of using Robert Floyd's algorithm over (say) Fisher-Yates? Also, more generally, is space complexity really a concern here? Can't we just user.shuffle(totalSize).take(sampleSize)for easy readability?EDIT: Please ignore my first concern. I misread the code.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I completely agree with you here. Except I was told earlier that iterating through a list the size of the executors was a concern. So this was to address time complexity.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
But isn't the time complexity same for both cases? It seems like they both only differ in terms of space complexity.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Note that, this logic is same as before, see https://github.com/apache/spark/pull/13932/files#diff-85cf7285f83b73c253480dc010b0013bL105