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 sz...@apache.org on 2012/02/27 00:32:14 UTC

svn commit: r1293964 [9/11] - in /hadoop/common/branches/branch-0.23-PB-merge/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/protocolPB/ src/...

Added: hadoop/common/branches/branch-0.23-PB-merge/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23-PB-merge/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto?rev=1293964&view=auto
==============================================================================
--- hadoop/common/branches/branch-0.23-PB-merge/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto (added)
+++ hadoop/common/branches/branch-0.23-PB-merge/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto Sun Feb 26 23:32:06 2012
@@ -0,0 +1,516 @@
+/**
+ * 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.hdfs.protocol.proto";
+option java_outer_classname = "ClientNamenodeProtocolProtos";
+option java_generic_services = true;
+option java_generate_equals_and_hash = true;
+
+import "hdfs.proto";
+
+/**
+ * The ClientNamenodeProtocol Service defines the interface between a client 
+ * (as runnign inside a MR Task) and the Namenode.
+ * See org.apache.hadoop.hdfs.protocol.ClientProtocol for the javadoc 
+ * for each of the methods.
+ * The exceptions declared in the above class also apply to this protocol.
+ * Exceptions are unwrapped and thrown by the  PB libraries.
+ */
+
+message GetBlockLocationsRequestProto {
+  required string src = 1;     // file name
+  required uint64 offset = 2;  // range start offset
+  required uint64 length = 3;  // range length
+}
+
+message GetBlockLocationsResponseProto {
+  optional LocatedBlocksProto locations = 1;
+}
+
+message GetServerDefaultsRequestProto { // No parameters
+}
+
+message GetServerDefaultsResponseProto {
+  required FsServerDefaultsProto serverDefaults = 1;
+}
+
+enum CreateFlagProto {
+  CREATE = 0x01;    // Create a file
+  OVERWRITE = 0x02; // Truncate/overwrite a file. Same as POSIX O_TRUNC
+  APPEND = 0x04;    // Append to a file
+}
+
+message CreateRequestProto {
+  required string src = 1;
+  required FsPermissionProto masked = 2;
+  required string clientName = 3;
+  required uint32 createFlag = 4;  // bits set using CreateFlag
+  required bool createParent = 5;
+  required uint32 replication = 6; // Short: Only 16 bits used
+  required uint64 blockSize = 7;
+}
+
+message CreateResponseProto { // void response
+}
+
+message AppendRequestProto {
+  required string src = 1;
+  required string clientName = 2;
+}
+
+message AppendResponseProto {
+  optional LocatedBlockProto block = 1;
+}
+
+message SetReplicationRequestProto {
+  required string src = 1;
+  required uint32 replication = 2; // Short: Only 16 bits used
+}
+
+message SetReplicationResponseProto {
+  required bool result = 1;
+}
+
+message SetPermissionRequestProto {
+  required string src = 1;
+  required FsPermissionProto permission = 2;
+}
+
+message SetPermissionResponseProto { // void response
+}
+
+message SetOwnerRequestProto {
+  required string src = 1;
+  optional string username = 2;
+  optional string groupname = 3;
+}
+
+message SetOwnerResponseProto { // void response
+}
+
+message AbandonBlockRequestProto {
+  required ExtendedBlockProto b = 1;
+  required string src = 2;
+  required string holder = 3;
+}
+
+message AbandonBlockResponseProto { // void response
+}
+
+message AddBlockRequestProto {
+  required string src = 1;
+  required string clientName = 2;
+  optional ExtendedBlockProto previous = 3;
+  repeated DatanodeInfoProto excludeNodes = 4;
+}
+
+message AddBlockResponseProto {
+  required LocatedBlockProto block = 1;
+}
+
+message GetAdditionalDatanodeRequestProto {
+  required string src = 1;
+  required ExtendedBlockProto blk = 2;
+  repeated DatanodeInfoProto existings = 3;
+  repeated DatanodeInfoProto excludes = 4;
+  required uint32 numAdditionalNodes = 5;
+  required string clientName = 6;
+}
+
+message GetAdditionalDatanodeResponseProto {
+  required LocatedBlockProto block = 1;
+}
+
+message CompleteRequestProto {
+  required string src = 1;
+  required string clientName = 2;
+  optional ExtendedBlockProto last = 3;
+}
+
+message CompleteResponseProto {
+  required bool result = 1;
+}
+
+message ReportBadBlocksRequestProto {
+  repeated LocatedBlockProto blocks = 1;
+}
+
+message ReportBadBlocksResponseProto { // void response
+}
+
+message ConcatRequestProto {
+  required string trg = 1;
+  repeated string srcs = 2;
+}
+
+message ConcatResponseProto { // void response
+}
+
+message RenameRequestProto {
+  required string src = 1;
+  required string dst = 2;
+}
+
+message RenameResponseProto { // void response
+  required bool result = 1;
+}
+
+
+message Rename2RequestProto {
+  required string src = 1;
+  required string dst = 2;
+  required bool overwriteDest = 3;
+}
+
+message Rename2ResponseProto { // void response
+}
+
+message DeleteRequestProto {
+  required string src = 1;
+  required bool recursive = 2;
+}
+
+message DeleteResponseProto {
+    required bool result = 1;
+}
+
+message MkdirsRequestProto {
+  required string src = 1;
+  required FsPermissionProto masked = 2;
+  required bool createParent = 3;
+}
+message MkdirsResponseProto {
+    required bool result = 1;
+}
+
+message GetListingRequestProto {
+  required string src = 1;
+  required bytes startAfter = 2;
+  required bool needLocation = 3;
+}
+message GetListingResponseProto {
+  optional DirectoryListingProto dirList = 1;
+}
+
+message RenewLeaseRequestProto {
+  required string clientName = 1;
+}
+
+message RenewLeaseResponseProto { //void response
+}
+
+message RecoverLeaseRequestProto {
+  required string src = 1;
+  required string clientName = 2;
+}
+message RecoverLeaseResponseProto {
+  required bool result = 1;
+}
+
+message GetFsStatusRequestProto { // no input paramters
+}
+
+message GetFsStatsResponseProto {
+  required uint64 capacity = 1;
+  required uint64 used = 2;
+  required uint64 remaining = 3;
+  required uint64 under_replicated = 4;
+  required uint64 corrupt_blocks = 5;
+  required uint64 missing_blocks = 6;
+}
+
+enum DatanodeReportTypeProto {  // type of the datanode report
+  ALL = 1;
+  LIVE = 2;
+  DEAD = 3;
+}
+
+message GetDatanodeReportRequestProto {
+  required DatanodeReportTypeProto type = 1;
+}
+
+message GetDatanodeReportResponseProto {
+  repeated DatanodeInfoProto di = 1;
+}
+
+message GetPreferredBlockSizeRequestProto {
+  required string filename = 1;
+}
+
+message GetPreferredBlockSizeResponseProto {
+  required uint64 bsize = 1;
+}
+
+enum SafeModeActionProto {
+  SAFEMODE_LEAVE = 1;
+  SAFEMODE_ENTER = 2;
+  SAFEMODE_GET = 3;
+}
+
+message SetSafeModeRequestProto {
+  required SafeModeActionProto action = 1;
+}
+
+message SetSafeModeResponseProto {
+  required bool result = 1;
+}
+
+message SaveNamespaceRequestProto { // no parameters
+}
+
+message SaveNamespaceResponseProto { // void response
+}
+
+message RestoreFailedStorageRequestProto {
+  required string arg = 1;
+}
+
+message RestoreFailedStorageResponseProto {
+    required bool result = 1;
+}
+
+message RefreshNodesRequestProto { // no parameters
+}
+
+message RefreshNodesResponseProto { // void response
+}
+
+message FinalizeUpgradeRequestProto { // no parameters
+}
+
+message FinalizeUpgradeResponseProto { // void response
+}
+
+enum UpgradeActionProto {
+  GET_STATUS = 1;
+  DETAILED_STATUS = 2;
+  FORCE_PROCEED = 3;
+}
+
+message DistributedUpgradeProgressRequestProto {
+  required UpgradeActionProto action = 1;
+}
+message DistributedUpgradeProgressResponseProto {
+  optional UpgradeStatusReportProto report = 1;
+}
+
+message ListCorruptFileBlocksRequestProto {
+  required string path = 1;
+  optional string cookie = 2;
+}
+
+message ListCorruptFileBlocksResponseProto {
+  required CorruptFileBlocksProto corrupt = 1;
+}
+
+message MetaSaveRequestProto {
+  required string filename = 1;
+}
+
+message MetaSaveResponseProto { // void response
+}
+
+message GetFileInfoRequestProto {
+  required string src = 1;
+}
+
+message GetFileInfoResponseProto {
+  optional HdfsFileStatusProto fs = 1;
+}
+
+message GetFileLinkInfoRequestProto {
+  required string src = 1;
+}
+
+message GetFileLinkInfoResponseProto {
+  optional HdfsFileStatusProto fs = 1;
+}
+
+message GetContentSummaryRequestProto {
+  required string path = 1;
+}
+
+message GetContentSummaryResponseProto {
+  required ContentSummaryProto summary = 1;
+}
+
+message SetQuotaRequestProto {
+  required string path = 1;
+  required uint64 namespaceQuota = 2;
+  required uint64 diskspaceQuota = 3;
+}
+
+message SetQuotaResponseProto { // void response
+}
+
+message FsyncRequestProto {
+  required string src = 1;
+  required string client = 2;
+}
+
+message FsyncResponseProto { // void response
+}
+
+message SetTimesRequestProto {
+  required string src = 1;
+  required uint64 mtime = 2;
+  required uint64 atime = 3;
+}
+
+message SetTimesResponseProto { // void response
+}
+
+message CreateSymlinkRequestProto {
+  required string target = 1;
+  required string link = 2;
+  required FsPermissionProto dirPerm = 3;
+  required bool createParent = 4;
+}
+
+message CreateSymlinkResponseProto { // void response
+}
+
+message GetLinkTargetRequestProto {
+  required string path = 1;
+}
+message GetLinkTargetResponseProto {
+  required string targetPath = 1;
+}
+
+message UpdateBlockForPipelineRequestProto {
+  required ExtendedBlockProto block = 1;
+  required string clientName = 2;
+}
+
+message UpdateBlockForPipelineResponseProto {
+  required LocatedBlockProto block = 1;
+}
+
+message UpdatePipelineRequestProto {
+  required string clientName = 1;
+  required ExtendedBlockProto oldBlock = 2;
+  required ExtendedBlockProto newBlock = 3;
+  repeated DatanodeIDProto newNodes = 4;
+}
+
+message UpdatePipelineResponseProto { // void response
+}
+
+message GetDelegationTokenRequestProto {
+  required string renewer = 1;
+}
+
+message GetDelegationTokenResponseProto {
+  required BlockTokenIdentifierProto token = 1;
+}
+
+message RenewDelegationTokenRequestProto {
+  required BlockTokenIdentifierProto token = 1;
+}
+
+message RenewDelegationTokenResponseProto {
+  required uint64 newExireTime = 1;
+}
+
+message CancelDelegationTokenRequestProto {
+  required BlockTokenIdentifierProto token = 1;
+}
+
+message CancelDelegationTokenResponseProto { // void response
+}
+
+message SetBalancerBandwidthRequestProto {
+  required int64 bandwidth = 1;
+}
+
+message SetBalancerBandwidthResponseProto { // void response
+}
+
+
+service ClientNamenodeProtocol {
+  rpc getBlockLocations(GetBlockLocationsRequestProto)
+      returns(GetBlockLocationsResponseProto);
+  rpc getServerDefaults(GetServerDefaultsRequestProto)
+      returns(GetServerDefaultsResponseProto);
+  rpc create(CreateRequestProto)returns(CreateResponseProto);
+  rpc append(AppendRequestProto) returns(AppendResponseProto);
+  rpc setReplication(SetReplicationRequestProto)
+      returns(SetReplicationResponseProto);
+  rpc setPermission(SetPermissionRequestProto)
+      returns(SetPermissionResponseProto);
+  rpc setOwner(SetOwnerRequestProto) returns(SetOwnerResponseProto);
+  rpc abandonBlock(AbandonBlockRequestProto) returns(AbandonBlockResponseProto);
+  rpc addBlock(AddBlockRequestProto) returns(AddBlockResponseProto);
+  rpc getAdditionalDatanode(GetAdditionalDatanodeRequestProto)
+      returns(GetAdditionalDatanodeResponseProto);
+  rpc complete(CompleteRequestProto) returns(CompleteResponseProto);
+  rpc reportBadBlocks(ReportBadBlocksRequestProto)
+      returns(ReportBadBlocksResponseProto);
+  rpc concat(ConcatRequestProto) returns(ConcatResponseProto);
+  rpc rename(RenameRequestProto) returns(RenameResponseProto);
+  rpc rename2(Rename2RequestProto) returns(Rename2ResponseProto);
+  rpc delete(DeleteRequestProto) returns(DeleteResponseProto);
+  rpc mkdirs(MkdirsRequestProto) returns(MkdirsResponseProto);
+  rpc getListing(GetListingRequestProto) returns(GetListingResponseProto);
+  rpc renewLease(RenewLeaseRequestProto) returns(RenewLeaseResponseProto);
+  rpc recoverLease(RecoverLeaseRequestProto)
+      returns(RecoverLeaseResponseProto);
+  rpc getFsStats(GetFsStatusRequestProto) returns(GetFsStatsResponseProto);
+  rpc getDatanodeReport(GetDatanodeReportRequestProto)
+      returns(GetDatanodeReportResponseProto);
+  rpc getPreferredBlockSize(GetPreferredBlockSizeRequestProto)
+      returns(GetPreferredBlockSizeResponseProto);
+  rpc setSafeMode(SetSafeModeRequestProto)
+      returns(SetSafeModeResponseProto);
+  rpc saveNamespace(SaveNamespaceRequestProto)
+      returns(SaveNamespaceResponseProto);
+  rpc restoreFailedStorage(RestoreFailedStorageRequestProto)
+      returns(RestoreFailedStorageResponseProto);
+  rpc refreshNodes(RefreshNodesRequestProto) returns(RefreshNodesResponseProto);
+  rpc finalizeUpgrade(FinalizeUpgradeRequestProto)
+      returns(FinalizeUpgradeResponseProto);
+  rpc distributedUpgradeProgress(DistributedUpgradeProgressRequestProto)
+      returns(DistributedUpgradeProgressResponseProto);
+  rpc listCorruptFileBlocks(ListCorruptFileBlocksRequestProto)
+      returns(ListCorruptFileBlocksResponseProto);
+  rpc metaSave(MetaSaveRequestProto) returns(MetaSaveResponseProto);
+  rpc getFileInfo(GetFileInfoRequestProto) returns(GetFileInfoResponseProto);
+  rpc getFileLinkInfo(GetFileLinkInfoRequestProto)
+      returns(GetFileLinkInfoResponseProto);
+  rpc getContentSummary(GetContentSummaryRequestProto)
+      returns(GetContentSummaryResponseProto);
+  rpc setQuota(SetQuotaRequestProto) returns(SetQuotaResponseProto);
+  rpc fsync(FsyncRequestProto) returns(FsyncResponseProto);
+  rpc setTimes(SetTimesRequestProto) returns(SetTimesResponseProto);
+  rpc createSymlink(CreateSymlinkRequestProto)
+      returns(CreateSymlinkResponseProto);
+  rpc getLinkTarget(GetLinkTargetRequestProto)
+      returns(GetLinkTargetResponseProto);
+  rpc updateBlockForPipeline(UpdateBlockForPipelineRequestProto)
+      returns(UpdateBlockForPipelineResponseProto);
+  rpc updatePipeline(UpdatePipelineRequestProto)
+      returns(UpdatePipelineResponseProto);
+  rpc getDelegationToken(GetDelegationTokenRequestProto)
+      returns(GetDelegationTokenResponseProto);
+  rpc renewDelegationToken(RenewDelegationTokenRequestProto)
+      returns(RenewDelegationTokenResponseProto);
+  rpc cancelDelegationToken(CancelDelegationTokenRequestProto)
+      returns(CancelDelegationTokenResponseProto);
+  rpc  setBalancerBandwidth(SetBalancerBandwidthRequestProto)
+      returns(SetBalancerBandwidthResponseProto);
+}

