You are viewing a plain text version of this content. The canonical link for it is here.
Posted to hdfs-commits@hadoop.apache.org by su...@apache.org on 2013/01/09 22:37:01 UTC

svn commit: r1431101 - in /hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs: ./ src/main/java/org/apache/hadoop/hdfs/ src/main/java/org/apache/hadoop/hdfs/protocol/ src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/ src/main/jav...

Author: suresh
Date: Wed Jan  9 21:37:01 2013
New Revision: 1431101

URL: http://svn.apache.org/viewvc?rev=1431101&view=rev
Log:
HDFS-4363. Merge change r1431088 from trunk

Removed:
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsProtoUtil.java
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocol/TestHdfsProtoUtil.java
Modified:
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader.java
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader2.java
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/DataTransferEncryptor.java
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/DataTransferProtoUtil.java
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PipelineAck.java
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Receiver.java
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Sender.java
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Balancer.java
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocolPB/TestPBHelper.java

Modified: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt?rev=1431101&r1=1431100&r2=1431101&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt Wed Jan  9 21:37:01 2013
@@ -176,6 +176,9 @@ Release 2.0.3-alpha - Unreleased
 
     HDFS-4035. LightWeightGSet and LightWeightHashSet increment a
     volatile without synchronization. (eli)
+    
+    HDFS-4363. Combine PBHelper and HdfsProtoUtil and remove redundant
+    methods. (suresh)
 
   OPTIMIZATIONS
 

Modified: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java?rev=1431101&r1=1431100&r2=1431101&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java Wed Jan  9 21:37:01 2013
@@ -116,7 +116,6 @@ import org.apache.hadoop.hdfs.protocol.H
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
-import org.apache.hadoop.hdfs.protocol.HdfsProtoUtil;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.NSQuotaExceededException;
@@ -130,6 +129,7 @@ import org.apache.hadoop.hdfs.protocol.p
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status;
 import org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey;
