-
Notifications
You must be signed in to change notification settings - Fork 100
Expand file tree
/
Copy pathStreamServerMessageDecoder.java
More file actions
388 lines (359 loc) · 15.3 KB
/
StreamServerMessageDecoder.java
File metadata and controls
388 lines (359 loc) · 15.3 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
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
/*
* This file is copied from Uber Remote Shuffle Service
* (https://github.com/uber/RemoteShuffleService) and modified.
*
* 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.decoders;
import com.uber.rss.common.DataBlockHeader;
import com.uber.rss.exceptions.RssException;
import com.uber.rss.exceptions.RssInvalidDataException;
import com.uber.rss.messages.*;
import com.uber.rss.metrics.M3Stats;
import com.uber.rss.metrics.NettyServerSideMetricGroupContainer;
import com.uber.rss.metrics.ServerHandlerMetrics;
import com.uber.rss.util.ByteBufUtils;
import com.uber.rss.util.LogUtils;
import com.uber.rss.util.NettyUtils;
import io.netty.buffer.ByteBuf;
import io.netty.buffer.ByteBufAllocator;
import io.netty.buffer.CompositeByteBuf;
import io.netty.buffer.PooledByteBufAllocator;
import io.netty.buffer.Unpooled;
import io.netty.buffer.UnpooledByteBufAllocator;
import io.netty.channel.ChannelHandlerContext;
import io.netty.handler.codec.ByteToMessageDecoder;
import com.uber.rss.messages.*;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.util.List;
/***
* This class is per client server connection, and contains state (data) per that connection.
* This decoder is shared in the stream server side to decode message for upload server, download server
* and control server. It will be only one of these roles based on the header message.
*/
public class StreamServerMessageDecoder extends ByteToMessageDecoder {
private static final Logger logger = LoggerFactory.getLogger(StreamServerMessageDecoder.class);
private static final int INVALID_CONTROL_MESSAGE_TYPE = 0;
private static final int INVALID_PARTITION_ID = -1;
private static final int INVALID_SESSION_ID = -1;
private static final int INVALID_TASK_ATTEMPT_ID = -1;
private static NettyServerSideMetricGroupContainer<ServerHandlerMetrics> metricGroupContainer =
new NettyServerSideMetricGroupContainer<>(ServerHandlerMetrics::new);
private enum State {
READ_MAGIC_BYTE_AND_VERSION,
READ_MESSAGE_TYPE,
READ_CONTROL_MESSAGE_LEN,
READ_CONTROL_MESSAGE_BYTES,
READ_TASK_ATTEMPT_ID,
READ_DATA_MESSAGE_LEN,
READ_DATA_MESSAGE_BYTES,
}
private State state = State.READ_MAGIC_BYTE_AND_VERSION;
private int requiredBytes = 0;
private final ByteBuf shuffleDataBuffer;
private int controlMessageType = INVALID_CONTROL_MESSAGE_TYPE;
private int partitionId = INVALID_PARTITION_ID;
private long taskAttemptId = INVALID_TASK_ATTEMPT_ID;
// store bytes for taskAttemptId to speed up serialization in DataBlockHeader.serializeToBytes
private final byte[] taskAttemptIdBytes = new byte[Long.BYTES];
private long startTime = System.currentTimeMillis();
private long numIncomingBytes = 0;
private String user = "uninitialized";
private ServerHandlerMetrics metrics = metricGroupContainer.getMetricGroup(user);
public StreamServerMessageDecoder(ByteBuf shuffleDataBuffer) {
super();
this.shuffleDataBuffer = shuffleDataBuffer;
}
@Override
public void channelInactive(ChannelHandlerContext ctx) throws Exception {
super.channelInactive(ctx);
if (shuffleDataBuffer != null) {
shuffleDataBuffer.release();
}
metricGroupContainer.removeMetricGroup(user);
String connectionInfo = NettyUtils.getServerConnectionInfo(ctx);
double dataSpeed = LogUtils
.calculateMegaBytesPerSecond(System.currentTimeMillis() - startTime, numIncomingBytes);
logger
.debug("Decoder finished, total bytes: {}, speed: {} mbs, {}", numIncomingBytes, dataSpeed,
connectionInfo);
}
@Override
protected void decode(ChannelHandlerContext ctx,
ByteBuf in,
List<Object> out) throws Exception {
int readableBytes = in.readableBytes();
try {
decodeImpl(ctx, in, out);
} finally {
int numReadBytes = readableBytes - in.readableBytes();
numIncomingBytes += numReadBytes;
metrics.getNumIncomingBytes().inc(numReadBytes);
}
}
private void decodeImpl(ChannelHandlerContext ctx,
ByteBuf in,
List<Object> out) {
metrics.getNumIncomingRequests().inc(1);
if (in.readableBytes() == 0) {
return;
}
switch (state) {
case READ_MAGIC_BYTE_AND_VERSION:
if (in.readableBytes() < 2 * Byte.BYTES) {
return;
}
byte magicByte = in.readByte();
byte version;
switch (magicByte) {
case MessageConstants.UPLOAD_UPLINK_MAGIC_BYTE:
version = in.readByte();
if (version != MessageConstants.UPLOAD_UPLINK_VERSION_3) {
String clientInfo = NettyUtils.getServerConnectionInfo(ctx);
logger.warn(
"Invalid notify version {} from client {}",
version, clientInfo);
ctx.close();
logger.debug("Closed connection to client {}", clientInfo);
return;
}
state = State.READ_MESSAGE_TYPE;
return;
case MessageConstants.DOWNLOAD_UPLINK_MAGIC_BYTE:
version = in.readByte();
if (version != MessageConstants.DOWNLOAD_UPLINK_VERSION_3) {
String clientInfo = NettyUtils.getServerConnectionInfo(ctx);
logger.warn(
"Invalid download version {} from client {}",
version, clientInfo);
ctx.close();
logger.debug("Closed connection to client {}", clientInfo);
return;
}
state = State.READ_MESSAGE_TYPE;
return;
case MessageConstants.NOTIFY_UPLINK_MAGIC_BYTE:
version = in.readByte();
if (version != MessageConstants.NOTIFY_UPLINK_VERSION_3) {
String clientInfo = NettyUtils.getServerConnectionInfo(ctx);
logger.warn(
"Invalid control version {} from client {}",
version, clientInfo);
ctx.close();
logger.debug("Closed connection to client {}", clientInfo);
return;
}
state = State.READ_MESSAGE_TYPE;
return;
case MessageConstants.REGISTRY_UPLINK_MAGIC_BYTE:
version = in.readByte();
if (version != MessageConstants.REGISTRY_UPLINK_VERSION_3) {
String clientInfo = NettyUtils.getServerConnectionInfo(ctx);
logger.warn(
"Invalid registry version {} from client {}",
version, clientInfo);
ctx.close();
logger.debug("Closed connection to client {}", clientInfo);
return;
}
state = State.READ_MESSAGE_TYPE;
return;
default:
String clientInfo = NettyUtils.getServerConnectionInfo(ctx);
logger.warn(
"Invalid magic byte {} from client {}",
magicByte, clientInfo);
ctx.close();
logger.debug("Closed connection to client {}", clientInfo);
return;
}
case READ_MESSAGE_TYPE:
if (in.readableBytes() < Integer.BYTES) {
return;
}
int messageType = in.readInt();
if (messageType < 0) {
controlMessageType = messageType;
state = State.READ_CONTROL_MESSAGE_LEN;
} else {
partitionId = messageType;
state = State.READ_TASK_ATTEMPT_ID;
}
return;
case READ_CONTROL_MESSAGE_LEN:
if (in.readableBytes() < Integer.BYTES) {
return;
}
requiredBytes = in.readInt();
if (requiredBytes < 0) {
throw new RssInvalidDataException(String.format(
"Invalid control message length: %s, %s",
requiredBytes, NettyUtils.getServerConnectionInfo(ctx)));
}
if (requiredBytes == 0) {
Object controlMessage = getControlMessage(ctx, controlMessageType, in);
out.add(controlMessage);
resetData();
state = State.READ_MESSAGE_TYPE;
} else {
state = State.READ_CONTROL_MESSAGE_BYTES;
}
return;
case READ_CONTROL_MESSAGE_BYTES:
if (in.readableBytes() < requiredBytes) {
return;
}
Object controlMessage = getControlMessage(ctx, controlMessageType, in);
out.add(controlMessage);
resetData();
state = State.READ_MESSAGE_TYPE;
return;
case READ_TASK_ATTEMPT_ID:
if (in.readableBytes() < Long.BYTES) {
return;
}
in.readBytes(taskAttemptIdBytes);
taskAttemptId = ByteBufUtils.readLong(taskAttemptIdBytes, 0);
if (taskAttemptId < 0) {
throw new RssInvalidDataException(String.format(
"Invalid task attempt id: %s, %s",
taskAttemptId, NettyUtils.getServerConnectionInfo(ctx)));
}
state = State.READ_DATA_MESSAGE_LEN;
return;
case READ_DATA_MESSAGE_LEN:
if (in.readableBytes() < Integer.BYTES) {
return;
}
int dataLen = in.readInt();
if (dataLen < 0) {
throw new RssInvalidDataException(String.format(
"Invalid data length: %s, %s",
dataLen, NettyUtils.getServerConnectionInfo(ctx)));
}
if (dataLen == 0) {
out.add(createShuffleDataWrapper(in, 0));
resetData();
requiredBytes = 0;
state = State.READ_MESSAGE_TYPE;
} else {
requiredBytes = dataLen;
state = State.READ_DATA_MESSAGE_BYTES;
shuffleDataBuffer.clear();
}
return;
case READ_DATA_MESSAGE_BYTES:
if (in.readableBytes() < requiredBytes) {
int count = in.readableBytes();
shuffleDataBuffer.ensureWritable(count);
in.readBytes(shuffleDataBuffer, count);
requiredBytes -= count;
} else {
shuffleDataBuffer.ensureWritable(requiredBytes);
in.readBytes(shuffleDataBuffer, requiredBytes);
out.add(createShuffleDataWrapper(shuffleDataBuffer, shuffleDataBuffer.readableBytes()));
requiredBytes = 0;
resetData();
state = State.READ_MESSAGE_TYPE;
}
return;
default:
throw new RssException(String.format(
"Should not get incoming data in state %s, client %s",
state, NettyUtils.getServerConnectionInfo(ctx)));
}
}
@Override
public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) throws Exception {
M3Stats.addException(cause, this.getClass().getSimpleName());
String connectionInfo = NettyUtils.getServerConnectionInfo(ctx);
String msg = "Got exception " + connectionInfo;
logger.warn(msg, cause);
ctx.close();
}
private Object getControlMessage(ChannelHandlerContext ctx,
int controlMessageType,
ByteBuf in) {
switch (controlMessageType) {
case MessageConstants.MESSAGE_ConnectUploadRequest:
ConnectUploadRequest connectUploadRequest = ConnectUploadRequest.deserialize(in);
metricGroupContainer.removeMetricGroup(user);
user = connectUploadRequest.getUser();
metrics = metricGroupContainer.getMetricGroup(user);
return connectUploadRequest;
case MessageConstants.MESSAGE_ConnectUploadResponse:
return ConnectUploadResponse.deserialize(in);
case MessageConstants.MESSAGE_StartUploadMessage:
return StartUploadMessage.deserialize(in);
case MessageConstants.MESSAGE_FinishUploadMessage:
return FinishUploadMessage.deserialize(in);
case MessageConstants.MESSAGE_HeartbeatMessage:
HeartbeatMessage heartbeatMessage = HeartbeatMessage.deserialize(in);
return heartbeatMessage;
case MessageConstants.MESSAGE_GetBusyStatusRequest:
GetBusyStatusRequest getBusyStatusRequest = GetBusyStatusRequest.deserialize(in);
return getBusyStatusRequest;
case MessageConstants.MESSAGE_GetBusyStatusResponse:
GetBusyStatusResponse getBusyStatusResponse = GetBusyStatusResponse.deserialize(in);
return getBusyStatusResponse;
case MessageConstants.MESSAGE_ConnectDownloadRequest:
ConnectDownloadRequest connectDownloadRequest = ConnectDownloadRequest.deserialize(in);
metricGroupContainer.removeMetricGroup(user);
user = connectDownloadRequest.getUser();
metrics = metricGroupContainer.getMetricGroup(user);
return connectDownloadRequest;
case MessageConstants.MESSAGE_ConnectDownloadResponse:
return ConnectDownloadResponse.deserialize(in);
case MessageConstants.MESSAGE_GetDataAvailabilityRequest:
return GetDataAvailabilityRequest.deserialize(in);
case MessageConstants.MESSAGE_GetDataAvailabilityResponse:
return GetDataAvailabilityResponse.deserialize(in);
case MessageConstants.MESSAGE_ConnectNotifyRequest:
return ConnectNotifyRequest.deserialize(in);
case MessageConstants.MESSAGE_ConnectNotifyResponse:
return ConnectNotifyResponse.deserialize(in);
case MessageConstants.MESSAGE_FinishApplicationJobRequest:
return FinishApplicationJobRequestMessage.deserialize(in);
case MessageConstants.MESSAGE_FinishApplicationAttemptRequest:
return FinishApplicationAttemptRequestMessage.deserialize(in);
case MessageConstants.MESSAGE_ConnectRegistryRequest:
return ConnectRegistryRequest.deserialize(in);
case MessageConstants.MESSAGE_ConnectRegistryResponse:
return ConnectRegistryResponse.deserialize(in);
case MessageConstants.MESSAGE_RegisterServerRequest:
return RegisterServerRequestMessage.deserialize(in);
case MessageConstants.MESSAGE_GetServersRequest:
return GetServersRequestMessage.deserialize(in);
default:
throw new RssException(String.format(
"Unsupported control message type %s from client %s",
controlMessageType,
NettyUtils.getServerConnectionInfo(ctx)));
}
}
private ShuffleDataWrapper createShuffleDataWrapper(ByteBuf in, int byteCount) {
metrics.getNumIncomingBlocks().inc(1);
// byte[] headerBytes = DataBlockHeader.serializeToBytes(taskAttemptIdBytes, byteCount);
// ByteBuf buffer = ByteBufAllocator.DEFAULT.buffer(headerBytes.length + byteCount);
// buffer.writeBytes(headerBytes);
// in.readBytes(buffer, headerBytes.length, byteCount);
CompositeByteBuf compositeBuffer = ByteBufAllocator.DEFAULT.compositeBuffer();
compositeBuffer.addComponent(true, DataBlockHeader.serializeToBuf(taskAttemptIdBytes, byteCount));
compositeBuffer.addComponent(true, in.readRetainedSlice(byteCount));
return new ShuffleDataWrapper(partitionId, taskAttemptId, compositeBuffer);
}
private void resetData() {
controlMessageType = INVALID_CONTROL_MESSAGE_TYPE;
partitionId = INVALID_PARTITION_ID;
taskAttemptId = INVALID_SESSION_ID;
}
}