浏览代码

HADOOP-9173. Merging changes r1428970 and r1428972 from trunk

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-2@1429030 13f79535-47bb-0310-9956-ffa450edef68
Suresh Srinivas 12 年之前
父节点
当前提交
b1fe072e04
共有 17 个文件被更改,包括 106 次插入63 次删除
  1. 2 0
      hadoop-common-project/hadoop-common/CHANGES.txt
  2. 4 0
      hadoop-common-project/hadoop-common/dev-support/findbugsExcludeFile.xml
  3. 1 0
      hadoop-common-project/hadoop-common/pom.xml
  4. 34 0
      hadoop-common-project/hadoop-common/src/main/proto/Security.proto
  5. 3 0
      hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
  6. 5 1
      hadoop-hdfs-project/hadoop-hdfs/pom.xml
  7. 26 28
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/pom.xml
  8. 4 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsProtoUtil.java
  9. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolServerSideTranslatorPB.java
  10. 3 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolTranslatorPB.java
  11. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
  12. 5 5
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
  13. 3 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientDatanodeProtocol.proto
  14. 4 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto
  15. 2 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/proto/datatransfer.proto
  16. 4 11
      hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto
  17. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocolPB/TestPBHelper.java

+ 2 - 0
hadoop-common-project/hadoop-common/CHANGES.txt

@@ -109,6 +109,8 @@ Release 2.0.3-alpha - Unreleased
     HADOOP-9162. Add utility to check native library availability.
     (Binglin Chang via suresh)
 
+    HADOOP-9173. Add security token protobuf definition to common and
+    use it in hdfs. (suresh)
 
   OPTIMIZATIONS
 

+ 4 - 0
hadoop-common-project/hadoop-common/dev-support/findbugsExcludeFile.xml

@@ -294,6 +294,10 @@
       <!-- protobuf generated code -->
       <Class name="~org\.apache\.hadoop\.ha\.proto\.ZKFCProtocolProtos.*"/>
     </Match>
+    <Match>
+      <!-- protobuf generated code -->
+      <Class name="~org\.apache\.hadoop\.security\.proto\.SecurityProtos.*"/>
+    </Match>
 
     <!--
        Manually checked, misses child thread manually syncing on parent's intrinsic lock.

+ 1 - 0
hadoop-common-project/hadoop-common/pom.xml

@@ -376,6 +376,7 @@
                 <argument>src/main/proto/RpcPayloadHeader.proto</argument>
                 <argument>src/main/proto/ZKFCProtocol.proto</argument>
                 <argument>src/main/proto/hadoop_rpc.proto</argument>
+                <argument>src/main/proto/Security.proto</argument>
               </arguments>
             </configuration>
           </execution>

+ 34 - 0
hadoop-common-project/hadoop-common/src/main/proto/Security.proto

@@ -0,0 +1,34 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+option java_package = "org.apache.hadoop.security.proto";
+option java_outer_classname = "SecurityProtos";
+option java_generic_services = true;
+option java_generate_equals_and_hash = true;
+package hadoop.common;
+
+/**
+ * Security token identifier
+ */
+message TokenProto {
+  required bytes identifier = 1;
+  required bytes password = 2;
+  required string kind = 3;
+  required string service = 4;
+}
+

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

@@ -158,6 +158,9 @@ Release 2.0.3-alpha - Unreleased
     that has reached its soft limit but not the hard limit.  (Derek Dagit via
     szetszwo)
 
+    HADOOP-9173. Add security token protobuf definition to common and
+    use it in hdfs. (suresh)
+
   OPTIMIZATIONS
 
   BUG FIXES 

+ 5 - 1
hadoop-hdfs-project/hadoop-hdfs/pom.xml

@@ -415,8 +415,10 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
             <configuration>
               <executable>protoc</executable>
               <arguments>
+                <argument>-I../../hadoop-common-project/hadoop-common/src/main/proto/</argument>
                 <argument>-Isrc/main/proto/</argument>
                 <argument>--java_out=target/generated-sources/java</argument>
