/
BlockManager.java
4222 lines (3856 loc) · 161 KB
/
BlockManager.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
*
* 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.hdfs.server.blockmanagement;
import static org.apache.hadoop.util.ExitUtil.terminate;
import java.io.IOException;
import java.io.PrintWriter;
import java.util.ArrayList;
import java.util.BitSet;
import java.util.Collection;
import java.util.Collections;
import java.util.EnumSet;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Iterator;
import java.util.LinkedList;
import java.util.List;
import java.util.Map;
import java.util.Queue;
import java.util.Set;
import java.util.TreeMap;
import java.util.TreeSet;
import java.util.concurrent.ThreadLocalRandom;
import java.util.concurrent.atomic.AtomicLong;
import org.apache.hadoop.HadoopIllegalArgumentException;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.StorageType;
import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.hdfs.HAUtil;
import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
import org.apache.hadoop.hdfs.protocol.BlockListAsLongs.BlockReportReplica;
import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.protocol.ErasureCodingZone;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
import org.apache.hadoop.fs.FileEncryptionInfo;
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock;
import org.apache.hadoop.hdfs.protocol.UnregisteredNodeException;
import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
import org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager;
import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier.AccessMode;
import org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey;
import org.apache.hadoop.hdfs.security.token.block.ExportedBlockKeys;
import org.apache.hadoop.hdfs.server.blockmanagement.CorruptReplicasMap.Reason;
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo.AddBlockResult;
import org.apache.hadoop.hdfs.server.blockmanagement.PendingDataNodeMessages.ReportedBlockInfo;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
import org.apache.hadoop.hdfs.server.namenode.NameNode;
import org.apache.hadoop.hdfs.server.namenode.NameNode.OperationCategory;
import org.apache.hadoop.hdfs.server.namenode.Namesystem;
import org.apache.hadoop.hdfs.server.namenode.metrics.NameNodeMetrics;
import org.apache.hadoop.hdfs.server.protocol.BlockCommand;
import org.apache.hadoop.hdfs.server.protocol.BlockReportContext;
import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations;
import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations.BlockWithLocations;
import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations.StripedBlockWithLocations;
import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage.State;
import org.apache.hadoop.hdfs.server.protocol.KeyUpdateCommand;
import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo;
import org.apache.hadoop.hdfs.server.protocol.StorageReceivedDeletedBlocks;
import org.apache.hadoop.hdfs.util.LightWeightLinkedSet;
import org.apache.hadoop.io.erasurecode.ECSchema;
import static org.apache.hadoop.hdfs.protocol.HdfsConstants.BLOCK_STRIPED_CELL_SIZE;
import static org.apache.hadoop.hdfs.util.StripedBlockUtil.getInternalBlockLength;
import org.apache.hadoop.net.Node;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.util.Daemon;
import org.apache.hadoop.util.LightWeightGSet;
import org.apache.hadoop.util.Time;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
import com.google.common.collect.Sets;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* Keeps information related to the blocks stored in the Hadoop cluster.
*/
@InterfaceAudience.Private
public class BlockManager {
public static final Logger LOG = LoggerFactory.getLogger(BlockManager.class);
public static final Logger blockLog = NameNode.blockStateChangeLog;
private static final String QUEUE_REASON_CORRUPT_STATE =
"it has the wrong state or generation stamp";
private static final String QUEUE_REASON_FUTURE_GENSTAMP =
"generation stamp is in the future";
private final Namesystem namesystem;
private final DatanodeManager datanodeManager;
private final HeartbeatManager heartbeatManager;
private final BlockTokenSecretManager blockTokenSecretManager;
private final PendingDataNodeMessages pendingDNMessages =
new PendingDataNodeMessages();
private volatile long pendingReplicationBlocksCount = 0L;
private volatile long corruptReplicaBlocksCount = 0L;
private volatile long underReplicatedBlocksCount = 0L;
private volatile long scheduledReplicationBlocksCount = 0L;
private final AtomicLong excessBlocksCount = new AtomicLong(0L);
private final AtomicLong postponedMisreplicatedBlocksCount = new AtomicLong(0L);
private final long startupDelayBlockDeletionInMs;
/** Used by metrics */
public long getPendingReplicationBlocksCount() {
return pendingReplicationBlocksCount;
}
/** Used by metrics */
public long getUnderReplicatedBlocksCount() {
return underReplicatedBlocksCount;
}
/** Used by metrics */
public long getCorruptReplicaBlocksCount() {
return corruptReplicaBlocksCount;
}
/** Used by metrics */
public long getScheduledReplicationBlocksCount() {
return scheduledReplicationBlocksCount;
}
/** Used by metrics */
public long getPendingDeletionBlocksCount() {
return invalidateBlocks.numBlocks();
}
/** Used by metrics */
public long getStartupDelayBlockDeletionInMs() {
return startupDelayBlockDeletionInMs;
}
/** Used by metrics */
public long getExcessBlocksCount() {
return excessBlocksCount.get();
}
/** Used by metrics */
public long getPostponedMisreplicatedBlocksCount() {
return postponedMisreplicatedBlocksCount.get();
}
/** Used by metrics */
public int getPendingDataNodeMessageCount() {
return pendingDNMessages.count();
}
/**replicationRecheckInterval is how often namenode checks for new replication work*/
private final long replicationRecheckInterval;
/**
* Mapping: Block -> { BlockCollection, datanodes, self ref }
* Updated only in response to client-sent information.
*/
final BlocksMap blocksMap;
/** Replication thread. */
final Daemon replicationThread = new Daemon(new ReplicationMonitor());
/** Store blocks -> datanodedescriptor(s) map of corrupt replicas */
final CorruptReplicasMap corruptReplicas = new CorruptReplicasMap();
/**
* Blocks to be invalidated.
* For a striped block to invalidate, we should track its individual internal
* blocks.
*/
private final InvalidateBlocks invalidateBlocks;
/**
* After a failover, over-replicated blocks may not be handled
* until all of the replicas have done a block report to the
* new active. This is to make sure that this NameNode has been
* notified of all block deletions that might have been pending
* when the failover happened.
*/
private final Set<Block> postponedMisreplicatedBlocks = Sets.newHashSet();
/**
* Maps a StorageID to the set of blocks that are "extra" for this
* DataNode. We'll eventually remove these extras.
*/
public final Map<String, LightWeightLinkedSet<BlockInfo>> excessReplicateMap =
new TreeMap<>();
/**
* Store set of Blocks that need to be replicated 1 or more times.
* We also store pending replication-orders.
*/
public final UnderReplicatedBlocks neededReplications = new UnderReplicatedBlocks();
@VisibleForTesting
final PendingReplicationBlocks pendingReplications;
/** The maximum number of replicas allowed for a block */
public final short maxReplication;
/**
* The maximum number of outgoing replication streams a given node should have
* at one time considering all but the highest priority replications needed.
*/
int maxReplicationStreams;
/**
* The maximum number of outgoing replication streams a given node should have
* at one time.
*/
int replicationStreamsHardLimit;
/** Minimum copies needed or else write is disallowed */
public final short minReplication;
/** Default number of replicas */
public final int defaultReplication;
/** value returned by MAX_CORRUPT_FILES_RETURNED */
final int maxCorruptFilesReturned;
final float blocksInvalidateWorkPct;
final int blocksReplWorkMultiplier;
/** variable to enable check for enough racks */
final boolean shouldCheckForEnoughRacks;
// whether or not to issue block encryption keys.
final boolean encryptDataTransfer;
// Max number of blocks to log info about during a block report.
private final long maxNumBlocksToLog;
/**
* When running inside a Standby node, the node may receive block reports
* from datanodes before receiving the corresponding namespace edits from
* the active NameNode. Thus, it will postpone them for later processing,
* instead of marking the blocks as corrupt.
*/
private boolean shouldPostponeBlocksFromFuture = false;
/**
* Process replication queues asynchronously to allow namenode safemode exit
* and failover to be faster. HDFS-5496
*/
private Daemon replicationQueuesInitializer = null;
/**
* Number of blocks to process asychronously for replication queues
* initialization once aquired the namesystem lock. Remaining blocks will be
* processed again after aquiring lock again.
*/
private int numBlocksPerIteration;
/**
* Progress of the Replication queues initialisation.
*/
private double replicationQueuesInitProgress = 0.0;
/** for block replicas placement */
private BlockPlacementPolicies placementPolicies;
private final BlockStoragePolicySuite storagePolicySuite;
/** Check whether name system is running before terminating */
private boolean checkNSRunning = true;
/** Check whether there are any non-EC blocks using StripedID */
private boolean hasNonEcBlockUsingStripedID = false;
public BlockManager(final Namesystem namesystem, final Configuration conf)
throws IOException {
this.namesystem = namesystem;
datanodeManager = new DatanodeManager(this, namesystem, conf);
heartbeatManager = datanodeManager.getHeartbeatManager();
startupDelayBlockDeletionInMs = conf.getLong(
DFSConfigKeys.DFS_NAMENODE_STARTUP_DELAY_BLOCK_DELETION_SEC_KEY,
DFSConfigKeys.DFS_NAMENODE_STARTUP_DELAY_BLOCK_DELETION_SEC_DEFAULT) * 1000L;
invalidateBlocks = new InvalidateBlocks(
datanodeManager.blockInvalidateLimit, startupDelayBlockDeletionInMs);
// Compute the map capacity by allocating 2% of total memory
blocksMap = new BlocksMap(
LightWeightGSet.computeCapacity(2.0, "BlocksMap"));
placementPolicies = new BlockPlacementPolicies(
conf, datanodeManager.getFSClusterStats(),
datanodeManager.getNetworkTopology(),
datanodeManager.getHost2DatanodeMap());
storagePolicySuite = BlockStoragePolicySuite.createDefaultSuite();
pendingReplications = new PendingReplicationBlocks(conf.getInt(
DFSConfigKeys.DFS_NAMENODE_REPLICATION_PENDING_TIMEOUT_SEC_KEY,
DFSConfigKeys.DFS_NAMENODE_REPLICATION_PENDING_TIMEOUT_SEC_DEFAULT) * 1000L);
blockTokenSecretManager = createBlockTokenSecretManager(conf);
this.maxCorruptFilesReturned = conf.getInt(
DFSConfigKeys.DFS_DEFAULT_MAX_CORRUPT_FILES_RETURNED_KEY,
DFSConfigKeys.DFS_DEFAULT_MAX_CORRUPT_FILES_RETURNED);
this.defaultReplication = conf.getInt(DFSConfigKeys.DFS_REPLICATION_KEY,
DFSConfigKeys.DFS_REPLICATION_DEFAULT);
final int maxR = conf.getInt(DFSConfigKeys.DFS_REPLICATION_MAX_KEY,
DFSConfigKeys.DFS_REPLICATION_MAX_DEFAULT);
final int minR = conf.getInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_MIN_KEY,
DFSConfigKeys.DFS_NAMENODE_REPLICATION_MIN_DEFAULT);
if (minR <= 0)
throw new IOException("Unexpected configuration parameters: "
+ DFSConfigKeys.DFS_NAMENODE_REPLICATION_MIN_KEY
+ " = " + minR + " <= 0");
if (maxR > Short.MAX_VALUE)
throw new IOException("Unexpected configuration parameters: "
+ DFSConfigKeys.DFS_REPLICATION_MAX_KEY
+ " = " + maxR + " > " + Short.MAX_VALUE);
if (minR > maxR)
throw new IOException("Unexpected configuration parameters: "
+ DFSConfigKeys.DFS_NAMENODE_REPLICATION_MIN_KEY
+ " = " + minR + " > "
+ DFSConfigKeys.DFS_REPLICATION_MAX_KEY
+ " = " + maxR);
this.minReplication = (short)minR;
this.maxReplication = (short)maxR;
this.maxReplicationStreams =
conf.getInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_MAX_STREAMS_KEY,
DFSConfigKeys.DFS_NAMENODE_REPLICATION_MAX_STREAMS_DEFAULT);
this.replicationStreamsHardLimit =
conf.getInt(
DFSConfigKeys.DFS_NAMENODE_REPLICATION_STREAMS_HARD_LIMIT_KEY,
DFSConfigKeys.DFS_NAMENODE_REPLICATION_STREAMS_HARD_LIMIT_DEFAULT);
this.shouldCheckForEnoughRacks =
conf.get(DFSConfigKeys.NET_TOPOLOGY_SCRIPT_FILE_NAME_KEY) == null
? false : true;
this.blocksInvalidateWorkPct = DFSUtil.getInvalidateWorkPctPerIteration(conf);
this.blocksReplWorkMultiplier = DFSUtil.getReplWorkMultiplier(conf);
this.replicationRecheckInterval =
conf.getInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_KEY,
DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_DEFAULT) * 1000L;
this.encryptDataTransfer =
conf.getBoolean(DFSConfigKeys.DFS_ENCRYPT_DATA_TRANSFER_KEY,
DFSConfigKeys.DFS_ENCRYPT_DATA_TRANSFER_DEFAULT);
this.maxNumBlocksToLog =
conf.getLong(DFSConfigKeys.DFS_MAX_NUM_BLOCKS_TO_LOG_KEY,
DFSConfigKeys.DFS_MAX_NUM_BLOCKS_TO_LOG_DEFAULT);
this.numBlocksPerIteration = conf.getInt(
DFSConfigKeys.DFS_BLOCK_MISREPLICATION_PROCESSING_LIMIT,
DFSConfigKeys.DFS_BLOCK_MISREPLICATION_PROCESSING_LIMIT_DEFAULT);
LOG.info("defaultReplication = " + defaultReplication);
LOG.info("maxReplication = " + maxReplication);
LOG.info("minReplication = " + minReplication);
LOG.info("maxReplicationStreams = " + maxReplicationStreams);
LOG.info("shouldCheckForEnoughRacks = " + shouldCheckForEnoughRacks);
LOG.info("replicationRecheckInterval = " + replicationRecheckInterval);
LOG.info("encryptDataTransfer = " + encryptDataTransfer);
LOG.info("maxNumBlocksToLog = " + maxNumBlocksToLog);
}
private static BlockTokenSecretManager createBlockTokenSecretManager(
final Configuration conf) throws IOException {
final boolean isEnabled = conf.getBoolean(
DFSConfigKeys.DFS_BLOCK_ACCESS_TOKEN_ENABLE_KEY,
DFSConfigKeys.DFS_BLOCK_ACCESS_TOKEN_ENABLE_DEFAULT);
LOG.info(DFSConfigKeys.DFS_BLOCK_ACCESS_TOKEN_ENABLE_KEY + "=" + isEnabled);
if (!isEnabled) {
if (UserGroupInformation.isSecurityEnabled()) {
String errMessage = "Security is enabled but block access tokens " +
"(via " + DFSConfigKeys.DFS_BLOCK_ACCESS_TOKEN_ENABLE_KEY + ") " +
"aren't enabled. This may cause issues " +
"when clients attempt to connect to a DataNode. Aborting NameNode";
throw new IOException(errMessage);
}
return null;
}
final long updateMin = conf.getLong(
DFSConfigKeys.DFS_BLOCK_ACCESS_KEY_UPDATE_INTERVAL_KEY,
DFSConfigKeys.DFS_BLOCK_ACCESS_KEY_UPDATE_INTERVAL_DEFAULT);
final long lifetimeMin = conf.getLong(
DFSConfigKeys.DFS_BLOCK_ACCESS_TOKEN_LIFETIME_KEY,
DFSConfigKeys.DFS_BLOCK_ACCESS_TOKEN_LIFETIME_DEFAULT);
final String encryptionAlgorithm = conf.get(
DFSConfigKeys.DFS_DATA_ENCRYPTION_ALGORITHM_KEY);
LOG.info(DFSConfigKeys.DFS_BLOCK_ACCESS_KEY_UPDATE_INTERVAL_KEY
+ "=" + updateMin + " min(s), "
+ DFSConfigKeys.DFS_BLOCK_ACCESS_TOKEN_LIFETIME_KEY
+ "=" + lifetimeMin + " min(s), "
+ DFSConfigKeys.DFS_DATA_ENCRYPTION_ALGORITHM_KEY
+ "=" + encryptionAlgorithm);
String nsId = DFSUtil.getNamenodeNameServiceId(conf);
boolean isHaEnabled = HAUtil.isHAEnabled(conf, nsId);
if (isHaEnabled) {
String thisNnId = HAUtil.getNameNodeId(conf, nsId);
String otherNnId = HAUtil.getNameNodeIdOfOtherNode(conf, nsId);
return new BlockTokenSecretManager(updateMin*60*1000L,
lifetimeMin*60*1000L, thisNnId.compareTo(otherNnId) < 0 ? 0 : 1, null,
encryptionAlgorithm);
} else {
return new BlockTokenSecretManager(updateMin*60*1000L,
lifetimeMin*60*1000L, 0, null, encryptionAlgorithm);
}
}
public BlockStoragePolicy getStoragePolicy(final String policyName) {
return storagePolicySuite.getPolicy(policyName);
}
public BlockStoragePolicy getStoragePolicy(final byte policyId) {
return storagePolicySuite.getPolicy(policyId);
}
public BlockStoragePolicy[] getStoragePolicies() {
return storagePolicySuite.getAllPolicies();
}
public void setBlockPoolId(String blockPoolId) {
if (isBlockTokenEnabled()) {
blockTokenSecretManager.setBlockPoolId(blockPoolId);
}
}
public BlockStoragePolicySuite getStoragePolicySuite() {
return storagePolicySuite;
}
/** get the BlockTokenSecretManager */
@VisibleForTesting
public BlockTokenSecretManager getBlockTokenSecretManager() {
return blockTokenSecretManager;
}
/** Allow silent termination of replication monitor for testing */
@VisibleForTesting
void enableRMTerminationForTesting() {
checkNSRunning = false;
}
private boolean isBlockTokenEnabled() {
return blockTokenSecretManager != null;
}
/** Should the access keys be updated? */
boolean shouldUpdateBlockKey(final long updateTime) throws IOException {
return isBlockTokenEnabled()? blockTokenSecretManager.updateKeys(updateTime)
: false;
}
public void activate(Configuration conf) {
pendingReplications.start();
datanodeManager.activate(conf);
this.replicationThread.start();
}
public void close() {
try {
replicationThread.interrupt();
replicationThread.join(3000);
} catch (InterruptedException ie) {
}
datanodeManager.close();
pendingReplications.stop();
blocksMap.close();
}
/** @return the datanodeManager */
public DatanodeManager getDatanodeManager() {
return datanodeManager;
}
@VisibleForTesting
public BlockPlacementPolicy getBlockPlacementPolicy() {
return placementPolicies.getPolicy(false);
}
/** Dump meta data to out. */
public void metaSave(PrintWriter out) {
assert namesystem.hasWriteLock();
final List<DatanodeDescriptor> live = new ArrayList<DatanodeDescriptor>();
final List<DatanodeDescriptor> dead = new ArrayList<DatanodeDescriptor>();
datanodeManager.fetchDatanodes(live, dead, false);
out.println("Live Datanodes: " + live.size());
out.println("Dead Datanodes: " + dead.size());
//
// Dump contents of neededReplication
//
synchronized (neededReplications) {
out.println("Metasave: Blocks waiting for replication: " +
neededReplications.size());
for (Block block : neededReplications) {
dumpBlockMeta(block, out);
}
}
// Dump any postponed over-replicated blocks
out.println("Mis-replicated blocks that have been postponed:");
for (Block block : postponedMisreplicatedBlocks) {
dumpBlockMeta(block, out);
}
// Dump blocks from pendingReplication
pendingReplications.metaSave(out);
// Dump blocks that are waiting to be deleted
invalidateBlocks.dump(out);
// Dump all datanodes
getDatanodeManager().datanodeDump(out);
}
/**
* Dump the metadata for the given block in a human-readable
* form.
*/
private void dumpBlockMeta(Block block, PrintWriter out) {
List<DatanodeDescriptor> containingNodes =
new ArrayList<DatanodeDescriptor>();
List<DatanodeStorageInfo> containingLiveReplicasNodes =
new ArrayList<DatanodeStorageInfo>();
NumberReplicas numReplicas = new NumberReplicas();
// source node returned is not used
chooseSourceDatanodes(getStoredBlock(block), containingNodes,
containingLiveReplicasNodes, numReplicas,
new LinkedList<Short>(), UnderReplicatedBlocks.LEVEL);
// containingLiveReplicasNodes can include READ_ONLY_SHARED replicas which are
// not included in the numReplicas.liveReplicas() count
assert containingLiveReplicasNodes.size() >= numReplicas.liveReplicas();
int usableReplicas = numReplicas.liveReplicas() +
numReplicas.decommissionedAndDecommissioning();
if (block instanceof BlockInfo) {
BlockCollection bc = ((BlockInfo) block).getBlockCollection();
String fileName = (bc == null) ? "[orphaned]" : bc.getName();
out.print(fileName + ": ");
}
// l: == live:, d: == decommissioned c: == corrupt e: == excess
out.print(block + ((usableReplicas > 0)? "" : " MISSING") +
" (replicas:" +
" l: " + numReplicas.liveReplicas() +
" d: " + numReplicas.decommissionedAndDecommissioning() +
" c: " + numReplicas.corruptReplicas() +
" e: " + numReplicas.excessReplicas() + ") ");
Collection<DatanodeDescriptor> corruptNodes =
corruptReplicas.getNodes(block);
for (DatanodeStorageInfo storage : blocksMap.getStorages(block)) {
final DatanodeDescriptor node = storage.getDatanodeDescriptor();
String state = "";
if (corruptNodes != null && corruptNodes.contains(node)) {
state = "(corrupt)";
} else if (node.isDecommissioned() ||
node.isDecommissionInProgress()) {
state = "(decommissioned)";
}
if (storage.areBlockContentsStale()) {
state += " (block deletions maybe out of date)";
}
out.print(" " + node + state + " : ");
}
out.println("");
}
/** @return maxReplicationStreams */
public int getMaxReplicationStreams() {
return maxReplicationStreams;
}
public int getDefaultStorageNum(BlockInfo block) {
if (block.isStriped()) {
return ((BlockInfoStriped) block).getRealTotalBlockNum();
} else {
return defaultReplication;
}
}
public short getMinStorageNum(BlockInfo block) {
if (block.isStriped()) {
return ((BlockInfoStriped) block).getRealDataBlockNum();
} else {
return minReplication;
}
}
public boolean hasMinStorage(BlockInfo block) {
return countNodes(block).liveReplicas() >= getMinStorageNum(block);
}
public boolean hasMinStorage(BlockInfo block, int liveNum) {
return liveNum >= getMinStorageNum(block);
}
/**
* Commit a block of a file
*
* @param block block to be committed
* @param commitBlock - contains client reported block length and generation
* @return true if the block is changed to committed state.
* @throws IOException if the block does not have at least a minimal number
* of replicas reported from data-nodes.
*/
private static boolean commitBlock(final BlockInfo block,
final Block commitBlock) throws IOException {
if (block instanceof BlockInfoUnderConstruction
&& block.getBlockUCState() != BlockUCState.COMMITTED) {
final BlockInfoUnderConstruction uc = (BlockInfoUnderConstruction)block;
assert block.getNumBytes() <= commitBlock.getNumBytes() :
"commitBlock length is less than the stored one "
+ commitBlock.getNumBytes() + " vs. " + block.getNumBytes();
uc.commitBlock(commitBlock);
return true;
}
return false;
}
/**
* Commit the last block of the file and mark it as complete if it has
* meets the minimum replication requirement
*
* @param bc block collection
* @param commitBlock - contains client reported block length and generation
* @return true if the last block is changed to committed state.
* @throws IOException if the block does not have at least a minimal number
* of replicas reported from data-nodes.
*/
public boolean commitOrCompleteLastBlock(BlockCollection bc,
Block commitBlock) throws IOException {
if(commitBlock == null)
return false; // not committing, this is a block allocation retry
BlockInfo lastBlock = bc.getLastBlock();
if(lastBlock == null)
return false; // no blocks in file yet
if(lastBlock.isComplete())
return false; // already completed (e.g. by syncBlock)
final boolean b = commitBlock(lastBlock, commitBlock);
if (hasMinStorage(lastBlock)) {
completeBlock(bc, bc.numBlocks() - 1, false);
}
return b;
}
/**
* Convert a specified block of the file to a complete block.
* @param bc file
* @param blkIndex block index in the file
* @throws IOException if the block does not have at least a minimal number
* of replicas reported from data-nodes.
*/
private BlockInfo completeBlock(final BlockCollection bc,
final int blkIndex, boolean force) throws IOException {
if (blkIndex < 0) {
return null;
}
BlockInfo curBlock = bc.getBlocks()[blkIndex];
if (curBlock.isComplete()) {
return curBlock;
}
int numNodes = curBlock.numNodes();
if (!force && !hasMinStorage(curBlock, numNodes)) {
throw new IOException("Cannot complete block: " +
"block does not satisfy minimal replication requirement.");
}
if (!force && curBlock.getBlockUCState() != BlockUCState.COMMITTED) {
throw new IOException(
"Cannot complete block: block has not been COMMITTED by the client");
}
final BlockInfo completeBlock
= !(curBlock instanceof BlockInfoUnderConstruction)? curBlock
: ((BlockInfoUnderConstruction)curBlock).convertToCompleteBlock();
// replace penultimate block in file
bc.setBlock(blkIndex, completeBlock);
// Since safe-mode only counts complete blocks, and we now have
// one more complete block, we need to adjust the total up, and
// also count it as safe, if we have at least the minimum replica
// count. (We may not have the minimum replica count yet if this is
// a "forced" completion when a file is getting closed by an
// OP_CLOSE edit on the standby).
namesystem.adjustSafeModeBlockTotals(0, 1);
final int minStorage = curBlock.isStriped() ?
((BlockInfoStriped) curBlock).getRealDataBlockNum() : minReplication;
namesystem.incrementSafeBlockCount(
Math.min(numNodes, minStorage), curBlock);
// replace block in the blocksMap
return blocksMap.replaceBlock(completeBlock);
}
private BlockInfo completeBlock(final BlockCollection bc,
final BlockInfo block, boolean force) throws IOException {
BlockInfo[] fileBlocks = bc.getBlocks();
for (int idx = 0; idx < fileBlocks.length; idx++) {
if (fileBlocks[idx] == block) {
return completeBlock(bc, idx, force);
}
}
return block;
}
/**
* Force the given block in the given file to be marked as complete,
* regardless of whether enough replicas are present. This is necessary
* when tailing edit logs as a Standby.
*/
public BlockInfo forceCompleteBlock(final BlockCollection bc,
final BlockInfo block) throws IOException {
if (block instanceof BlockInfoUnderConstruction) {
((BlockInfoUnderConstruction)block).commitBlock(block);
}
return completeBlock(bc, block, true);
}
/**
* Convert the last block of the file to an under construction block.<p>
* The block is converted only if the file has blocks and the last one
* is a partial block (its size is less than the preferred block size).
* The converted block is returned to the client.
* The client uses the returned block locations to form the data pipeline
* for this block.<br>
* The methods returns null if there is no partial block at the end.
* The client is supposed to allocate a new block with the next call.
*
* @param bc file
* @param bytesToRemove num of bytes to remove from block
* @return the last block locations if the block is partial or null otherwise
*/
public LocatedBlock convertLastBlockToUnderConstruction(
BlockCollection bc, long bytesToRemove) throws IOException {
BlockInfo oldBlock = bc.getLastBlock();
if(oldBlock == null ||
bc.getPreferredBlockSize() == oldBlock.getNumBytes() - bytesToRemove)
return null;
assert oldBlock == getStoredBlock(oldBlock) :
"last block of the file is not in blocksMap";
DatanodeStorageInfo[] targets = getStorages(oldBlock);
// convert the last block to UC
bc.convertLastBlockToUC(oldBlock, targets);
// get the new created uc block
BlockInfo ucBlock = bc.getLastBlock();
blocksMap.replaceBlock(ucBlock);
// Remove block from replication queue.
NumberReplicas replicas = countNodes(ucBlock);
neededReplications.remove(ucBlock, replicas.liveReplicas(),
replicas.decommissionedAndDecommissioning(), getReplication(ucBlock));
pendingReplications.remove(ucBlock);
// remove this block from the list of pending blocks to be deleted.
for (DatanodeStorageInfo storage : targets) {
invalidateBlocks.remove(storage.getDatanodeDescriptor(), oldBlock);
}
// Adjust safe-mode totals, since under-construction blocks don't
// count in safe-mode.
namesystem.adjustSafeModeBlockTotals(
// decrement safe if we had enough
hasMinStorage(oldBlock, targets.length) ? -1 : 0,
// always decrement total blocks
-1);
final long fileLength = bc.computeContentSummary(getStoragePolicySuite()).getLength();
final long pos = fileLength - ucBlock.getNumBytes();
return createLocatedBlock(ucBlock, pos, BlockTokenIdentifier.AccessMode.WRITE);
}
/**
* Get all valid locations of the block
*/
private List<DatanodeStorageInfo> getValidLocations(Block block) {
final List<DatanodeStorageInfo> locations
= new ArrayList<DatanodeStorageInfo>(blocksMap.numNodes(block));
for(DatanodeStorageInfo storage : blocksMap.getStorages(block)) {
// filter invalidate replicas
if(!invalidateBlocks.contains(storage.getDatanodeDescriptor(), block)) {
locations.add(storage);
}
}
return locations;
}
private List<LocatedBlock> createLocatedBlockList(final BlockInfo[] blocks,
final long offset, final long length, final int nrBlocksToReturn,
final AccessMode mode) throws IOException {
int curBlk;
long curPos = 0, blkSize = 0;
int nrBlocks = (blocks[0].getNumBytes() == 0) ? 0 : blocks.length;
for (curBlk = 0; curBlk < nrBlocks; curBlk++) {
blkSize = blocks[curBlk].getNumBytes();
assert blkSize > 0 : "Block of size 0";
if (curPos + blkSize > offset) {
break;
}
curPos += blkSize;
}
if (nrBlocks > 0 && curBlk == nrBlocks) // offset >= end of file
return Collections.emptyList();
long endOff = offset + length;
List<LocatedBlock> results = new ArrayList<>(blocks.length);
do {
results.add(createLocatedBlock(blocks[curBlk], curPos, mode));
curPos += blocks[curBlk].getNumBytes();
curBlk++;
} while (curPos < endOff
&& curBlk < blocks.length
&& results.size() < nrBlocksToReturn);
return results;
}
private LocatedBlock createLocatedBlock(final BlockInfo[] blocks,
final long endPos, final AccessMode mode) throws IOException {
int curBlk;
long curPos = 0;
int nrBlocks = (blocks[0].getNumBytes() == 0) ? 0 : blocks.length;
for (curBlk = 0; curBlk < nrBlocks; curBlk++) {
long blkSize = blocks[curBlk].getNumBytes();
if (curPos + blkSize >= endPos) {
break;
}
curPos += blkSize;
}
return createLocatedBlock(blocks[curBlk], curPos, mode);
}
private LocatedBlock createLocatedBlock(final BlockInfo blk, final long pos,
final AccessMode mode) throws IOException {
final LocatedBlock lb = createLocatedBlock(blk, pos);
if (mode != null) {
setBlockToken(lb, mode);
}
return lb;
}
/** @return a LocatedBlock for the given block */
private LocatedBlock createLocatedBlock(final BlockInfo blk, final long pos) {
if (!blk.isComplete()) {
if (blk.isStriped()) {
final BlockInfoStripedUnderConstruction uc =
(BlockInfoStripedUnderConstruction) blk;
final DatanodeStorageInfo[] storages = uc.getExpectedStorageLocations();
final ExtendedBlock eb = new ExtendedBlock(namesystem.getBlockPoolId(),
blk);
return newLocatedStripedBlock(eb, storages, uc.getBlockIndices(), pos,
false);
} else {
assert blk instanceof BlockInfoContiguousUnderConstruction;
final BlockInfoContiguousUnderConstruction uc =
(BlockInfoContiguousUnderConstruction) blk;
final DatanodeStorageInfo[] storages = uc.getExpectedStorageLocations();
final ExtendedBlock eb = new ExtendedBlock(namesystem.getBlockPoolId(),
blk);
return newLocatedBlock(eb, storages, pos, false);
}
}
// get block locations
final int numCorruptNodes = countNodes(blk).corruptReplicas();
final int numCorruptReplicas = corruptReplicas.numCorruptReplicas(blk);
if (numCorruptNodes != numCorruptReplicas) {
LOG.warn("Inconsistent number of corrupt replicas for "
+ blk + " blockMap has " + numCorruptNodes
+ " but corrupt replicas map has " + numCorruptReplicas);
}
final int numNodes = blocksMap.numNodes(blk);
final boolean isCorrupt = numCorruptNodes != 0 &&
numCorruptNodes == numNodes;
final int numMachines = isCorrupt ? numNodes: numNodes - numCorruptNodes;
final DatanodeStorageInfo[] machines = new DatanodeStorageInfo[numMachines];
final int[] blockIndices = blk.isStriped() ? new int[numMachines] : null;
int j = 0, i = 0;
if (numMachines > 0) {
for(DatanodeStorageInfo storage : blocksMap.getStorages(blk)) {
final DatanodeDescriptor d = storage.getDatanodeDescriptor();
final boolean replicaCorrupt = corruptReplicas.isReplicaCorrupt(blk, d);
if (isCorrupt || (!replicaCorrupt)) {
machines[j++] = storage;
// TODO this can be more efficient
if (blockIndices != null) {
int index = ((BlockInfoStriped) blk).getStorageBlockIndex(storage);
assert index >= 0;
blockIndices[i++] = index;
}
}
}
}
assert j == machines.length :
"isCorrupt: " + isCorrupt +
" numMachines: " + numMachines +
" numNodes: " + numNodes +
" numCorrupt: " + numCorruptNodes +
" numCorruptRepls: " + numCorruptReplicas;
final ExtendedBlock eb = new ExtendedBlock(namesystem.getBlockPoolId(), blk);
return blockIndices == null ?
newLocatedBlock(eb, machines, pos, isCorrupt) :
newLocatedStripedBlock(eb, machines, blockIndices, pos, isCorrupt);
}
/** Create a LocatedBlocks. */
public LocatedBlocks createLocatedBlocks(final BlockInfo[] blocks,
final long fileSizeExcludeBlocksUnderConstruction,
final boolean isFileUnderConstruction, final long offset,
final long length, final boolean needBlockToken,
final boolean inSnapshot, FileEncryptionInfo feInfo,
ErasureCodingZone ecZone)
throws IOException {
assert namesystem.hasReadLock();
final ECSchema schema = ecZone != null ? ecZone.getSchema() : null;
final int cellSize = ecZone != null ? ecZone.getCellSize() : 0;
if (blocks == null) {
return null;
} else if (blocks.length == 0) {
return new LocatedBlocks(0, isFileUnderConstruction,
Collections.<LocatedBlock> emptyList(), null, false, feInfo, schema,
cellSize);
} else {
if (LOG.isDebugEnabled()) {
LOG.debug("blocks = " + java.util.Arrays.asList(blocks));
}
final AccessMode mode = needBlockToken? BlockTokenIdentifier.AccessMode.READ: null;
final List<LocatedBlock> locatedblocks = createLocatedBlockList(
blocks, offset, length, Integer.MAX_VALUE, mode);
final LocatedBlock lastlb;
final boolean isComplete;
if (!inSnapshot) {
final BlockInfo last = blocks[blocks.length - 1];
final long lastPos = last.isComplete()?
fileSizeExcludeBlocksUnderConstruction - last.getNumBytes()
: fileSizeExcludeBlocksUnderConstruction;
lastlb = createLocatedBlock(last, lastPos, mode);
isComplete = last.isComplete();
} else {
lastlb = createLocatedBlock(blocks,
fileSizeExcludeBlocksUnderConstruction, mode);
isComplete = true;
}
return new LocatedBlocks(fileSizeExcludeBlocksUnderConstruction,
isFileUnderConstruction, locatedblocks, lastlb, isComplete, feInfo,
schema, cellSize);
}
}
/** @return current access keys. */
public ExportedBlockKeys getBlockKeys() {
return isBlockTokenEnabled()? blockTokenSecretManager.exportKeys()
: ExportedBlockKeys.DUMMY_KEYS;
}
/** Generate a block token for the located block. */
public void setBlockToken(final LocatedBlock b,
final AccessMode mode) throws IOException {
if (isBlockTokenEnabled()) {
// Use cached UGI if serving RPC calls.
b.setBlockToken(blockTokenSecretManager.generateToken(
NameNode.getRemoteUser().getShortUserName(),
b.getBlock(), EnumSet.of(mode)));
}
}
void addKeyUpdateCommand(final List<DatanodeCommand> cmds,
final DatanodeDescriptor nodeinfo) {
// check access key update