Browse Source

HDFS-7439. Add BlockOpResponseProto's message to the exception messages. Contributed by Takanobu Asanuma

Tsz-Wo Nicholas Sze 10 years ago
parent
commit
a5f3156b30

+ 3 - 0
hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt

@@ -393,6 +393,9 @@ Release 2.7.0 - UNRELEASED
     HDFS-5853. Add "hadoop.user.group.metrics.percentiles.intervals" to
     hdfs-default.xml. (aajisaka)
 
+    HDFS-7439. Add BlockOpResponseProto's message to the exception messages.
+    (Takanobu Asanuma via szetszwo)
+
   OPTIMIZATIONS
 
     HDFS-7454. Reduce memory footprint for AclEntries in NameNode.

+ 7 - 19
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java

@@ -175,6 +175,7 @@ import org.apache.hadoop.hdfs.protocol.SnapshotAccessControlException;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
 import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
 import org.apache.hadoop.hdfs.protocol.UnresolvedPathException;
+import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtoUtil;
 import org.apache.hadoop.hdfs.protocol.datatransfer.IOStreamPair;
 import org.apache.hadoop.hdfs.protocol.datatransfer.Op;
 import org.apache.hadoop.hdfs.protocol.datatransfer.ReplaceDatanodeOnFailure;
@@ -2259,15 +2260,9 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
           final BlockOpResponseProto reply =
             BlockOpResponseProto.parseFrom(PBHelper.vintPrefixed(in));
 
-          if (reply.getStatus() != Status.SUCCESS) {
-            if (reply.getStatus() == Status.ERROR_ACCESS_TOKEN) {
-              throw new InvalidBlockTokenException();
-            } else {
-              throw new IOException("Bad response " + reply + " for block "
-                  + block + " from datanode " + datanodes[j]);
-            }
-          }
-          
+          String logInfo = "for block " + block + " from datanode " + datanodes[j];
+          DataTransferProtoUtil.checkBlockOpStatus(reply, logInfo);
+
           OpBlockChecksumResponseProto checksumData =
             reply.getChecksumResponse();
 
@@ -2424,16 +2419,9 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
           0, 1, true, CachingStrategy.newDefaultStrategy());
       final BlockOpResponseProto reply =
           BlockOpResponseProto.parseFrom(PBHelper.vintPrefixed(in));
-      
-      if (reply.getStatus() != Status.SUCCESS) {
-        if (reply.getStatus() == Status.ERROR_ACCESS_TOKEN) {
-          throw new InvalidBlockTokenException();
-        } else {
-          throw new IOException("Bad response " + reply + " trying to read "
-              + lb.getBlock() + " from datanode " + dn);
-        }
-      }
-      
+      String logInfo = "trying to read " + lb.getBlock() + " from datanode " + dn;
+      DataTransferProtoUtil.checkBlockOpStatus(reply, logInfo);
+
       return PBHelper.convert(reply.getReadOpChecksumInfo().getChecksum().getType());
     } finally {
       IOUtils.cleanup(null, pair.in, pair.out);

+ 5 - 10
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java

@@ -69,6 +69,7 @@ import org.apache.hadoop.hdfs.protocol.SnapshotAccessControlException;
 import org.apache.hadoop.hdfs.protocol.UnresolvedPathException;
 import org.apache.hadoop.hdfs.protocol.datatransfer.BlockConstructionStage;
 import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtocol;
+import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtoUtil;
 import org.apache.hadoop.hdfs.protocol.datatransfer.IOStreamPair;
 import org.apache.hadoop.hdfs.protocol.datatransfer.InvalidEncryptionKeyException;
 import org.apache.hadoop.hdfs.protocol.datatransfer.PacketHeader;
@@ -1469,16 +1470,10 @@ public class DFSOutputStream extends FSOutputSummer
             checkRestart = true;
             throw new IOException("A datanode is restarting.");
           }
-          if (pipelineStatus != SUCCESS) {
-            if (pipelineStatus == Status.ERROR_ACCESS_TOKEN) {
-              throw new InvalidBlockTokenException(
-                  "Got access token error for connect ack with firstBadLink as "
-                      + firstBadLink);
-            } else {
-              throw new IOException("Bad connect ack with firstBadLink as "
-                  + firstBadLink);
-            }
-          }
+
+          String logInfo = "ack with firstBadLink as " + firstBadLink;
+          DataTransferProtoUtil.checkBlockOpStatus(resp, logInfo);
+
           assert null == blockStream : "Previous blockStream unclosed";
           blockStream = out;
           result =  true; // success

+ 7 - 17
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader2.java

@@ -45,7 +45,6 @@ import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ReadOpChecksumIn
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status;
 import org.apache.hadoop.hdfs.protocolPB.PBHelper;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
-import org.apache.hadoop.hdfs.security.token.block.InvalidBlockTokenException;
 import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
 import org.apache.hadoop.hdfs.shortcircuit.ClientMmap;
 import org.apache.hadoop.net.NetUtils;