Added: hadoop/common/branches/branch-0.23-PB-merge/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23-PB-merge/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto?rev=1293964&view=auto
==============================================================================
--- hadoop/common/branches/branch-0.23-PB-merge/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto (added)
+++ hadoop/common/branches/branch-0.23-PB-merge/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto Sun Feb 26 23:32:06 2012
@@ -0,0 +1,385 @@
+/**
+ * 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.
+ */
+
+// 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 = "DatanodeProtocolProtos";
+option java_generic_services = true;
+option java_generate_equals_and_hash = true;
+
+import "hdfs.proto";
+
+/**
+ * Information to identify a datanode to a namenode
+ */
+message DatanodeRegistrationProto {
+  required DatanodeIDProto datanodeID = 1;    // Datanode information
+  required StorageInfoProto storageInfo = 2;  // Node information
+  required ExportedBlockKeysProto keys = 3;   // Block keys
+}
+
+/**
+ * Represents a storage available on the datanode
+ */
+message DatanodeStorageProto {
+  enum StorageState {
+    NORMAL = 0;
+    READ_ONLY = 1;
+  }
+
+  required string storageID = 1;    // Unique identifier for the storage
+  optional StorageState state = 2 [default = NORMAL];
+}
+
+/**
+ * Commands sent from namenode to the datanodes
+ */
+message DatanodeCommandProto {
+  enum Type {
+    BalancerBandwidthCommand = 0;
+    BlockCommand = 1;
+    BlockRecoveryCommand = 2;
+    FinalizeCommand = 3;
+    KeyUpdateCommand = 4;
+    RegisterCommand = 5;
+    UpgradeCommand = 6;
+    NullDatanodeCommand = 7;
+  }
+
+  required Type cmdType = 1;    // Type of the command
+
+  // One of the following command is available when the corresponding
+  // cmdType is set
+  optional BalancerBandwidthCommandProto balancerCmd = 2;
+  optional BlockCommandProto blkCmd = 3;
+  optional BlockRecoveryCommandProto recoveryCmd = 4;
+  optional FinalizeCommandProto finalizeCmd = 5;
+  optional KeyUpdateCommandProto keyUpdateCmd = 6;
+  optional RegisterCommandProto registerCmd = 7;
+  optional UpgradeCommandProto upgradeCmd = 8;
+}
+
+/**
+ * Command sent from namenode to datanode to set the
+ * maximum bandwidth to be used for balancing.
+ */
+message BalancerBandwidthCommandProto {
+
+  // Maximum bandwidth to be used by datanode for balancing
+  required uint64 bandwidth = 1;
+}
+
+/**
+ * Command to instruct datanodes to perform certain action
+ * on the given set of blocks.
+ */
+message BlockCommandProto {
+  enum Action {  
+    TRANSFER = 1;   // Transfer blocks to another datanode
+    INVALIDATE = 2; // Invalidate blocks
+    SHUTDOWN = 3; // Shutdown the datanode
+  }
+  required Action action = 1;
+  required string blockPoolId = 2;
+  repeated BlockProto blocks = 3;
+  repeated DatanodeInfosProto targets = 4;
+}
+
+/**
+ * List of blocks to be recovered by the datanode
+ */
+message BlockRecoveryCommandProto {
+  repeated RecoveringBlockProto blocks = 1;
+}
+
+/**
+ * Finalize the upgrade at the datanode
+ */
+message FinalizeCommandProto {
+  required string blockPoolId = 1; // Block pool to be finalized
+}
+
+/**
+ * Update the block keys at the datanode
+ */
+message KeyUpdateCommandProto {
+  required ExportedBlockKeysProto keys = 1;
+}
+
+/**
+ * Instruct datanode to register with the namenode
+ */
+message RegisterCommandProto {
+  // void
+}
+
+/**
+ * Generic distributed upgrade Command
+ */
+message UpgradeCommandProto {
+  enum Action {
+    UNKNOWN = 0;          // Unknown action
+    REPORT_STATUS = 100;  // Report upgrade status
+    START_UPGRADE = 101;  // Start upgrade
+  }
+  required Action action = 1;  // Upgrade action
+  required uint32 version = 2; // Version of the upgrade
+  required uint32 upgradeStatus = 3; // % completed in range 0 & 100
+}
+
+/**
+ * registration - Information of the datanode registering with the namenode
+ */
+message RegisterDatanodeRequestProto {
+  required DatanodeRegistrationProto registration = 1; // Datanode info
+  repeated DatanodeStorageProto storages = 2; // Storages on the datanode
+}
+
+/**
+ * registration - Update registration of the datanode that successfully 
+ *                registered. StorageInfo will be updated to include new 
+ *                storage ID if the datanode did not have one in the request.
+ */
+message RegisterDatanodeResponseProto {
+  required DatanodeRegistrationProto registration = 1; // Datanode info
+}
+
+/**
+ * registration - datanode registration information
+ * capacity - total storage capacity available at the datanode
+ * dfsUsed - storage used by HDFS
+ * remaining - remaining storage available for HDFS
+ * blockPoolUsed - storage used by the block pool
+ * xmitsInProgress - number of transfers from this datanode to others
+ * xceiverCount - number of active transceiver threads
+ * failedVolumes - number of failed volumes
+ */
+message HeartbeatRequestProto {
+  required DatanodeRegistrationProto registration = 1; // Datanode info
+  repeated StorageReportProto reports = 2;
+  optional uint32 xmitsInProgress = 3 [ default = 0 ];
+  optional uint32 xceiverCount = 4 [ default = 0 ];
+  optional uint32 failedVolumes = 5 [ default = 0 ];
+}
+
+message StorageReportProto {
+  required string storageID = 1;
+  optional bool failed = 2 [ default = false ];
+  optional uint64 capacity = 3 [ default = 0 ];
+  optional uint64 dfsUsed = 4 [ default = 0 ];
+  optional uint64 remaining = 5 [ default = 0 ];
+  optional uint64 blockPoolUsed = 6 [ default = 0 ];
+}
+
+/**
+ * cmds - Commands from namenode to datanode.
+ */
+message HeartbeatResponseProto {
+  repeated DatanodeCommandProto cmds = 1; // Returned commands can be null
+}
+
+/**
+ * registration - datanode registration information
+ * blockPoolID  - block pool ID of the reported blocks
+ * blocks       - each block is represented as two longs in the array.
+ *                first long represents block ID
+ *                second long represents length
+ */
+message BlockReportRequestProto {
+  required DatanodeRegistrationProto registration = 1;
+  required string blockPoolId = 2;
+  repeated StorageBlockReportProto reports = 3;
+}
+
+/**
+ * Report of blocks in a storage
+ */
+message StorageBlockReportProto {
+  required string storageID = 1;    // Storage ID
+  repeated uint64 blocks = 2 [packed=true];
+}
+
+/**
+ * cmd - Command from namenode to the datanode
+ */
+message BlockReportResponseProto {
+  optional DatanodeCommandProto cmd = 1;
+} 
+
+/**
+ * Data structure to send received or deleted block information
+ * from datanode to namenode.
+ *
+ * deleteHint set to "-" indicates block deletion.
+ * other deleteHint indicates block addition.
+ */
+message ReceivedDeletedBlockInfoProto {
+  required BlockProto block = 1;
+  optional string deleteHint = 2;
+}
+
+/**
+ * List of blocks received and deleted for a storage.
+ */
+message StorageReceivedDeletedBlocksProto {
+  required string storageID = 1;
+  repeated ReceivedDeletedBlockInfoProto blocks = 2;
+}
+
+/**
+ * registration - datanode registration information
+ * blockPoolID  - block pool ID of the reported blocks
+ * blocks       - Received/deleted block list
+ */
+message BlockReceivedAndDeletedRequestProto {
+  required DatanodeRegistrationProto registration = 1;
+  required string blockPoolId = 2;
+  repeated StorageReceivedDeletedBlocksProto blocks = 3;
+}
+
+/**
+ * void response
+ */
+message BlockReceivedAndDeletedResponseProto {
+}
+
+/**
+ * registartion - Datanode reporting the error
+ * errorCode - error code indicating the error
+ * msg - Free text description of the error
+ */
+message ErrorReportRequestProto {
+  enum ErrorCode {
+    NOTIFY = 0;           // Error report to be logged at the namenode
+    DISK_ERROR = 1;       // DN has disk errors but still has valid volumes
+    INVALID_BLOCK = 2;    // Command from namenode has invalid block ID
+    FATAL_DISK_ERROR = 3; // No valid volumes left on datanode
+  }
+  required DatanodeRegistrationProto registartion = 1; // Registartion info
+  required uint32 errorCode = 2;  // Error code
+  required string msg = 3;        // Error message
+}
+
+/**
+ * void response
+ */
+message ErrorReportResponseProto {
+}
+
+/**
+ * cmd - Upgrade command sent from datanode to namenode
+ */
+message ProcessUpgradeRequestProto {
+  optional UpgradeCommandProto cmd = 1;
+}
+
+/**
+ * cmd - Upgrade command sent from namenode to datanode
+ */
+message ProcessUpgradeResponseProto {
+  optional UpgradeCommandProto cmd = 1;
+}
+
+/**
+ * blocks - list of blocks that are reported as corrupt
+ */
+message ReportBadBlocksRequestProto {
+  repeated LocatedBlockProto blocks = 1;
+}
+
+/**
+ * void response
+ */
+message ReportBadBlocksResponseProto {
+}
+
+/**
+ * Commit block synchronization request during lease recovery
+ */
+message CommitBlockSynchronizationRequestProto {
+  required ExtendedBlockProto block = 1;
+  required uint64 newGenStamp = 2;
+  required uint64 newLength = 3;
+  required bool closeFile = 4;
+  required bool deleteBlock = 5;
+  repeated DatanodeIDProto newTaragets = 6;
+}
+
+/**
+ * void response
+ */
+message CommitBlockSynchronizationResponseProto {
+}
+
+/**
+ * Protocol used from datanode to the namenode
+ * See the request and response for details of rpc call.
+ */
+service DatanodeProtocolService {
+  /**
+   * Register a datanode at a namenode
+   */
+  rpc registerDatanode(RegisterDatanodeRequestProto)
+      returns(RegisterDatanodeResponseProto);
+
+  /**
+   * Send heartbeat from datanode to namenode
+   */
+  rpc sendHeartbeat(HeartbeatRequestProto) returns(HeartbeatResponseProto);
+
+  /**
+   * Report blocks at a given datanode to the namenode
+   */
+  rpc blockReport(BlockReportRequestProto) returns(BlockReportResponseProto);
+
+  /**
+   * Report from datanode about recently received or deleted block
+   */
+  rpc blockReceivedAndDeleted(BlockReceivedAndDeletedRequestProto) 
+      returns(BlockReceivedAndDeletedResponseProto);
+
+  /**
+   * Report from a datanode of an error to the active namenode.
+   * Used for debugging.
+   */
+  rpc errorReport(ErrorReportRequestProto) returns(ErrorReportResponseProto);
+  
+  /**
+   * Request the version
+   */
+  rpc versionRequest(VersionRequestProto) returns(VersionResponseProto);
+
+  /**
+   * Generic way to send commands from datanode to namenode during
+   * distributed upgrade process.
+   */
+  rpc processUpgrade(ProcessUpgradeRequestProto) returns(ProcessUpgradeResponseProto);
+
+  /**
+   * Report corrupt blocks at the specified location
+   */
+  rpc reportBadBlocks(ReportBadBlocksRequestProto) returns(ReportBadBlocksResponseProto);
+
+  /**
+   * Commit block synchronization during lease recovery.
+   */
+  rpc commitBlockSynchronization(CommitBlockSynchronizationRequestProto)
+      returns(CommitBlockSynchronizationResponseProto);
+}

