/
RssShuffleWriter.java
323 lines (303 loc) · 13.2 KB
/
RssShuffleWriter.java
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
/*
* Tencent is pleased to support the open source community by making
* Firestorm-Spark remote shuffle server available.
*
* Copyright (C) 2021 THL A29 Limited, a Tencent company. All rights reserved.
*
* 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
*
* https://opensource.org/licenses/Apache-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 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.writer;
import java.io.IOException;
import java.util.Arrays;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.Future;
import java.util.concurrent.TimeUnit;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import com.google.common.collect.Sets;
import com.google.common.util.concurrent.Uninterruptibles;
import org.apache.spark.Partitioner;
import org.apache.spark.ShuffleDependency;
import org.apache.spark.SparkConf;
import org.apache.spark.executor.ShuffleWriteMetrics;
import org.apache.spark.scheduler.MapStatus;
import org.apache.spark.shuffle.RssShuffleHandle;
import org.apache.spark.shuffle.RssShuffleManager;
import org.apache.spark.shuffle.RssSparkConfig;
import org.apache.spark.shuffle.ShuffleWriter;
import org.apache.spark.storage.BlockManagerId;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import scala.Function1;
import scala.Option;
import scala.Product2;
import scala.collection.Iterator;
import com.tencent.rss.client.api.ShuffleWriteClient;
import com.tencent.rss.common.ShuffleBlockInfo;
import com.tencent.rss.common.ShuffleServerInfo;
import com.tencent.rss.common.exception.RssException;
import com.tencent.rss.storage.util.StorageType;
public class RssShuffleWriter<K, V, C> extends ShuffleWriter<K, V> {
private static final Logger LOG = LoggerFactory.getLogger(RssShuffleWriter.class);
private static final String DUMMY_HOST = "dummy_host";
private static final int DUMMY_PORT = 99999;
private final String appId;
private final int shuffleId;
private final WriteBufferManager bufferManager;
private final String taskId;
private final long taskAttemptId;
private final int numMaps;
private final ShuffleDependency<K, V, C> shuffleDependency;
private final ShuffleWriteMetrics shuffleWriteMetrics;
private final Partitioner partitioner;
private final RssShuffleManager shuffleManager;
private final boolean shouldPartition;
private final long sendCheckTimeout;
private final long sendCheckInterval;
private final long sendSizeLimit;
private final int bitmapSplitNum;
private final Map<Integer, Set<Long>> partitionToBlockIds;
private final ShuffleWriteClient shuffleWriteClient;
private final Map<Integer, List<ShuffleServerInfo>> partitionToServers;
private final Set shuffleServersForData;
private final long[] partitionLengths;
private boolean isMemoryShuffleEnabled;
public RssShuffleWriter(
String appId,
int shuffleId,
String taskId,
long taskAttemptId,
WriteBufferManager bufferManager,
ShuffleWriteMetrics shuffleWriteMetrics,
RssShuffleManager shuffleManager,
SparkConf sparkConf,
ShuffleWriteClient shuffleWriteClient,
RssShuffleHandle rssHandle) {
LOG.warn("RssShuffle start write taskAttemptId data" + taskAttemptId);
this.shuffleManager = shuffleManager;
this.appId = appId;
this.bufferManager = bufferManager;
this.shuffleId = shuffleId;
this.taskId = taskId;
this.taskAttemptId = taskAttemptId;
this.numMaps = rssHandle.getNumMaps();
this.shuffleWriteMetrics = shuffleWriteMetrics;
this.shuffleDependency = rssHandle.getDependency();
this.partitioner = shuffleDependency.partitioner();
this.shouldPartition = partitioner.numPartitions() > 1;
this.sendCheckInterval = sparkConf.getLong(RssSparkConfig.RSS_CLIENT_SEND_CHECK_INTERVAL_MS,
RssSparkConfig.RSS_CLIENT_SEND_CHECK_INTERVAL_MS_DEFAULT_VALUE);
this.sendCheckTimeout = sparkConf.getLong(RssSparkConfig.RSS_CLIENT_SEND_CHECK_TIMEOUT_MS,
RssSparkConfig.RSS_CLIENT_SEND_CHECK_TIMEOUT_MS_DEFAULT_VALUE);
this.sendSizeLimit = sparkConf.getSizeAsBytes(RssSparkConfig.RSS_CLIENT_SEND_SIZE_LIMIT,
RssSparkConfig.RSS_CLIENT_SEND_SIZE_LIMIT_DEFAULT_VALUE);
this.bitmapSplitNum = sparkConf.getInt(RssSparkConfig.RSS_CLIENT_BITMAP_SPLIT_NUM,
RssSparkConfig.RSS_CLIENT_BITMAP_SPLIT_NUM_DEFAULT_VALUE);
this.partitionToBlockIds = Maps.newConcurrentMap();
this.shuffleWriteClient = shuffleWriteClient;
this.shuffleServersForData = rssHandle.getShuffleServersForData();
this.partitionLengths = new long[partitioner.numPartitions()];
Arrays.fill(partitionLengths, 0);
partitionToServers = rssHandle.getPartitionToServers();
this.isMemoryShuffleEnabled = isMemoryShuffleEnabled(
sparkConf.get(RssSparkConfig.RSS_STORAGE_TYPE));
}
private boolean isMemoryShuffleEnabled(String storageType) {
return StorageType.MEMORY_LOCALFILE.name().equals(storageType)
|| StorageType.MEMORY_HDFS.name().equals(storageType)
|| StorageType.MEMORY_LOCALFILE_HDFS.name().equals(storageType);
}
@Override
public void write(Iterator<Product2<K, V>> records) throws IOException {
List<ShuffleBlockInfo> shuffleBlockInfos = null;
Set<Long> blockIds = Sets.newConcurrentHashSet();
while (records.hasNext()) {
Product2<K, V> record = records.next();
K key = record._1();
int partition = getPartition(key);
boolean isCombine = shuffleDependency.mapSideCombine();
if (isCombine) {
Function1 createCombiner = shuffleDependency.aggregator().get().createCombiner();
Object c = createCombiner.apply(record._2());
shuffleBlockInfos = bufferManager.addRecord(partition, record._1(), c);
} else {
shuffleBlockInfos = bufferManager.addRecord(partition, record._1(), record._2());
}
if (shuffleBlockInfos != null && !shuffleBlockInfos.isEmpty()) {
processShuffleBlockInfos(shuffleBlockInfos, blockIds);
}
}
final long start = System.currentTimeMillis();
shuffleBlockInfos = bufferManager.clear();
if (shuffleBlockInfos != null && !shuffleBlockInfos.isEmpty()) {
processShuffleBlockInfos(shuffleBlockInfos, blockIds);
}
long checkStartTs = System.currentTimeMillis();
checkBlockSendResult(blockIds);
long commitStartTs = System.currentTimeMillis();
long checkDuration = commitStartTs - checkStartTs;
if (!isMemoryShuffleEnabled) {
sendCommit();
}
long writeDurationMs = bufferManager.getWriteTime() + (System.currentTimeMillis() - start);
shuffleWriteMetrics.incWriteTime(TimeUnit.MILLISECONDS.toNanos(writeDurationMs));
LOG.info("Finish write shuffle for appId[" + appId + "], shuffleId[" + shuffleId
+ "], taskId[" + taskId + "] with write " + writeDurationMs + " ms, include checkSendResult["
+ checkDuration + "], commit[" + (System.currentTimeMillis() - commitStartTs) + "], "
+ bufferManager.getManagerCostInfo());
}
// only push-based shuffle use this interface, but rss won't be used when push-based shuffle is enabled.
public long[] getPartitionLengths() {
return new long[0];
}
private void processShuffleBlockInfos(List<ShuffleBlockInfo> shuffleBlockInfoList, Set<Long> blockIds) {
if (shuffleBlockInfoList != null && !shuffleBlockInfoList.isEmpty()) {
shuffleBlockInfoList.forEach(sbi -> {
long blockId = sbi.getBlockId();
// add blockId to set, check if it is send later
blockIds.add(blockId);
// update [partition, blockIds], it will be sent to shuffle server
int partitionId = sbi.getPartitionId();
partitionToBlockIds.putIfAbsent(partitionId, Sets.newConcurrentHashSet());
partitionToBlockIds.get(partitionId).add(blockId);
partitionLengths[partitionId] += sbi.getLength();
});
postBlockEvent(shuffleBlockInfoList);
}
}
protected void postBlockEvent(List<ShuffleBlockInfo> shuffleBlockInfoList) {
long totalSize = 0;
List<ShuffleBlockInfo> shuffleBlockInfosPerEvent = Lists.newArrayList();
for (ShuffleBlockInfo sbi : shuffleBlockInfoList) {
totalSize += sbi.getSize();
shuffleBlockInfosPerEvent.add(sbi);
// split shuffle data according to the size
if (totalSize > sendSizeLimit) {
LOG.info("Post event to queue with " + shuffleBlockInfosPerEvent.size()
+ " blocks and " + totalSize + " bytes");
shuffleManager.postEvent(
new AddBlockEvent(taskId, shuffleBlockInfosPerEvent));
shuffleBlockInfosPerEvent = Lists.newArrayList();
totalSize = 0;
}
}
if (!shuffleBlockInfosPerEvent.isEmpty()) {
LOG.info("Post event to queue with " + shuffleBlockInfosPerEvent.size()
+ " blocks and " + totalSize + " bytes");
shuffleManager.postEvent(
new AddBlockEvent(taskId, shuffleBlockInfosPerEvent));
}
}
@VisibleForTesting
protected void checkBlockSendResult(Set<Long> blockIds) throws RuntimeException {
long start = System.currentTimeMillis();
while (true) {
Set<Long> successBlockIds = shuffleManager.getSuccessBlockIds(taskId);
Set<Long> failedBlockIds = shuffleManager.getFailedBlockIds(taskId);
if (!failedBlockIds.isEmpty()) {
String errorMsg = "Send failed: Task[" + taskId + "]"
+ " failed because " + failedBlockIds.size()
+ " blocks can't be sent to shuffle server.";
LOG.error(errorMsg);
throw new RssException(errorMsg);
}
blockIds.removeAll(successBlockIds);
if (blockIds.isEmpty()) {
break;
}
LOG.info("Wait " + blockIds.size() + " blocks sent to shuffle server");
Uninterruptibles.sleepUninterruptibly(sendCheckInterval, TimeUnit.MILLISECONDS);
if (System.currentTimeMillis() - start > sendCheckTimeout) {
String errorMsg = "Timeout: Task[" + taskId + "] failed because " + blockIds.size()
+ " blocks can't be sent to shuffle server in " + sendCheckTimeout + " ms.";
LOG.error(errorMsg);
throw new RssException(errorMsg);
}
}
}
@VisibleForTesting
protected void sendCommit() {
ExecutorService executor = Executors.newSingleThreadExecutor();
Future<Boolean> future = executor.submit(
() -> shuffleWriteClient.sendCommit(shuffleServersForData, appId, shuffleId, numMaps));
int maxWait = 5000;
int currentWait = 200;
long start = System.currentTimeMillis();
while (!future.isDone()) {
LOG.info("Wait commit to shuffle server for task[" + taskAttemptId + "] cost "
+ (System.currentTimeMillis() - start) + " ms");
Uninterruptibles.sleepUninterruptibly(currentWait, TimeUnit.MILLISECONDS);
currentWait = Math.min(currentWait * 2, maxWait);
}
try {
if (!future.get()) {
throw new RssException("Failed to commit task to shuffle server");
}
} catch (InterruptedException ie) {
LOG.warn("Ignore the InterruptedException which should be caused by internal killed");
} catch (Exception e) {
throw new RuntimeException("Exception happened when get commit status", e);
} finally {
executor.shutdown();
}
}
@VisibleForTesting
protected <K> int getPartition(K key) {
int result = 0;
if (shouldPartition) {
result = partitioner.getPartition(key);
}
return result;
}
@Override
public Option<MapStatus> stop(boolean success) {
try {
if (success) {
Map<Integer, List<Long>> ptb = Maps.newHashMap();
for (Map.Entry<Integer, Set<Long>> entry : partitionToBlockIds.entrySet()) {
ptb.put(entry.getKey(), Lists.newArrayList(entry.getValue()));
}
long start = System.currentTimeMillis();
shuffleWriteClient.reportShuffleResult(partitionToServers, appId, shuffleId,
taskAttemptId, ptb, bitmapSplitNum);
LOG.info("Report shuffle result for task[{}] with bitmapNum[{}] cost {} ms",
taskAttemptId, bitmapSplitNum, (System.currentTimeMillis() - start));
// todo: we can replace the dummy host and port with the real shuffle server which we prefer to read
final BlockManagerId blockManagerId = BlockManagerId.apply(appId + "_" + taskId,
DUMMY_HOST,
DUMMY_PORT,
Option.apply(Long.toString(taskAttemptId)));
MapStatus mapStatus = MapStatus.apply(blockManagerId, partitionLengths, taskAttemptId);
return Option.apply(mapStatus);
} else {
return Option.empty();
}
} finally {
// free all memory & metadata, or memory leak happen in executor
if (bufferManager != null) {
bufferManager.freeAllMemory();
}
if (shuffleManager != null) {
shuffleManager.clearTaskMeta(taskId);
}
}
}
@VisibleForTesting
Map<Integer, Set<Long>> getPartitionToBlockIds() {
return partitionToBlockIds;
}
}