+                <argument>src/main/proto/hdfs.proto</argument>
                 <argument>src/main/proto/GetUserMappingsProtocol.proto</argument>
                 <argument>src/main/proto/HAZKInfo.proto</argument>
                 <argument>src/main/proto/InterDatanodeProtocol.proto</argument>
@@ -424,7 +426,6 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
                 <argument>src/main/proto/RefreshAuthorizationPolicyProtocol.proto</argument>
                 <argument>src/main/proto/RefreshUserMappingsProtocol.proto</argument>
                 <argument>src/main/proto/datatransfer.proto</argument>
-                <argument>src/main/proto/hdfs.proto</argument>
               </arguments>
             </configuration>
           </execution>
@@ -437,6 +438,7 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
             <configuration>
               <executable>protoc</executable>
               <arguments>
+                <argument>-I../../hadoop-common-project/hadoop-common/src/main/proto/</argument>
                 <argument>-Isrc/main/proto/</argument>
                 <argument>--java_out=target/generated-sources/java</argument>
                 <argument>src/main/proto/ClientDatanodeProtocol.proto</argument>
@@ -453,6 +455,7 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
             <configuration>
               <executable>protoc</executable>
               <arguments>
+                <argument>-I../../hadoop-common-project/hadoop-common/src/main/proto/</argument>
                 <argument>-Isrc/main/proto/</argument>
                 <argument>--java_out=target/generated-sources/java</argument>
                 <argument>src/main/proto/ClientNamenodeProtocol.proto</argument>
@@ -469,6 +472,7 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
             <configuration>
               <executable>protoc</executable>
               <arguments>
+                <argument>-I../../hadoop-common-project/hadoop-common/src/main/proto/</argument>
                 <argument>-Isrc/main/proto/</argument>
                 <argument>--java_out=target/generated-sources/java</argument>
                 <argument>src/main/proto/QJournalProtocol.proto</argument>

+ 26 - 28
hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/pom.xml

@@ -117,44 +117,42 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
         </configuration>
         <executions>
           <execution>
-            <id>compile-proto</id>
-            <phase>generate-sources</phase>
+            <id>create-protobuf-generated-sources-directory</id>
+            <phase>initialize</phase>
             <goals>
               <goal>run</goal>
             </goals>
             <configuration>
               <target>
-                <echo file="target/compile-proto.sh">
-                    PROTO_DIR=src/main/proto
-                    INCLUDE_DIR=../../main/proto
-                    JAVA_DIR=target/generated-sources/java
-                    which cygpath 2&gt; /dev/null
-                    if [ $? = 1 ]; then
-                      IS_WIN=false
-                    else
-                      IS_WIN=true
-                      WIN_PROTO_DIR=`cygpath --windows $PROTO_DIR`
-                      WIN_JAVA_DIR=`cygpath --windows $JAVA_DIR`
-                      WIN_INCLUDE_DIR=`cygpath --windows $INCLUDE_DIR`
-                    fi
-                    mkdir -p $JAVA_DIR 2&gt; /dev/null
-                    for PROTO_FILE in `ls $PROTO_DIR/*.proto 2&gt; /dev/null`
-                    do
-                        if [ "$IS_WIN" = "true" ]; then
-                          protoc -I$WIN_PROTO_DIR -I$WIN_INCLUDE_DIR --java_out=$WIN_JAVA_DIR $PROTO_FILE
-                        else
-                          protoc -I$PROTO_DIR -I$INCLUDE_DIR --java_out=$JAVA_DIR $PROTO_FILE
-                        fi
-                    done
-                </echo>
-                <exec executable="sh" dir="${basedir}" failonerror="true">
-                  <arg line="target/compile-proto.sh"/>
-                </exec>
+                <mkdir dir="target/generated-sources/java" />
               </target>
             </configuration>
           </execution>
         </executions>
       </plugin>
