-
Notifications
You must be signed in to change notification settings - Fork 477
/
SCMNodeManager.java
1686 lines (1564 loc) · 60.6 KB
/
SCMNodeManager.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
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
/**
* 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
* <p>
* http://www.apache.org/licenses/LICENSE-2.0
* <p>
* 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.hdds.scm.node;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
import com.google.common.base.Strings;
import org.apache.hadoop.hdds.DFSConfigKeysLegacy;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeOperationalState;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState;
import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos;
import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.SCMCommandProto;
import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.CommandQueueReportProto;
import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.LayoutVersionProto;
import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.NodeReportProto;
import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.PipelineReportsProto;
import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.SCMRegisteredResponseProto.ErrorCode;
import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.SCMVersionRequestProto;
import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.StorageReportProto;
import org.apache.hadoop.hdds.scm.ScmConfigKeys;
import org.apache.hadoop.hdds.scm.VersionInfo;
import org.apache.hadoop.hdds.scm.container.ContainerID;
import org.apache.hadoop.hdds.scm.container.placement.metrics.SCMNodeMetric;
import org.apache.hadoop.hdds.scm.container.placement.metrics.SCMNodeStat;
import org.apache.hadoop.hdds.scm.events.SCMEvents;
import org.apache.hadoop.hdds.scm.ha.SCMContext;
import org.apache.hadoop.hdds.scm.net.NetworkTopology;
import org.apache.hadoop.hdds.scm.node.states.NodeAlreadyExistsException;
import org.apache.hadoop.hdds.scm.node.states.NodeNotFoundException;
import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
import org.apache.hadoop.hdds.scm.pipeline.PipelineID;
import org.apache.hadoop.hdds.scm.pipeline.PipelineManager;
import org.apache.hadoop.hdds.scm.pipeline.PipelineNotFoundException;
import org.apache.hadoop.hdds.scm.server.SCMStorageConfig;
import org.apache.hadoop.hdds.scm.server.upgrade.FinalizationManager;
import org.apache.hadoop.hdds.server.events.EventPublisher;
import org.apache.hadoop.hdds.upgrade.HDDSLayoutVersionManager;
import org.apache.hadoop.ipc.Server;
import org.apache.hadoop.metrics2.util.MBeans;
import org.apache.hadoop.ozone.OzoneConsts;
import org.apache.hadoop.ozone.protocol.VersionResponse;
import org.apache.hadoop.ozone.protocol.commands.CommandForDatanode;
import org.apache.hadoop.ozone.protocol.commands.FinalizeNewLayoutVersionCommand;
import org.apache.hadoop.ozone.protocol.commands.RefreshVolumeUsageCommand;
import org.apache.hadoop.ozone.protocol.commands.RegisteredCommand;
import org.apache.hadoop.ozone.protocol.commands.SCMCommand;
import org.apache.hadoop.ozone.protocol.commands.SetNodeOperationalStateCommand;
import org.apache.hadoop.util.Time;
import org.apache.ratis.protocol.exceptions.NotLeaderException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import javax.management.ObjectName;
import java.io.IOException;
import java.math.RoundingMode;
import java.net.InetAddress;
import java.text.DecimalFormat;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.LinkedList;
import java.util.List;
import java.util.Map;
import java.util.Objects;
import java.util.Set;
import java.util.UUID;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ScheduledFuture;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.locks.ReentrantReadWriteLock;
import java.util.function.BiConsumer;
import java.util.function.Function;
import java.util.stream.Collectors;
import static org.apache.hadoop.hdds.protocol.DatanodeDetails.Port.Name.HTTP;
import static org.apache.hadoop.hdds.protocol.DatanodeDetails.Port.Name.HTTPS;
import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeOperationalState.IN_SERVICE;
import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState.HEALTHY;
import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState.HEALTHY_READONLY;
/**
* Maintains information about the Datanodes on SCM side.
* <p>
* Heartbeats under SCM is very simple compared to HDFS heartbeatManager.
* <p>
* The getNode(byState) functions make copy of node maps and then creates a list
* based on that. It should be assumed that these get functions always report
* *stale* information. For example, getting the deadNodeCount followed by
* getNodes(DEAD) could very well produce totally different count. Also
* getNodeCount(HEALTHY) + getNodeCount(DEAD) + getNodeCode(STALE), is not
* guaranteed to add up to the total nodes that we know off. Please treat all
* get functions in this file as a snap-shot of information that is inconsistent
* as soon as you read it.
*/
public class SCMNodeManager implements NodeManager {
public static final Logger LOG =
LoggerFactory.getLogger(SCMNodeManager.class);
private final NodeStateManager nodeStateManager;
private final VersionInfo version;
private final CommandQueue commandQueue;
private final SCMNodeMetrics metrics;
// Node manager MXBean
private ObjectName nmInfoBean;
private final SCMStorageConfig scmStorageConfig;
private final NetworkTopology clusterMap;
private final Function<String, String> nodeResolver;
private final boolean useHostname;
private final Map<String, Set<UUID>> dnsToUuidMap = new ConcurrentHashMap<>();
private final int numPipelinesPerMetadataVolume;
private final int heavyNodeCriteria;
private final HDDSLayoutVersionManager scmLayoutVersionManager;
private final EventPublisher scmNodeEventPublisher;
private final SCMContext scmContext;
private final Map<SCMCommandProto.Type,
BiConsumer<DatanodeDetails, SCMCommand<?>>> sendCommandNotifyMap;
/**
* Lock used to synchronize some operation in Node manager to ensure a
* consistent view of the node state.
*/
private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
private static final String OPESTATE = "OPSTATE";
private static final String COMSTATE = "COMSTATE";
private static final String LASTHEARTBEAT = "LASTHEARTBEAT";
private static final String USEDSPACEPERCENT = "USEDSPACEPERCENT";
private static final String TOTALCAPACITY = "CAPACITY";
/**
* Constructs SCM machine Manager.
*/
public SCMNodeManager(
OzoneConfiguration conf,
SCMStorageConfig scmStorageConfig,
EventPublisher eventPublisher,
NetworkTopology networkTopology,
SCMContext scmContext,
HDDSLayoutVersionManager layoutVersionManager) {
this(conf, scmStorageConfig, eventPublisher, networkTopology, scmContext,
layoutVersionManager, hostname -> null);
}
public SCMNodeManager(
OzoneConfiguration conf,
SCMStorageConfig scmStorageConfig,
EventPublisher eventPublisher,
NetworkTopology networkTopology,
SCMContext scmContext,
HDDSLayoutVersionManager layoutVersionManager,
Function<String, String> nodeResolver) {
this.scmNodeEventPublisher = eventPublisher;
this.nodeStateManager = new NodeStateManager(conf, eventPublisher,
layoutVersionManager, scmContext);
this.version = VersionInfo.getLatestVersion();
this.commandQueue = new CommandQueue();
this.scmStorageConfig = scmStorageConfig;
this.scmLayoutVersionManager = layoutVersionManager;
LOG.info("Entering startup safe mode.");
registerMXBean();
this.metrics = SCMNodeMetrics.create(this);
this.clusterMap = networkTopology;
this.nodeResolver = nodeResolver;
this.useHostname = conf.getBoolean(
DFSConfigKeysLegacy.DFS_DATANODE_USE_DN_HOSTNAME,
DFSConfigKeysLegacy.DFS_DATANODE_USE_DN_HOSTNAME_DEFAULT);
this.numPipelinesPerMetadataVolume =
conf.getInt(ScmConfigKeys.OZONE_SCM_PIPELINE_PER_METADATA_VOLUME,
ScmConfigKeys.OZONE_SCM_PIPELINE_PER_METADATA_VOLUME_DEFAULT);
String dnLimit = conf.get(ScmConfigKeys.OZONE_DATANODE_PIPELINE_LIMIT);
this.heavyNodeCriteria = dnLimit == null ? 0 : Integer.parseInt(dnLimit);
this.scmContext = scmContext;
this.sendCommandNotifyMap = new HashMap<>();
}
@Override
public void registerSendCommandNotify(SCMCommandProto.Type type,
BiConsumer<DatanodeDetails, SCMCommand<?>> scmCommand) {
this.sendCommandNotifyMap.put(type, scmCommand);
}
private void registerMXBean() {
this.nmInfoBean = MBeans.register("SCMNodeManager",
"SCMNodeManagerInfo", this);
}
private void unregisterMXBean() {
if (this.nmInfoBean != null) {
MBeans.unregister(this.nmInfoBean);
this.nmInfoBean = null;
}
}
protected NodeStateManager getNodeStateManager() {
return nodeStateManager;
}
/**
* Returns all datanode that are in the given state. This function works by
* taking a snapshot of the current collection and then returning the list
* from that collection. This means that real map might have changed by the
* time we return this list.
*
* @return List of Datanodes that are known to SCM in the requested state.
*/
@Override
public List<DatanodeDetails> getNodes(NodeStatus nodeStatus) {
return nodeStateManager.getNodes(nodeStatus)
.stream()
.map(node -> (DatanodeDetails)node).collect(Collectors.toList());
}
/**
* Returns all datanode that are in the given states. Passing null for one of
* of the states acts like a wildcard for that state. This function works by
* taking a snapshot of the current collection and then returning the list
* from that collection. This means that real map might have changed by the
* time we return this list.
*
* @param opState The operational state of the node
* @param health The health of the node
* @return List of Datanodes that are known to SCM in the requested states.
*/
@Override
public List<DatanodeDetails> getNodes(
NodeOperationalState opState, NodeState health) {
return nodeStateManager.getNodes(opState, health)
.stream()
.map(node -> (DatanodeDetails)node).collect(Collectors.toList());
}
/**
* Returns all datanodes that are known to SCM.
*
* @return List of DatanodeDetails
*/
@Override
public List<DatanodeDetails> getAllNodes() {
return nodeStateManager.getAllNodes().stream()
.map(node -> (DatanodeDetails) node).collect(Collectors.toList());
}
/**
* Returns the Number of Datanodes by State they are in.
*
* @return count
*/
@Override
public int getNodeCount(NodeStatus nodeStatus) {
return nodeStateManager.getNodeCount(nodeStatus);
}
/**
* Returns the Number of Datanodes by State they are in. Passing null for
* either of the states acts like a wildcard for that state.
*
* @param nodeOpState - The Operational State of the node
* @param health - The health of the node
* @return count
*/
@Override
public int getNodeCount(NodeOperationalState nodeOpState, NodeState health) {
return nodeStateManager.getNodeCount(nodeOpState, health);
}
/**
* Returns the node status of a specific node.
*
* @param datanodeDetails Datanode Details
* @return NodeStatus for the node
*/
@Override
public NodeStatus getNodeStatus(DatanodeDetails datanodeDetails)
throws NodeNotFoundException {
return nodeStateManager.getNodeStatus(datanodeDetails);
}
/**
* Set the operation state of a node.
* @param datanodeDetails The datanode to set the new state for
* @param newState The new operational state for the node
*/
@Override
public void setNodeOperationalState(DatanodeDetails datanodeDetails,
NodeOperationalState newState) throws NodeNotFoundException {
setNodeOperationalState(datanodeDetails, newState, 0);
}
/**
* Set the operation state of a node.
* @param datanodeDetails The datanode to set the new state for
* @param newState The new operational state for the node
* @param opStateExpiryEpocSec Seconds from the epoch when the operational
* state should end. Zero indicates the state
* never end.
*/
@Override
public void setNodeOperationalState(DatanodeDetails datanodeDetails,
NodeOperationalState newState, long opStateExpiryEpocSec)
throws NodeNotFoundException {
nodeStateManager.setNodeOperationalState(
datanodeDetails, newState, opStateExpiryEpocSec);
}
/**
* Closes this stream and releases any system resources associated with it. If
* the stream is already closed then invoking this method has no effect.
*
* @throws IOException if an I/O error occurs
*/
@Override
public void close() throws IOException {
unregisterMXBean();
metrics.unRegister();
nodeStateManager.close();
}
/**
* Gets the version info from SCM.
*
* @param versionRequest - version Request.
* @return - returns SCM version info and other required information needed by
* datanode.
*/
@Override
public VersionResponse getVersion(SCMVersionRequestProto versionRequest) {
return VersionResponse.newBuilder()
.setVersion(this.version.getVersion())
.addValue(OzoneConsts.SCM_ID,
this.scmStorageConfig.getScmId())
.addValue(OzoneConsts.CLUSTER_ID, this.scmStorageConfig.getClusterID())
.build();
}
@Override
public RegisteredCommand register(
DatanodeDetails datanodeDetails, NodeReportProto nodeReport,
PipelineReportsProto pipelineReportsProto) {
return register(datanodeDetails, nodeReport, pipelineReportsProto,
LayoutVersionProto.newBuilder()
.setMetadataLayoutVersion(
scmLayoutVersionManager.getMetadataLayoutVersion())
.setSoftwareLayoutVersion(
scmLayoutVersionManager.getSoftwareLayoutVersion())
.build());
}
/**
* Register the node if the node finds that it is not registered with any
* SCM.
*
* @param datanodeDetails - Send datanodeDetails with Node info.
* This function generates and assigns new datanode ID
* for the datanode. This allows SCM to be run
* independent
* of Namenode if required.
* @param nodeReport NodeReport.
* @return SCMRegisteredResponseProto
*/
@Override
public RegisteredCommand register(
DatanodeDetails datanodeDetails, NodeReportProto nodeReport,
PipelineReportsProto pipelineReportsProto,
LayoutVersionProto layoutInfo) {
if (layoutInfo.getSoftwareLayoutVersion() !=
scmLayoutVersionManager.getSoftwareLayoutVersion()) {
return RegisteredCommand.newBuilder()
.setErrorCode(ErrorCode.errorNodeNotPermitted)
.setDatanode(datanodeDetails)
.setClusterID(this.scmStorageConfig.getClusterID())
.build();
}
InetAddress dnAddress = Server.getRemoteIp();
if (dnAddress != null) {
// Mostly called inside an RPC, update ip
if (!useHostname) {
datanodeDetails.setHostName(dnAddress.getHostName());
}
datanodeDetails.setIpAddress(dnAddress.getHostAddress());
}
final String ipAddress = datanodeDetails.getIpAddress();
final String hostName = datanodeDetails.getHostName();
datanodeDetails.setNetworkName(datanodeDetails.getUuidString());
String networkLocation = nodeResolver.apply(
useHostname ? hostName : ipAddress);
if (networkLocation != null) {
datanodeDetails.setNetworkLocation(networkLocation);
}
final UUID uuid = datanodeDetails.getUuid();
if (!isNodeRegistered(datanodeDetails)) {
try {
clusterMap.add(datanodeDetails);
nodeStateManager.addNode(datanodeDetails, layoutInfo);
// Check that datanode in nodeStateManager has topology parent set
DatanodeDetails dn = nodeStateManager.getNode(datanodeDetails);
Preconditions.checkState(dn.getParent() != null);
addToDnsToUuidMap(uuid, ipAddress, hostName);
// Updating Node Report, as registration is successful
processNodeReport(datanodeDetails, nodeReport);
LOG.info("Registered datanode: {}", datanodeDetails.toDebugString());
scmNodeEventPublisher.fireEvent(SCMEvents.NEW_NODE, datanodeDetails);
} catch (NodeAlreadyExistsException e) {
if (LOG.isTraceEnabled()) {
LOG.trace("Datanode is already registered: {}",
datanodeDetails);
}
} catch (NodeNotFoundException e) {
LOG.error("Cannot find datanode {} from nodeStateManager",
datanodeDetails);
}
} else {
// Update datanode if it is registered but the ip or hostname changes
try {
final DatanodeInfo oldNode = nodeStateManager.getNode(datanodeDetails);
if (updateDnsToUuidMap(oldNode.getHostName(), oldNode.getIpAddress(),
hostName, ipAddress, uuid)) {
LOG.info("Updating datanode {} from {} to {}",
datanodeDetails.getUuidString(),
oldNode,
datanodeDetails);
clusterMap.update(oldNode, datanodeDetails);
nodeStateManager.updateNode(datanodeDetails, layoutInfo);
DatanodeDetails dn = nodeStateManager.getNode(datanodeDetails);
Preconditions.checkState(dn.getParent() != null);
processNodeReport(datanodeDetails, nodeReport);
LOG.info("Updated datanode to: {}", dn);
scmNodeEventPublisher.fireEvent(SCMEvents.NODE_ADDRESS_UPDATE, dn);
}
} catch (NodeNotFoundException e) {
LOG.error("Cannot find datanode {} from nodeStateManager",
datanodeDetails);
}
}
return RegisteredCommand.newBuilder().setErrorCode(ErrorCode.success)
.setDatanode(datanodeDetails)
.setClusterID(this.scmStorageConfig.getClusterID())
.build();
}
/**
* Add an entry to the dnsToUuidMap, which maps hostname / IP to the DNs
* running on that host. As each address can have many DNs running on it,
* and each host can have multiple addresses,
* this is a many to many mapping.
*
* @param uuid the UUID of the registered node.
* @param addresses hostname and/or IP of the node
*/
private synchronized void addToDnsToUuidMap(UUID uuid, String... addresses) {
for (String addr : addresses) {
if (!Strings.isNullOrEmpty(addr)) {
dnsToUuidMap.computeIfAbsent(addr, k -> ConcurrentHashMap.newKeySet())
.add(uuid);
}
}
}
private synchronized void removeFromDnsToUuidMap(UUID uuid, String address) {
if (address != null) {
Set<UUID> dnSet = dnsToUuidMap.get(address);
if (dnSet != null && dnSet.remove(uuid) && dnSet.isEmpty()) {
dnsToUuidMap.remove(address);
}
}
}
private boolean updateDnsToUuidMap(
String oldHostName, String oldIpAddress,
String newHostName, String newIpAddress,
UUID uuid) {
final boolean ipChanged = !Objects.equals(oldIpAddress, newIpAddress);
final boolean hostNameChanged = !Objects.equals(oldHostName, newHostName);
if (ipChanged || hostNameChanged) {
synchronized (this) {
if (ipChanged) {
removeFromDnsToUuidMap(uuid, oldIpAddress);
addToDnsToUuidMap(uuid, newIpAddress);
}
if (hostNameChanged) {
removeFromDnsToUuidMap(uuid, oldHostName);
addToDnsToUuidMap(uuid, newHostName);
}
}
}
return ipChanged || hostNameChanged;
}
/**
* Send heartbeat to indicate the datanode is alive and doing well.
*
* @param datanodeDetails - DatanodeDetailsProto.
* @return SCMheartbeat response.
*/
@Override
public List<SCMCommand> processHeartbeat(DatanodeDetails datanodeDetails,
CommandQueueReportProto queueReport) {
Preconditions.checkNotNull(datanodeDetails, "Heartbeat is missing " +
"DatanodeDetails.");
try {
nodeStateManager.updateLastHeartbeatTime(datanodeDetails);
metrics.incNumHBProcessed();
updateDatanodeOpState(datanodeDetails);
} catch (NodeNotFoundException e) {
metrics.incNumHBProcessingFailed();
LOG.error("SCM trying to process heartbeat from an " +
"unregistered node {}. Ignoring the heartbeat.", datanodeDetails);
}
writeLock().lock();
try {
Map<SCMCommandProto.Type, Integer> summary =
commandQueue.getDatanodeCommandSummary(datanodeDetails.getUuid());
List<SCMCommand> commands =
commandQueue.getCommand(datanodeDetails.getUuid());
// Update the SCMCommand of deleteBlocksCommand Status
for (SCMCommand<?> command : commands) {
if (sendCommandNotifyMap.get(command.getType()) != null) {
sendCommandNotifyMap.get(command.getType())
.accept(datanodeDetails, command);
}
}
if (queueReport != null) {
processNodeCommandQueueReport(datanodeDetails, queueReport, summary);
}
return commands;
} finally {
writeLock().unlock();
}
}
boolean opStateDiffers(DatanodeDetails dnDetails, NodeStatus nodeStatus) {
return nodeStatus.getOperationalState() != dnDetails.getPersistedOpState()
|| nodeStatus.getOpStateExpiryEpochSeconds()
!= dnDetails.getPersistedOpStateExpiryEpochSec();
}
/**
* This method should only be called when processing the heartbeat.
*
* On leader SCM, for a registered node, the information stored in SCM is
* the source of truth. If the operational state or expiry reported in the
* datanode heartbeat do not match those store in SCM, queue a command to
* update the state persisted on the datanode. Additionally, ensure the
* datanodeDetails stored in SCM match those reported in the heartbeat.
*
* On follower SCM, datanode notifies follower SCM its latest operational
* state or expiry via heartbeat. If the operational state or expiry
* reported in the datanode heartbeat do not match those stored in SCM,
* just update the state in follower SCM accordingly.
*
* @param reportedDn The DatanodeDetails taken from the node heartbeat.
* @throws NodeNotFoundException
*/
protected void updateDatanodeOpState(DatanodeDetails reportedDn)
throws NodeNotFoundException {
NodeStatus scmStatus = getNodeStatus(reportedDn);
if (opStateDiffers(reportedDn, scmStatus)) {
if (scmContext.isLeader()) {
LOG.info("Scheduling a command to update the operationalState " +
"persisted on {} as the reported value ({}, {}) does not " +
"match the value stored in SCM ({}, {})",
reportedDn,
reportedDn.getPersistedOpState(),
reportedDn.getPersistedOpStateExpiryEpochSec(),
scmStatus.getOperationalState(),
scmStatus.getOpStateExpiryEpochSeconds());
try {
SCMCommand<?> command = new SetNodeOperationalStateCommand(
Time.monotonicNow(),
scmStatus.getOperationalState(),
scmStatus.getOpStateExpiryEpochSeconds());
command.setTerm(scmContext.getTermOfLeader());
addDatanodeCommand(reportedDn.getUuid(), command);
} catch (NotLeaderException nle) {
LOG.warn("Skip sending SetNodeOperationalStateCommand,"
+ " since current SCM is not leader.", nle);
return;
}
} else {
LOG.info("Update the operationalState saved in follower SCM " +
"for {} as the reported value ({}, {}) does not " +
"match the value stored in SCM ({}, {})",
reportedDn,
reportedDn.getPersistedOpState(),
reportedDn.getPersistedOpStateExpiryEpochSec(),
scmStatus.getOperationalState(),
scmStatus.getOpStateExpiryEpochSeconds());
setNodeOperationalState(reportedDn, reportedDn.getPersistedOpState(),
reportedDn.getPersistedOpStateExpiryEpochSec());
}
}
DatanodeDetails scmDnd = nodeStateManager.getNode(reportedDn);
scmDnd.setPersistedOpStateExpiryEpochSec(
reportedDn.getPersistedOpStateExpiryEpochSec());
scmDnd.setPersistedOpState(reportedDn.getPersistedOpState());
}
@Override
public Boolean isNodeRegistered(DatanodeDetails datanodeDetails) {
try {
nodeStateManager.getNode(datanodeDetails);
return true;
} catch (NodeNotFoundException e) {
return false;
}
}
/**
* Process node report.
*
* @param datanodeDetails
* @param nodeReport
*/
@Override
public void processNodeReport(DatanodeDetails datanodeDetails,
NodeReportProto nodeReport) {
if (LOG.isDebugEnabled()) {
LOG.debug("Processing node report from [datanode={}]",
datanodeDetails.getHostName());
}
if (LOG.isTraceEnabled() && nodeReport != null) {
LOG.trace("HB is received from [datanode={}]: <json>{}</json>",
datanodeDetails.getHostName(),
nodeReport.toString().replaceAll("\n", "\\\\n"));
}
try {
DatanodeInfo datanodeInfo = nodeStateManager.getNode(datanodeDetails);
if (nodeReport != null) {
datanodeInfo.updateStorageReports(nodeReport.getStorageReportList());
datanodeInfo.updateMetaDataStorageReports(nodeReport.
getMetadataStorageReportList());
metrics.incNumNodeReportProcessed();
}
} catch (NodeNotFoundException e) {
metrics.incNumNodeReportProcessingFailed();
LOG.warn("Got node report from unregistered datanode {}",
datanodeDetails);
}
}
/**
* Process Layout Version report.
*
* @param datanodeDetails
* @param layoutVersionReport
*/
@Override
public void processLayoutVersionReport(DatanodeDetails datanodeDetails,
LayoutVersionProto layoutVersionReport) {
if (LOG.isDebugEnabled()) {
LOG.debug("Processing Layout Version report from [datanode={}]",
datanodeDetails.getHostName());
}
if (LOG.isTraceEnabled()) {
LOG.trace("HB is received from [datanode={}]: <json>{}</json>",
datanodeDetails.getHostName(),
layoutVersionReport.toString().replaceAll("\n", "\\\\n"));
}
try {
nodeStateManager.updateLastKnownLayoutVersion(datanodeDetails,
layoutVersionReport);
} catch (NodeNotFoundException e) {
LOG.error("SCM trying to process Layout Version from an " +
"unregistered node {}.", datanodeDetails);
return;
}
sendFinalizeToDatanodeIfNeeded(datanodeDetails, layoutVersionReport);
}
protected void sendFinalizeToDatanodeIfNeeded(DatanodeDetails datanodeDetails,
LayoutVersionProto layoutVersionReport) {
// Software layout version is hardcoded to the SCM.
int scmSlv = scmLayoutVersionManager.getSoftwareLayoutVersion();
int dnSlv = layoutVersionReport.getSoftwareLayoutVersion();
int dnMlv = layoutVersionReport.getMetadataLayoutVersion();
// A datanode with a larger software layout version is from a future
// version of ozone. It should not have been added to the cluster.
if (dnSlv > scmSlv) {
LOG.error("Invalid data node in the cluster : {}. " +
"DataNode SoftwareLayoutVersion = {}, SCM " +
"SoftwareLayoutVersion = {}",
datanodeDetails.getHostName(), dnSlv, scmSlv);
}
if (FinalizationManager.shouldTellDatanodesToFinalize(
scmContext.getFinalizationCheckpoint())) {
// Because we have crossed the MLV_EQUALS_SLV checkpoint, SCM metadata
// layout version will not change. We can now compare it to the
// datanodes' metadata layout versions to tell them to finalize.
int scmMlv = scmLayoutVersionManager.getMetadataLayoutVersion();
// If the datanode mlv < scm mlv, it can not be allowed to be part of
// any pipeline. However it can be allowed to join the cluster
if (dnMlv < scmMlv) {
LOG.warn("Data node {} can not be used in any pipeline in the " +
"cluster. " + "DataNode MetadataLayoutVersion = {}, SCM " +
"MetadataLayoutVersion = {}",
datanodeDetails.getHostName(), dnMlv, scmMlv);
FinalizeNewLayoutVersionCommand finalizeCmd =
new FinalizeNewLayoutVersionCommand(true,
LayoutVersionProto.newBuilder()
.setSoftwareLayoutVersion(dnSlv)
.setMetadataLayoutVersion(dnSlv).build());
if (scmContext.isLeader()) {
try {
finalizeCmd.setTerm(scmContext.getTermOfLeader());
// Send Finalize command to the data node. Its OK to
// send Finalize command multiple times.
scmNodeEventPublisher.fireEvent(SCMEvents.DATANODE_COMMAND,
new CommandForDatanode<>(datanodeDetails.getUuid(),
finalizeCmd));
} catch (NotLeaderException ex) {
LOG.warn("Skip sending finalize upgrade command since current SCM" +
" is not leader.", ex);
}
}
}
}
}
/**
* Process Command Queue Reports from the Datanode Heartbeat.
*
* @param datanodeDetails
* @param commandQueueReportProto
* @param commandsToBeSent
*/
private void processNodeCommandQueueReport(DatanodeDetails datanodeDetails,
CommandQueueReportProto commandQueueReportProto,
Map<SCMCommandProto.Type, Integer> commandsToBeSent) {
LOG.debug("Processing Command Queue Report from [datanode={}]",
datanodeDetails.getHostName());
if (commandQueueReportProto == null) {
LOG.debug("The Command Queue Report from [datanode={}] is null",
datanodeDetails.getHostName());
return;
}
if (LOG.isTraceEnabled()) {
LOG.trace("Command Queue Report is received from [datanode={}]: " +
"<json>{}</json>", datanodeDetails.getHostName(),
commandQueueReportProto.toString().replaceAll("\n", "\\\\n"));
}
try {
DatanodeInfo datanodeInfo = nodeStateManager.getNode(datanodeDetails);
datanodeInfo.setCommandCounts(commandQueueReportProto,
commandsToBeSent);
metrics.incNumNodeCommandQueueReportProcessed();
scmNodeEventPublisher.fireEvent(
SCMEvents.DATANODE_COMMAND_COUNT_UPDATED, datanodeDetails);
} catch (NodeNotFoundException e) {
metrics.incNumNodeCommandQueueReportProcessingFailed();
LOG.warn("Got Command Queue Report from unregistered datanode {}",
datanodeDetails);
}
}
/**
* Get the number of commands of the given type queued on the datanode at the
* last heartbeat. If the Datanode has not reported information for the given
* command type, -1 will be returned.
* @param cmdType
* @return The queued count or -1 if no data has been received from the DN.
*/
@Override
public int getNodeQueuedCommandCount(DatanodeDetails datanodeDetails,
SCMCommandProto.Type cmdType) throws NodeNotFoundException {
readLock().lock();
try {
DatanodeInfo datanodeInfo = nodeStateManager.getNode(datanodeDetails);
return datanodeInfo.getCommandCount(cmdType);
} finally {
readLock().unlock();
}
}
/**
* Get the number of commands of the given type queued in the SCM CommandQueue
* for the given datanode.
* @param dnID The UUID of the datanode.
* @param cmdType The Type of command to query the current count for.
* @return The count of commands queued, or zero if none.
*/
@Override
public int getCommandQueueCount(UUID dnID, SCMCommandProto.Type cmdType) {
readLock().lock();
try {
return commandQueue.getDatanodeCommandCount(dnID, cmdType);
} finally {
readLock().unlock();
}
}
/**
* Get the total number of pending commands of the given type on the given
* datanode. This includes both the number of commands queued in SCM which
* will be sent to the datanode on the next heartbeat, and the number of
* commands reported by the datanode in the last heartbeat.
* If the datanode has not reported any information for the given command,
* zero is assumed.
* @param datanodeDetails The datanode to query.
* @param cmdType The command Type To query.
* @return The number of commands of the given type pending on the datanode.
* @throws NodeNotFoundException
*/
@Override
public int getTotalDatanodeCommandCount(DatanodeDetails datanodeDetails,
SCMCommandProto.Type cmdType) throws NodeNotFoundException {
readLock().lock();
try {
int dnCount = getNodeQueuedCommandCount(datanodeDetails, cmdType);
if (dnCount == -1) {
LOG.warn("No command count information for datanode {} and command {}" +
". Assuming zero", datanodeDetails, cmdType);
dnCount = 0;
}
return getCommandQueueCount(datanodeDetails.getUuid(), cmdType) + dnCount;
} finally {
readLock().unlock();
}
}
/**
* Get the total number of pending commands of the given types on the given
* datanode. For each command, this includes both the number of commands
* queued in SCM which will be sent to the datanode on the next heartbeat,
* and the number of commands reported by the datanode in the last heartbeat.
* If the datanode has not reported any information for the given command,
* zero is assumed.
* All commands are retrieved under a single read lock, so the counts are
* consistent.
* @param datanodeDetails The datanode to query.
* @param cmdType The list of command Types To query.
* @return A Map of commandType to Integer with an entry for each command type
* passed.
* @throws NodeNotFoundException
*/
@Override
public Map<SCMCommandProto.Type, Integer> getTotalDatanodeCommandCounts(
DatanodeDetails datanodeDetails, SCMCommandProto.Type... cmdType)
throws NodeNotFoundException {
Map<SCMCommandProto.Type, Integer> counts = new HashMap<>();
readLock().lock();
try {
for (SCMCommandProto.Type type : cmdType) {
counts.put(type, getTotalDatanodeCommandCount(datanodeDetails, type));
}
return counts;
} finally {
readLock().unlock();
}
}
/**
* Returns the aggregated node stats.
*
* @return the aggregated node stats.
*/
@Override
public SCMNodeStat getStats() {
long capacity = 0L;
long used = 0L;
long remaining = 0L;
long committed = 0L;
long freeSpaceToSpare = 0L;
for (SCMNodeStat stat : getNodeStats().values()) {
capacity += stat.getCapacity().get();
used += stat.getScmUsed().get();
remaining += stat.getRemaining().get();
committed += stat.getCommitted().get();
freeSpaceToSpare += stat.getFreeSpaceToSpare().get();
}
return new SCMNodeStat(capacity, used, remaining, committed,
freeSpaceToSpare);
}
/**
* Return a map of node stats.
*
* @return a map of individual node stats (live/stale but not dead).
*/
@Override
public Map<DatanodeDetails, SCMNodeStat> getNodeStats() {
final Map<DatanodeDetails, SCMNodeStat> nodeStats = new HashMap<>();
final List<DatanodeInfo> healthyNodes = nodeStateManager
.getNodes(null, HEALTHY);
final List<DatanodeInfo> healthyReadOnlyNodes = nodeStateManager
.getNodes(null, HEALTHY_READONLY);
final List<DatanodeInfo> staleNodes = nodeStateManager
.getStaleNodes();
final List<DatanodeInfo> datanodes = new ArrayList<>(healthyNodes);
datanodes.addAll(healthyReadOnlyNodes);
datanodes.addAll(staleNodes);
for (DatanodeInfo dnInfo : datanodes) {
SCMNodeStat nodeStat = getNodeStatInternal(dnInfo);
if (nodeStat != null) {
nodeStats.put(dnInfo, nodeStat);
}
}
return nodeStats;
}
/**
* Gets a sorted list of most or least used DatanodeUsageInfo containing
* healthy, in-service nodes. If the specified mostUsed is true, the returned
* list is in descending order of usage. Otherwise, the returned list is in
* ascending order of usage.
*
* @param mostUsed true if most used, false if least used
* @return List of DatanodeUsageInfo
*/
@Override
public List<DatanodeUsageInfo> getMostOrLeastUsedDatanodes(
boolean mostUsed) {
List<DatanodeDetails> healthyNodes =
getNodes(IN_SERVICE, NodeState.HEALTHY);
List<DatanodeUsageInfo> datanodeUsageInfoList =
new ArrayList<>(healthyNodes.size());
// create a DatanodeUsageInfo from each DatanodeDetails and add it to the
// list
for (DatanodeDetails node : healthyNodes) {
DatanodeUsageInfo datanodeUsageInfo = getUsageInfo(node);
datanodeUsageInfoList.add(datanodeUsageInfo);
}
// sort the list according to appropriate comparator
if (mostUsed) {
datanodeUsageInfoList.sort(
DatanodeUsageInfo.getMostUtilized().reversed());
} else {
datanodeUsageInfoList.sort(
DatanodeUsageInfo.getMostUtilized());
}
return datanodeUsageInfoList;
}
/**
* Get the usage info of a specified datanode.
*
* @param dn the usage of which we want to get
* @return DatanodeUsageInfo of the specified datanode
*/
@Override
public DatanodeUsageInfo getUsageInfo(DatanodeDetails dn) {
SCMNodeStat stat = getNodeStatInternal(dn);
DatanodeUsageInfo usageInfo = new DatanodeUsageInfo(dn, stat);
try {
usageInfo.setContainerCount(getContainerCount(dn));
} catch (NodeNotFoundException ex) {
LOG.error("Unknown datanode {}.", dn, ex);
}
return usageInfo;
}
/**
* Return the node stat of the specified datanode.
*
* @param datanodeDetails - datanode ID.
* @return node stat if it is live/stale, null if it is decommissioned or
* doesn't exist.
*/
@Override
public SCMNodeMetric getNodeStat(DatanodeDetails datanodeDetails) {
final SCMNodeStat nodeStat = getNodeStatInternal(datanodeDetails);
return nodeStat != null ? new SCMNodeMetric(nodeStat) : null;