浏览代码

HDFS-16717. Replace NPE with IOException in DataNode.class (#4699). Contributed by ZanderXu.

Signed-off-by: He Xiaoqiao <hexiaoqiao@apache.org>
ZanderXu 2 年之前
父节点
当前提交
183f09b1da

+ 25 - 11
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java

@@ -889,8 +889,11 @@ public class DataNode extends ReconfigurableBase
     String result = null;
     String result = null;
     try {
     try {
       LOG.info("Reconfiguring {} to {}", property, newVal);
       LOG.info("Reconfiguring {} to {}", property, newVal);
+      if (data == null) {
+        LOG.debug("FsDatasetSpi has not been initialized.");
+        throw new IOException("FsDatasetSpi has not been initialized");
+      }
       if (property.equals(FS_DU_INTERVAL_KEY)) {
       if (property.equals(FS_DU_INTERVAL_KEY)) {
-        Preconditions.checkNotNull(data, "FsDatasetSpi has not been initialized.");
         long interval = (newVal == null ? FS_DU_INTERVAL_DEFAULT :
         long interval = (newVal == null ? FS_DU_INTERVAL_DEFAULT :
             Long.parseLong(newVal));
             Long.parseLong(newVal));
         result = Long.toString(interval);
         result = Long.toString(interval);
@@ -902,7 +905,6 @@ public class DataNode extends ReconfigurableBase
           }
           }
         }
         }
       } else if (property.equals(FS_GETSPACEUSED_JITTER_KEY)) {
       } else if (property.equals(FS_GETSPACEUSED_JITTER_KEY)) {
-        Preconditions.checkNotNull(data, "FsDatasetSpi has not been initialized.");
         long jitter = (newVal == null ? FS_GETSPACEUSED_JITTER_DEFAULT :
         long jitter = (newVal == null ? FS_GETSPACEUSED_JITTER_DEFAULT :
             Long.parseLong(newVal));
             Long.parseLong(newVal));
         result = Long.toString(jitter);
         result = Long.toString(jitter);
@@ -914,7 +916,6 @@ public class DataNode extends ReconfigurableBase
           }
           }
         }
         }
       } else if (property.equals(FS_GETSPACEUSED_CLASSNAME)) {
       } else if (property.equals(FS_GETSPACEUSED_CLASSNAME)) {
-        Preconditions.checkNotNull(data, "FsDatasetSpi has not been initialized.");
         Class<? extends GetSpaceUsed> klass;
         Class<? extends GetSpaceUsed> klass;
         if (newVal == null) {
         if (newVal == null) {
           if (Shell.WINDOWS) {
           if (Shell.WINDOWS) {
@@ -1174,7 +1175,7 @@ public class DataNode extends ReconfigurableBase
               .newFixedThreadPool(changedVolumes.newLocations.size());
               .newFixedThreadPool(changedVolumes.newLocations.size());
           List<Future<IOException>> exceptions = Lists.newArrayList();
           List<Future<IOException>> exceptions = Lists.newArrayList();
 
 
-          Preconditions.checkNotNull(data, "Storage not yet initialized");
+          checkStorageState("refreshVolumes");
           for (final StorageLocation location : changedVolumes.newLocations) {
           for (final StorageLocation location : changedVolumes.newLocations) {
             exceptions.add(service.submit(new Callable<IOException>() {
             exceptions.add(service.submit(new Callable<IOException>() {
               @Override
               @Override
@@ -1274,7 +1275,7 @@ public class DataNode extends ReconfigurableBase
         clearFailure, Joiner.on(",").join(storageLocations)));
         clearFailure, Joiner.on(",").join(storageLocations)));
 
 
     IOException ioe = null;
     IOException ioe = null;
-    Preconditions.checkNotNull(data, "Storage not yet initialized");
+    checkStorageState("removeVolumes");
     // Remove volumes and block infos from FsDataset.
     // Remove volumes and block infos from FsDataset.
     data.removeVolumes(storageLocations, clearFailure);
     data.removeVolumes(storageLocations, clearFailure);
 
 
@@ -2301,7 +2302,7 @@ public class DataNode extends ReconfigurableBase
       Token<BlockTokenIdentifier> token) throws IOException {
       Token<BlockTokenIdentifier> token) throws IOException {
     checkBlockLocalPathAccess();
     checkBlockLocalPathAccess();
     checkBlockToken(block, token, BlockTokenIdentifier.AccessMode.READ);
     checkBlockToken(block, token, BlockTokenIdentifier.AccessMode.READ);
-    Preconditions.checkNotNull(data, "Storage not yet initialized");
+    checkStorageState("getBlockLocalPathInfo");
     BlockLocalPathInfo info = data.getBlockLocalPathInfo(block);
     BlockLocalPathInfo info = data.getBlockLocalPathInfo(block);
     if (info != null) {
     if (info != null) {
       LOG.trace("getBlockLocalPathInfo successful " +
       LOG.trace("getBlockLocalPathInfo successful " +
@@ -2351,7 +2352,7 @@ public class DataNode extends ReconfigurableBase
     FileInputStream fis[] = new FileInputStream[2];
     FileInputStream fis[] = new FileInputStream[2];
     
     
     try {
     try {
-      Preconditions.checkNotNull(data, "Storage not yet initialized");
+      checkStorageState("requestShortCircuitFdsForRead");
       fis[0] = (FileInputStream)data.getBlockInputStream(blk, 0);
       fis[0] = (FileInputStream)data.getBlockInputStream(blk, 0);
       fis[1] = DatanodeUtil.getMetaDataInputStream(blk, data);
       fis[1] = DatanodeUtil.getMetaDataInputStream(blk, data);
     } catch (ClassCastException e) {
     } catch (ClassCastException e) {
@@ -3382,7 +3383,7 @@ public class DataNode extends ReconfigurableBase
   @Override // InterDatanodeProtocol
   @Override // InterDatanodeProtocol
   public ReplicaRecoveryInfo initReplicaRecovery(RecoveringBlock rBlock)
   public ReplicaRecoveryInfo initReplicaRecovery(RecoveringBlock rBlock)
       throws IOException {
       throws IOException {
-    Preconditions.checkNotNull(data, "Storage not yet initialized");
+    checkStorageState("initReplicaRecovery");
     return data.initReplicaRecovery(rBlock);
     return data.initReplicaRecovery(rBlock);
   }
   }
 
 
@@ -3393,7 +3394,7 @@ public class DataNode extends ReconfigurableBase
   public String updateReplicaUnderRecovery(final ExtendedBlock oldBlock,
   public String updateReplicaUnderRecovery(final ExtendedBlock oldBlock,
       final long recoveryId, final long newBlockId, final long newLength)
       final long recoveryId, final long newBlockId, final long newLength)
       throws IOException {
       throws IOException {
-    Preconditions.checkNotNull(data, "Storage not yet initialized");
+    checkStorageState("updateReplicaUnderRecovery");
     final Replica r = data.updateReplicaUnderRecovery(oldBlock,
     final Replica r = data.updateReplicaUnderRecovery(oldBlock,
         recoveryId, newBlockId, newLength);
         recoveryId, newBlockId, newLength);
     // Notify the namenode of the updated block info. This is important
     // Notify the namenode of the updated block info. This is important
@@ -3679,10 +3680,23 @@ public class DataNode extends ReconfigurableBase
           "The block pool is still running. First do a refreshNamenodes to " +
           "The block pool is still running. First do a refreshNamenodes to " +
           "shutdown the block pool service");
           "shutdown the block pool service");
     }
     }
-    Preconditions.checkNotNull(data, "Storage not yet initialized");
+    checkStorageState("deleteBlockPool");
     data.deleteBlockPool(blockPoolId, force);
     data.deleteBlockPool(blockPoolId, force);
   }
   }
 
 
+  /**
+   * Check if storage has been initialized.
+   * @param methodName caller name
+   * @throws IOException throw IOException if not yet initialized.
+   */
+  private void checkStorageState(String methodName) throws IOException {
+    if (data == null) {
+      String message = "Storage not yet initialized for " + methodName;
+      LOG.debug(message);
+      throw new IOException(message);
+    }
+  }
+
   @Override // ClientDatanodeProtocol
   @Override // ClientDatanodeProtocol
   public synchronized void shutdownDatanode(boolean forUpgrade) throws IOException {
   public synchronized void shutdownDatanode(boolean forUpgrade) throws IOException {
     checkSuperuserPrivilege();
     checkSuperuserPrivilege();
@@ -4131,7 +4145,7 @@ public class DataNode extends ReconfigurableBase
   @Override
   @Override
   public List<DatanodeVolumeInfo> getVolumeReport() throws IOException {
   public List<DatanodeVolumeInfo> getVolumeReport() throws IOException {
     checkSuperuserPrivilege();
     checkSuperuserPrivilege();
-    Preconditions.checkNotNull(data, "Storage not yet initialized");
+    checkStorageState("getVolumeReport");
     Map<String, Object> volumeInfoMap = data.getVolumeInfoMap();
     Map<String, Object> volumeInfoMap = data.getVolumeInfoMap();
     if (volumeInfoMap == null) {
     if (volumeInfoMap == null) {
       LOG.warn("DataNode volume info not available.");
       LOG.warn("DataNode volume info not available.");