-
Notifications
You must be signed in to change notification settings - Fork 100
/
RssShuffleManager.scala
466 lines (399 loc) · 21.5 KB
/
RssShuffleManager.scala
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
/*
* Copyright (c) 2020 Uber Technologies, Inc.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
* http://www.apache.org/licenses/LICENSE-2.0
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.spark.shuffle
import java.util
import java.util.Random
import java.util.function.Supplier
import com.uber.rss.{RssBuildInfo, StreamServerConfig}
import com.uber.rss.clients.{MultiServerAsyncWriteClient, MultiServerHeartbeatClient, MultiServerSyncWriteClient, MultiServerWriteClient, PooledWriteClientFactory, ServerConnectionCacheUpdateRefresher, ServerConnectionStringCache, ServerConnectionStringResolver, ServerReplicationGroupUtil, ShuffleWriteConfig}
import com.uber.rss.common.{AppShuffleId, AppTaskAttemptId, ServerDetail, ServerList}
import com.uber.rss.exceptions.{RssException, RssInvalidStateException, RssNoServerAvailableException, RssServerResolveException}
import com.uber.rss.metadata.{ServiceRegistry, ServiceRegistryUtils, StandaloneServiceRegistryClient, ZooKeeperServiceRegistry}
import com.uber.rss.metrics.{M3Stats, ShuffleClientStageMetrics, ShuffleClientStageMetricsKey}
import com.uber.rss.util.{ExceptionUtils, RetryUtils, ServerHostAndPort, ThreadUtils}
import org.apache.commons.lang3.StringUtils
import org.apache.spark._
import org.apache.spark.internal.Logging
import org.apache.spark.shuffle.rss.{BufferManagerOptions, RssSparkListener, RssUtils}
import scala.collection.JavaConverters
class RssShuffleManager(conf: SparkConf) extends ShuffleManager with Logging {
logInfo(s"Creating ShuffleManager instance: ${this.getClass.getSimpleName}, version: ${RssBuildInfo.Version}, revision: ${RssBuildInfo.Revision}")
private val SparkYarnQueueConfigKey = "spark.yarn.queue"
private val NumRssServersMetricName = "numRssServers2"
private val FailToGetRssServersMetricName = "failToGetRssServers2"
private val RssDataCenterTagName = "rssDataCenter"
private val RssClusterTagName = "rssCluster"
private val UserMetricTagName = "user"
private val networkTimeoutMillis = conf.get(RssOpts.networkTimeout).toInt
private val networkRetries = conf.get(RssOpts.networkRetries).toInt
private val pollInterval = conf.get(RssOpts.pollInterval)
private val dataAvailableWaitTime = conf.get(RssOpts.readerDataAvailableWaitTime)
private var shuffleClientStageMetrics: ShuffleClientStageMetrics = null
private val serviceRegistry = createServiceRegistry
private val dataCenter = getDataCenter
private val cluster = conf.get(RssOpts.cluster)
private val executorCores = conf.getInt("spark.executor.cores", 1)
private val writerBufferSize = conf.get(RssOpts.writerBufferSize)
private val writerBufferMax = conf.get(RssOpts.writerBufferMax)
private val writerBufferSpill = conf.get(RssOpts.writerBufferSpill)
private val writerBufferSpillByEachExecutor = if (executorCores >= 1) {
writerBufferSpill/executorCores
} else {
writerBufferSpill
}
private val bufferOptions = BufferManagerOptions(
individualBufferSize = writerBufferSize,
individualBufferMax = writerBufferMax,
bufferSpillThreshold = writerBufferSpillByEachExecutor
)
private def getSparkContext = {
SparkContext.getActive.get
}
// This method is called in Spark driver side, and Spark driver will make some decision, e.g. determining what
// RSS servers to use. Then Spark driver will return a ShuffleHandle and pass that ShuffleHandle to executors (getWriter/getReader).
override def registerShuffle[K, V, C](shuffleId: Int, numMaps: Int, dependency: ShuffleDependency[K, V, C]): ShuffleHandle = {
// RSS does not support speculation yet, due to the random task attempt ids (finished map task attempt id not always increasing).
// We will fall back to SortShuffleManager if speculation is configured to true.
val useSpeculation = conf.getBoolean("spark.speculation", false)
if (useSpeculation) {
throw new RssException("Do not support speculation in Remote Shuffle Service")
}
logInfo(s"Use ShuffleManager: ${this.getClass().getSimpleName()}")
val numPartitions = dependency.partitioner.numPartitions;
val sparkContext = getSparkContext
val user = sparkContext.sparkUser
val queue = conf.get(SparkYarnQueueConfigKey, "")
val appId = conf.getAppId
val appAttempt = sparkContext.applicationAttemptId.getOrElse("0")
val heartbeatClient = MultiServerHeartbeatClient.getInstance();
heartbeatClient.setAppContext(user, appId, appAttempt)
if (!heartbeatClient.hasServerConnectionRefresher) {
heartbeatClient.setServerConnectionRefresher(createServerConnectionRefresher4Heartbeat())
}
var rssServerSelectionResult: RssServerSelectionResult = null
val excludeHostsConfigValue = conf.get(RssOpts.excludeHosts)
val excludeHosts = excludeHostsConfigValue.split(",").filter(!_.isEmpty).distinct
rssServerSelectionResult = getRssServers(numMaps, numPartitions, excludeHosts)
val rssServers = rssServerSelectionResult.servers
logInfo(s"Selected ${rssServers.size} RSS servers for shuffle $shuffleId, maps: $numMaps, partitions: $numPartitions, replicas: ${rssServerSelectionResult.replicas}, partition fanout: ${rssServerSelectionResult.partitionFanout}, ${rssServers.mkString(",")}")
val tagMap = new java.util.HashMap[String, String]()
tagMap.put(RssDataCenterTagName, dataCenter)
tagMap.put(RssClusterTagName, cluster)
tagMap.put(UserMetricTagName, user)
M3Stats.getDefaultScope.tagged(tagMap).gauge(NumRssServersMetricName).update(rssServers.length)
RssSparkListener.registerSparkListenerOnlyOnce(sparkContext, () =>
new RssSparkListener(
user,
conf.getAppId,
appAttempt,
rssServerSelectionResult.servers.map(_.getConnectionString()),
networkTimeoutMillis))
val shuffleClientStageMetricsKey = new ShuffleClientStageMetricsKey(user, queue)
shuffleClientStageMetrics = new ShuffleClientStageMetrics(shuffleClientStageMetricsKey)
shuffleClientStageMetrics.getNumRegisterShuffle.inc(1)
shuffleClientStageMetrics.getNumMappers().recordValue(numMaps)
shuffleClientStageMetrics.getNumReducers().recordValue(numPartitions)
val dependencyInfo = s"numPartitions: ${dependency.partitioner.numPartitions}, " +
s"serializer: ${dependency.serializer.getClass().getSimpleName()}, " +
s"keyOrdering: ${dependency.keyOrdering}, " +
s"aggregator: ${dependency.aggregator}, " +
s"mapSideCombine: ${dependency.mapSideCombine}, " +
s"keyClassName: ${dependency.keyClassName}, " +
s"valueClassName: ${dependency.valueClassName}"
logInfo(s"registerShuffle: $appId, $appAttempt, $shuffleId, $numMaps, $dependencyInfo")
val rssServerHandles = rssServerSelectionResult.servers.map(t => new RssShuffleServerHandle(t.getServerId(), t.getRunningVersion(), t.getConnectionString())).toArray
new RssShuffleHandle(shuffleId, appId, appAttempt, numMaps, user, queue, dependency, rssServerHandles, rssServerSelectionResult.partitionFanout)
}
// This method is called in Spark executor, getting information from Spark driver via the ShuffleHandle.
override def getWriter[K, V](handle: ShuffleHandle, mapId: Int, context: TaskContext): ShuffleWriter[K, V] = {
logInfo(s"getWriter: Use ShuffleManager: ${this.getClass().getSimpleName()}, $handle, mapId: $mapId, stageId: ${context.stageId()}, shuffleId: ${handle.shuffleId}")
handle match {
case rssShuffleHandle: RssShuffleHandle[K@unchecked, V@unchecked, _] => {
val writerQueueSize = conf.get(RssOpts.writerQueueSize)
val mapInfo = new AppTaskAttemptId(
conf.getAppId,
rssShuffleHandle.appAttempt,
handle.shuffleId,
mapId,
context.taskAttemptId()
)
logDebug( s"getWriter $mapInfo" )
createShuffleClientStageMetricsIfNeeded( rssShuffleHandle )
val serializer = rssShuffleHandle.dependency.serializer
val maxWaitMillis = conf.get( RssOpts.maxWaitTime )
val useConnectionPool = conf.get(RssOpts.useConnectionPool)
val rssMapsPerSplit = conf.get(RssOpts.mapsPerSplit)
var rssNumSplits = Math.ceil(rssShuffleHandle.numMaps.toDouble/rssMapsPerSplit.toDouble).toInt
val rssMinSplits = conf.get(RssOpts.minSplits)
val rssMaxSplits = conf.get(RssOpts.maxSplits)
if (rssNumSplits < rssMinSplits) {
rssNumSplits = rssMinSplits
} else if (rssNumSplits > rssMaxSplits) {
rssNumSplits = rssMaxSplits
}
val shuffleWriteConfig = new ShuffleWriteConfig(rssNumSplits.toShort)
val rssReplicas = conf.get(RssOpts.replicas)
if (rssReplicas <= 0) {
throw new RssException(s"Invalid config value for ${RssOpts.replicas.key}: $rssReplicas")
}
val rssServers: ServerList = ServerConnectionStringCache.getInstance().getServerList(rssShuffleHandle.getServerList)
val serverReplicationGroups = ServerReplicationGroupUtil.createReplicationGroups(rssServers.getSevers, rssReplicas)
val serverConnectionResolver = new ServerConnectionStringResolver {
override def resolveConnection(serverId: String): ServerDetail = {
val serverDetailInShuffleHandle = rssShuffleHandle.getServerList.getSeverDetail(serverId)
if (serverDetailInShuffleHandle == null) {
throw new FetchFailedException(
bmAddress = RssUtils.createMapTaskDummyBlockManagerId(mapInfo.getMapId, mapInfo.getTaskAttemptId),
shuffleId = rssShuffleHandle.shuffleId,
mapId = -1,
reduceId = 0,
message = s"Failed to get server detail for $serverId from shuffle handle: $rssShuffleHandle")
}
// random sleep some time to avoid request spike on service registry
val random = new Random()
val randomWaitMillis = random.nextInt(pollInterval)
ThreadUtils.sleep(randomWaitMillis)
val lookupResult = serviceRegistry.lookupServers(dataCenter, cluster, util.Arrays.asList(serverId))
if (lookupResult == null) {
throw new RssServerResolveException(s"Got null when looking up server for $serverId")
}
if (lookupResult.size() != 1) {
throw new RssInvalidStateException(s"Invalid result $lookupResult when looking up server for $serverId")
}
val refreshedServer: ServerDetail = lookupResult.get(0)
// add refreshed server into cache so future server lookup from the cache will get latest server.
ServerConnectionStringCache.getInstance().updateServer(serverId, refreshedServer)
if (!refreshedServer.equals(serverDetailInShuffleHandle)) {
throw new FetchFailedException(
bmAddress = RssUtils.createMapTaskDummyBlockManagerId(mapInfo.getMapId, mapInfo.getTaskAttemptId),
shuffleId = rssShuffleHandle.shuffleId,
mapId = -1,
reduceId = 0,
message = s"Detected server restart, current server: $refreshedServer, previous server: $serverDetailInShuffleHandle")
}
refreshedServer
}
}
val serverConnectionRefresher = new ServerConnectionCacheUpdateRefresher(serverConnectionResolver, ServerConnectionStringCache.getInstance())
val writerAsyncFinish = conf.get(RssOpts.writerAsyncFinish)
val finishUploadAck = !writerAsyncFinish
RetryUtils.retry(pollInterval, pollInterval * 10, maxWaitMillis, "create write client", new Supplier[ShuffleWriter[K, V]] {
override def get(): ShuffleWriter[K, V] = {
val writeClient: MultiServerWriteClient =
if (writerQueueSize == 0) {
logInfo(s"Use replicated sync writer, $rssNumSplits splits, ${rssShuffleHandle.partitionFanout} partition fanout, $serverReplicationGroups, finishUploadAck: $finishUploadAck")
new MultiServerSyncWriteClient(
serverReplicationGroups,
rssShuffleHandle.partitionFanout,
networkTimeoutMillis,
maxWaitMillis,
serverConnectionRefresher,
finishUploadAck,
useConnectionPool,
rssShuffleHandle.user,
rssShuffleHandle.appId,
rssShuffleHandle.appAttempt,
shuffleWriteConfig)
} else {
val maxThreads = conf.get(RssOpts.writerMaxThreads)
val serverThreadRatio = 8.0
val numThreadsBasedOnShuffleServers = Math.ceil(rssShuffleHandle.rssServers.length.toDouble / serverThreadRatio)
val numThreads = Math.min(numThreadsBasedOnShuffleServers, maxThreads).toInt
logInfo(s"Use replicated async writer with queue size $writerQueueSize threads $numThreads, $rssNumSplits splits, ${rssShuffleHandle.partitionFanout} partition fanout, $serverReplicationGroups, finishUploadAck: $finishUploadAck")
new MultiServerAsyncWriteClient(
serverReplicationGroups,
rssShuffleHandle.partitionFanout,
networkTimeoutMillis,
maxWaitMillis,
serverConnectionRefresher,
finishUploadAck,
useConnectionPool,
writerQueueSize,
numThreads,
rssShuffleHandle.user,
rssShuffleHandle.appId,
rssShuffleHandle.appAttempt,
shuffleWriteConfig)
}
try {
writeClient.connect()
new RssShuffleWriter(
rssShuffleHandle.user,
new ServerList(rssShuffleHandle.rssServers.map(_.toServerDetail()).toArray),
writeClient,
mapInfo,
rssShuffleHandle.numMaps,
serializer,
bufferOptions,
rssShuffleHandle.dependency,
shuffleClientStageMetrics,
context.taskMetrics().shuffleWriteMetrics)
} catch {
case ex: Throwable => {
ExceptionUtils.closeWithoutException(writeClient)
throw ex
}
}
}
})
}
}
}
// This method is called in Spark executor, getting information from Spark driver via the ShuffleHandle.
override def getReader[K, C](handle: ShuffleHandle, startPartition: Int, endPartition: Int, context: TaskContext): ShuffleReader[K, C] = {
logInfo(s"getReader: Use ShuffleManager: ${this.getClass().getSimpleName()}, $handle, partitions: [$startPartition, $endPartition)")
val rssShuffleHandle = handle.asInstanceOf[RssShuffleHandle[K, _, C]]
val shuffleInfo = new AppShuffleId(
conf.getAppId,
rssShuffleHandle.appAttempt,
handle.shuffleId
)
if (rssShuffleHandle.numMaps == 0) {
return new RssEmptyShuffleReader(
shuffleInfo,
startPartition,
endPartition)
}
val serializer = rssShuffleHandle.dependency.serializer
val rssReplicas = conf.get(RssOpts.replicas)
val rssCheckReplicaConsistency = conf.get(RssOpts.checkReplicaConsistency)
val maxWaitMillis = conf.get(RssOpts.maxWaitTime)
val rssServers = ServerConnectionStringCache.getInstance().getServerList(rssShuffleHandle.getServerList)
new RssShuffleReader(
user = rssShuffleHandle.user,
shuffleInfo = shuffleInfo,
startPartition = startPartition,
endPartition = endPartition,
serializer = serializer,
context = context,
shuffleDependency = rssShuffleHandle.dependency,
numMaps = rssShuffleHandle.numMaps,
rssServers = rssServers,
partitionFanout = rssShuffleHandle.partitionFanout,
serviceRegistry = serviceRegistry,
serviceRegistryDataCenter = dataCenter,
serviceRegistryCluster = cluster,
timeoutMillis = networkTimeoutMillis,
maxRetryMillis = maxWaitMillis.toInt,
dataAvailablePollInterval = pollInterval,
dataAvailableWaitTime = dataAvailableWaitTime,
shuffleReplicas = rssReplicas,
checkShuffleReplicaConsistency = rssCheckReplicaConsistency)
}
override def unregisterShuffle(shuffleId: Int): Boolean = {
if (shuffleClientStageMetrics != null) {
shuffleClientStageMetrics.close()
}
true
}
override def shuffleBlockResolver: ShuffleBlockResolver = {
new RssShuffleBlockResolver()
}
override def stop(): Unit = {
PooledWriteClientFactory.getInstance().shutdown();
serviceRegistry.close()
M3Stats.closeDefaultScope()
}
private def createServiceRegistry: ServiceRegistry = {
val serviceRegistryType = conf.get(RssOpts.serviceRegistryType)
logInfo(s"Service registry type: $serviceRegistryType")
serviceRegistryType match {
case ServiceRegistry.TYPE_ZOOKEEPER =>
val zkServers = getZooKeeperServers
ZooKeeperServiceRegistry.createTimingInstance(zkServers, networkTimeoutMillis, networkRetries)
case ServiceRegistry.TYPE_STANDALONE =>
val serviceRegistryServer = conf.get(RssOpts.serviceRegistryServer)
if (serviceRegistryServer == null || serviceRegistryServer.isEmpty) {
throw new RssException(s"${RssOpts.serviceRegistryServer.key} configure is not set")
}
val hostAndPort = ServerHostAndPort.fromString(serviceRegistryServer)
new StandaloneServiceRegistryClient(hostAndPort.getHost, hostAndPort.getPort, networkTimeoutMillis, "rss")
case _ => throw new RuntimeException(s"Invalid service registry type: $serviceRegistryType" )
}
}
private def getDataCenter: String = {
var dataCenterValue = conf.get(RssOpts.dataCenter)
if (StringUtils.isBlank(dataCenterValue)) {
dataCenterValue = StreamServerConfig.DEFAULT_DATA_CENTER;
}
dataCenterValue
}
private def getZooKeeperServers: String = {
val serversValue = conf.get(RssOpts.serviceRegistryZKServers)
serversValue
}
private def getRssServers(numMaps: Int, numPartitions: Int, excludeHosts: Seq[String]): RssServerSelectionResult = {
val maxServerCount = conf.get(RssOpts.maxServerCount)
val minServerCount = conf.get(RssOpts.minServerCount)
var selectedServerCount = maxServerCount
val shuffleServerRatio = conf.get(RssOpts.serverRatio)
val serverCountEstimate = Math.ceil(Math.max(numMaps, numPartitions).doubleValue()/shuffleServerRatio).intValue()
if (selectedServerCount > serverCountEstimate) {
selectedServerCount = serverCountEstimate
}
if (selectedServerCount > numPartitions) {
selectedServerCount = numPartitions
}
if (selectedServerCount <= 0) {
selectedServerCount = 1
}
val rssReplicas = conf.get(RssOpts.replicas)
selectedServerCount = selectedServerCount * rssReplicas
if (selectedServerCount < minServerCount) {
selectedServerCount = minServerCount
}
val excludeHostsJavaCollection = JavaConverters.asJavaCollectionConverter(excludeHosts).asJavaCollection
val servers = ServiceRegistryUtils.getReachableServers(serviceRegistry, selectedServerCount, networkTimeoutMillis, dataCenter, cluster, excludeHostsJavaCollection)
if (servers.isEmpty) {
throw new RssNoServerAvailableException("There is no reachable RSS server")
}
MultiServerHeartbeatClient.getInstance().addServers(servers)
val serverArray = servers.toArray(new Array[ServerDetail](0))
var partitionFanout = 1
if (minServerCount > 1) {
// if min server count is configured, try to distribute a single partition on multiple servers
val numReplicationGroups = serverArray.length / rssReplicas
if (numReplicationGroups > numPartitions) {
partitionFanout = numReplicationGroups / numPartitions
}
}
RssServerSelectionResult(serverArray, rssReplicas, partitionFanout)
}
private def createShuffleClientStageMetricsIfNeeded(rssShuffleHandle: RssShuffleHandle[_, _, _]) = {
if (shuffleClientStageMetrics == null) {
val shuffleClientStageMetricsKey = new ShuffleClientStageMetricsKey(rssShuffleHandle.user, rssShuffleHandle.queue)
shuffleClientStageMetrics = new ShuffleClientStageMetrics(shuffleClientStageMetricsKey)
}
}
private def createServerConnectionRefresher4Heartbeat() = {
val serverConnectionResolver = new ServerConnectionStringResolver {
override def resolveConnection(serverId: String): ServerDetail = {
// random sleep some time to avoid request spike on service registry
val random = new Random()
val randomWaitMillis = random.nextInt(pollInterval)
ThreadUtils.sleep(randomWaitMillis)
val lookupResult = serviceRegistry.lookupServers(dataCenter, cluster, util.Arrays.asList(serverId))
if (lookupResult == null) {
throw new RssServerResolveException(s"Got null when looking up server for $serverId")
}
if (lookupResult.size() != 1) {
throw new RssInvalidStateException(s"Invalid result $lookupResult when looking up server for $serverId")
}
val refreshedServer: ServerDetail = lookupResult.get(0)
refreshedServer
}
}
new ServerConnectionCacheUpdateRefresher(serverConnectionResolver, MultiServerHeartbeatClient.getServerCache)
}
}