-
Notifications
You must be signed in to change notification settings - Fork 475
/
DatanodeStateMachine.java
507 lines (460 loc) · 16.2 KB
/
DatanodeStateMachine.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
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
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
/**
* 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.container.common.statemachine;
import java.io.Closeable;
import java.io.IOException;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicLong;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.PipelineReportsProto;
import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.CommandStatusReportsProto;
import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.ContainerReportsProto;
import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.NodeReportProto;
import org.apache.hadoop.hdds.security.x509.certificate.client.CertificateClient;
import org.apache.hadoop.ozone.HddsDatanodeStopService;
import org.apache.hadoop.ozone.container.common.report.ReportManager;
import org.apache.hadoop.ozone.container.common.statemachine.commandhandler
.CloseContainerCommandHandler;
import org.apache.hadoop.ozone.container.common.statemachine.commandhandler
.ClosePipelineCommandHandler;
import org.apache.hadoop.ozone.container.common.statemachine.commandhandler
.CommandDispatcher;
import org.apache.hadoop.ozone.container.common.statemachine.commandhandler
.CreatePipelineCommandHandler;
import org.apache.hadoop.ozone.container.common.statemachine.commandhandler
.DeleteBlocksCommandHandler;
import org.apache.hadoop.ozone.container.common.statemachine.commandhandler
.DeleteContainerCommandHandler;
import org.apache.hadoop.ozone.container.common.statemachine.commandhandler
.ReplicateContainerCommandHandler;
import org.apache.hadoop.ozone.container.keyvalue.TarContainerPacker;
import org.apache.hadoop.ozone.container.ozoneimpl.OzoneContainer;
import org.apache.hadoop.ozone.container.replication.ContainerReplicator;
import org.apache.hadoop.ozone.container.replication.DownloadAndImportReplicator;
import org.apache.hadoop.ozone.container.replication.ReplicationSupervisor;
import org.apache.hadoop.ozone.container.replication.SimpleContainerDownloader;
import org.apache.hadoop.ozone.protocol.commands.SCMCommand;
import org.apache.hadoop.util.JvmPauseMonitor;
import org.apache.hadoop.util.Time;
import org.apache.hadoop.util.concurrent.HadoopExecutors;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.util.concurrent.ThreadFactoryBuilder;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* State Machine Class.
*/
public class DatanodeStateMachine implements Closeable {
@VisibleForTesting
static final Logger LOG =
LoggerFactory.getLogger(DatanodeStateMachine.class);
private final ExecutorService executorService;
private final Configuration conf;
private final SCMConnectionManager connectionManager;
private StateContext context;
private final OzoneContainer container;
private DatanodeDetails datanodeDetails;
private final CommandDispatcher commandDispatcher;
private final ReportManager reportManager;
private long commandsHandled;
private AtomicLong nextHB;
private Thread stateMachineThread = null;
private Thread cmdProcessThread = null;
private final ReplicationSupervisor supervisor;
private JvmPauseMonitor jvmPauseMonitor;
private CertificateClient dnCertClient;
private final HddsDatanodeStopService hddsDatanodeStopService;
/**
* Constructs a a datanode state machine.
* @param datanodeDetails - DatanodeDetails used to identify a datanode
* @param conf - Configuration.
* @param certClient - Datanode Certificate client, required if security is
* enabled
*/
public DatanodeStateMachine(DatanodeDetails datanodeDetails,
Configuration conf, CertificateClient certClient,
HddsDatanodeStopService hddsDatanodeStopService) throws IOException {
OzoneConfiguration ozoneConf = new OzoneConfiguration(conf);
DatanodeConfiguration dnConf =
ozoneConf.getObject(DatanodeConfiguration.class);
this.hddsDatanodeStopService = hddsDatanodeStopService;
this.conf = conf;
this.datanodeDetails = datanodeDetails;
executorService = HadoopExecutors.newCachedThreadPool(
new ThreadFactoryBuilder().setDaemon(true)
.setNameFormat("Datanode State Machine Thread - %d").build());
connectionManager = new SCMConnectionManager(conf);
context = new StateContext(this.conf, DatanodeStates.getInitState(), this);
container = new OzoneContainer(this.datanodeDetails,
ozoneConf, context, certClient);
dnCertClient = certClient;
nextHB = new AtomicLong(Time.monotonicNow());
ContainerReplicator replicator =
new DownloadAndImportReplicator(container.getContainerSet(),
container.getController(),
new SimpleContainerDownloader(conf), new TarContainerPacker());
supervisor =
new ReplicationSupervisor(container.getContainerSet(), replicator,
dnConf.getReplicationMaxStreams());
// When we add new handlers just adding a new handler here should do the
// trick.
commandDispatcher = CommandDispatcher.newBuilder()
.addHandler(new CloseContainerCommandHandler())
.addHandler(new DeleteBlocksCommandHandler(container.getContainerSet(),
conf))
.addHandler(new ReplicateContainerCommandHandler(conf, supervisor))
.addHandler(new DeleteContainerCommandHandler(
dnConf.getContainerDeleteThreads()))
.addHandler(new ClosePipelineCommandHandler())
.addHandler(new CreatePipelineCommandHandler(conf))
.setConnectionManager(connectionManager)
.setContainer(container)
.setContext(context)
.build();
reportManager = ReportManager.newBuilder(conf)
.setStateContext(context)
.addPublisherFor(NodeReportProto.class)
.addPublisherFor(ContainerReportsProto.class)
.addPublisherFor(CommandStatusReportsProto.class)
.addPublisherFor(PipelineReportsProto.class)
.build();
}
/**
*
* Return DatanodeDetails if set, return null otherwise.
*
* @return DatanodeDetails
*/
public DatanodeDetails getDatanodeDetails() {
return datanodeDetails;
}
/**
* Returns the Connection manager for this state machine.
*
* @return - SCMConnectionManager.
*/
public SCMConnectionManager getConnectionManager() {
return connectionManager;
}
public OzoneContainer getContainer() {
return this.container;
}
/**
* Runs the state machine at a fixed frequency.
*/
private void start() throws IOException {
long now = 0;
reportManager.init();
initCommandHandlerThread(conf);
// Start jvm monitor
jvmPauseMonitor = new JvmPauseMonitor();
jvmPauseMonitor.init(conf);
jvmPauseMonitor.start();
while (context.getState() != DatanodeStates.SHUTDOWN) {
try {
LOG.debug("Executing cycle Number : {}", context.getExecutionCount());
long heartbeatFrequency = context.getHeartbeatFrequency();
nextHB.set(Time.monotonicNow() + heartbeatFrequency);
context.execute(executorService, heartbeatFrequency,
TimeUnit.MILLISECONDS);
now = Time.monotonicNow();
if (now < nextHB.get()) {
if(!Thread.interrupted()) {
Thread.sleep(nextHB.get() - now);
}
}
} catch (InterruptedException e) {
// Some one has sent interrupt signal, this could be because
// 1. Trigger heartbeat immediately
// 2. Shutdown has be initiated.
} catch (Exception e) {
LOG.error("Unable to finish the execution.", e);
}
}
// If we have got some exception in stateMachine we set the state to
// shutdown to stop the stateMachine thread. Along with this we should
// also stop the datanode.
if (context.getShutdownOnError()) {
LOG.error("DatanodeStateMachine Shutdown due to an critical error");
hddsDatanodeStopService.stopService();
}
}
/**
* Gets the current context.
*
* @return StateContext
*/
public StateContext getContext() {
return context;
}
/**
* Sets the current context.
*
* @param context - Context
*/
public void setContext(StateContext context) {
this.context = context;
}
/**
* Closes this stream and releases any system resources associated with it. If
* the stream is already closed then invoking this method has no effect.
* <p>
* <p> As noted in {@link AutoCloseable#close()}, cases where the close may
* fail require careful attention. It is strongly advised to relinquish the
* underlying resources and to internally <em>mark</em> the {@code Closeable}
* as closed, prior to throwing the {@code IOException}.
*
* @throws IOException if an I/O error occurs
*/
@Override
public void close() throws IOException {
if (stateMachineThread != null) {
stateMachineThread.interrupt();
}
if (cmdProcessThread != null) {
cmdProcessThread.interrupt();
}
context.setState(DatanodeStates.getLastState());
executorService.shutdown();
try {
if (!executorService.awaitTermination(5, TimeUnit.SECONDS)) {
executorService.shutdownNow();
}
if (!executorService.awaitTermination(5, TimeUnit.SECONDS)) {
LOG.error("Unable to shutdown state machine properly.");
}
} catch (InterruptedException e) {
LOG.error("Error attempting to shutdown.", e);
executorService.shutdownNow();
Thread.currentThread().interrupt();
}
if (connectionManager != null) {
connectionManager.close();
}
if(container != null) {
container.stop();
}
if (jvmPauseMonitor != null) {
jvmPauseMonitor.stop();
}
if (commandDispatcher != null) {
commandDispatcher.stop();
}
}
/**
* States that a datanode can be in. GetNextState will move this enum from
* getInitState to getLastState.
*/
public enum DatanodeStates {
INIT(1),
RUNNING(2),
SHUTDOWN(3);
private final int value;
/**
* Constructs states.
*
* @param value Enum Value
*/
DatanodeStates(int value) {
this.value = value;
}
/**
* Returns the first State.
*
* @return First State.
*/
public static DatanodeStates getInitState() {
return INIT;
}
/**
* The last state of endpoint states.
*
* @return last state.
*/
public static DatanodeStates getLastState() {
return SHUTDOWN;
}
/**
* returns the numeric value associated with the endPoint.
*
* @return int.
*/
public int getValue() {
return value;
}
/**
* Returns the next logical state that endPoint should move to. This
* function assumes the States are sequentially numbered.
*
* @return NextState.
*/
public DatanodeStates getNextState() {
if (this.value < getLastState().getValue()) {
int stateValue = this.getValue() + 1;
for (DatanodeStates iter : values()) {
if (stateValue == iter.getValue()) {
return iter;
}
}
}
return getLastState();
}
}
/**
* Start datanode state machine as a single thread daemon.
*/
public void startDaemon() {
Runnable startStateMachineTask = () -> {
try {
start();
LOG.info("Ozone container server started.");
} catch (Exception ex) {
LOG.error("Unable to start the DatanodeState Machine", ex);
}
};
stateMachineThread = new ThreadFactoryBuilder()
.setDaemon(true)
.setNameFormat("Datanode State Machine Thread - %d")
.build().newThread(startStateMachineTask);
stateMachineThread.start();
}
/**
* Calling this will immediately trigger a heartbeat to the SCMs.
* This heartbeat will also include all the reports which are ready to
* be sent by datanode.
*/
public void triggerHeartbeat() {
if (stateMachineThread != null) {
stateMachineThread.interrupt();
}
}
/**
* Waits for DatanodeStateMachine to exit.
*
* @throws InterruptedException
*/
public void join() throws InterruptedException {
if (stateMachineThread != null) {
stateMachineThread.join();
}
if (cmdProcessThread != null) {
cmdProcessThread.join();
}
}
/**
* Stop the daemon thread of the datanode state machine.
*/
public synchronized void stopDaemon() {
try {
supervisor.stop();
context.setState(DatanodeStates.SHUTDOWN);
reportManager.shutdown();
this.close();
LOG.info("Ozone container server stopped.");
} catch (IOException e) {
LOG.error("Stop ozone container server failed.", e);
}
}
/**
*
* Check if the datanode state machine daemon is stopped.
*
* @return True if datanode state machine daemon is stopped
* and false otherwise.
*/
@VisibleForTesting
public boolean isDaemonStopped() {
return this.executorService.isShutdown()
&& this.getContext().getState() == DatanodeStates.SHUTDOWN;
}
/**
* Create a command handler thread.
*
* @param config
*/
private void initCommandHandlerThread(Configuration config) {
/**
* Task that periodically checks if we have any outstanding commands.
* It is assumed that commands can be processed slowly and in order.
* This assumption might change in future. Right now due to this assumption
* we have single command queue process thread.
*/
Runnable processCommandQueue = () -> {
long now;
while (getContext().getState() != DatanodeStates.SHUTDOWN) {
SCMCommand command = getContext().getNextCommand();
if (command != null) {
commandDispatcher.handle(command);
commandsHandled++;
} else {
try {
// Sleep till the next HB + 1 second.
now = Time.monotonicNow();
if (nextHB.get() > now) {
Thread.sleep((nextHB.get() - now) + 1000L);
}
} catch (InterruptedException e) {
// Ignore this exception.
}
}
}
};
// We will have only one thread for command processing in a datanode.
cmdProcessThread = getCommandHandlerThread(processCommandQueue);
cmdProcessThread.start();
}
private Thread getCommandHandlerThread(Runnable processCommandQueue) {
Thread handlerThread = new Thread(processCommandQueue);
handlerThread.setDaemon(true);
handlerThread.setName("Command processor thread");
handlerThread.setUncaughtExceptionHandler((Thread t, Throwable e) -> {
// Let us just restart this thread after logging a critical error.
// if this thread is not running we cannot handle commands from SCM.
LOG.error("Critical Error : Command processor thread encountered an " +
"error. Thread: {}", t.toString(), e);
getCommandHandlerThread(processCommandQueue).start();
});
return handlerThread;
}
/**
* Returns the number of commands handled by the datanode.
* @return count
*/
@VisibleForTesting
public long getCommandHandled() {
return commandsHandled;
}
/**
* returns the Command Dispatcher.
* @return CommandDispatcher
*/
@VisibleForTesting
public CommandDispatcher getCommandDispatcher() {
return commandDispatcher;
}
@VisibleForTesting
public ReplicationSupervisor getSupervisor() {
return supervisor;
}
}