-
Notifications
You must be signed in to change notification settings - Fork 100
Expand file tree
/
Copy pathUploadServerHandler.java
More file actions
162 lines (133 loc) · 7.1 KB
/
UploadServerHandler.java
File metadata and controls
162 lines (133 loc) · 7.1 KB
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
/*
* 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 com.uber.rss.handlers;
import com.uber.rss.clients.ShuffleWriteConfig;
import com.uber.rss.common.AppMapId;
import com.uber.rss.common.AppShuffleId;
import com.uber.rss.common.AppTaskAttemptId;
import com.uber.rss.exceptions.RssInvalidDataException;
import com.uber.rss.exceptions.RssInvalidStageException;
import com.uber.rss.exceptions.RssInvalidStateException;
import com.uber.rss.exceptions.RssMaxConnectionsException;
import com.uber.rss.execution.ShuffleDataWrapper;
import com.uber.rss.execution.ShuffleExecutor;
import com.uber.rss.util.NettyUtils;
import io.netty.buffer.Unpooled;
import io.netty.channel.ChannelHandlerContext;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.util.concurrent.ConcurrentHashMap;
/***
* This class handles messages for shuffle writer to upload data.
*/
public class UploadServerHandler {
private static final Logger logger = LoggerFactory.getLogger(UploadServerHandler.class);
private final ShuffleExecutor executor;
private final UploadChannelManager channelManager;
private final ConcurrentHashMap<Long, AppMapId> taskAttemptMap = new ConcurrentHashMap<>();
private String connectionInfo;
private int numPartitions;
private ShuffleWriteConfig writeConfig;
private final ConcurrentHashMap<Long, Boolean> taskAttemptUploadStarted = new ConcurrentHashMap<>();
public UploadServerHandler(ShuffleExecutor executor, UploadChannelManager channelManager) {
this.executor = executor;
this.channelManager = channelManager;
channelManager.incNumConnections();
}
public void initializeAppTaskAttempt(AppTaskAttemptId appTaskAttemptId,
int numPartitions,
ShuffleWriteConfig writeConfig,
ChannelHandlerContext ctx) {
initializeAppTaskAttemptImpl(appTaskAttemptId, numPartitions, writeConfig, ctx);
}
private void initializeAppTaskAttemptImpl(AppTaskAttemptId appTaskAttemptId,
int numPartitions,
ShuffleWriteConfig writeConfig,
ChannelHandlerContext ctx) {
this.connectionInfo = NettyUtils.getServerConnectionInfo(ctx.channel());
this.numPartitions = numPartitions;
this.writeConfig = writeConfig;
AppMapId newAppMapIdValue = appTaskAttemptId.getAppMapId();
AppMapId oldAppMapIdValue = this.taskAttemptMap.put(appTaskAttemptId.getTaskAttemptId(), newAppMapIdValue);
if (oldAppMapIdValue != null) {
if (!oldAppMapIdValue.equals(newAppMapIdValue)) {
throw new RssInvalidStateException(String.format(
"There was already value %s with task attempt %s, but trying to set a different value %s",
oldAppMapIdValue, appTaskAttemptId.getTaskAttemptId(), newAppMapIdValue));
}
}
}
public void checkMaxConnections() throws RssMaxConnectionsException {
channelManager.checkMaxConnections();
}
public void updateLiveness(String appId) {
executor.updateLiveness(appId);
}
public void checkAppMaxWriteBytes(String appId) {
executor.checkAppMaxWriteBytes(appId);
}
public void onChannelInactive() {
channelManager.decNumConnections();
}
public void writeRecord(com.uber.rss.messages.ShuffleDataWrapper shuffleDataWrapper) {
AppMapId appMapId = getAppMapId(shuffleDataWrapper.getTaskAttemptId());
lazyStartUpload(new AppTaskAttemptId(appMapId, shuffleDataWrapper.getTaskAttemptId()));
if (shuffleDataWrapper.getPartitionId() < 0 || shuffleDataWrapper.getPartitionId() > numPartitions) {
throw new RssInvalidDataException(String.format("Invalid partition: %s, %s",
shuffleDataWrapper.getPartitionId(), connectionInfo));
}
executor.writeData(new ShuffleDataWrapper(
appMapId.getAppShuffleId(), appMapId.getMapId(), shuffleDataWrapper.getTaskAttemptId(),
shuffleDataWrapper.getPartitionId(), Unpooled.wrappedBuffer(shuffleDataWrapper.getBytes())));
}
public void finishUpload(long taskAttemptId) {
AppMapId appMapId = getAppMapId(taskAttemptId);
AppTaskAttemptId appTaskAttemptIdToFinishUpload = new AppTaskAttemptId(appMapId, taskAttemptId);
finishUploadImpl(appTaskAttemptIdToFinishUpload);
}
public void registerStageId(int stageId, AppShuffleId appShuffleId) {
// stageId default is -1. this would only occur in cases if method not called for StartUploadMessage or error occurred
if (stageId != -1) {
executor.registerShuffleId(stageId, appShuffleId);
} else {
String error = String.format("registerStageId called not using StartUploadMessage or stageId never set for shuffle=%s", appShuffleId);
logger.error(error);
throw new RssInvalidStageException(error);
}
}
private void finishUploadImpl(AppTaskAttemptId appTaskAttemptIdToFinishUpload) {
lazyStartUpload(appTaskAttemptIdToFinishUpload);
executor.finishUpload(appTaskAttemptIdToFinishUpload.getAppShuffleId(),
appTaskAttemptIdToFinishUpload.getTaskAttemptId());
taskAttemptMap.remove(appTaskAttemptIdToFinishUpload.getTaskAttemptId());
taskAttemptUploadStarted.remove(appTaskAttemptIdToFinishUpload.getTaskAttemptId());
}
// lazy initialize on executor when only upload the first record, so same map task
// could retry connecting to the server without really start the upload
private void lazyStartUpload(AppTaskAttemptId appTaskAttemptIdToStartUpload) {
if (!taskAttemptUploadStarted.getOrDefault(appTaskAttemptIdToStartUpload.getTaskAttemptId(), false)) {
executor.registerShuffle(appTaskAttemptIdToStartUpload.getAppShuffleId(), numPartitions, writeConfig);
executor.startUpload(appTaskAttemptIdToStartUpload);
taskAttemptUploadStarted.put(appTaskAttemptIdToStartUpload.getTaskAttemptId(), true);
}
}
private AppMapId getAppMapId(long taskAttemptId) {
AppMapId appMapId = taskAttemptMap.get(taskAttemptId);
if (appMapId == null) {
throw new RssInvalidStateException(String.format("Did not get app map id for task attempt %s, %s",
taskAttemptId, connectionInfo));
}
return appMapId;
}
}