@@ -448,22 +447,13 @@ public class RemoteBlockReader2  implements BlockReader {
       BlockOpResponseProto status, Peer peer,
       ExtendedBlock block, String file)
       throws IOException {
-    if (status.getStatus() != Status.SUCCESS) {
-      if (status.getStatus() == Status.ERROR_ACCESS_TOKEN) {
-        throw new InvalidBlockTokenException(
-            "Got access token error for OP_READ_BLOCK, self="
-                + peer.getLocalAddressString() + ", remote="
-                + peer.getRemoteAddressString() + ", for file " + file
-                + ", for pool " + block.getBlockPoolId() + " block " 
-                + block.getBlockId() + "_" + block.getGenerationStamp());
-      } else {
-        throw new IOException("Got error for OP_READ_BLOCK, self="
-            + peer.getLocalAddressString() + ", remote="
-            + peer.getRemoteAddressString() + ", for file " + file
-            + ", for pool " + block.getBlockPoolId() + " block " 
-            + block.getBlockId() + "_" + block.getGenerationStamp());
-      }
-    }
+    String logInfo = "for OP_READ_BLOCK"
+      + ", self=" + peer.getLocalAddressString()
+      + ", remote=" + peer.getRemoteAddressString()
+      + ", for file " + file
+      + ", for pool " + block.getBlockPoolId()
+      + " block " + block.getBlockId() + "_" + block.getGenerationStamp();
+    DataTransferProtoUtil.checkBlockOpStatus(status, logInfo);
   }
   
   @Override

+ 26 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/DataTransferProtoUtil.java

@@ -17,11 +17,16 @@
  */
 package org.apache.hadoop.hdfs.protocol.datatransfer;
 
+import java.io.IOException;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hdfs.net.Peer;
+import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto;
+import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status;
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ChecksumProto;
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProto;
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpWriteBlockProto;
@@ -29,6 +34,7 @@ import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.DataTransferTrac
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ChecksumTypeProto;
 import org.apache.hadoop.hdfs.protocolPB.PBHelper;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
+import org.apache.hadoop.hdfs.security.token.block.InvalidBlockTokenException;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.util.DataChecksum;
 import org.apache.htrace.Span;
@@ -119,4 +125,24 @@ public abstract class DataTransferProtoUtil {
     }
     return scope;
   }
+
+  public static void checkBlockOpStatus(
+          BlockOpResponseProto response,
+          String logInfo) throws IOException {
+    if (response.getStatus() != Status.SUCCESS) {
+      if (response.getStatus() == Status.ERROR_ACCESS_TOKEN) {
+        throw new InvalidBlockTokenException(
+          "Got access token error"
+          + ", status message " + response.getMessage()
+          + ", " + logInfo
+        );
+      } else {
+        throw new IOException(
+          "Got error"
+          + ", status message " + response.getMessage()
+          + ", " + logInfo
+        );
+      }
+    }
+  }
 }

+ 3 - 6
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Dispatcher.java

@@ -55,6 +55,7 @@ import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtoUtil;
 import org.apache.hadoop.hdfs.protocol.datatransfer.IOStreamPair;
 import org.apache.hadoop.hdfs.protocol.datatransfer.Sender;
 import org.apache.hadoop.hdfs.protocol.datatransfer.TrustedChannelResolver;
@@ -359,12 +360,8 @@ public class Dispatcher {
         // read intermediate responses
         response = BlockOpResponseProto.parseFrom(vintPrefixed(in));
       }
-      if (response.getStatus() != Status.SUCCESS) {
-        if (response.getStatus() == Status.ERROR_ACCESS_TOKEN) {
-          throw new IOException("block move failed due to access token error");
-        }
-        throw new IOException("block move is failed: " + response.getMessage());
-      }
+      String logInfo = "block move is failed";
+      DataTransferProtoUtil.checkBlockOpStatus(response, logInfo);
     }
 
     /** reset the object */

+ 4 - 10
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java

@@ -1116,16 +1116,10 @@ class DataXceiver extends Receiver implements Runnable {
         BlockOpResponseProto copyResponse = BlockOpResponseProto.parseFrom(
             PBHelper.vintPrefixed(proxyReply));
         
-        if (copyResponse.getStatus() != SUCCESS) {
-          if (copyResponse.getStatus() == ERROR_ACCESS_TOKEN) {
-            throw new IOException("Copy block " + block + " from "
-                + proxySock.getRemoteSocketAddress()
-                + " failed due to access token error");
-          }
-          throw new IOException("Copy block " + block + " from "
-              + proxySock.getRemoteSocketAddress() + " failed");
-        }
-        
+        String logInfo = "copy block " + block + " from "
+            + proxySock.getRemoteSocketAddress();
+        DataTransferProtoUtil.checkBlockOpStatus(copyResponse, logInfo);
+
         // get checksum info about the block we're copying
         ReadOpChecksumInfoProto checksumInfo = copyResponse.getReadOpChecksumInfo();
         DataChecksum remoteChecksum = DataTransferProtoUtil.fromProto(