36
36
import static org .apache .hadoop .hdfs .DFSConfigKeys .DFS_DATANODE_ALLOW_SAME_DISK_TIERING ;
37
37
import static org .apache .hadoop .hdfs .DFSConfigKeys .DFS_DATANODE_ALLOW_SAME_DISK_TIERING_DEFAULT ;
38
38
import static org .apache .hadoop .hdfs .DFSConfigKeys .DFS_DATANODE_DATA_DIR_KEY ;
39
+ import static org .apache .hadoop .hdfs .DFSConfigKeys .DFS_DATANODE_DATA_DIR_TO_ADD_KEY ;
40
+ import static org .apache .hadoop .hdfs .DFSConfigKeys .DFS_DATANODE_DATA_DIR_TO_REMOVE_KEY ;
39
41
import static org .apache .hadoop .hdfs .DFSConfigKeys .DFS_DATANODE_DATA_TRANSFER_BANDWIDTHPERSEC_DEFAULT ;
40
42
import static org .apache .hadoop .hdfs .DFSConfigKeys .DFS_DATANODE_DATA_TRANSFER_BANDWIDTHPERSEC_KEY ;
41
43
import static org .apache .hadoop .hdfs .DFSConfigKeys .DFS_DATANODE_DATA_WRITE_BANDWIDTHPERSEC_DEFAULT ;
138
140
import java .util .Map .Entry ;
139
141
import java .util .Set ;
140
142
import java .util .UUID ;
141
- import java .util .concurrent .Callable ;
142
143
import java .util .concurrent .ConcurrentHashMap ;
143
144
import java .util .concurrent .ExecutionException ;
144
145
import java .util .concurrent .ExecutorService ;
@@ -351,6 +352,8 @@ public class DataNode extends ReconfigurableBase
351
352
Collections .unmodifiableList (
352
353
Arrays .asList (
353
354
DFS_DATANODE_DATA_DIR_KEY ,
355
+ DFS_DATANODE_DATA_DIR_TO_ADD_KEY ,
356
+ DFS_DATANODE_DATA_DIR_TO_REMOVE_KEY ,
354
357
DFS_DATANODE_BALANCE_MAX_NUM_CONCURRENT_MOVES_KEY ,
355
358
DFS_BLOCKREPORT_INTERVAL_MSEC_KEY ,
356
359
DFS_BLOCKREPORT_SPLIT_THRESHOLD_KEY ,
@@ -639,34 +642,10 @@ protected Configuration getNewConf() {
639
642
public String reconfigurePropertyImpl (String property , String newVal )
640
643
throws ReconfigurationException {
641
644
switch (property ) {
642
- case DFS_DATANODE_DATA_DIR_KEY : {
643
- IOException rootException = null ;
644
- try {
645
- LOG .info ("Reconfiguring {} to {}" , property , newVal );
646
- this .refreshVolumes (newVal );
647
- return getConf ().get (DFS_DATANODE_DATA_DIR_KEY );
648
- } catch (IOException e ) {
649
- rootException = e ;
650
- } finally {
651
- // Send a full block report to let NN acknowledge the volume changes.
652
- try {
653
- triggerBlockReport (
654
- new BlockReportOptions .Factory ().setIncremental (false ).build ());
655
- } catch (IOException e ) {
656
- LOG .warn ("Exception while sending the block report after refreshing"
657
- + " volumes {} to {}" , property , newVal , e );
658
- if (rootException == null ) {
659
- rootException = e ;
660
- }
661
- } finally {
662
- if (rootException != null ) {
663
- throw new ReconfigurationException (property , newVal ,
664
- getConf ().get (property ), rootException );
665
- }
666
- }
667
- }
668
- break ;
669
- }
645
+ case DFS_DATANODE_DATA_DIR_KEY :
646
+ case DFS_DATANODE_DATA_DIR_TO_ADD_KEY :
647
+ case DFS_DATANODE_DATA_DIR_TO_REMOVE_KEY :
648
+ return reconfDataDirsParameters (property , newVal );
670
649
case DFS_DATANODE_BALANCE_MAX_NUM_CONCURRENT_MOVES_KEY : {
671
650
ReconfigurationException rootException = null ;
672
651
try {
@@ -1079,6 +1058,43 @@ private String reconfSlowIoWarningThresholdParameters(String property, String ne
1079
1058
}
1080
1059
}
1081
1060
1061
+ private String reconfDataDirsParameters (String property , String newVal )
1062
+ throws ReconfigurationException {
1063
+ LOG .info ("Reconfiguring {} to {}" , property , newVal );
1064
+ Set <String > allDataDirs = new HashSet <>();
1065
+ List <StorageLocation > storageLocations = getStorageLocations (getConf ());
1066
+ for (StorageLocation location : storageLocations ) {
1067
+ allDataDirs .add (location .getNormalizedUri ().getPath ());
1068
+ }
1069
+ if (property .equals (DFS_DATANODE_DATA_DIR_TO_ADD_KEY )) {
1070
+ Set <String > toAdd = new HashSet <>(Arrays .asList (newVal .split ("," )));
1071
+ allDataDirs .addAll (toAdd );
1072
+ } else if (property .equals (DFS_DATANODE_DATA_DIR_TO_REMOVE_KEY )) {
1073
+ Set <String > toRemove = new HashSet <>(Arrays .asList (newVal .split ("," )));
1074
+ allDataDirs .removeAll (toRemove );
1075
+ }
1076
+ String newDirs = String .join ("," , allDataDirs );
1077
+ IOException rootException = null ;
1078
+ try {
1079
+ this .refreshVolumes (newDirs );
1080
+ return getConf ().get (DFS_DATANODE_DATA_DIR_KEY );
1081
+ } catch (IOException e ) {
1082
+ rootException = e ;
1083
+ } finally {
1084
+ // Send a full block report to let NN acknowledge the volume changes.
1085
+ try {
1086
+ triggerBlockReport (new BlockReportOptions .Factory ().setIncremental (false ).build ());
1087
+ } catch (IOException e ) {
1088
+ LOG .warn ("Exception while sending the block report after refreshing volumes {} to {}" ,
1089
+ property , newVal , e );
1090
+ if (rootException == null ) {
1091
+ rootException = e ;
1092
+ }
1093
+ }
1094
+ }
1095
+ throw new ReconfigurationException (property , newVal , getConf ().get (property ), rootException );
1096
+ }
1097
+
1082
1098
/**
1083
1099
* Get a list of the keys of the re-configurable properties in configuration.
1084
1100
*/
@@ -1286,10 +1302,9 @@ private void refreshVolumes(String newVolumes) throws IOException {
1286
1302
for (BPOfferService bpos : blockPoolManager .getAllNamenodeThreads ()) {
1287
1303
nsInfos .add (bpos .getNamespaceInfo ());
1288
1304
}
1289
- synchronized (this ) {
1305
+ synchronized (this ) {
1290
1306
Configuration conf = getConf ();
1291
1307
conf .set (DFS_DATANODE_DATA_DIR_KEY , newVolumes );
1292
- ExecutorService service = null ;
1293
1308
int numOldDataDirs = dataDirs .size ();
1294
1309
ChangedVolumes changedVolumes = parseChangedVolumes (newVolumes );
1295
1310
StringBuilder errorMessageBuilder = new StringBuilder ();
@@ -1305,49 +1320,9 @@ private void refreshVolumes(String newVolumes) throws IOException {
1305
1320
throw new IOException ("Attempt to remove all volumes." );
1306
1321
}
1307
1322
if (!changedVolumes .newLocations .isEmpty ()) {
1308
- LOG .info ("Adding new volumes: {}" ,
1309
- Joiner .on ("," ).join (changedVolumes .newLocations ));
1310
-
1311
- service = Executors
1312
- .newFixedThreadPool (changedVolumes .newLocations .size ());
1313
- List <Future <IOException >> exceptions = Lists .newArrayList ();
1314
-
1315
- checkStorageState ("refreshVolumes" );
1316
- for (final StorageLocation location : changedVolumes .newLocations ) {
1317
- exceptions .add (service .submit (new Callable <IOException >() {
1318
- @ Override
1319
- public IOException call () {
1320
- try {
1321
- data .addVolume (location , nsInfos );
1322
- } catch (IOException e ) {
1323
- return e ;
1324
- }
1325
- return null ;
1326
- }
1327
- }));
1328
- }
1329
-
1330
- for (int i = 0 ; i < changedVolumes .newLocations .size (); i ++) {
1331
- StorageLocation volume = changedVolumes .newLocations .get (i );
1332
- Future <IOException > ioExceptionFuture = exceptions .get (i );
1333
- try {
1334
- IOException ioe = ioExceptionFuture .get ();
1335
- if (ioe != null ) {
1336
- errorMessageBuilder .append (
1337
- String .format ("FAILED TO ADD: %s: %s%n" ,
1338
- volume , ioe .getMessage ()));
1339
- LOG .error ("Failed to add volume: {}" , volume , ioe );
1340
- } else {
1341
- effectiveVolumes .add (volume .toString ());
1342
- LOG .info ("Successfully added volume: {}" , volume );
1343
- }
1344
- } catch (Exception e ) {
1345
- errorMessageBuilder .append (
1346
- String .format ("FAILED to ADD: %s: %s%n" , volume ,
1347
- e .toString ()));
1348
- LOG .error ("Failed to add volume: {}" , volume , e );
1349
- }
1350
- }
1323
+ List <String > addedVolumes =
1324
+ addVolumes (changedVolumes .newLocations , nsInfos , errorMessageBuilder );
1325
+ effectiveVolumes .addAll (addedVolumes );
1351
1326
}
1352
1327
1353
1328
try {
@@ -1361,16 +1336,65 @@ public IOException call() {
1361
1336
throw new IOException (errorMessageBuilder .toString ());
1362
1337
}
1363
1338
} finally {
1364
- if (service != null ) {
1365
- service .shutdown ();
1366
- }
1367
1339
conf .set (DFS_DATANODE_DATA_DIR_KEY ,
1368
1340
Joiner .on ("," ).join (effectiveVolumes ));
1369
1341
dataDirs = getStorageLocations (conf );
1370
1342
}
1371
1343
}
1372
1344
}
1373
1345
1346
+ /**
1347
+ * Add volumes from DataNode.
1348
+ *
1349
+ * @param locations the StorageLocations of the volumes to be added.
1350
+ * @throws IOException storage not yet initialized
1351
+ */
1352
+ private List <String > addVolumes (final List <StorageLocation > locations ,
1353
+ List <NamespaceInfo > nsInfos , StringBuilder errorMessageBuilder ) throws IOException {
1354
+ LOG .info ("Adding new volumes: {}" , Joiner .on ("," ).join (locations ));
1355
+ List <String > effectiveVolumes = new ArrayList <>();
1356
+ ExecutorService service = null ;
1357
+ List <Future <IOException >> exceptions = Lists .newArrayList ();
1358
+ checkStorageState ("refreshVolumes" );
1359
+ try {
1360
+ service = Executors .newFixedThreadPool (locations .size ());
1361
+ for (final StorageLocation location : locations ) {
1362
+ exceptions .add (service .submit (() -> {
1363
+ try {
1364
+ data .addVolume (location , nsInfos );
1365
+ } catch (IOException e ) {
1366
+ return e ;
1367
+ }
1368
+ return null ;
1369
+ }));
1370
+ }
1371
+
1372
+ for (int i = 0 ; i < locations .size (); i ++) {
1373
+ StorageLocation volume = locations .get (i );
1374
+ Future <IOException > ioExceptionFuture = exceptions .get (i );
1375
+ try {
1376
+ IOException ioe = ioExceptionFuture .get ();
1377
+ if (ioe != null ) {
1378
+ errorMessageBuilder .append (
1379
+ String .format ("FAILED TO ADD: %s: %s%n" , volume , ioe .getMessage ()));
1380
+ LOG .error ("Failed to add volume: {}" , volume , ioe );
1381
+ } else {
1382
+ effectiveVolumes .add (volume .toString ());
1383
+ LOG .info ("Successfully added volume: {}" , volume );
1384
+ }
1385
+ } catch (Exception e ) {
1386
+ errorMessageBuilder .append (String .format ("FAILED to ADD: %s: %s%n" , volume , e ));
1387
+ LOG .error ("Failed to add volume: {}" , volume , e );
1388
+ }
1389
+ }
1390
+ } finally {
1391
+ if (service != null ) {
1392
+ service .shutdown ();
1393
+ }
1394
+ }
1395
+ return effectiveVolumes ;
1396
+ }
1397
+
1374
1398
/**
1375
1399
* Remove volumes from DataNode.
1376
1400
* See {@link #removeVolumes(Collection, boolean)} for details.
@@ -3308,6 +3332,19 @@ private static boolean checkFileSystemWithConfigured(
3308
3332
public static List <StorageLocation > getStorageLocations (Configuration conf ) {
3309
3333
Collection <String > rawLocations =
3310
3334
conf .getTrimmedStringCollection (DFS_DATANODE_DATA_DIR_KEY );
3335
+ // Compatible with local conf for service restarts
3336
+ Collection <String > addedList =
3337
+ conf .getTrimmedStringCollection (DFS_DATANODE_DATA_DIR_TO_ADD_KEY );
3338
+ for (String location : addedList ) {
3339
+ if (!rawLocations .contains (location )) {
3340
+ rawLocations .add (location );
3341
+ }
3342
+ }
3343
+ Collection <String > removedList =
3344
+ conf .getTrimmedStringCollection (DFS_DATANODE_DATA_DIR_TO_REMOVE_KEY );
3345
+ for (String location : removedList ) {
3346
+ rawLocations .remove (location );
3347
+ }
3311
3348
List <StorageLocation > locations =
3312
3349
new ArrayList <StorageLocation >(rawLocations .size ());
3313
3350
0 commit comments