+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.security.token.delegation.DelegationTokenIdentifier;
@@ -364,7 +364,7 @@ public class DFSClient implements java.i
 
   /**
    * Same as this(nameNodeUri, conf, null);
-   * @see #DFSClient(InetSocketAddress, Configuration, org.apache.hadoop.fs.FileSystem.Statistics)
+   * @see #DFSClient(URI, Configuration, FileSystem.Statistics)
    */
   public DFSClient(URI nameNodeUri, Configuration conf
       ) throws IOException {
@@ -373,7 +373,7 @@ public class DFSClient implements java.i
 
   /**
    * Same as this(nameNodeUri, null, conf, stats);
-   * @see #DFSClient(InetSocketAddress, ClientProtocol, Configuration, org.apache.hadoop.fs.FileSystem.Statistics) 
+   * @see #DFSClient(URI, ClientProtocol, Configuration, FileSystem.Statistics) 
    */
   public DFSClient(URI nameNodeUri, Configuration conf,
                    FileSystem.Statistics stats)
@@ -1183,7 +1183,7 @@ public class DFSClient implements java.i
 
   /**
    * Call {@link #create(String, FsPermission, EnumSet, short, long, 
-   * Progressable, int)} with default <code>permission</code>
+   * Progressable, int, ChecksumOpt)} with default <code>permission</code>
    * {@link FsPermission#getDefault()}.
    * 
    * @param src File name
@@ -1294,7 +1294,7 @@ public class DFSClient implements java.i
   
   /**
    * Same as {{@link #create(String, FsPermission, EnumSet, short, long,
-   *  Progressable, int)} except that the permission
+   *  Progressable, int, ChecksumOpt)} except that the permission
    *  is absolute (ie has already been masked with umask.
    */
   public DFSOutputStream primitiveCreate(String src, 
@@ -1479,7 +1479,7 @@ public class DFSClient implements java.i
   }
   /**
    * Delete file or directory.
-   * See {@link ClientProtocol#delete(String)}. 
+   * See {@link ClientProtocol#delete(String, boolean)}. 
    */
   @Deprecated
   public boolean delete(String src) throws IOException {
@@ -1704,7 +1704,7 @@ public class DFSClient implements java.i
           new Sender(out).blockChecksum(block, lb.getBlockToken());
 
           final BlockOpResponseProto reply =
-            BlockOpResponseProto.parseFrom(HdfsProtoUtil.vintPrefixed(in));
+            BlockOpResponseProto.parseFrom(PBHelper.vintPrefixed(in));
 
           if (reply.getStatus() != Status.SUCCESS) {
             if (reply.getStatus() == Status.ERROR_ACCESS_TOKEN
@@ -1751,8 +1751,8 @@ public class DFSClient implements java.i
           md5.write(md5out);
           
           // read crc-type
-          final DataChecksum.Type ct = HdfsProtoUtil.
-              fromProto(checksumData.getCrcType());
+          final DataChecksum.Type ct = PBHelper.convert(checksumData
+              .getCrcType());
           if (i == 0) { // first block
             crcType = ct;
           } else if (crcType != DataChecksum.Type.MIXED
@@ -1914,7 +1914,7 @@ public class DFSClient implements java.i
    * @param isChecked
    *          If true, then check only active namenode's safemode status, else
    *          check first namenode's status.
-   * @see ClientProtocol#setSafeMode(HdfsConstants.SafeModeActio,boolean)
+   * @see ClientProtocol#setSafeMode(HdfsConstants.SafeModeAction, boolean)
    */
   public boolean setSafeMode(SafeModeAction action, boolean isChecked) throws IOException{
     return namenode.setSafeMode(action, isChecked);    

Modified: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java?rev=1431101&r1=1431100&r2=1431101&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java Wed Jan  9 21:37:01 2013
@@ -52,7 +52,6 @@ import org.apache.hadoop.hdfs.protocol.D
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
-import org.apache.hadoop.hdfs.protocol.HdfsProtoUtil;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.NSQuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.UnresolvedPathException;
@@ -66,6 +65,7 @@ import org.apache.hadoop.hdfs.protocol.d
 import org.apache.hadoop.hdfs.protocol.datatransfer.Sender;
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto;
 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.namenode.NotReplicatedYetException;
@@ -888,7 +888,7 @@ public class DFSOutputStream extends FSO
 
         //ack
         BlockOpResponseProto response =
-          BlockOpResponseProto.parseFrom(HdfsProtoUtil.vintPrefixed(in));
+          BlockOpResponseProto.parseFrom(PBHelper.vintPrefixed(in));
         if (SUCCESS != response.getStatus()) {
           throw new IOException("Failed to add a datanode");
         }
@@ -1078,7 +1078,7 @@ public class DFSOutputStream extends FSO
   
           // receive ack for connect
           BlockOpResponseProto resp = BlockOpResponseProto.parseFrom(
-              HdfsProtoUtil.vintPrefixed(blockReplyStream));
+              PBHelper.vintPrefixed(blockReplyStream));
           pipelineStatus = resp.getStatus();
           firstBadLink = resp.getFirstBadLink();
           

Modified: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader.java?rev=1431101&r1=1431100&r2=1431101&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader.java (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader.java Wed Jan  9 21:37:01 2013
@@ -17,8 +17,6 @@
  */
 package org.apache.hadoop.hdfs;
 
-import static org.apache.hadoop.hdfs.protocol.HdfsProtoUtil.vintPrefixed;
-
 import java.io.BufferedInputStream;
 import java.io.BufferedOutputStream;
 import java.io.DataInputStream;
@@ -39,6 +37,7 @@ import org.apache.hadoop.hdfs.protocol.d
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ReadOpChecksumInfoProto;
 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.server.common.HdfsServerConstants;
 import org.apache.hadoop.io.IOUtils;
@@ -392,7 +391,7 @@ public class RemoteBlockReader extends F
                                 bufferSize));
     
     BlockOpResponseProto status = BlockOpResponseProto.parseFrom(
-        vintPrefixed(in));
+        PBHelper.vintPrefixed(in));
     RemoteBlockReader2.checkSuccess(status, sock, block, file);
     ReadOpChecksumInfoProto checksumInfo =
       status.getReadOpChecksumInfo();

Modified: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader2.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader2.java?rev=1431101&r1=1431100&r2=1431101&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader2.java (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader2.java Wed Jan  9 21:37:01 2013
@@ -17,8 +17,6 @@
  */
 package org.apache.hadoop.hdfs;
 
-import static org.apache.hadoop.hdfs.protocol.HdfsProtoUtil.vintPrefixed;
-
 import java.io.BufferedOutputStream;
 import java.io.DataInputStream;
 import java.io.DataOutputStream;
@@ -43,6 +41,7 @@ import org.apache.hadoop.hdfs.protocol.p
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ReadOpChecksumInfoProto;
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status;
 import org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey;
+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.net.SocketInputWrapper;
@@ -401,7 +400,7 @@ public class RemoteBlockReader2  impleme
     DataInputStream in = new DataInputStream(ioStreams.in);
 
     BlockOpResponseProto status = BlockOpResponseProto.parseFrom(
-        vintPrefixed(in));
+        PBHelper.vintPrefixed(in));
     checkSuccess(status, sock, block, file);
     ReadOpChecksumInfoProto checksumInfo =
       status.getReadOpChecksumInfo();

Modified: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/DataTransferEncryptor.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/DataTransferEncryptor.java?rev=1431101&r1=1431100&r2=1431101&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/DataTransferEncryptor.java (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/DataTransferEncryptor.java Wed Jan  9 21:37:01 2013
@@ -17,7 +17,7 @@
  */
 package org.apache.hadoop.hdfs.protocol.datatransfer;
 
-import static org.apache.hadoop.hdfs.protocol.HdfsProtoUtil.vintPrefixed;
+import static org.apache.hadoop.hdfs.protocolPB.PBHelper.vintPrefixed;
 
 import java.io.DataInputStream;
 import java.io.DataOutputStream;

Modified: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/DataTransferProtoUtil.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/DataTransferProtoUtil.java?rev=1431101&r1=1431100&r2=1431101&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/DataTransferProtoUtil.java (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/DataTransferProtoUtil.java Wed Jan  9 21:37:01 2013
@@ -21,12 +21,12 @@ package org.apache.hadoop.hdfs.protocol.
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
-import org.apache.hadoop.hdfs.protocol.HdfsProtoUtil;
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto;
 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;
 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.security.token.Token;
 import org.apache.hadoop.util.DataChecksum;
@@ -41,18 +41,16 @@ import org.apache.hadoop.util.DataChecks
 public abstract class DataTransferProtoUtil {
   static BlockConstructionStage fromProto(
       OpWriteBlockProto.BlockConstructionStage stage) {
-    return BlockConstructionStage.valueOf(BlockConstructionStage.class,
-        stage.name());
+    return BlockConstructionStage.valueOf(stage.name());
   }
 
   static OpWriteBlockProto.BlockConstructionStage toProto(
       BlockConstructionStage stage) {
-    return OpWriteBlockProto.BlockConstructionStage.valueOf(
-        stage.name());
+    return OpWriteBlockProto.BlockConstructionStage.valueOf(stage.name());
   }
 
   public static ChecksumProto toProto(DataChecksum checksum) {
-    ChecksumTypeProto type = HdfsProtoUtil.toProto(checksum.getChecksumType());
+    ChecksumTypeProto type = PBHelper.convert(checksum.getChecksumType());
     // ChecksumType#valueOf never returns null
     return ChecksumProto.newBuilder()
       .setBytesPerChecksum(checksum.getBytesPerChecksum())
@@ -64,8 +62,7 @@ public abstract class DataTransferProtoU
     if (proto == null) return null;
 
     int bytesPerChecksum = proto.getBytesPerChecksum();
-    DataChecksum.Type type = HdfsProtoUtil.fromProto(proto.getType());
-    
+    DataChecksum.Type type = PBHelper.convert(proto.getType());
     return DataChecksum.newDataChecksum(type, bytesPerChecksum);
   }
 
@@ -82,8 +79,8 @@ public abstract class DataTransferProtoU
   static BaseHeaderProto buildBaseHeader(ExtendedBlock blk,
       Token<BlockTokenIdentifier> blockToken) {
     return BaseHeaderProto.newBuilder()
-      .setBlock(HdfsProtoUtil.toProto(blk))
-      .setToken(HdfsProtoUtil.toProto(blockToken))
+      .setBlock(PBHelper.convert(blk))
+      .setToken(PBHelper.convert(blockToken))
       .build();
   }
 }

Modified: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PipelineAck.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PipelineAck.java?rev=1431101&r1=1431100&r2=1431101&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PipelineAck.java (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PipelineAck.java Wed Jan  9 21:37:01 2013
@@ -17,7 +17,7 @@
  */
 package org.apache.hadoop.hdfs.protocol.datatransfer;
 
-import static org.apache.hadoop.hdfs.protocol.HdfsProtoUtil.vintPrefixed;
+import static org.apache.hadoop.hdfs.protocolPB.PBHelper.vintPrefixed;
 
 import java.io.IOException;
 import java.io.InputStream;

Modified: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Receiver.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Receiver.java?rev=1431101&r1=1431100&r2=1431101&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Receiver.java (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Receiver.java Wed Jan  9 21:37:01 2013
@@ -17,9 +17,7 @@
  */
 package org.apache.hadoop.hdfs.protocol.datatransfer;
 
-import static org.apache.hadoop.hdfs.protocol.HdfsProtoUtil.fromProto;
-import static org.apache.hadoop.hdfs.protocol.HdfsProtoUtil.fromProtos;
-import static org.apache.hadoop.hdfs.protocol.HdfsProtoUtil.vintPrefixed;
+import static org.apache.hadoop.hdfs.protocolPB.PBHelper.vintPrefixed;
 import static org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtoUtil.fromProto;
 
 import java.io.DataInputStream;
@@ -33,6 +31,7 @@ import org.apache.hadoop.hdfs.protocol.p
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpReplaceBlockProto;
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpTransferBlockProto;
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpWriteBlockProto;
+import org.apache.hadoop.hdfs.protocolPB.PBHelper;
 
 /** Receiver */
 @InterfaceAudience.Private
@@ -85,8 +84,8 @@ public abstract class Receiver implement
   /** Receive OP_READ_BLOCK */
   private void opReadBlock() throws IOException {
     OpReadBlockProto proto = OpReadBlockProto.parseFrom(vintPrefixed(in));
-    readBlock(fromProto(proto.getHeader().getBaseHeader().getBlock()),
-        fromProto(proto.getHeader().getBaseHeader().getToken()),
+    readBlock(PBHelper.convert(proto.getHeader().getBaseHeader().getBlock()),
+        PBHelper.convert(proto.getHeader().getBaseHeader().getToken()),
         proto.getHeader().getClientName(),
         proto.getOffset(),
         proto.getLen());
@@ -95,11 +94,11 @@ public abstract class Receiver implement
   /** Receive OP_WRITE_BLOCK */
   private void opWriteBlock(DataInputStream in) throws IOException {
     final OpWriteBlockProto proto = OpWriteBlockProto.parseFrom(vintPrefixed(in));
-    writeBlock(fromProto(proto.getHeader().getBaseHeader().getBlock()),
-        fromProto(proto.getHeader().getBaseHeader().getToken()),
+    writeBlock(PBHelper.convert(proto.getHeader().getBaseHeader().getBlock()),
+        PBHelper.convert(proto.getHeader().getBaseHeader().getToken()),
         proto.getHeader().getClientName(),
-        fromProtos(proto.getTargetsList()),
-        fromProto(proto.getSource()),
+        PBHelper.convert(proto.getTargetsList()),
+        PBHelper.convert(proto.getSource()),
         fromProto(proto.getStage()),
         proto.getPipelineSize(),
         proto.getMinBytesRcvd(), proto.getMaxBytesRcvd(),
@@ -111,33 +110,33 @@ public abstract class Receiver implement
   private void opTransferBlock(DataInputStream in) throws IOException {
     final OpTransferBlockProto proto =
       OpTransferBlockProto.parseFrom(vintPrefixed(in));
-    transferBlock(fromProto(proto.getHeader().getBaseHeader().getBlock()),
-        fromProto(proto.getHeader().getBaseHeader().getToken()),
+    transferBlock(PBHelper.convert(proto.getHeader().getBaseHeader().getBlock()),
+        PBHelper.convert(proto.getHeader().getBaseHeader().getToken()),
         proto.getHeader().getClientName(),
-        fromProtos(proto.getTargetsList()));
+        PBHelper.convert(proto.getTargetsList()));
   }
 
   /** Receive OP_REPLACE_BLOCK */
   private void opReplaceBlock(DataInputStream in) throws IOException {
     OpReplaceBlockProto proto = OpReplaceBlockProto.parseFrom(vintPrefixed(in));
-    replaceBlock(fromProto(proto.getHeader().getBlock()),
-        fromProto(proto.getHeader().getToken()),
+    replaceBlock(PBHelper.convert(proto.getHeader().getBlock()),
+        PBHelper.convert(proto.getHeader().getToken()),
         proto.getDelHint(),
-        fromProto(proto.getSource()));
+        PBHelper.convert(proto.getSource()));
   }
 
   /** Receive OP_COPY_BLOCK */
   private void opCopyBlock(DataInputStream in) throws IOException {
     OpCopyBlockProto proto = OpCopyBlockProto.parseFrom(vintPrefixed(in));
-    copyBlock(fromProto(proto.getHeader().getBlock()),
-        fromProto(proto.getHeader().getToken()));
+    copyBlock(PBHelper.convert(proto.getHeader().getBlock()),
+        PBHelper.convert(proto.getHeader().getToken()));
   }
 
   /** Receive OP_BLOCK_CHECKSUM */
   private void opBlockChecksum(DataInputStream in) throws IOException {
     OpBlockChecksumProto proto = OpBlockChecksumProto.parseFrom(vintPrefixed(in));
     
-    blockChecksum(fromProto(proto.getHeader().getBlock()),
-        fromProto(proto.getHeader().getToken()));
+    blockChecksum(PBHelper.convert(proto.getHeader().getBlock()),
+        PBHelper.convert(proto.getHeader().getToken()));
   }
 }

Modified: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Sender.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Sender.java?rev=1431101&r1=1431100&r2=1431101&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Sender.java (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Sender.java Wed Jan  9 21:37:01 2013
@@ -17,8 +17,6 @@
  */
 package org.apache.hadoop.hdfs.protocol.datatransfer;
 
-import static org.apache.hadoop.hdfs.protocol.HdfsProtoUtil.toProto;
-import static org.apache.hadoop.hdfs.protocol.HdfsProtoUtil.toProtos;
 import static org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtoUtil.toProto;
 
 import java.io.DataOutput;
@@ -37,6 +35,7 @@ import org.apache.hadoop.hdfs.protocol.p
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpReplaceBlockProto;
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpTransferBlockProto;
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpWriteBlockProto;
+import org.apache.hadoop.hdfs.protocolPB.PBHelper;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.util.DataChecksum;
@@ -105,7 +104,7 @@ public class Sender implements DataTrans
 
     OpWriteBlockProto.Builder proto = OpWriteBlockProto.newBuilder()
       .setHeader(header)
-      .addAllTargets(toProtos(targets, 1))
+      .addAllTargets(PBHelper.convert(targets, 1))
       .setStage(toProto(stage))
       .setPipelineSize(pipelineSize)
       .setMinBytesRcvd(minBytesRcvd)
@@ -114,7 +113,7 @@ public class Sender implements DataTrans
       .setRequestedChecksum(checksumProto);
     
     if (source != null) {
-      proto.setSource(toProto(source));
+      proto.setSource(PBHelper.convertDatanodeInfo(source));
     }
 
     send(out, Op.WRITE_BLOCK, proto.build());
@@ -129,7 +128,7 @@ public class Sender implements DataTrans
     OpTransferBlockProto proto = OpTransferBlockProto.newBuilder()
       .setHeader(DataTransferProtoUtil.buildClientHeader(
           blk, clientName, blockToken))
-      .addAllTargets(toProtos(targets, 0))
+      .addAllTargets(PBHelper.convert(targets))
       .build();
 
     send(out, Op.TRANSFER_BLOCK, proto);
@@ -143,7 +142,7 @@ public class Sender implements DataTrans
     OpReplaceBlockProto proto = OpReplaceBlockProto.newBuilder()
       .setHeader(DataTransferProtoUtil.buildBaseHeader(blk, blockToken))
       .setDelHint(delHint)
-      .setSource(toProto(source))
+      .setSource(PBHelper.convertDatanodeInfo(source))
       .build();
     
     send(out, Op.REPLACE_BLOCK, proto);

Modified: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java?rev=1431101&r1=1431100&r2=1431101&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java Wed Jan  9 21:37:01 2013
@@ -18,7 +18,6 @@
 package org.apache.hadoop.hdfs.protocolPB;
 
 import java.io.IOException;
-import java.util.Arrays;
 import java.util.List;
 
 import org.apache.hadoop.classification.InterfaceAudience;
@@ -131,7 +130,6 @@ import org.apache.hadoop.hdfs.protocol.p
 import org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.io.Text;
-import org.apache.hadoop.security.proto.SecurityProtos.TokenProto;
 import org.apache.hadoop.security.token.Token;
 
 import com.google.protobuf.RpcController;
@@ -494,10 +492,10 @@ public class ClientNamenodeProtocolServe
       RpcController controller, GetDatanodeReportRequestProto req)
       throws ServiceException {
     try {
-      DatanodeInfoProto[] result = PBHelper.convert(server
+      List<? extends DatanodeInfoProto> result = PBHelper.convert(server
           .getDatanodeReport(PBHelper.convert(req.getType())));
       return GetDatanodeReportResponseProto.newBuilder()
-          .addAllDi(Arrays.asList(result)).build();
+          .addAllDi(result).build();
     } catch (IOException e) {
       throw new ServiceException(e);
     }

Modified: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java?rev=1431101&r1=1431100&r2=1431101&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java Wed Jan  9 21:37:01 2013
@@ -282,7 +282,7 @@ public class ClientNamenodeProtocolTrans
     if (previous != null) 
       req.setPrevious(PBHelper.convert(previous)); 
     if (excludeNodes != null) 
-      req.addAllExcludeNodes(Arrays.asList(PBHelper.convert(excludeNodes)));
+      req.addAllExcludeNodes(PBHelper.convert(excludeNodes));
     try {
       return PBHelper.convert(rpcProxy.addBlock(null, req.build()).getBlock());
     } catch (ServiceException e) {
@@ -300,8 +300,8 @@ public class ClientNamenodeProtocolTrans
         .newBuilder()
         .setSrc(src)
         .setBlk(PBHelper.convert(blk))
-        .addAllExistings(Arrays.asList(PBHelper.convert(existings)))
-        .addAllExcludes(Arrays.asList(PBHelper.convert(excludes)))
+        .addAllExistings(PBHelper.convert(existings))
+        .addAllExcludes(PBHelper.convert(excludes))
         .setNumAdditionalNodes(numAdditionalNodes)
         .setClientName(clientName)
         .build();

Modified: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java?rev=1431101&r1=1431100&r2=1431101&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java Wed Jan  9 21:37:01 2013
@@ -17,6 +17,9 @@
  */
 package org.apache.hadoop.hdfs.protocolPB;
 
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.InputStream;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.EnumSet;
@@ -40,10 +43,10 @@ import org.apache.hadoop.hdfs.protocol.H
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.HdfsLocatedFileStatus;
-import org.apache.hadoop.hdfs.protocol.HdfsProtoUtil;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CreateFlagProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.DatanodeReportTypeProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFsStatsResponseProto;
@@ -127,15 +130,20 @@ import org.apache.hadoop.hdfs.server.pro
 import org.apache.hadoop.hdfs.server.protocol.RegisterCommand;
 import org.apache.hadoop.hdfs.server.protocol.RemoteEditLog;
 import org.apache.hadoop.hdfs.server.protocol.RemoteEditLogManifest;
+import org.apache.hadoop.hdfs.util.ExactSizeInputStream;
 import org.apache.hadoop.io.EnumSetWritable;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.security.proto.SecurityProtos.TokenProto;
 import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.util.DataChecksum;
 
+import com.google.common.collect.Lists;
 import com.google.protobuf.ByteString;
+import com.google.protobuf.CodedInputStream;
 
 /**
- * Utilities for converting protobuf classes to and from implementation classes.
+ * Utilities for converting protobuf classes to and from implementation classes
+ * and other helper utilities to help in dealing with protobuf.
  * 
  * Note that when converting from an internal type to protobuf type, the
  * converter never return null for protobuf type. The check for internal type
@@ -219,7 +227,8 @@ public class PBHelper {
 
   // Arrays of DatanodeId
   public static DatanodeIDProto[] convert(DatanodeID[] did) {
-    if (did == null) return null;
+    if (did == null)
+      return null;
     final int len = did.length;
     DatanodeIDProto[] result = new DatanodeIDProto[len];
     for (int i = 0; i < len; ++i) {
@@ -482,14 +491,26 @@ public class PBHelper {
     }    
     return result;
   }
+
+  public static List<? extends HdfsProtos.DatanodeInfoProto> convert(
+      DatanodeInfo[] dnInfos) {
+    return convert(dnInfos, 0);
+  }
   
-  static public DatanodeInfoProto[] convert(DatanodeInfo[] di) {
-    if (di == null) return null;
-    DatanodeInfoProto[] result = new DatanodeInfoProto[di.length];
-    for (int i = 0; i < di.length; i++) {
-      result[i] = PBHelper.convertDatanodeInfo(di[i]);
+  /**
+   * Copy from {@code dnInfos} to a target of list of same size starting at
+   * {@code startIdx}.
+   */
+  public static List<? extends HdfsProtos.DatanodeInfoProto> convert(
+      DatanodeInfo[] dnInfos, int startIdx) {
+    if (dnInfos == null)
+      return null;
+    ArrayList<HdfsProtos.DatanodeInfoProto> protos = Lists
+        .newArrayListWithCapacity(dnInfos.length);
+    for (int i = startIdx; i < dnInfos.length; i++) {
+      protos.add(convert(dnInfos[i]));
     }
-    return result;
+    return protos;
   }
 
   public static DatanodeInfo[] convert(List<DatanodeInfoProto> list) {
@@ -694,7 +715,7 @@ public class PBHelper {
     DatanodeInfosProto[] ret = new DatanodeInfosProto[targets.length];
     for (int i = 0; i < targets.length; i++) {
       ret[i] = DatanodeInfosProto.newBuilder()
-          .addAllDatanodes(Arrays.asList(PBHelper.convert(targets[i]))).build();
+          .addAllDatanodes(PBHelper.convert(targets[i])).build();
     }
     return Arrays.asList(ret);
   }
@@ -963,7 +984,7 @@ public class PBHelper {
         fs.getFileBufferSize(),
         fs.getEncryptDataTransfer(),
         fs.getTrashInterval(),
-        HdfsProtoUtil.fromProto(fs.getChecksumType()));
+        PBHelper.convert(fs.getChecksumType()));
   }
   
   public static FsServerDefaultsProto convert(FsServerDefaults fs) {
@@ -976,7 +997,7 @@ public class PBHelper {
       .setFileBufferSize(fs.getFileBufferSize())
       .setEncryptDataTransfer(fs.getEncryptDataTransfer())
       .setTrashInterval(fs.getTrashInterval())
-      .setChecksumType(HdfsProtoUtil.toProto(fs.getChecksumType()))
+      .setChecksumType(PBHelper.convert(fs.getChecksumType()))
       .build();
   }
   
@@ -1314,4 +1335,24 @@ public class PBHelper {
         .setLayoutVersion(j.getLayoutVersion())
         .setNamespaceID(j.getNamespaceId()).build();
   }
+
+  public static DataChecksum.Type convert(HdfsProtos.ChecksumTypeProto type) {
+    return DataChecksum.Type.valueOf(type.getNumber());
+  }
+
+  public static HdfsProtos.ChecksumTypeProto convert(DataChecksum.Type type) {
+    return HdfsProtos.ChecksumTypeProto.valueOf(type.id);
+  }
+
+  public static InputStream vintPrefixed(final InputStream input)
+      throws IOException {
+    final int firstByte = input.read();
+    if (firstByte == -1) {
+      throw new EOFException("Premature EOF: no length prefix available");
+    }
+
+    int size = CodedInputStream.readRawVarint32(firstByte, input);
+    assert size >= 0;
+    return new ExactSizeInputStream(input, size);
+  }
 }

Modified: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Balancer.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Balancer.java?rev=1431101&r1=1431100&r2=1431101&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Balancer.java (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Balancer.java Wed Jan  9 21:37:01 2013
@@ -18,7 +18,8 @@
 package org.apache.hadoop.hdfs.server.balancer;
 
 import static com.google.common.base.Preconditions.checkArgument;
-import static org.apache.hadoop.hdfs.protocol.HdfsProtoUtil.vintPrefixed;
+
+import static org.apache.hadoop.hdfs.protocolPB.PBHelper.vintPrefixed;
 
 import java.io.BufferedInputStream;
 import java.io.BufferedOutputStream;

Modified: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java?rev=1431101&r1=1431100&r2=1431101&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java Wed Jan  9 21:37:01 2013
@@ -99,7 +99,6 @@ import org.apache.hadoop.hdfs.protocol.D
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsBlocksMetadata;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
-import org.apache.hadoop.hdfs.protocol.HdfsProtoUtil;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.RecoveryInProgressException;
 import org.apache.hadoop.hdfs.protocol.datatransfer.BlockConstructionStage;
@@ -117,6 +116,7 @@ import org.apache.hadoop.hdfs.protocolPB
 import org.apache.hadoop.hdfs.protocolPB.InterDatanodeProtocolPB;
 import org.apache.hadoop.hdfs.protocolPB.InterDatanodeProtocolServerSideTranslatorPB;
 import org.apache.hadoop.hdfs.protocolPB.InterDatanodeProtocolTranslatorPB;
+import org.apache.hadoop.hdfs.protocolPB.PBHelper;
 import org.apache.hadoop.hdfs.security.token.block.BlockPoolTokenSecretManager;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager;
@@ -1468,7 +1468,7 @@ public class DataNode extends Configured
         // read ack
         if (isClient) {
           DNTransferAckProto closeAck = DNTransferAckProto.parseFrom(
-              HdfsProtoUtil.vintPrefixed(in));
+              PBHelper.vintPrefixed(in));
           if (LOG.isDebugEnabled()) {
             LOG.debug(getClass().getSimpleName() + ": close-ack=" + closeAck);
           }

Modified: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java?rev=1431101&r1=1431100&r2=1431101&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java Wed Jan  9 21:37:01 2013
@@ -42,7 +42,6 @@ import org.apache.commons.logging.Log;
 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.HdfsProtoUtil;
 import org.apache.hadoop.hdfs.protocol.datatransfer.BlockConstructionStage;
 import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferEncryptor.InvalidMagicNumberException;
 import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtoUtil;
@@ -56,6 +55,7 @@ import org.apache.hadoop.hdfs.protocol.p
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ReadOpChecksumInfoProto;
 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.BlockTokenSecretManager;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
@@ -144,7 +144,7 @@ class DataXceiver extends Receiver imple
   /** Return the datanode object. */
   DataNode getDataNode() {return datanode;}
   
-  private OutputStream getOutputStream() throws IOException {
+  private OutputStream getOutputStream() {
     return socketOut;
   }
 
@@ -284,7 +284,7 @@ class DataXceiver extends Receiver imple
         // to respond with a Status enum.
         try {
           ClientReadStatusProto stat = ClientReadStatusProto.parseFrom(
-              HdfsProtoUtil.vintPrefixed(in));
+              PBHelper.vintPrefixed(in));
           if (!stat.hasStatus()) {
             LOG.warn("Client " + s.getInetAddress() + " did not send a valid status " +
                      "code after reading. Will close connection.");
@@ -445,7 +445,7 @@ class DataXceiver extends Receiver imple
           // read connect ack (only for clients, not for replication req)
           if (isClient) {
             BlockOpResponseProto connectAck =
-              BlockOpResponseProto.parseFrom(HdfsProtoUtil.vintPrefixed(mirrorIn));
+              BlockOpResponseProto.parseFrom(PBHelper.vintPrefixed(mirrorIn));
             mirrorInStatus = connectAck.getStatus();
             firstBadLink = connectAck.getFirstBadLink();
             if (LOG.isDebugEnabled() || mirrorInStatus != SUCCESS) {
@@ -606,7 +606,7 @@ class DataXceiver extends Receiver imple
           .setBytesPerCrc(bytesPerCRC)
           .setCrcPerBlock(crcPerBlock)
           .setMd5(ByteString.copyFrom(md5.getDigest()))
-          .setCrcType(HdfsProtoUtil.toProto(checksum.getChecksumType()))
+          .setCrcType(PBHelper.convert(checksum.getChecksumType()))
           )
         .build()
         .writeDelimitedTo(out);
@@ -765,7 +765,7 @@ class DataXceiver extends Receiver imple
       // receive the response from the proxy
       
       BlockOpResponseProto copyResponse = BlockOpResponseProto.parseFrom(
-          HdfsProtoUtil.vintPrefixed(proxyReply));
+          PBHelper.vintPrefixed(proxyReply));
 
       if (copyResponse.getStatus() != SUCCESS) {
         if (copyResponse.getStatus() == ERROR_ACCESS_TOKEN) {

Modified: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocolPB/TestPBHelper.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocolPB/TestPBHelper.java?rev=1431101&r1=1431100&r2=1431101&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocolPB/TestPBHelper.java (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocolPB/TestPBHelper.java Wed Jan  9 21:37:01 2013
@@ -31,6 +31,7 @@ import org.apache.hadoop.hdfs.protocol.D
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo.AdminStates;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockCommandProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockRecoveryCommandProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto;
@@ -70,6 +71,7 @@ import org.apache.hadoop.hdfs.server.pro
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.security.proto.SecurityProtos.TokenProto;
 import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.util.DataChecksum;
 import org.junit.Test;
 
 import com.google.common.base.Joiner;
@@ -471,4 +473,20 @@ public class TestPBHelper {
       }
     }
   }
+  
+  @Test
+  public void testChecksumTypeProto() {
+    assertEquals(DataChecksum.Type.NULL,
+        PBHelper.convert(HdfsProtos.ChecksumTypeProto.CHECKSUM_NULL));
+    assertEquals(DataChecksum.Type.CRC32,
+        PBHelper.convert(HdfsProtos.ChecksumTypeProto.CHECKSUM_CRC32));
+    assertEquals(DataChecksum.Type.CRC32C,
+        PBHelper.convert(HdfsProtos.ChecksumTypeProto.CHECKSUM_CRC32C));
+    assertEquals(PBHelper.convert(DataChecksum.Type.NULL),
+        HdfsProtos.ChecksumTypeProto.CHECKSUM_NULL);
+    assertEquals(PBHelper.convert(DataChecksum.Type.CRC32),
+        HdfsProtos.ChecksumTypeProto.CHECKSUM_CRC32);
+    assertEquals(PBHelper.convert(DataChecksum.Type.CRC32C),
+        HdfsProtos.ChecksumTypeProto.CHECKSUM_CRC32C);
+  }
 }