-
Notifications
You must be signed in to change notification settings - Fork 8.8k
/
OzoneManagerRatisServer.java
277 lines (243 loc) · 10.5 KB
/
OzoneManagerRatisServer.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
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you 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.hadoop.ozone.om.ratis;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Strings;
import java.io.File;
import java.io.IOException;
import java.net.InetSocketAddress;
import java.net.ServerSocket;
import java.net.SocketAddress;
import java.util.Collections;
import java.util.Objects;
import java.util.concurrent.TimeUnit;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.conf.StorageUnit;
import org.apache.hadoop.hdds.scm.HddsServerUtil;
import org.apache.hadoop.ozone.om.OMConfigKeys;
import org.apache.ratis.RaftConfigKeys;
import org.apache.ratis.client.RaftClientConfigKeys;
import org.apache.ratis.conf.RaftProperties;
import org.apache.ratis.grpc.GrpcConfigKeys;
import org.apache.ratis.netty.NettyConfigKeys;
import org.apache.ratis.protocol.RaftGroupId;
import org.apache.ratis.protocol.RaftPeerId;
import org.apache.ratis.rpc.RpcType;
import org.apache.ratis.rpc.SupportedRpcType;
import org.apache.ratis.server.RaftServer;
import org.apache.ratis.server.RaftServerConfigKeys;
import org.apache.ratis.statemachine.impl.BaseStateMachine;
import org.apache.ratis.util.LifeCycle;
import org.apache.ratis.util.SizeInBytes;
import org.apache.ratis.util.TimeDuration;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* Creates a Ratis server endpoint for OM.
*/
public final class OzoneManagerRatisServer {
private static final Logger LOG = LoggerFactory
.getLogger(OzoneManagerRatisServer.class);
private final int port;
private final RaftServer server;
private OzoneManagerRatisServer(String omId, int port, Configuration conf)
throws IOException {
Objects.requireNonNull(omId, "omId == null");
this.port = port;
RaftProperties serverProperties = newRaftProperties(conf);
this.server = RaftServer.newBuilder()
.setServerId(RaftPeerId.valueOf(omId))
.setProperties(serverProperties)
.setStateMachineRegistry(this::getStateMachine)
.build();
}
public static OzoneManagerRatisServer newOMRatisServer(String omId,
Configuration ozoneConf) throws IOException {
int localPort = ozoneConf.getInt(
OMConfigKeys.OZONE_OM_RATIS_PORT_KEY,
OMConfigKeys.OZONE_OM_RATIS_PORT_DEFAULT);
// Get an available port on current node and
// use that as the container port
if (ozoneConf.getBoolean(
OMConfigKeys.OZONE_OM_RATIS_RANDOM_PORT_KEY,
OMConfigKeys.OZONE_OM_RATIS_RANDOM_PORT_KEY_DEFAULT)) {
try (ServerSocket socket = new ServerSocket()) {
socket.setReuseAddress(true);
SocketAddress address = new InetSocketAddress(0);
socket.bind(address);
localPort = socket.getLocalPort();
LOG.info("Found a free port for the OM Ratis server : {}", localPort);
} catch (IOException e) {
LOG.error("Unable find a random free port for the server, "
+ "fallback to use default port {}", localPort, e);
}
}
return new OzoneManagerRatisServer(omId, localPort, ozoneConf);
}
/**
* Return a dummy StateMachine.
* TODO: Implement a state machine on OM.
*/
private BaseStateMachine getStateMachine(RaftGroupId gid) {
return new BaseStateMachine();
}
public void start() throws IOException {
LOG.info("Starting {} {} at port {}", getClass().getSimpleName(),
server.getId(), port);
server.start();
}
public void stop() {
try {
server.close();
} catch (IOException e) {
throw new RuntimeException(e);
}
}
private RaftProperties newRaftProperties(Configuration conf) {
final RaftProperties properties = new RaftProperties();
// Set RPC type
final String rpcType = conf.get(
OMConfigKeys.OZONE_OM_RATIS_RPC_TYPE_KEY,
OMConfigKeys.OZONE_OM_RATIS_RPC_TYPE_DEFAULT);
final RpcType rpc = SupportedRpcType.valueOfIgnoreCase(rpcType);
RaftConfigKeys.Rpc.setType(properties, rpc);
// Set the ratis port number
if (rpc == SupportedRpcType.GRPC) {
GrpcConfigKeys.Server.setPort(properties, port);
} else if (rpc == SupportedRpcType.NETTY) {
NettyConfigKeys.Server.setPort(properties, port);
}
// Set Ratis storage directory
String storageDir = getOMRatisDirectory(conf);
RaftServerConfigKeys.setStorageDirs(properties,
Collections.singletonList(new File(storageDir)));
// Set RAFT segment size
final int raftSegmentSize = (int) conf.getStorageSize(
OMConfigKeys.OZONE_OM_RATIS_SEGMENT_SIZE_KEY,
OMConfigKeys.OZONE_OM_RATIS_SEGMENT_SIZE_DEFAULT,
StorageUnit.BYTES);
RaftServerConfigKeys.Log.setSegmentSizeMax(properties,
SizeInBytes.valueOf(raftSegmentSize));
// Set RAFT segment pre-allocated size
final int raftSegmentPreallocatedSize = (int) conf.getStorageSize(
OMConfigKeys.OZONE_OM_RATIS_SEGMENT_PREALLOCATED_SIZE_KEY,
OMConfigKeys.OZONE_OM_RATIS_SEGMENT_PREALLOCATED_SIZE_DEFAULT,
StorageUnit.BYTES);
int logAppenderQueueNumElements = conf.getInt(
OMConfigKeys.OZONE_OM_RATIS_LOG_APPENDER_QUEUE_NUM_ELEMENTS,
OMConfigKeys.OZONE_OM_RATIS_LOG_APPENDER_QUEUE_NUM_ELEMENTS_DEFAULT);
final int logAppenderQueueByteLimit = (int) conf.getStorageSize(
OMConfigKeys.OZONE_OM_RATIS_LOG_APPENDER_QUEUE_BYTE_LIMIT,
OMConfigKeys.OZONE_OM_RATIS_LOG_APPENDER_QUEUE_BYTE_LIMIT_DEFAULT,
StorageUnit.BYTES);
RaftServerConfigKeys.Log.Appender
.setBufferElementLimit(properties, logAppenderQueueNumElements);
RaftServerConfigKeys.Log.Appender.setBufferByteLimit(properties,
SizeInBytes.valueOf(logAppenderQueueByteLimit));
RaftServerConfigKeys.Log.setPreallocatedSize(properties,
SizeInBytes.valueOf(raftSegmentPreallocatedSize));
// For grpc set the maximum message size
// TODO: calculate the max message size based on the max size of a
// PutSmallFileRequest's file size limit
GrpcConfigKeys.setMessageSizeMax(properties,
SizeInBytes.valueOf(logAppenderQueueByteLimit));
// Set the server request timeout
TimeUnit serverRequestTimeoutUnit =
OMConfigKeys.OZONE_OM_RATIS_SERVER_REQUEST_TIMEOUT_DEFAULT.getUnit();
long serverRequestTimeoutDuration = conf.getTimeDuration(
OMConfigKeys.OZONE_OM_RATIS_SERVER_REQUEST_TIMEOUT_KEY,
OMConfigKeys.OZONE_OM_RATIS_SERVER_REQUEST_TIMEOUT_DEFAULT
.getDuration(), serverRequestTimeoutUnit);
final TimeDuration serverRequestTimeout = TimeDuration.valueOf(
serverRequestTimeoutDuration, serverRequestTimeoutUnit);
RaftServerConfigKeys.Rpc.setRequestTimeout(properties,
serverRequestTimeout);
// Set timeout for server retry cache entry
TimeUnit retryCacheTimeoutUnit = OMConfigKeys
.OZONE_OM_RATIS_SERVER_RETRY_CACHE_TIMEOUT_DEFAULT.getUnit();
long retryCacheTimeoutDuration = conf.getTimeDuration(
OMConfigKeys.OZONE_OM_RATIS_SERVER_RETRY_CACHE_TIMEOUT_KEY,
OMConfigKeys.OZONE_OM_RATIS_SERVER_RETRY_CACHE_TIMEOUT_DEFAULT
.getDuration(), retryCacheTimeoutUnit);
final TimeDuration retryCacheTimeout = TimeDuration.valueOf(
retryCacheTimeoutDuration, retryCacheTimeoutUnit);
RaftServerConfigKeys.RetryCache
.setExpiryTime(properties, retryCacheTimeout);
// Set the server min and max timeout
TimeUnit serverMinTimeoutUnit =
OMConfigKeys.OZONE_OM_RATIS_MINIMUM_TIMEOUT_DEFAULT.getUnit();
long serverMinTimeoutDuration = conf.getTimeDuration(
OMConfigKeys.OZONE_OM_RATIS_MINIMUM_TIMEOUT_KEY,
OMConfigKeys.OZONE_OM_RATIS_MINIMUM_TIMEOUT_DEFAULT
.getDuration(), serverMinTimeoutUnit);
final TimeDuration serverMinTimeout = TimeDuration.valueOf(
serverMinTimeoutDuration, serverMinTimeoutUnit);
long serverMaxTimeoutDuration =
serverMinTimeout.toLong(TimeUnit.MILLISECONDS) + 200;
final TimeDuration serverMaxTimeout = TimeDuration.valueOf(
serverMaxTimeoutDuration, serverMinTimeoutUnit);
RaftServerConfigKeys.Rpc.setTimeoutMin(properties,
serverMinTimeout);
RaftServerConfigKeys.Rpc.setTimeoutMax(properties,
serverMaxTimeout);
// Set the number of maximum cached segments
RaftServerConfigKeys.Log.setMaxCachedSegmentNum(properties, 2);
// Set the client request timeout
TimeUnit clientRequestTimeoutUnit =
OMConfigKeys.OZONE_OM_RATIS_CLIENT_REQUEST_TIMEOUT_DEFAULT.getUnit();
long clientRequestTimeoutDuration = conf.getTimeDuration(
OMConfigKeys.OZONE_OM_RATIS_CLIENT_REQUEST_TIMEOUT_KEY,
OMConfigKeys.OZONE_OM_RATIS_CLIENT_REQUEST_TIMEOUT_DEFAULT
.getDuration(), clientRequestTimeoutUnit);
final TimeDuration clientRequestTimeout = TimeDuration.valueOf(
clientRequestTimeoutDuration, clientRequestTimeoutUnit);
RaftClientConfigKeys.Rpc.setRequestTimeout(properties,
clientRequestTimeout);
// TODO: set max write buffer size
/**
* TODO: when state machine is implemented, enable StateMachineData sync
* and set sync timeout and number of sync retries.
*/
/**
* TODO: set following ratis leader election related configs when
* replicated ratis server is implemented.
* 1. leader election timeout
* 2. node failure timeout
* 3.
*/
/**
* TODO: when ratis snapshots are implemented, set snapshot threshold and
* queue size.
*/
return properties;
}
public int getServerPort() {
return port;
}
@VisibleForTesting
public LifeCycle.State getServerState() {
return server.getLifeCycleState();
}
public static String getOMRatisDirectory(Configuration conf) {
String storageDir = conf.get(OMConfigKeys.OZONE_OM_RATIS_STORAGE_DIR);
if (Strings.isNullOrEmpty(storageDir)) {
storageDir = HddsServerUtil.getDefaultRatisDirectory(conf);
}
return storageDir;
}
}