|
@@ -400,6 +400,7 @@ public class DataNode extends ReconfigurableBase
|
|
private final DatasetVolumeChecker volumeChecker;
|
|
private final DatasetVolumeChecker volumeChecker;
|
|
|
|
|
|
private final SocketFactory socketFactory;
|
|
private final SocketFactory socketFactory;
|
|
|
|
+
|
|
private DatanodeStateMachine datanodeStateMachine;
|
|
private DatanodeStateMachine datanodeStateMachine;
|
|
|
|
|
|
private static Tracer createTracer(Configuration conf) {
|
|
private static Tracer createTracer(Configuration conf) {
|
|
@@ -1543,7 +1544,15 @@ public class DataNode extends ReconfigurableBase
|
|
+ bpRegistration.getDatanodeUuid()
|
|
+ bpRegistration.getDatanodeUuid()
|
|
+ ". Expecting " + storage.getDatanodeUuid());
|
|
+ ". Expecting " + storage.getDatanodeUuid());
|
|
}
|
|
}
|
|
-
|
|
|
|
|
|
+
|
|
|
|
+ if (isOzoneEnabled()) {
|
|
|
|
+ if (datanodeStateMachine == null) {
|
|
|
|
+ datanodeStateMachine = new DatanodeStateMachine(
|
|
|
|
+ getDatanodeId(),
|
|
|
|
+ getConf());
|
|
|
|
+ datanodeStateMachine.startDaemon();
|
|
|
|
+ }
|
|
|
|
+ }
|
|
registerBlockPoolWithSecretManager(bpRegistration, blockPoolId);
|
|
registerBlockPoolWithSecretManager(bpRegistration, blockPoolId);
|
|
}
|
|
}
|
|
|
|
|
|
@@ -1643,14 +1652,6 @@ public class DataNode extends ReconfigurableBase
|
|
data.addBlockPool(nsInfo.getBlockPoolID(), getConf());
|
|
data.addBlockPool(nsInfo.getBlockPoolID(), getConf());
|
|
blockScanner.enableBlockPoolId(bpos.getBlockPoolId());
|
|
blockScanner.enableBlockPoolId(bpos.getBlockPoolId());
|
|
initDirectoryScanner(getConf());
|
|
initDirectoryScanner(getConf());
|
|
- if(this.ozoneEnabled) {
|
|
|
|
- try {
|
|
|
|
- datanodeStateMachine = DatanodeStateMachine.initStateMachine(getConf());
|
|
|
|
- LOG.info("Ozone container server started.");
|
|
|
|
- } catch (IOException ex) {
|
|
|
|
- LOG.error("Unable to start Ozone. ex: {}", ex);
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
initDiskBalancer(data, getConf());
|
|
initDiskBalancer(data, getConf());
|
|
}
|
|
}
|
|
|
|
|
|
@@ -1661,11 +1662,11 @@ public class DataNode extends ReconfigurableBase
|
|
BPOfferService getBPOfferService(String bpid){
|
|
BPOfferService getBPOfferService(String bpid){
|
|
return blockPoolManager.get(bpid);
|
|
return blockPoolManager.get(bpid);
|
|
}
|
|
}
|
|
-
|
|
|
|
|
|
+
|
|
int getBpOsCount() {
|
|
int getBpOsCount() {
|
|
return blockPoolManager.getAllNamenodeThreads().size();
|
|
return blockPoolManager.getAllNamenodeThreads().size();
|
|
}
|
|
}
|
|
-
|
|
|
|
|
|
+
|
|
/**
|
|
/**
|
|
* Initializes the {@link #data}. The initialization is done only once, when
|
|
* Initializes the {@link #data}. The initialization is done only once, when
|
|
* handshake with the the first namenode is completed.
|
|
* handshake with the the first namenode is completed.
|
|
@@ -1998,125 +1999,119 @@ public class DataNode extends ReconfigurableBase
|
|
}
|
|
}
|
|
|
|
|
|
// Stop the object store handler
|
|
// Stop the object store handler
|
|
- if (this.objectStoreHandler != null) {
|
|
|
|
- this.objectStoreHandler.close();
|
|
|
|
|
|
+ if (isOzoneEnabled()) {
|
|
|
|
+ if (this.objectStoreHandler != null) {
|
|
|
|
+ this.objectStoreHandler.close();
|
|
|
|
+ }
|
|
}
|
|
}
|
|
|
|
|
|
- if (this.ozoneEnabled) {
|
|
|
|
- if (datanodeStateMachine != null) {
|
|
|
|
- try {
|
|
|
|
- datanodeStateMachine.close();
|
|
|
|
- } catch (Exception e) {
|
|
|
|
- LOG.error("Error is ozone shutdown. ex {}", e.toString());
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
- volumeChecker.shutdownAndWait(1, TimeUnit.SECONDS);
|
|
|
|
|
|
+ volumeChecker.shutdownAndWait(1, TimeUnit.SECONDS);
|
|
|
|
|
|
- if (storageLocationChecker != null) {
|
|
|
|
- storageLocationChecker.shutdownAndWait(1, TimeUnit.SECONDS);
|
|
|
|
- }
|
|
|
|
|
|
+ if (storageLocationChecker != null) {
|
|
|
|
+ storageLocationChecker.shutdownAndWait(1, TimeUnit.SECONDS);
|
|
|
|
+ }
|
|
|
|
|
|
- if (pauseMonitor != null) {
|
|
|
|
- pauseMonitor.stop();
|
|
|
|
- }
|
|
|
|
|
|
+ if (pauseMonitor != null) {
|
|
|
|
+ pauseMonitor.stop();
|
|
|
|
+ }
|
|
|
|
|
|
- // shouldRun is set to false here to prevent certain threads from exiting
|
|
|
|
- // before the restart prep is done.
|
|
|
|
- this.shouldRun = false;
|
|
|
|
|
|
+ // shouldRun is set to false here to prevent certain threads from exiting
|
|
|
|
+ // before the restart prep is done.
|
|
|
|
+ this.shouldRun = false;
|
|
|
|
|
|
- // wait reconfiguration thread, if any, to exit
|
|
|
|
- shutdownReconfigurationTask();
|
|
|
|
-
|
|
|
|
- // wait for all data receiver threads to exit
|
|
|
|
- if (this.threadGroup != null) {
|
|
|
|
- int sleepMs = 2;
|
|
|
|
- while (true) {
|
|
|
|
- // When shutting down for restart, wait 2.5 seconds before forcing
|
|
|
|
- // termination of receiver threads.
|
|
|
|
- if (!this.shutdownForUpgrade ||
|
|
|
|
- (this.shutdownForUpgrade && (Time.monotonicNow() - timeNotified
|
|
|
|
- > 1000))) {
|
|
|
|
- this.threadGroup.interrupt();
|
|
|
|
- break;
|
|
|
|
- }
|
|
|
|
- LOG.info("Waiting for threadgroup to exit, active threads is " +
|
|
|
|
- this.threadGroup.activeCount());
|
|
|
|
- if (this.threadGroup.activeCount() == 0) {
|
|
|
|
- break;
|
|
|
|
- }
|
|
|
|
- try {
|
|
|
|
- Thread.sleep(sleepMs);
|
|
|
|
- } catch (InterruptedException e) {
|
|
|
|
- }
|
|
|
|
- sleepMs = sleepMs * 3 / 2; // exponential backoff
|
|
|
|
- if (sleepMs > 200) {
|
|
|
|
- sleepMs = 200;
|
|
|
|
- }
|
|
|
|
|
|
+ // wait reconfiguration thread, if any, to exit
|
|
|
|
+ shutdownReconfigurationTask();
|
|
|
|
+
|
|
|
|
+ // wait for all data receiver threads to exit
|
|
|
|
+ if (this.threadGroup != null) {
|
|
|
|
+ int sleepMs = 2;
|
|
|
|
+ while (true) {
|
|
|
|
+ // When shutting down for restart, wait 2.5 seconds before forcing
|
|
|
|
+ // termination of receiver threads.
|
|
|
|
+ if (!this.shutdownForUpgrade || (this.shutdownForUpgrade && (
|
|
|
|
+ Time.monotonicNow() - timeNotified > 1000))) {
|
|
|
|
+ this.threadGroup.interrupt();
|
|
|
|
+ break;
|
|
}
|
|
}
|
|
- this.threadGroup = null;
|
|
|
|
- }
|
|
|
|
- if (this.dataXceiverServer != null) {
|
|
|
|
- // wait for dataXceiverServer to terminate
|
|
|
|
- try {
|
|
|
|
- this.dataXceiverServer.join();
|
|
|
|
- } catch (InterruptedException ie) {
|
|
|
|
|
|
+ LOG.info("Waiting for threadgroup to exit, active threads is "
|
|
|
|
+ + this.threadGroup.activeCount());
|
|
|
|
+ if (this.threadGroup.activeCount() == 0) {
|
|
|
|
+ break;
|
|
}
|
|
}
|
|
- }
|
|
|
|
- if (this.localDataXceiverServer != null) {
|
|
|
|
- // wait for localDataXceiverServer to terminate
|
|
|
|
try {
|
|
try {
|
|
- this.localDataXceiverServer.join();
|
|
|
|
- } catch (InterruptedException ie) {
|
|
|
|
|
|
+ Thread.sleep(sleepMs);
|
|
|
|
+ } catch (InterruptedException e) {
|
|
}
|
|
}
|
|
|
|
+ sleepMs = sleepMs * 3 / 2; // exponential backoff
|
|
|
|
+ if (sleepMs > 200) {
|
|
|
|
+ sleepMs = 200;
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+ this.threadGroup = null;
|
|
|
|
+ }
|
|
|
|
+ if (this.dataXceiverServer != null) {
|
|
|
|
+ // wait for dataXceiverServer to terminate
|
|
|
|
+ try {
|
|
|
|
+ this.dataXceiverServer.join();
|
|
|
|
+ } catch (InterruptedException ie) {
|
|
}
|
|
}
|
|
- if (metrics != null) {
|
|
|
|
- metrics.setDataNodeActiveXceiversCount(0);
|
|
|
|
|
|
+ }
|
|
|
|
+ if (this.localDataXceiverServer != null) {
|
|
|
|
+ // wait for localDataXceiverServer to terminate
|
|
|
|
+ try {
|
|
|
|
+ this.localDataXceiverServer.join();
|
|
|
|
+ } catch (InterruptedException ie) {
|
|
}
|
|
}
|
|
|
|
+ }
|
|
|
|
+ if (metrics != null) {
|
|
|
|
+ metrics.setDataNodeActiveXceiversCount(0);
|
|
|
|
+ }
|
|
|
|
|
|
- // IPC server needs to be shutdown late in the process, otherwise
|
|
|
|
- // shutdown command response won't get sent.
|
|
|
|
- if (ipcServer != null) {
|
|
|
|
- ipcServer.stop();
|
|
|
|
- }
|
|
|
|
|
|
+ // IPC server needs to be shutdown late in the process, otherwise
|
|
|
|
+ // shutdown command response won't get sent.
|
|
|
|
+ if (ipcServer != null) {
|
|
|
|
+ ipcServer.stop();
|
|
|
|
+ }
|
|
|
|
|
|
- if (blockPoolManager != null) {
|
|
|
|
- try {
|
|
|
|
- this.blockPoolManager.shutDownAll(bposArray);
|
|
|
|
- } catch (InterruptedException ie) {
|
|
|
|
- LOG.warn("Received exception in BlockPoolManager#shutDownAll: ", ie);
|
|
|
|
- }
|
|
|
|
|
|
+ if (blockPoolManager != null) {
|
|
|
|
+ try {
|
|
|
|
+ this.blockPoolManager.shutDownAll(bposArray);
|
|
|
|
+ } catch (InterruptedException ie) {
|
|
|
|
+ LOG.warn("Received exception in BlockPoolManager#shutDownAll: ", ie);
|
|
}
|
|
}
|
|
|
|
+ }
|
|
|
|
|
|
- if (storage != null) {
|
|
|
|
- try {
|
|
|
|
- this.storage.unlockAll();
|
|
|
|
- } catch (IOException ie) {
|
|
|
|
- LOG.warn("Exception when unlocking storage: " + ie, ie);
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
- if (data != null) {
|
|
|
|
- data.shutdown();
|
|
|
|
- }
|
|
|
|
- if (metrics != null) {
|
|
|
|
- metrics.shutdown();
|
|
|
|
- }
|
|
|
|
- if (diskMetrics != null) {
|
|
|
|
- diskMetrics.shutdownAndWait();
|
|
|
|
- }
|
|
|
|
- if (dataNodeInfoBeanName != null) {
|
|
|
|
- MBeans.unregister(dataNodeInfoBeanName);
|
|
|
|
- dataNodeInfoBeanName = null;
|
|
|
|
- }
|
|
|
|
- if (shortCircuitRegistry != null) shortCircuitRegistry.shutdown();
|
|
|
|
- LOG.info("Shutdown complete.");
|
|
|
|
- synchronized (this) {
|
|
|
|
- // it is already false, but setting it again to avoid a findbug warning.
|
|
|
|
- this.shouldRun = false;
|
|
|
|
- // Notify the main thread.
|
|
|
|
- notifyAll();
|
|
|
|
|
|
+ if (storage != null) {
|
|
|
|
+ try {
|
|
|
|
+ this.storage.unlockAll();
|
|
|
|
+ } catch (IOException ie) {
|
|
|
|
+ LOG.warn("Exception when unlocking storage: " + ie, ie);
|
|
}
|
|
}
|
|
- tracer.close();
|
|
|
|
}
|
|
}
|
|
|
|
+ if (data != null) {
|
|
|
|
+ data.shutdown();
|
|
|
|
+ }
|
|
|
|
+ if (metrics != null) {
|
|
|
|
+ metrics.shutdown();
|
|
|
|
+ }
|
|
|
|
+ if (diskMetrics != null) {
|
|
|
|
+ diskMetrics.shutdownAndWait();
|
|
|
|
+ }
|
|
|
|
+ if (dataNodeInfoBeanName != null) {
|
|
|
|
+ MBeans.unregister(dataNodeInfoBeanName);
|
|
|
|
+ dataNodeInfoBeanName = null;
|
|
|
|
+ }
|
|
|
|
+ if (shortCircuitRegistry != null) {
|
|
|
|
+ shortCircuitRegistry.shutdown();
|
|
|
|
+ }
|
|
|
|
+ LOG.info("Shutdown complete.");
|
|
|
|
+ synchronized (this) {
|
|
|
|
+ // it is already false, but setting it again to avoid a findbug warning.
|
|
|
|
+ this.shouldRun = false;
|
|
|
|
+ // Notify the main thread.
|
|
|
|
+ notifyAll();
|
|
|
|
+ }
|
|
|
|
+ tracer.close();
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
@@ -3149,6 +3144,12 @@ public class DataNode extends ReconfigurableBase
|
|
} catch (InterruptedException ie) { }
|
|
} catch (InterruptedException ie) { }
|
|
}
|
|
}
|
|
shutdown();
|
|
shutdown();
|
|
|
|
+
|
|
|
|
+ if (isOzoneEnabled()) {
|
|
|
|
+ if(datanodeStateMachine != null) {
|
|
|
|
+ datanodeStateMachine.stopDaemon();
|
|
|
|
+ }
|
|
|
|
+ }
|
|
}
|
|
}
|
|
};
|
|
};
|
|
|
|
|