Copied: hadoop/common/branches/branch-0.23-PB-merge/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/GetUserMappingsProtocol.proto (from r1293950, hadoop/common/branches/branch-0.23-PB-merge/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/security/token/block/BlockKey.java)
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23-PB-merge/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/GetUserMappingsProtocol.proto?p2=hadoop/common/branches/branch-0.23-PB-merge/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/GetUserMappingsProtocol.proto&p1=hadoop/common/branches/branch-0.23-PB-merge/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/security/token/block/BlockKey.java&r1=1293950&r2=1293964&rev=1293964&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.23-PB-merge/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/security/token/block/BlockKey.java (original)
+++ hadoop/common/branches/branch-0.23-PB-merge/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/GetUserMappingsProtocol.proto Sun Feb 26 23:32:06 2012
@@ -16,24 +16,34 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.hdfs.security.token.block;
+option java_package = "org.apache.hadoop.hdfs.protocol.proto";
+option java_outer_classname = "GetUserMappingsProtocolProtos";
+option java_generic_services = true;
+option java_generate_equals_and_hash = true;
 
-import javax.crypto.SecretKey;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.security.token.delegation.DelegationKey;
+/**
+ *  Get groups for user request.
+ */
+message GetGroupsForUserRequestProto {
+  required string user = 1;
+}
 
 /**
- * Key used for generating and verifying block tokens
+ * Response for get groups.
  */