+      <plugin>
+        <groupId>org.codehaus.mojo</groupId>
+        <artifactId>exec-maven-plugin</artifactId>
+        <executions>
+          <execution>
+            <id>compile-proto</id>
+            <phase>generate-sources</phase>
+            <goals>
+              <goal>exec</goal>
+            </goals>
+            <configuration>
+              <executable>protoc</executable>
+              <arguments>
+                <argument>-I../../../../../hadoop-common-project/hadoop-common/src/main/proto/</argument>
+                <argument>-Isrc/main/proto/</argument>
+                <argument>-I../../main/proto</argument>
+                <argument>--java_out=target/generated-sources/java</argument>
+                <argument>src/main/proto/bkjournal.proto</argument>
+              </arguments>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
       <plugin>
         <groupId>org.codehaus.mojo</groupId>
         <artifactId>findbugs-maven-plugin</artifactId>

+ 4 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsProtoUtil.java

@@ -30,6 +30,7 @@ import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 import org.apache.hadoop.hdfs.util.ExactSizeInputStream;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.util.DataChecksum;
+import org.apache.hadoop.security.proto.SecurityProtos.TokenProto;
 import org.apache.hadoop.security.token.Token;
 
 import com.google.common.collect.Lists;
@@ -47,8 +48,8 @@ public abstract class HdfsProtoUtil {
   
   //// Block Token ////
   
-  public static HdfsProtos.BlockTokenIdentifierProto toProto(Token<?> blockToken) {
-    return HdfsProtos.BlockTokenIdentifierProto.newBuilder()
+  public static TokenProto toProto(Token<?> blockToken) {
+    return TokenProto.newBuilder()
       .setIdentifier(ByteString.copyFrom(blockToken.getIdentifier()))
       .setPassword(ByteString.copyFrom(blockToken.getPassword()))
       .setKind(blockToken.getKind().toString())
@@ -56,7 +57,7 @@ public abstract class HdfsProtoUtil {
       .build();
   }
 
-  public static Token<BlockTokenIdentifier> fromProto(HdfsProtos.BlockTokenIdentifierProto proto) {
+  public static Token<BlockTokenIdentifier> fromProto(TokenProto proto) {
     return new Token<BlockTokenIdentifier>(proto.getIdentifier().toByteArray(),
         proto.getPassword().toByteArray(),
         new Text(proto.getKind()),

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolServerSideTranslatorPB.java

@@ -37,9 +37,9 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetRep
 import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetReplicaVisibleLengthResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.RefreshNamenodesRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.RefreshNamenodesResponseProto;
-import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockTokenIdentifierProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProto;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
+import org.apache.hadoop.security.proto.SecurityProtos.TokenProto;
 import org.apache.hadoop.security.token.Token;
 
 import com.google.protobuf.ByteString;
@@ -133,7 +133,7 @@ public class ClientDatanodeProtocolServerSideTranslatorPB implements
       }
       List<Token<BlockTokenIdentifier>> tokens = 
           new ArrayList<Token<BlockTokenIdentifier>>(request.getTokensCount());
-      for (BlockTokenIdentifierProto b : request.getTokensList()) {
+      for (TokenProto b : request.getTokensList()) {
         tokens.add(PBHelper.convert(b));
       }
       // Call the real implementation

+ 3 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolTranslatorPB.java

@@ -44,7 +44,6 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetHdf
 import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetHdfsBlockLocationsResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetReplicaVisibleLengthRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.RefreshNamenodesRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockTokenIdentifierProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProto;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 import org.apache.hadoop.ipc.ProtobufHelper;
@@ -55,6 +54,7 @@ import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.ipc.RpcClientUtil;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.proto.SecurityProtos.TokenProto;
 import org.apache.hadoop.security.token.Token;
 
 import com.google.protobuf.ByteString;
@@ -222,8 +222,8 @@ public class ClientDatanodeProtocolTranslatorPB implements
     // Convert to proto objects
     List<ExtendedBlockProto> blocksProtos = 
         new ArrayList<ExtendedBlockProto>(blocks.size());
-    List<BlockTokenIdentifierProto> tokensProtos = 
-        new ArrayList<BlockTokenIdentifierProto>(tokens.size());
+    List<TokenProto> tokensProtos = 
+        new ArrayList<TokenProto>(tokens.size());
     for (ExtendedBlock b : blocks) {
       blocksProtos.add(PBHelper.convert(b));
     }

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java

@@ -125,12 +125,12 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.Update
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.UpdateBlockForPipelineResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.UpdatePipelineRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.UpdatePipelineResponseProto;
-import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockTokenIdentifierProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.LocatedBlockProto;
 import org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey;
 import org.apache.hadoop.io.Text;
+import org.apache.hadoop.security.proto.SecurityProtos.TokenProto;
 
 import com.google.protobuf.RpcController;
 import com.google.protobuf.ServiceException;
@@ -774,7 +774,7 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
       RpcController controller, GetDelegationTokenRequestProto req)
       throws ServiceException {
     try {
-      BlockTokenIdentifierProto result = PBHelper.convert(server
+      TokenProto result = PBHelper.convert(server
           .getDelegationToken(new Text(req.getRenewer())));
       return GetDelegationTokenResponseProto.newBuilder().setToken(result)
           .build();

+ 5 - 5
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java

@@ -64,7 +64,6 @@ import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.StorageRepor
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DataEncryptionKeyProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockKeyProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto;
-import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockTokenIdentifierProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockWithLocationsProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlocksWithLocationsProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.CheckpointCommandProto;
@@ -130,6 +129,7 @@ import org.apache.hadoop.hdfs.server.protocol.RemoteEditLog;
 import org.apache.hadoop.hdfs.server.protocol.RemoteEditLogManifest;
 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 com.google.protobuf.ByteString;
@@ -552,8 +552,8 @@ public class PBHelper {
     return lb;
   }
 
-  public static BlockTokenIdentifierProto convert(Token<?> tok) {
-    return BlockTokenIdentifierProto.newBuilder().
+  public static TokenProto convert(Token<?> tok) {
+    return TokenProto.newBuilder().
               setIdentifier(ByteString.copyFrom(tok.getIdentifier())).
               setPassword(ByteString.copyFrom(tok.getPassword())).
               setKind(tok.getKind().toString()).
@@ -561,7 +561,7 @@ public class PBHelper {
   }
   
   public static Token<BlockTokenIdentifier> convert(
-      BlockTokenIdentifierProto blockToken) {
+      TokenProto blockToken) {
     return new Token<BlockTokenIdentifier>(blockToken.getIdentifier()
         .toByteArray(), blockToken.getPassword().toByteArray(), new Text(
         blockToken.getKind()), new Text(blockToken.getService()));
@@ -569,7 +569,7 @@ public class PBHelper {
 
   
   public static Token<DelegationTokenIdentifier> convertDelegationToken(
-      BlockTokenIdentifierProto blockToken) {
+      TokenProto blockToken) {
     return new Token<DelegationTokenIdentifier>(blockToken.getIdentifier()
         .toByteArray(), blockToken.getPassword().toByteArray(), new Text(
         blockToken.getKind()), new Text(blockToken.getService()));

+ 3 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientDatanodeProtocol.proto

@@ -25,6 +25,7 @@ option java_generic_services = true;
 option java_generate_equals_and_hash = true;
 package hadoop.hdfs;
 
+import "Security.proto";
 import "hdfs.proto";
 
 /**
@@ -76,7 +77,7 @@ message DeleteBlockPoolResponseProto {
  */
 message GetBlockLocalPathInfoRequestProto {
   required ExtendedBlockProto block = 1;
-  required BlockTokenIdentifierProto token = 2;
+  required hadoop.common.TokenProto token = 2;
 }
 
 /**
@@ -96,7 +97,7 @@ message GetBlockLocalPathInfoResponseProto {
  */
 message GetHdfsBlockLocationsRequestProto {
   repeated ExtendedBlockProto blocks = 1;
-  repeated BlockTokenIdentifierProto tokens = 2;
+  repeated hadoop.common.TokenProto tokens = 2;
 }
 
 /**

+ 4 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto

@@ -22,6 +22,7 @@ option java_generic_services = true;
 option java_generate_equals_and_hash = true;
 package hadoop.hdfs;
 
+import "Security.proto";
 import "hdfs.proto";
 
 /**
@@ -413,11 +414,11 @@ message GetDelegationTokenRequestProto {
 }
 
 message GetDelegationTokenResponseProto {
-  required BlockTokenIdentifierProto token = 1;
+  required hadoop.common.TokenProto token = 1;
 }
 
 message RenewDelegationTokenRequestProto {
-  required BlockTokenIdentifierProto token = 1;
+  required hadoop.common.TokenProto token = 1;
 }
 
 message RenewDelegationTokenResponseProto {
@@ -425,7 +426,7 @@ message RenewDelegationTokenResponseProto {
 }
 
 message CancelDelegationTokenRequestProto {
-  required BlockTokenIdentifierProto token = 1;
+  required hadoop.common.TokenProto token = 1;
 }
 
 message CancelDelegationTokenResponseProto { // void response

+ 2 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/proto/datatransfer.proto

@@ -24,6 +24,7 @@ option java_outer_classname = "DataTransferProtos";
 option java_generate_equals_and_hash = true;
 package hadoop.hdfs;
 
+import "Security.proto";
 import "hdfs.proto";
 
 message DataTransferEncryptorMessageProto {
@@ -39,7 +40,7 @@ message DataTransferEncryptorMessageProto {
 
 message BaseHeaderProto {
   required ExtendedBlockProto block = 1;
-  optional BlockTokenIdentifierProto token = 2;
+  optional hadoop.common.TokenProto token = 2;
 }
 
 message ClientOperationHeaderProto {

+ 4 - 11
hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto

@@ -19,11 +19,14 @@
 // This file contains protocol buffers that are used throughout HDFS -- i.e.
 // by the client, server, and data transfer protocols.
 
+
 option java_package = "org.apache.hadoop.hdfs.protocol.proto";
 option java_outer_classname = "HdfsProtos";
 option java_generate_equals_and_hash = true;
 package hadoop.hdfs;
 
+import "Security.proto";
+
 /**
  * Extended block idenfies a block
  */
@@ -35,16 +38,6 @@ message ExtendedBlockProto {
                                                // here for historical reasons
 }
 
-/**
- * Block Token
- */
-message BlockTokenIdentifierProto {
-  required bytes identifier = 1;
-  required bytes password = 2;
-  required string kind = 3;
-  required string service = 4;
-}
-
 /**
  * Identifies a Datanode
  */
@@ -126,7 +119,7 @@ message LocatedBlockProto {
                                         // If block has few corrupt replicas, they are filtered and 
                                         // their locations are not part of this object
 
-  required BlockTokenIdentifierProto blockToken = 5;
+  required hadoop.common.TokenProto blockToken = 5;
 }
 
 message DataEncryptionKeyProto {

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocolPB/TestPBHelper.java

@@ -36,7 +36,6 @@ import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockRecover
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockKeyProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto;
-import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockTokenIdentifierProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockWithLocationsProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlocksWithLocationsProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.CheckpointSignatureProto;
@@ -69,6 +68,7 @@ import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
 import org.apache.hadoop.hdfs.server.protocol.RemoteEditLog;
 import org.apache.hadoop.hdfs.server.protocol.RemoteEditLogManifest;
 import org.apache.hadoop.io.Text;
+import org.apache.hadoop.security.proto.SecurityProtos.TokenProto;
 import org.apache.hadoop.security.token.Token;
 import org.junit.Test;
 
@@ -374,7 +374,7 @@ public class TestPBHelper {
     Token<BlockTokenIdentifier> token = new Token<BlockTokenIdentifier>(
         "identifier".getBytes(), "password".getBytes(), new Text("kind"),
         new Text("service"));
-    BlockTokenIdentifierProto tokenProto = PBHelper.convert(token);
+    TokenProto tokenProto = PBHelper.convert(token);
     Token<BlockTokenIdentifier> token2 = PBHelper.convert(tokenProto);
     compare(token, token2);
   }