-@InterfaceAudience.Private
-public class BlockKey extends DelegationKey {
+message GetGroupsForUserResponseProto {
+  repeated string groups = 1;
+}
 
-  public BlockKey() {
-    super();
-  }
 
-  public BlockKey(int keyId, long expiryDate, SecretKey key) {
-    super(keyId, expiryDate, key);
-  }
+/**
+ * Protocol implemented by the Name Node and Job Tracker which maps users to
+ * groups.
+ */
+service GetUserMappingsProtocolService {
+  /**
+   * Get the groups which are mapped to the given user.
+   */
+  rpc getGroupsForUser(GetGroupsForUserRequestProto)
+      returns(GetGroupsForUserResponseProto);
 }

Added: hadoop/common/branches/branch-0.23-PB-merge/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/InterDatanodeProtocol.proto
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23-PB-merge/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/InterDatanodeProtocol.proto?rev=1293964&view=auto
==============================================================================
--- hadoop/common/branches/branch-0.23-PB-merge/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/InterDatanodeProtocol.proto (added)
+++ hadoop/common/branches/branch-0.23-PB-merge/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/InterDatanodeProtocol.proto Sun Feb 26 23:32:06 2012
@@ -0,0 +1,78 @@
+/**
+ * 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.
+ */
+
+// 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 = "InterDatanodeProtocolProtos";
+option java_generic_services = true;
+option java_generate_equals_and_hash = true;
+
+import "hdfs.proto";
+
+/**
+ * Block with location information and new generation stamp
+ * to be used for recovery.
+ */
+message InitReplicaRecoveryRequestProto {
+  required RecoveringBlockProto block = 1;
+}
+
+/**
+ * Repica recovery information
+ */
+message InitReplicaRecoveryResponseProto {
+  required ReplicaStateProto state = 1; // State of the replica
+  required BlockProto block = 2;   // block information
+}
+
+/**
+ * Update replica with new generation stamp and length
+ */
+message UpdateReplicaUnderRecoveryRequestProto {
+  required ExtendedBlockProto block = 1; // Block identifier
+  required uint64 recoveryId = 2;        // New genstamp of the replica
+  required uint64 newLength = 3;         // New length of the replica
+}
+
+/**
+ * Response returns updated block information
+ */
+message UpdateReplicaUnderRecoveryResponseProto {
+  required ExtendedBlockProto block = 1; // Updated block information
+}
+
+/**
+ * Protocol used between datanodes for block recovery.
+ *
+ * See the request and response for details of rpc call.
+ */
+service InterDatanodeProtocolService {
+  /**
+   * Initialize recovery of a replica
+   */
+  rpc initReplicaRecovery(InitReplicaRecoveryRequestProto)
+      returns(InitReplicaRecoveryResponseProto);
+
+  /**
+   * Update a replica with new generation stamp and length
+   */
+  rpc updateReplicaUnderRecovery(UpdateReplicaUnderRecoveryRequestProto)
+      returns(UpdateReplicaUnderRecoveryResponseProto);
+}

Added: hadoop/common/branches/branch-0.23-PB-merge/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/JournalProtocol.proto
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23-PB-merge/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/JournalProtocol.proto?rev=1293964&view=auto
==============================================================================
--- hadoop/common/branches/branch-0.23-PB-merge/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/JournalProtocol.proto (added)
+++ hadoop/common/branches/branch-0.23-PB-merge/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/JournalProtocol.proto Sun Feb 26 23:32:06 2012
@@ -0,0 +1,83 @@
+/**
+ * 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.
+ */
+
+// 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 = "JournalProtocolProtos";
+option java_generic_services = true;
+option java_generate_equals_and_hash = true;
+
+import "hdfs.proto";
+
+/**
+ * registration - the registration info of the active NameNode
+ * firstTxnId - the first txid in the rolled edit log
+ * numTxns - Number of transactions in editlog
+ * records - bytes containing serialized journal records
+ */
+message JournalRequestProto {
+  required NamenodeRegistrationProto registration = 1; // Registration info
+  required uint64 firstTxnId = 2; // Transaction ID
+  required uint32 numTxns = 3;    // Transaction ID
+  required bytes records = 4;     // Journal record
+}
+
+/**
+ * void response
+ */
+message JournalResponseProto { 
+}
+
+/**
+ * registration - the registration info of the active NameNode
+ * txid - first txid in the new log
+ */
+message StartLogSegmentRequestProto {
+  required NamenodeRegistrationProto registration = 1; // Registration info
+  required uint64 txid = 2; // Transaction ID
+}
+
+/**
+ * void response
+ */
+message StartLogSegmentResponseProto { 
+}
+
+/**
+ * Protocol used to journal edits to a remote node. Currently,
+ * this is used to publish edits from the NameNode to a BackupNode.
+ *
+ * See the request and response for details of rpc call.
+ */
+service JournalProtocolService {
+  /**
+   * Request sent by active namenode to backup node via 
+   * EditLogBackupOutputStream to stream editlog records.
+   */
+  rpc journal(JournalRequestProto) returns (JournalResponseProto);
+
+  /**
+   * Request sent by active namenode to backup node to notify 
+   * that the NameNode has rolled its edit logs and is now writing a 
+   * new log segment.
+   */
+  rpc startLogSegment(StartLogSegmentRequestProto) 
+      returns (StartLogSegmentResponseProto);
+}

Added: hadoop/common/branches/branch-0.23-PB-merge/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/NamenodeProtocol.proto
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23-PB-merge/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/NamenodeProtocol.proto?rev=1293964&view=auto
==============================================================================
--- hadoop/common/branches/branch-0.23-PB-merge/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/NamenodeProtocol.proto (added)
+++ hadoop/common/branches/branch-0.23-PB-merge/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/NamenodeProtocol.proto Sun Feb 26 23:32:06 2012
@@ -0,0 +1,230 @@
+/**
+ * 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.
+ */
+
+// 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 = "NamenodeProtocolProtos";
+option java_generic_services = true;
+option java_generate_equals_and_hash = true;
+
+import "hdfs.proto";
+
+/**
+ * Get list of blocks for a given datanode with the total length 
+ * of adding up to given size
+ * datanode - Datanode ID to get list of block from
+ * size - size to which the block lengths must add up to
+ */
+message GetBlocksRequestProto {
+  required DatanodeIDProto datanode = 1; // Datanode ID
+  required uint64 size = 2;              // Size in bytes
+}
+
+ 
+/**
+ * blocks - List of returned blocks
+ */
+message GetBlocksResponseProto {
+  required BlocksWithLocationsProto blocks = 1; // List of blocks
+}
+
+/**
+ * void request
+ */
+message GetBlockKeysRequestProto {
+}
+
+/**
+ * keys - Information about block keys at the active namenode
+ */
+message GetBlockKeysResponseProto {
+  required ExportedBlockKeysProto keys = 1;
+}
+
+/**
+ * void request
+ */
+message GetTransactionIdRequestProto {
+}
+
+/**
+ * txId - Transaction ID of the most recently persisted edit log record
+ */
+message GetTransactionIdResponseProto {
+  required uint64 txId = 1;   // Transaction ID
+}
+
+/**
+ * void request
+ */
+message RollEditLogRequestProto {
+}
+
+/**
+ * signature - A unique token to identify checkpoint transaction
+ */
+message RollEditLogResponseProto {
+  required CheckpointSignatureProto signature = 1;
+}
+
+/**
+ * registration - Namenode reporting the error
+ * errorCode - error code indicating the error
+ * msg - Free text description of the error
+ */
+message ErrorReportRequestProto {
+  required NamenodeRegistrationProto registration = 1; // Registration info
+  required uint32 errorCode = 2;  // Error code
+  required string msg = 3;        // Error message
+}
+
+/**
+ * void response
+ */
+message ErrorReportResponseProto {
+}
+
+/**
+ * registration - Information of the namenode registering with primary namenode
+ */
+message RegisterRequestProto {
+  required NamenodeRegistrationProto registration = 1; // Registration info
+}
+
+/**
+ * registration - Updated registration information of the newly registered
+ *                datanode.
+ */
+message RegisterResponseProto {
+  required NamenodeRegistrationProto registration = 1; // Registration info
+}
+
+/**
+ * Start checkpoint request
+ * registration - Namenode that is starting the checkpoint
+ */
+message StartCheckpointRequestProto {
+  required NamenodeRegistrationProto registration = 1; // Registration info
+}
+
+/**
+ * command - Command returned by the active namenode to be
+ *           be handled by the caller.
+ */
+message StartCheckpointResponseProto {
+  required NamenodeCommandProto command = 1;
+}
+
+/**
+ * End or finalize the previously started checkpoint
+ * registration - Namenode that is ending the checkpoint
+ * signature - unique token to identify checkpoint transaction,
+ *             that was received when checkpoint was started.
+ */
+message EndCheckpointRequestProto {
+  required NamenodeRegistrationProto registration = 1; // Registration info
+  required CheckpointSignatureProto signature = 2;
+}
+
+/**
+ * void response
+ */
+message EndCheckpointResponseProto {
+}
+
+/**
+ * sinceTxId - return the editlog information for transactions >= sinceTxId
+ */
+message GetEditLogManifestRequestProto {
+  required uint64 sinceTxId = 1;  // Transaction ID
+}
+
+/**
+ * manifest - Enumeration of editlogs from namenode for 
+ *            logs >= sinceTxId in the request
+ */
+message GetEditLogManifestResponseProto {
+  required RemoteEditLogManifestProto manifest = 1; 
+}
+
+/**
+ * Protocol used by the sub-ordinate namenode to send requests
+ * the active/primary namenode.
+ *
+ * See the request and response for details of rpc call.
+ */
+service NamenodeProtocolService {
+  /**
+   * Get list of blocks for a given datanode with length
+   * of blocks adding up to given size.
+   */
+  rpc getBlocks(GetBlocksRequestProto) returns(GetBlocksResponseProto);
+
+  /**
+   * Get the current block keys
+   */
+  rpc getBlockKeys(GetBlockKeysRequestProto) returns(GetBlockKeysResponseProto);
+
+  /**
+   * Get the transaction ID of the most recently persisted editlog record
+   */
+  rpc getTransactionId(GetTransactionIdRequestProto) 
+      returns(GetTransactionIdResponseProto);
+
+  /**
+   * Close the current editlog and open a new one for checkpointing purposes
+   */
+  rpc rollEditLog(RollEditLogRequestProto) returns(RollEditLogResponseProto);
+
+  /**
+   * Close the current editlog and open a new one for checkpointing purposes
+   */
+  rpc versionRequest(VersionRequestProto) returns(VersionResponseProto);
+
+  /**
+   * Report from a sub-ordinate namenode of an error to the active namenode.
+   * Active namenode may decide to unregister the reporting namenode 
+   * depending on the error.
+   */
+  rpc errorReport(ErrorReportRequestProto) returns(ErrorReportResponseProto);
+
+  /**
+   * Request to register a sub-ordinate namenode
+   */
+  rpc register(RegisterRequestProto) returns(RegisterResponseProto);
+
+  /**
+   * Request to start a checkpoint. 
+   */
+  rpc startCheckpoint(StartCheckpointRequestProto) 
+      returns(StartCheckpointResponseProto);
+
+  /**
+   * End of finalize the previously started checkpoint
+   */
+  rpc endCheckpoint(EndCheckpointRequestProto) 
+      returns(EndCheckpointResponseProto);
+
+  /**
+   * Get editlog manifests from the active namenode for all the editlogs
+   */
+  rpc getEditLogManifest(GetEditLogManifestRequestProto) 
+      returns(GetEditLogManifestResponseProto);
+}

Copied: hadoop/common/branches/branch-0.23-PB-merge/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/RefreshAuthorizationPolicyProtocol.proto (from r1293950, hadoop/common/branches/branch-0.23-PB-merge/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/security/token/block/BlockKey.java)
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23-PB-merge/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/RefreshAuthorizationPolicyProtocol.proto?p2=hadoop/common/branches/branch-0.23-PB-merge/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/RefreshAuthorizationPolicyProtocol.proto&p1=hadoop/common/branches/branch-0.23-PB-merge/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/security/token/block/BlockKey.java&r1=1293950&r2=1293964&rev=1293964&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.23-PB-merge/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/security/token/block/BlockKey.java (original)
+++ hadoop/common/branches/branch-0.23-PB-merge/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/RefreshAuthorizationPolicyProtocol.proto Sun Feb 26 23:32:06 2012
@@ -16,24 +16,30 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.hdfs.security.token.block;
-
-import javax.crypto.SecretKey;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.security.token.delegation.DelegationKey;
+option java_package = "org.apache.hadoop.hdfs.protocol.proto";
+option java_outer_classname = "RefreshAuthorizationPolicyProtocolProtos";
+option java_generic_services = true;
+option java_generate_equals_and_hash = true;
 
 /**
- * Key used for generating and verifying block tokens
+ *  Refresh service acl request.
  */
-@InterfaceAudience.Private
-public class BlockKey extends DelegationKey {
+message RefreshServiceAclRequestProto {
+}
 
-  public BlockKey() {
-    super();
-  }
+/**
+ * void response
+ */
+message RefreshServiceAclResponseProto {
+}
 
-  public BlockKey(int keyId, long expiryDate, SecretKey key) {
-    super(keyId, expiryDate, key);
-  }
+/**
+ * Protocol which is used to refresh the authorization policy in use currently.
+ */
+service RefreshAuthorizationPolicyProtocolService {
+  /**
+   * Refresh the service-level authorization policy in-effect.
+   */
+  rpc refreshServiceAcl(RefreshServiceAclRequestProto)
+      returns(RefreshServiceAclResponseProto);
 }

Added: hadoop/common/branches/branch-0.23-PB-merge/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/RefreshUserMappingsProtocol.proto
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23-PB-merge/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/RefreshUserMappingsProtocol.proto?rev=1293964&view=auto
==============================================================================
--- hadoop/common/branches/branch-0.23-PB-merge/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/RefreshUserMappingsProtocol.proto (added)
+++ hadoop/common/branches/branch-0.23-PB-merge/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/RefreshUserMappingsProtocol.proto Sun Feb 26 23:32:06 2012
@@ -0,0 +1,63 @@
+/**
+ * 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.hdfs.protocol.proto";
+option java_outer_classname = "RefreshUserMappingsProtocolProtos";
+option java_generic_services = true;
+option java_generate_equals_and_hash = true;
+
+/**
+ *  Refresh user to group mappings request.
+ */
+message RefreshUserToGroupsMappingsRequestProto {
+}
+
+/**
+ * void response
+ */
+message RefreshUserToGroupsMappingsResponseProto {
+}
+
+/**
+ * Refresh superuser configuration request.
+ */
+message RefreshSuperUserGroupsConfigurationRequestProto {
+}
+
+/**
+ * void response
+ */
+message RefreshSuperUserGroupsConfigurationResponseProto {
+}
+
+/**
+ * Protocol to refresh the user mappings.
+ */
+service RefreshUserMappingsProtocolService {
+  /**
+   * Refresh user to group mappings.
+   */
+  rpc refreshUserToGroupsMappings(RefreshUserToGroupsMappingsRequestProto)
+      returns(RefreshUserToGroupsMappingsResponseProto);
+
+  /**
+   * Refresh superuser proxy group list.
+   */
+  rpc refreshSuperUserGroupsConfiguration(RefreshSuperUserGroupsConfigurationRequestProto)
+      returns(RefreshSuperUserGroupsConfigurationResponseProto);
+}

Modified: hadoop/common/branches/branch-0.23-PB-merge/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23-PB-merge/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto?rev=1293964&r1=1293963&r2=1293964&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.23-PB-merge/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto (original)
+++ hadoop/common/branches/branch-0.23-PB-merge/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto Sun Feb 26 23:32:06 2012
@@ -23,13 +23,20 @@ option java_package = "org.apache.hadoop
 option java_outer_classname = "HdfsProtos";
 option java_generate_equals_and_hash = true;
 
+/**
+ * Extended block idenfies a block
+ */
 message ExtendedBlockProto {
-  required string poolId = 1;
-  required uint64 blockId = 2;
-  required uint64 numBytes = 3;
-  required uint64 generationStamp = 4;
+  required string poolId = 1;   // Block pool id - gloablly unique across clusters
+  required uint64 blockId = 2;  // the local id within a pool
+  required uint64 generationStamp = 3;
+  optional uint64 numBytes = 4 [default = 0];  // len does not belong in ebid 
+                                               // here for historical reasons
 }
 
+/**
+ * Block Token
+ */
 message BlockTokenIdentifierProto {
   required bytes identifier = 1;
   required bytes password = 2;
@@ -37,20 +44,34 @@ message BlockTokenIdentifierProto {
   required string service = 4;
 }
 
+/**
+ * Identifies a Datanode
+ */
 message DatanodeIDProto {
-  required string name = 1;
-  required string storageID = 2;
-  required uint32 infoPort = 3;
+  required string name = 1;      // hostname:portNumber
+  required string storageID = 2; // Unique storage id
+  required uint32 infoPort = 3;  // the port where the infoserver is running
+  required uint32 ipcPort = 4;   // the port where the ipc Server is running
 }
 
+/**
+ * DatanodeInfo array
+ */
+message DatanodeInfosProto {
+  repeated DatanodeInfoProto datanodes = 1;
+}
+
+/**
+ * The status of a Datanode
+ */
 message DatanodeInfoProto {
   required DatanodeIDProto id = 1;
-  optional uint64 capacity = 2;
-  optional uint64 dfsUsed = 3;
-  optional uint64 remaining = 4;
-  optional uint64 blockPoolUsed = 5;
-  optional uint64 lastUpdate = 6;
-  optional uint32 xceiverCount = 7;
+  optional uint64 capacity = 2 [default = 0];
+  optional uint64 dfsUsed = 3 [default = 0];
+  optional uint64 remaining = 4 [default = 0];
+  optional uint64 blockPoolUsed = 5 [default = 0];
+  optional uint64 lastUpdate = 6 [default = 0];
+  optional uint32 xceiverCount = 7 [default = 0];
   optional string location = 8;
   optional string hostName = 9;
   enum AdminState {
@@ -59,6 +80,283 @@ message DatanodeInfoProto {
     DECOMMISSIONED = 2;
   }
 
-  optional AdminState adminState = 10;
+  optional AdminState adminState = 10 [default = NORMAL];
+}
+
+/**
+ * Summary of a file or directory
+ */
+message ContentSummaryProto {
+  required uint64 length = 1;
+  required uint64 fileCount = 2;
+  required uint64 directoryCount = 3;
+  required uint64 quota = 4;
+  required uint64 spaceConsumed = 5;
+  required uint64 spaceQuota = 6;
+}
+
+/**
+ * Contains a list of paths corresponding to corrupt files and a cookie
+ * used for iterative calls to NameNode.listCorruptFileBlocks.
+ *
+ */
+message CorruptFileBlocksProto {
+ repeated string files = 1;
+ required string   cookie = 2;
+}
+
+/**
+ * File or Directory permision - same spec as posix
+ */
+message FsPermissionProto {
+  required uint32 perm = 1;       // Actually a short - only 16bits used
+}
+
+
+/**
+ * A LocatedBlock gives information about a block and its location.
+ */ 
+message LocatedBlockProto {
+  required ExtendedBlockProto b  = 1;
+  required uint64 offset = 2;           // offset of first byte of block in the file
+  repeated DatanodeInfoProto locs = 3;  // Locations ordered by proximity to client ip
+  required bool corrupt = 4;            // true if all replicas of a block are corrupt, else false
+                                        // If block has few corrupt replicas, they are filtered and 
+                                        // their locations are not part of this object
+
+  required BlockTokenIdentifierProto blockToken = 5;
+ }
+
+
+/**
+ * A set of file blocks and their locations.
+ */
+message LocatedBlocksProto {
+  required uint64 fileLength = 1;
+  repeated LocatedBlockProto blocks = 2;
+  required bool underConstruction = 3;
+  optional LocatedBlockProto lastBlock = 4;
+  required bool isLastBlockComplete = 5;
+}
+
+
+/**
+ * Status of a file, directory  or symlink
+ * Optionally includes a file's block locations if requested by client on the rpc call.
+ */
+message HdfsFileStatusProto {
+  enum FileType {
+    IS_DIR = 1;
+    IS_FILE = 2;
+    IS_SYMLINK = 3;
+  }
+  required FileType fileType = 1;
+  required bytes path = 2;          // local name of inode encoded java UTF8
+  required uint64 length = 3;
+  required FsPermissionProto permission = 4;
+  required string owner = 5;
+  required string group = 6;
+  required uint64 modification_time = 7;
+  required uint64 access_time = 8;
+
+  // Optional fields for symlink
+  optional bytes symlink = 9;             // if symlink, target encoded java UTF8 
+
+  // Optional fields for file
+  optional uint32 block_replication = 10 [default = 0]; // only 16bits used
+  optional uint64 blocksize = 11 [default = 0];
+  optional LocatedBlocksProto locations = 12;  // suppled only if asked by client
+} 
+
+/**
+ * HDFS Server Defaults
+ */
+message FsServerDefaultsProto {
+  required uint64 blockSize = 1;
+  required uint32 bytesPerChecksum = 2;
+  required uint32 writePacketSize = 3;
+  required uint32 replication = 4; // Actually a short - only 16 bits used
+  required uint32 fileBufferSize = 5;
+}
+
+
+/**
+ * Directory listing
+ */
+message DirectoryListingProto {
+  repeated HdfsFileStatusProto partialListing = 1;
+  required uint32 remainingEntries  = 2;
+}
+
+/**
+ * Status of current cluster upgrade from one version to another
+ */
+message UpgradeStatusReportProto {
+  required uint32 version = 1;;
+  required uint32 upgradeStatus = 2; // % completed in range 0 & 100
+	required bool finalized = 3;
+}
+
+/**
+ * Common node information shared by all the nodes in the cluster
+ */
+message StorageInfoProto {
+  required uint32 layoutVersion = 1; // Layout version of the file system
+  required uint32 namespceID = 2;    // File system namespace ID
+  required string clusterID = 3;     // ID of the cluster
+  required uint64 cTime = 4;         // File system creation time
+}
+
+/**
+ * Information sent by a namenode to identify itself to the primary namenode.
+ */
+message NamenodeRegistrationProto {
+  required string rpcAddress = 1;    // host:port of the namenode RPC address
+  required string httpAddress = 2;   // host:port of the namenode http server
+  enum NamenodeRoleProto {
+    NAMENODE = 1;
+    BACKUP = 2;
+    CHECKPOINT = 3;
+  }
+  required StorageInfoProto storageInfo = 3;  // Node information
+  optional NamenodeRoleProto role = 4 [default = NAMENODE];        // Namenode role
+}
+
+/**
+ * Unique signature to identify checkpoint transactions.
+ */
+message CheckpointSignatureProto {
+  required string blockPoolId = 1;
+  required uint64 mostRecentCheckpointTxId = 2;
+  required uint64 curSegmentTxId = 3;
+  required StorageInfoProto storageInfo = 4;
+}
+
+/**
+ * Command sent from one namenode to another namenode.
+ */
+message NamenodeCommandProto {
+  enum Type {
+    NamenodeCommand = 0;      // Base command
+    CheckPointCommand = 1;    // Check point command
+  }
+  required uint32 action = 1;
+  required Type type = 2;
+  optional CheckpointCommandProto checkpointCmd = 3;
+}
+
+/**
+ * Command returned from primary to checkpointing namenode.
+ * This command has checkpoint signature that identifies
+ * checkpoint transaction and is needed for further
+ * communication related to checkpointing.
+ */
+message CheckpointCommandProto {
+  // Unique signature to identify checkpoint transation
+  required CheckpointSignatureProto signature = 1; 
+
+  // If true, return transfer image to primary upon the completion of checkpoint
+  required bool needToReturnImage = 2;
+}
+
+/**
+ * Block information
+ */
+message BlockProto {
+  required uint64 blockId = 1;
+  required uint64 genStamp = 2;
+  optional uint64 numBytes = 3 [default = 0];
+}
+
+/**
+ * Block and datanodes where is it located
+ */
+message BlockWithLocationsProto {
+  required BlockProto block = 1;   // Block
+  repeated string datanodeIDs = 2; // Datanodes with replicas of the block
+}
+
+/**
+ * List of block with locations
+ */
+message BlocksWithLocationsProto {
+  repeated BlockWithLocationsProto blocks = 1;
+}
+
+/**
+ * Editlog information with available transactions
+ */
+message RemoteEditLogProto {
+  required uint64 startTxId = 1;  // Starting available edit log transaction
+  required uint64 endTxId = 2;    // Ending available edit log transaction
+}
+
+/**
+ * Enumeration of editlogs available on a remote namenode
+ */
+message RemoteEditLogManifestProto {
+  repeated RemoteEditLogProto logs = 1;
+}
+
+/**
+ * Namespace information that describes namespace on a namenode
+ */
+message NamespaceInfoProto {
+  required string buildVersion = 1;         // Software build version
+  required uint32 distUpgradeVersion = 2;   // Distributed upgrade version
+  required string blockPoolID = 3;          // block pool used by the namespace
+  required StorageInfoProto storageInfo = 4;// Noe information
+}
+
+/**
+ * Block access token information
+ */
+message BlockKeyProto {
+  required uint32 keyId = 1;      // Key identifier
+  required uint64 expiryDate = 2; // Expiry time in milliseconds
+  optional bytes keyBytes = 3;    // Key secret
+}
+
+/**
+ * Current key and set of block keys at the namenode.
+ */
+message ExportedBlockKeysProto {
+  required bool isBlockTokenEnabled = 1;
+  required uint64 keyUpdateInterval = 2;
+  required uint64 tokenLifeTime = 3;
+  required BlockKeyProto currentKey = 4;
+  repeated BlockKeyProto allKeys = 5;
+}
+
+/**
+ * State of a block replica at a datanode
+ */
+enum ReplicaStateProto {
+  FINALIZED = 0;  // State of a replica when it is not modified
+  RBW = 1;        // State of replica that is being written to
+  RWR = 2;        // State of replica that is waiting to be recovered
+  RUR = 3;        // State of replica that is under recovery
+  TEMPORARY = 4;  // State of replica that is created for replication
+}
+
+/**
+ * Block that needs to be recovered with at a given location
+ */
+message RecoveringBlockProto {
+  required uint64 newGenStamp = 1;      // New genstamp post recovery
+  required LocatedBlockProto block = 2; // Block to be recovered
+}
+
+/**
+ * void request
+ */
+message VersionRequestProto {
+}
+
+/**
+ * Version response from namenode.
+ */
+message VersionResponseProto {
+  required NamespaceInfoProto info = 1;
 }
 

Modified: hadoop/common/branches/branch-0.23-PB-merge/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23-PB-merge/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java?rev=1293964&r1=1293963&r2=1293964&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.23-PB-merge/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java (original)
+++ hadoop/common/branches/branch-0.23-PB-merge/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java Sun Feb 26 23:32:06 2012
@@ -44,12 +44,11 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
-import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
-import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.common.Storage;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
 import org.apache.hadoop.hdfs.server.datanode.DataStorage;
@@ -58,20 +57,15 @@ import org.apache.hadoop.hdfs.server.dat
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
-import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
-import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
 import org.apache.hadoop.hdfs.tools.DFSAdmin;
 import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
 import org.apache.hadoop.net.DNSToSwitchMapping;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.net.StaticMapping;
-import org.apache.hadoop.security.RefreshUserMappingsProtocol;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.authorize.ProxyUsers;
-import org.apache.hadoop.security.authorize.RefreshAuthorizationPolicyProtocol;
-import org.apache.hadoop.tools.GetUserMappingsProtocol;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.ToolRunner;
 
@@ -504,29 +498,6 @@ public class MiniDFSCluster {
     this.federation = federation;
     this.waitSafeMode = waitSafeMode;
     
-    // use alternate RPC engine if spec'd
-    String rpcEngineName = System.getProperty("hdfs.rpc.engine");
-    if (rpcEngineName != null && !"".equals(rpcEngineName)) {
-      
-      LOG.info("HDFS using RPCEngine: " + rpcEngineName);
-      try {
-        Class<?> rpcEngine = conf.getClassByName(rpcEngineName);
-        setRpcEngine(conf, NamenodeProtocols.class, rpcEngine);
-        setRpcEngine(conf, NamenodeProtocol.class, rpcEngine);
-        setRpcEngine(conf, ClientProtocol.class, rpcEngine);
-        setRpcEngine(conf, DatanodeProtocol.class, rpcEngine);
-        setRpcEngine(conf, RefreshAuthorizationPolicyProtocol.class, rpcEngine);
-        setRpcEngine(conf, RefreshUserMappingsProtocol.class, rpcEngine);
-        setRpcEngine(conf, GetUserMappingsProtocol.class, rpcEngine);
-      } catch (ClassNotFoundException e) {
-        throw new RuntimeException(e);
-      }
-
-      // disable service authorization, as it does not work with tunnelled RPC
-      conf.setBoolean(CommonConfigurationKeys.HADOOP_SECURITY_AUTHORIZATION,
-                      false);
-    }
-    
     int replication = conf.getInt(DFSConfigKeys.DFS_REPLICATION_KEY, 3);
     conf.setInt(DFSConfigKeys.DFS_REPLICATION_KEY, Math.min(replication, numDataNodes));
     conf.setInt(DFSConfigKeys.DFS_NAMENODE_SAFEMODE_EXTENSION_KEY, 0);
@@ -659,10 +630,6 @@ public class MiniDFSCluster {
     nameNodes[nnIndex] = new NameNodeInfo(nn, new Configuration(conf));
   }
 
-  private void setRpcEngine(Configuration conf, Class<?> protocol, Class<?> engine) {
-    conf.setClass("rpc.engine."+protocol.getName(), engine, Object.class);
-  }
-
   /**
    * @return URI of the namenode from a single namenode MiniDFSCluster
    */

Modified: hadoop/common/branches/branch-0.23-PB-merge/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientRetries.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23-PB-merge/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientRetries.java?rev=1293964&r1=1293963&r2=1293964&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.23-PB-merge/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientRetries.java (original)
+++ hadoop/common/branches/branch-0.23-PB-merge/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientRetries.java Sun Feb 26 23:32:06 2012
@@ -61,6 +61,7 @@ import org.apache.hadoop.hdfs.server.pro
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.ipc.Server;
+import org.apache.hadoop.ipc.RpcPayloadHeader.RpcKind;
 import org.apache.hadoop.net.NetUtils;
 import org.mockito.internal.stubbing.answers.ThrowsException;
 import org.mockito.invocation.InvocationOnMock;
@@ -96,7 +97,7 @@ public class TestDFSClientRetries extend
     }
 
     @Override
-    public Writable call(Class<?> protocol, Writable param, long receiveTime)
+    public Writable call(RpcKind rpcKind, String protocol, Writable param, long receiveTime)
         throws IOException {
       if (sleep) {
         // sleep a bit
@@ -636,7 +637,7 @@ public class TestDFSClientRetries extend
       proxy = DFSUtil.createClientDatanodeProtocolProxy(
           fakeDnId, conf, 500, fakeBlock);
 
-      proxy.getReplicaVisibleLength(null);
+      proxy.getReplicaVisibleLength(new ExtendedBlock("bpid", 1));
       fail ("Did not get expected exception: SocketTimeoutException");
     } catch (SocketTimeoutException e) {
       LOG.info("Got the expected Exception: SocketTimeoutException");

Modified: hadoop/common/branches/branch-0.23-PB-merge/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSRollback.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23-PB-merge/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSRollback.java?rev=1293964&r1=1293963&r2=1293964&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.23-PB-merge/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSRollback.java (original)
+++ hadoop/common/branches/branch-0.23-PB-merge/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSRollback.java Sun Feb 26 23:32:06 2012
@@ -248,7 +248,7 @@ public class TestDFSRollback extends Tes
       baseDirs = UpgradeUtilities.createNameNodeStorageDirs(nameNodeDirs, "previous");
       deleteMatchingFiles(baseDirs, "edits.*");
       startNameNodeShouldFail(StartupOption.ROLLBACK,
-          "but there are no logs to load");
+          "No non-corrupt logs for txid ");
       UpgradeUtilities.createEmptyDirs(nameNodeDirs);
       
       log("NameNode rollback with no image file", numDirs);

Modified: hadoop/common/branches/branch-0.23-PB-merge/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestGetBlocks.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23-PB-merge/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestGetBlocks.java?rev=1293964&r1=1293963&r2=1293964&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.23-PB-merge/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestGetBlocks.java (original)
+++ hadoop/common/branches/branch-0.23-PB-merge/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestGetBlocks.java Sun Feb 26 23:32:06 2012
@@ -25,6 +25,7 @@ import org.apache.hadoop.conf.Configurat
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.protocolPB.NamenodeProtocolTranslatorPB;
 import org.apache.hadoop.hdfs.server.common.GenerationStamp;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol;
@@ -32,9 +33,7 @@ import org.apache.hadoop.hdfs.server.pro
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.ipc.RemoteException;
-import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.UserGroupInformation;
 
 import junit.framework.TestCase;
@@ -98,10 +97,8 @@ public class TestGetBlocks extends TestC
       // get RPC client to namenode
       InetSocketAddress addr = new InetSocketAddress("localhost",
           cluster.getNameNodePort());
-      NamenodeProtocol namenode = (NamenodeProtocol) RPC.getProxy(
-          NamenodeProtocol.class, NamenodeProtocol.versionID, addr,
-          UserGroupInformation.getCurrentUser(), CONF,
-          NetUtils.getDefaultSocketFactory(CONF));
+      NamenodeProtocol namenode = new NamenodeProtocolTranslatorPB(addr, CONF,
+          UserGroupInformation.getCurrentUser());
 
       // get blocks of size fileLen from dataNodes[0]
       BlockWithLocations[] locs;

Added: hadoop/common/branches/branch-0.23-PB-merge/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestIsMethodSupported.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23-PB-merge/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestIsMethodSupported.java?rev=1293964&view=auto
==============================================================================
--- hadoop/common/branches/branch-0.23-PB-merge/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestIsMethodSupported.java (added)
+++ hadoop/common/branches/branch-0.23-PB-merge/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestIsMethodSupported.java Sun Feb 26 23:32:06 2012
@@ -0,0 +1,154 @@
+/**
+ * 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.
+ */
+package org.apache.hadoop.hdfs;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+
+import junit.framework.Assert;
+
+import org.apache.hadoop.hdfs.protocolPB.ClientDatanodeProtocolTranslatorPB;
+import org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolTranslatorPB;
+import org.apache.hadoop.hdfs.protocolPB.DatanodeProtocolClientSideTranslatorPB;
+import org.apache.hadoop.hdfs.protocolPB.GetUserMappingsProtocolClientSideTranslatorPB;
+import org.apache.hadoop.hdfs.protocolPB.InterDatanodeProtocolTranslatorPB;
+import org.apache.hadoop.hdfs.protocolPB.JournalProtocolTranslatorPB;
+import org.apache.hadoop.hdfs.protocolPB.NamenodeProtocolTranslatorPB;
+import org.apache.hadoop.hdfs.protocolPB.RefreshAuthorizationPolicyProtocolClientSideTranslatorPB;
+import org.apache.hadoop.hdfs.protocolPB.RefreshUserMappingsProtocolClientSideTranslatorPB;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+/**
+ * Test cases to verify that client side translators correctly implement the
+ * isMethodSupported method in ProtocolMetaInterface.
+ */
+public class TestIsMethodSupported {
+  private static MiniDFSCluster cluster = null;
+  private static HdfsConfiguration conf = new HdfsConfiguration();
+  private static InetSocketAddress nnAddress = null;
+  private static InetSocketAddress dnAddress = null;
+  
+  @BeforeClass
+  public static void setUp() throws Exception {
+    cluster = (new MiniDFSCluster.Builder(conf)).numNameNodes(1)
+        .numDataNodes(1).build();
+    nnAddress = cluster.getNameNode().getNameNodeAddress();
+    dnAddress = new InetSocketAddress(cluster.getDataNodes().get(0)
+        .getDatanodeId().getHost(), cluster.getDataNodes().get(0).getIpcPort());
+  }
+
+  @AfterClass
+  public static void tearDown() throws Exception {
+    if (cluster != null) {
+      cluster.shutdown();
+    }
+  }
+
+  @Test
+  public void testNamenodeProtocol() throws IOException {
+    NamenodeProtocolTranslatorPB translator =
+        new NamenodeProtocolTranslatorPB(nnAddress, conf,
+            UserGroupInformation.getCurrentUser());
+    boolean exists = translator.isMethodSupported("rollEditLog");
+    Assert.assertTrue(exists);
+    exists = translator.isMethodSupported("bogusMethod");
+    Assert.assertFalse(exists);
+  }
+  
+  @Test
+  public void testDatanodeProtocol() throws IOException {
+    DatanodeProtocolClientSideTranslatorPB translator = 
+        new DatanodeProtocolClientSideTranslatorPB(nnAddress, conf);
+    Assert.assertTrue(translator.isMethodSupported("sendHeartbeat"));
+  }
+  
+  @Test
+  public void testClientDatanodeProtocol() throws IOException {
+    ClientDatanodeProtocolTranslatorPB translator = 
+        new ClientDatanodeProtocolTranslatorPB(nnAddress, 
+            UserGroupInformation.getCurrentUser(), conf,
+        NetUtils.getDefaultSocketFactory(conf));
+    //Namenode doesn't implement ClientDatanodeProtocol
+    Assert.assertFalse(translator.isMethodSupported("refreshNamenodes"));
+    
+    translator = new ClientDatanodeProtocolTranslatorPB(
+        dnAddress, UserGroupInformation.getCurrentUser(), conf,
+        NetUtils.getDefaultSocketFactory(conf));
+    Assert.assertTrue(translator.isMethodSupported("refreshNamenodes"));
+  }
+  
+  @Test
+  public void testClientNamenodeProtocol() throws IOException {
+    ClientNamenodeProtocolTranslatorPB translator = 
+        new ClientNamenodeProtocolTranslatorPB(nnAddress, conf, 
+            UserGroupInformation.getCurrentUser()); 
+    Assert.assertTrue(translator.isMethodSupported("mkdirs"));
+  }
+  
+  @Test
+  public void tesJournalProtocol() throws IOException {
+    JournalProtocolTranslatorPB translator = 
+        new JournalProtocolTranslatorPB(nnAddress, conf);
+    //Nameode doesn't implement JournalProtocol
+    Assert.assertFalse(translator.isMethodSupported("startLogSegment"));
+  }
+  
+  @Test
+  public void testInterDatanodeProtocol() throws IOException {
+    InterDatanodeProtocolTranslatorPB translator = 
+        new InterDatanodeProtocolTranslatorPB(
+            nnAddress, UserGroupInformation.getCurrentUser(), conf,
+            NetUtils.getDefaultSocketFactory(conf), 0);
+    //Not supported at namenode
+    Assert.assertFalse(translator.isMethodSupported("initReplicaRecovery"));
+    
+    translator = new InterDatanodeProtocolTranslatorPB(
+        dnAddress, UserGroupInformation.getCurrentUser(), conf,
+        NetUtils.getDefaultSocketFactory(conf), 0);
+    Assert.assertTrue(translator.isMethodSupported("initReplicaRecovery"));
+  }
+  
+  @Test
+  public void testGetUserMappingsProtocol() throws IOException {
+    GetUserMappingsProtocolClientSideTranslatorPB translator = 
+        new GetUserMappingsProtocolClientSideTranslatorPB(
+            nnAddress, UserGroupInformation.getCurrentUser(), conf);
+    Assert.assertTrue(translator.isMethodSupported("getGroupsForUser"));
+  }
+  
+  @Test
+  public void testRefreshAuthorizationPolicyProtocol() throws IOException {
+    RefreshAuthorizationPolicyProtocolClientSideTranslatorPB translator =
+        new RefreshAuthorizationPolicyProtocolClientSideTranslatorPB(
+            nnAddress, UserGroupInformation.getCurrentUser(), conf);
+    Assert.assertTrue(translator.isMethodSupported("refreshServiceAcl"));
+  }
+  
+  @Test
+  public void testRefreshUserMappingsProtocol() throws IOException {
+    RefreshUserMappingsProtocolClientSideTranslatorPB translator =
+        new RefreshUserMappingsProtocolClientSideTranslatorPB(
+            nnAddress, UserGroupInformation.getCurrentUser(), conf);
+    Assert.assertTrue(
+        translator.isMethodSupported("refreshUserToGroupsMappings"));
+  }
+}