浏览代码

svn merge -c 1187505 from trunk for HDFS-2480.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-0.23-PB@1229487 13f79535-47bb-0310-9956-ffa450edef68
Tsz-wo Sze 13 年之前
父节点
当前提交
429be99307
共有 16 个文件被更改,包括 1309 次插入17 次删除
  1. 2 0
      hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
  2. 82 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/BlockWritable.java
  3. 129 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/BlocksWithLocationsWritable.java
  4. 92 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/CheckpointSignatureWritable.java
  5. 20 10
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/DatanodeInfoWritable.java
  6. 104 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/ExportedBlockKeysWritable.java
  7. 70 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/NamenodeCommandWritable.java
  8. 163 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/NamenodeProtocolServerSideTranslatorR23.java
  9. 186 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/NamenodeProtocolTranslatorR23.java
  10. 169 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/NamenodeWireProtocol.java
  11. 100 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/NamespaceInfoWritable.java
  12. 90 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/RemoteEditLogManifestWritable.java
  13. 69 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/RemoteEditLogWritable.java
  14. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/security/token/block/ExportedBlockKeys.java
  15. 16 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/CheckpointSignature.java
  16. 16 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/NamenodeProtocol.java

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

@@ -17,6 +17,8 @@ Release 0.23-PB - Unreleased
 
     HDFS-2459. Separate datatypes for Journal Protocol. (suresh)
 
+    HDFS-2480. Separate datatypes for NamenodeProtocol. (suresh)
+
 Release 0.23.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

+ 82 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/BlockWritable.java

@@ -0,0 +1,82 @@
+/**
+ * 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.protocolR23Compatible;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableFactories;
+import org.apache.hadoop.io.WritableFactory;
+
+/**************************************************
+ * A Block is a Hadoop FS primitive, identified by a long.
+ **************************************************/
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class BlockWritable implements Writable {
+  static {                                      // register a ctor
+    WritableFactories.setFactory
+      (BlockWritable.class,
+       new WritableFactory() {
+         public Writable newInstance() { return new BlockWritable(); }
+       });
+  }
+
+  private long blockId;
+  private long numBytes;
+  private long generationStamp;
+
+  public BlockWritable() {this(0, 0, 0);}
+
+  public BlockWritable(final long blkid, final long len, final long genStamp) {
+    this.blockId = blkid;
+    this.numBytes = len;
+    this.generationStamp = genStamp;
+  }
+
+  /////////////////////////////////////
+  // Writable
+  /////////////////////////////////////
+  @Override // Writable
+  public void write(DataOutput out) throws IOException {
+    out.writeLong(blockId);
+    out.writeLong(numBytes);
+    out.writeLong(generationStamp);
+  }
+
+  @Override // Writable
+  public void readFields(DataInput in) throws IOException {
+    this.blockId = in.readLong();
+    this.numBytes = in.readLong();
+    this.generationStamp = in.readLong();
+  }
+
+  public static BlockWritable convert(Block b) {
+    return new BlockWritable(b.getBlockId(), b.getNumBytes(),
+        b.getGenerationStamp());
+  }
+
+  public Block convert() {
+    return new Block(blockId, numBytes, generationStamp);
+  }
+}

+ 129 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/BlocksWithLocationsWritable.java

@@ -0,0 +1,129 @@
+/**
+ * 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.protocolR23Compatible;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations;
+import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations.BlockWithLocations;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableUtils;
+
+/** A class to implement an array of BlockLocations
+ *  It provide efficient customized serialization/deserialization methods
+ *  in stead of using the default array (de)serialization provided by RPC
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class BlocksWithLocationsWritable implements Writable {
+
+  /**
+   * A class to keep track of a block and its locations
+   */
+  @InterfaceAudience.Private
+  @InterfaceStability.Evolving
+  public static class BlockWithLocationsWritable  implements Writable {
+    private BlockWritable block;
+    private String datanodeIDs[];
+    
+    /** default constructor */
+    public BlockWithLocationsWritable() {
+      block = new BlockWritable();
+      datanodeIDs = null;
+    }
+    
+    /** constructor */
+    public BlockWithLocationsWritable(BlockWritable b, String[] datanodes) {
+      block = b;
+      datanodeIDs = datanodes;
+    }
+    
+    /** deserialization method */
+    public void readFields(DataInput in) throws IOException {
+      block.readFields(in);
+      int len = WritableUtils.readVInt(in); // variable length integer
+      datanodeIDs = new String[len];
+      for(int i=0; i<len; i++) {
+        datanodeIDs[i] = Text.readString(in);
+      }
+    }
+    
+    /** serialization method */
+    public void write(DataOutput out) throws IOException {
+      block.write(out);
+      WritableUtils.writeVInt(out, datanodeIDs.length); // variable length int
+      for(String id:datanodeIDs) {
+        Text.writeString(out, id);
+      }
+    }
+  }
+
+  private BlockWithLocationsWritable[] blocks;
+
+  /** default constructor */
+  BlocksWithLocationsWritable() {
+  }
+
+  /** Constructor with one parameter */
+  public BlocksWithLocationsWritable( BlockWithLocationsWritable[] blocks ) {
+    this.blocks = blocks;
+  }
+
+  /** serialization method */
+  public void write( DataOutput out ) throws IOException {
+    WritableUtils.writeVInt(out, blocks.length);
+    for(int i=0; i<blocks.length; i++) {
+      blocks[i].write(out);
+    }
+  }
+
+  /** deserialization method */
+  public void readFields(DataInput in) throws IOException {
+    int len = WritableUtils.readVInt(in);
+    blocks = new BlockWithLocationsWritable[len];
+    for(int i=0; i<len; i++) {
+      blocks[i] = new BlockWithLocationsWritable();
+      blocks[i].readFields(in);
+    }
+  }
+
+  public static BlocksWithLocationsWritable convert(BlocksWithLocations locs) {
+    BlockWithLocations[] blocks = locs.getBlocks();
+    BlockWithLocationsWritable[] blocksWritable = 
+        new BlockWithLocationsWritable[blocks.length];
+    for (int i = 0; i < blocks.length; i++) {
+      blocksWritable[i] = new BlockWithLocationsWritable(
+          BlockWritable.convert(blocks[i].getBlock()), blocks[i].getDatanodes());
+    }
+    return new BlocksWithLocationsWritable(blocksWritable);
+  }
+  
+  public BlocksWithLocations convert() {
+    BlockWithLocations[] locs = new BlockWithLocations[blocks.length];
+    for (int i = 0; i < blocks.length; i++) {
+      locs[i] = new BlockWithLocations(blocks[i].block.convert(),
+          blocks[i].datanodeIDs);
+    }
+    return new BlocksWithLocations(locs);
+  }
+}

+ 92 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/CheckpointSignatureWritable.java

@@ -0,0 +1,92 @@
+/**
+ * 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.protocolR23Compatible;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hdfs.server.namenode.CheckpointSignature;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableFactories;
+import org.apache.hadoop.io.WritableFactory;
+import org.apache.hadoop.io.WritableUtils;
+
+/**
+ * A unique signature intended to identify checkpoint transactions.
+ */
+@InterfaceAudience.Private
+public class CheckpointSignatureWritable implements Writable { 
+  private String blockpoolID = "";
+  private long mostRecentCheckpointTxId;
+  private long curSegmentTxId;
+  private StorageInfoWritable storageInfo;
+
+  public CheckpointSignatureWritable() {}
+
+  CheckpointSignatureWritable(long mostRecentCheckpointTxId,
+      long curSegmentTxId, int layoutVersion, int namespaceID, String bpid,
+      String clusterID, long cTime) {
+    this.blockpoolID = bpid;
+    this.mostRecentCheckpointTxId = mostRecentCheckpointTxId;
+    this.curSegmentTxId = curSegmentTxId;
+    this.storageInfo = new StorageInfoWritable(layoutVersion, namespaceID,
+        clusterID, cTime);
+  }
+
+  /////////////////////////////////////////////////
+  // Writable
+  /////////////////////////////////////////////////
+  static {
+    WritableFactories.setFactory(CheckpointSignatureWritable.class,
+        new WritableFactory() {
+          public Writable newInstance() {
+            return new CheckpointSignatureWritable();
+          }
+        });
+  }
+  
+  @Override
+  public void write(DataOutput out) throws IOException {
+    storageInfo.write(out);
+    WritableUtils.writeString(out, blockpoolID);
+    out.writeLong(mostRecentCheckpointTxId);
+    out.writeLong(curSegmentTxId);
+  }
+
+  @Override
+  public void readFields(DataInput in) throws IOException {
+    storageInfo.readFields(in);
+    blockpoolID = WritableUtils.readString(in);
+    mostRecentCheckpointTxId = in.readLong();
+    curSegmentTxId = in.readLong();
+  }
+
+  public static CheckpointSignatureWritable convert(
+      CheckpointSignature sig) {
+    return new CheckpointSignatureWritable(sig.getMostRecentCheckpointTxId(),
+        sig.getCurSegmentTxId(), sig.getLayoutVersion(), sig.getNamespaceID(),
+        sig.getBlockpoolID(), sig.getClusterID(), sig.getCTime());
+  }
+
+  public CheckpointSignature convert() {
+    return new CheckpointSignature(storageInfo.convert(), blockpoolID,
+        mostRecentCheckpointTxId, curSegmentTxId);
+  }
+}

+ 20 - 10
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/DatanodeInfoWritable.java

@@ -24,6 +24,7 @@ import java.io.IOException;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableFactories;
@@ -59,9 +60,9 @@ public class DatanodeInfoWritable extends DatanodeIDWritable  {
   
   // administrative states of a datanode
   public enum AdminStates {
-    NORMAL(org.apache.hadoop.hdfs.protocol.DatanodeInfo.AdminStates.NORMAL.toString()), 
-    DECOMMISSION_INPROGRESS(org.apache.hadoop.hdfs.protocol.DatanodeInfo.AdminStates.DECOMMISSION_INPROGRESS.toString()), 
-    DECOMMISSIONED(org.apache.hadoop.hdfs.protocol.DatanodeInfo.AdminStates.DECOMMISSIONED.toString());
+    NORMAL(DatanodeInfo.AdminStates.NORMAL.toString()), 
+    DECOMMISSION_INPROGRESS(DatanodeInfo.AdminStates.DECOMMISSION_INPROGRESS.toString()), 
+    DECOMMISSIONED(DatanodeInfo.AdminStates.DECOMMISSIONED.toString());
 
     final String value;
 
@@ -84,27 +85,27 @@ public class DatanodeInfoWritable extends DatanodeIDWritable  {
   @Nullable
   protected AdminStates adminState;
   
-  static public org.apache.hadoop.hdfs.protocol.DatanodeInfo convertDatanodeInfo(DatanodeInfoWritable di) {
+  static public DatanodeInfo convertDatanodeInfo(DatanodeInfoWritable di) {
     if (di == null) return null;
-    return new org.apache.hadoop.hdfs.protocol.DatanodeInfo(
+    return new DatanodeInfo(
         new org.apache.hadoop.hdfs.protocol.DatanodeID(di.getName(), di.getStorageID(), di.getInfoPort(), di.getIpcPort()),
         di.getNetworkLocation(), di.getHostName(),
          di.getCapacity(),  di.getDfsUsed(),  di.getRemaining(),
         di.getBlockPoolUsed()  ,  di.getLastUpdate() , di.getXceiverCount() ,
-        org.apache.hadoop.hdfs.protocol.DatanodeInfo.AdminStates.fromValue(di.getAdminState().value)); 
+        DatanodeInfo.AdminStates.fromValue(di.getAdminState().value)); 
   }
   
   
-  static public org.apache.hadoop.hdfs.protocol.DatanodeInfo[] convertDatanodeInfo(DatanodeInfoWritable di[]) {
+  static public DatanodeInfo[] convertDatanodeInfo(DatanodeInfoWritable di[]) {
     if (di == null) return null;
-    org.apache.hadoop.hdfs.protocol.DatanodeInfo[] result = new org.apache.hadoop.hdfs.protocol.DatanodeInfo[di.length];
+    DatanodeInfo[] result = new DatanodeInfo[di.length];
     for (int i = 0; i < di.length; i++) {
       result[i] = convertDatanodeInfo(di[i]);
     }    
     return result;
   }
   
-  static public DatanodeInfoWritable[] convertDatanodeInfo(org.apache.hadoop.hdfs.protocol.DatanodeInfo[] di) {
+  static public DatanodeInfoWritable[] convertDatanodeInfo(DatanodeInfo[] di) {
     if (di == null) return null;
     DatanodeInfoWritable[] result = new DatanodeInfoWritable[di.length];
     for (int i = 0; i < di.length; i++) {
@@ -115,7 +116,16 @@ public class DatanodeInfoWritable extends DatanodeIDWritable  {
           AdminStates.fromValue(di[i].getAdminState().toString()));
     }    
     return result;
-    
+  }
+  
+  static public DatanodeInfoWritable convertDatanodeInfo(DatanodeInfo di) {
+    if (di == null) return null;
+    return new DatanodeInfoWritable(new DatanodeIDWritable(di.getName(),
+        di.getStorageID(), di.getInfoPort(), di.getIpcPort()),
+        di.getNetworkLocation(), di.getHostName(), di.getCapacity(),
+        di.getDfsUsed(), di.getRemaining(), di.getBlockPoolUsed(),
+        di.getLastUpdate(), di.getXceiverCount(), 
+        AdminStates.fromValue(di.getAdminState().toString()));
   }
 
   public DatanodeInfoWritable() {

+ 104 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/ExportedBlockKeysWritable.java

@@ -0,0 +1,104 @@
+/**
+ * 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.protocolR23Compatible;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hdfs.security.token.block.BlockKey;
+import org.apache.hadoop.hdfs.security.token.block.ExportedBlockKeys;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableFactories;
+import org.apache.hadoop.io.WritableFactory;
+
+/**
+ * Object for passing block keys
+ */
+@InterfaceAudience.Private
+public class ExportedBlockKeysWritable implements Writable {
+  private boolean isBlockTokenEnabled;
+  private long keyUpdateInterval;
+  private long tokenLifetime;
+  private BlockKey currentKey;
+  private BlockKey[] allKeys;
+
+  public ExportedBlockKeysWritable() {
+    this(false, 0, 0, new BlockKey(), new BlockKey[0]);
+  }
+
+  ExportedBlockKeysWritable(boolean isBlockTokenEnabled, long keyUpdateInterval,
+      long tokenLifetime, BlockKey currentKey, BlockKey[] allKeys) {
+    this.isBlockTokenEnabled = isBlockTokenEnabled;
+    this.keyUpdateInterval = keyUpdateInterval;
+    this.tokenLifetime = tokenLifetime;
+    this.currentKey = currentKey == null ? new BlockKey() : currentKey;
+    this.allKeys = allKeys == null ? new BlockKey[0] : allKeys;
+  }
+
+  // ///////////////////////////////////////////////
+  // Writable
+  // ///////////////////////////////////////////////
+  static { // register a ctor
+    WritableFactories.setFactory(ExportedBlockKeysWritable.class,
+        new WritableFactory() {
+          public Writable newInstance() {
+            return new ExportedBlockKeysWritable();
+          }
+        });
+  }
+
+  @Override
+  public void write(DataOutput out) throws IOException {
+    out.writeBoolean(isBlockTokenEnabled);
+    out.writeLong(keyUpdateInterval);
+    out.writeLong(tokenLifetime);
+    currentKey.write(out);
+    out.writeInt(allKeys.length);
+    for (int i = 0; i < allKeys.length; i++) {
+      allKeys[i].write(out);
+    }
+  }
+
+  @Override
+  public void readFields(DataInput in) throws IOException {
+    isBlockTokenEnabled = in.readBoolean();
+    keyUpdateInterval = in.readLong();
+    tokenLifetime = in.readLong();
+    currentKey.readFields(in);
+    this.allKeys = new BlockKey[in.readInt()];
+    for (int i = 0; i < allKeys.length; i++) {
+      allKeys[i] = new BlockKey();
+      allKeys[i].readFields(in);
+    }
+  }
+
+  public static ExportedBlockKeysWritable convert(ExportedBlockKeys blockKeys) {
+    if (blockKeys == null) return null;
+    return new ExportedBlockKeysWritable(blockKeys.isBlockTokenEnabled(),
+        blockKeys.getKeyUpdateInterval(), blockKeys.getTokenLifetime(),
+        blockKeys.getCurrentKey(), blockKeys.getAllKeys());
+  }
+  
+  public ExportedBlockKeys convert() {
+    return new ExportedBlockKeys(isBlockTokenEnabled, keyUpdateInterval,
+        tokenLifetime, currentKey, allKeys);
+  }
+}

+ 70 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/NamenodeCommandWritable.java

@@ -0,0 +1,70 @@
+/**
+ * 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.protocolR23Compatible;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hdfs.server.protocol.NamenodeCommand;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableFactories;
+import org.apache.hadoop.io.WritableFactory;
+
+/**
+ * Base class for name-node command.
+ * Issued by the name-node to notify other name-nodes what should be done.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class NamenodeCommandWritable implements Writable {
+  private int action;
+  static {
+    WritableFactories.setFactory(NamenodeCommandWritable.class,
+        new WritableFactory() {
+          public Writable newInstance() {return new NamenodeCommandWritable();}
+        });
+  }
+
+  public NamenodeCommandWritable() {
+  }
+
+  public NamenodeCommandWritable(int action) {
+    this.action = action;
+  }
+
+  @Override
+  public void write(DataOutput out) throws IOException {
+    out.writeInt(this.action);
+  }
+
+  @Override
+  public void readFields(DataInput in) throws IOException {
+    this.action = in.readInt();
+  }
+
+  public static NamenodeCommandWritable convert(NamenodeCommand cmd) {
+    return new NamenodeCommandWritable(cmd.getAction());
+  }
+
+  public NamenodeCommand convert() {
+    return new NamenodeCommand(action);
+  }
+}

+ 163 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/NamenodeProtocolServerSideTranslatorR23.java

@@ -0,0 +1,163 @@
+/**
+ * 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.protocolR23Compatible;
+
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations;
+import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol;
+import org.apache.hadoop.ipc.ProtocolSignature;
+import org.apache.hadoop.ipc.RPC;
+
+/**
+ * This class is used on the server side.
+ * Calls come across the wire for the protocol family of Release 23 onwards.
+ * This class translates the R23 data types to the internal data types used
+ * inside the DN as specified in the generic NamenodeProtocol.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Stable
+public class NamenodeProtocolServerSideTranslatorR23 implements
+  NamenodeWireProtocol {
+  
+  final private NamenodeProtocol server;
+
+  /**
+   * @param server - the NN server
+   * @throws IOException
+   */
+  public NamenodeProtocolServerSideTranslatorR23(
+      NamenodeProtocol server) throws IOException {
+    this.server = server;
+  }
+  
+  /**
+   * the client side will redirect getProtocolSignature to 
+   * getProtocolSignature2.
+   * 
+   * However the RPC layer below on the Server side will call
+   * getProtocolVersion and possibly in the future getProtocolSignature.
+   * Hence we still implement it even though the end client's call will
+   * never reach here.
+   */
+  @Override
+  public ProtocolSignature getProtocolSignature(String protocol,
+      long clientVersion, int clientMethodsHash) throws IOException {
+    /**
+     * Don't forward this to the server. The protocol version and
+     * signature is that of  {@link NamenodeProtocol}
+     */
+    if (!protocol.equals(RPC.getProtocolName(
+        NamenodeWireProtocol.class))) {
+      throw new IOException("Namenode Serverside implements " + 
+          NamenodeWireProtocol.class + 
+          ". The following requested protocol is unknown: " + protocol);
+    }
+    
+    return ProtocolSignature.getProtocolSignature(clientMethodsHash, 
+        NamenodeWireProtocol.versionID, 
+        NamenodeWireProtocol.class);
+  }
+
+  @Override
+  public ProtocolSignatureWritable getProtocolSignature2(String protocol,
+      long clientVersion, int clientMethodsHash) throws IOException {
+    /**
+     * Don't forward this to the server. The protocol version and
+     * signature is that of  {@link ClientNamenodeProtocol}
+     */
+   return ProtocolSignatureWritable.convert(
+        this.getProtocolSignature(protocol, clientVersion, clientMethodsHash));
+  }
+
+  @Override
+  public long getProtocolVersion(String protocol, long clientVersion)
+      throws IOException {
+    if (protocol.equals(RPC.getProtocolName(
+        NamenodeWireProtocol.class))) {
+      return NamenodeWireProtocol.versionID; 
+    }
+    throw new IOException("Datanode Serverside implements " + 
+        NamenodeWireProtocol.class + 
+        ". The following requested protocol is unknown: " + protocol);
+  }
+
+  @Override
+  public BlocksWithLocationsWritable getBlocks(DatanodeInfoWritable datanode,
+      long size) throws IOException {
+    BlocksWithLocations locs = server.getBlocks(
+        DatanodeInfoWritable.convertDatanodeInfo(datanode), size);
+    return BlocksWithLocationsWritable.convert(locs);
+  }
+
+  @Override
+  public ExportedBlockKeysWritable getBlockKeys() throws IOException {
+    return ExportedBlockKeysWritable.convert(server.getBlockKeys());
+  }
+
+  @Override
+  public long getTransactionID() throws IOException {
+    return server.getTransactionID();
+  }
+
+  @Override
+  @SuppressWarnings("deprecation")
+  public CheckpointSignatureWritable rollEditLog() throws IOException {
+    return CheckpointSignatureWritable.convert(server.rollEditLog());
+  }
+
+  @Override
+  public NamespaceInfoWritable versionRequest() throws IOException {
+    return NamespaceInfoWritable.convert(server.versionRequest());
+  }
+
+  @Override
+  public void errorReport(NamenodeRegistrationWritable registration,
+      int errorCode, String msg) throws IOException {
+    server.errorReport(registration.convert(), errorCode, msg);
+  }
+
+  @Override
+  public NamenodeRegistrationWritable register(
+      NamenodeRegistrationWritable registration) throws IOException {
+    return NamenodeRegistrationWritable.convert(server
+        .register(registration.convert()));
+  }
+
+  @Override
+  public NamenodeCommandWritable startCheckpoint(
+      NamenodeRegistrationWritable registration) throws IOException {
+    return NamenodeCommandWritable.convert(server.startCheckpoint(registration
+        .convert()));
+  }
+
+  @Override
+  public void endCheckpoint(NamenodeRegistrationWritable registration,
+      CheckpointSignatureWritable sig) throws IOException {
+    server.endCheckpoint(registration.convert(), sig.convert());
+  }
+
+  @Override
+  public RemoteEditLogManifestWritable getEditLogManifest(long sinceTxId)
+      throws IOException {
+    return RemoteEditLogManifestWritable.convert(server
+        .getEditLogManifest(sinceTxId));
+  }
+}

+ 186 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/NamenodeProtocolTranslatorR23.java

@@ -0,0 +1,186 @@
+/**
+ * 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.protocolR23Compatible;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.security.token.block.ExportedBlockKeys;
+import org.apache.hadoop.hdfs.server.namenode.CheckpointSignature;
+import org.apache.hadoop.hdfs.server.namenode.NameNode;
+import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations;
+import org.apache.hadoop.hdfs.server.protocol.NamenodeCommand;
+import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol;
+import org.apache.hadoop.hdfs.server.protocol.NamenodeRegistration;
+import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
+import org.apache.hadoop.hdfs.server.protocol.RemoteEditLogManifest;
+import org.apache.hadoop.io.retry.RetryPolicies;
+import org.apache.hadoop.io.retry.RetryPolicy;
+import org.apache.hadoop.io.retry.RetryProxy;
+import org.apache.hadoop.ipc.ProtocolSignature;
+import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.ipc.RemoteException;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.security.UserGroupInformation;
+
+/**
+ * This class forwards NN's NamenodeProtocol calls as RPC calls to the NN server
+ * while translating from the parameter types used in NamenodeProtocol to those
+ * used in protocolR23Compatile.*.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Stable
+public class NamenodeProtocolTranslatorR23 implements
+    NamenodeProtocol, Closeable {
+  final private NamenodeWireProtocol rpcProxyWithoutRetry;
+  final private NamenodeWireProtocol rpcProxy;
+
+  private static NamenodeWireProtocol createNamenode(
+      InetSocketAddress nameNodeAddr, Configuration conf,
+      UserGroupInformation ugi) throws IOException {
+    return RPC.getProxy(NamenodeWireProtocol.class,
+        NamenodeWireProtocol.versionID, nameNodeAddr, ugi, conf,
+        NetUtils.getSocketFactory(conf, NamenodeWireProtocol.class));
+  }
+
+  /** Create a {@link NameNode} proxy */
+  static NamenodeWireProtocol createNamenodeWithRetry(
+      NamenodeWireProtocol rpcNamenode) {
+    RetryPolicy createPolicy = RetryPolicies
+        .retryUpToMaximumCountWithFixedSleep(5,
+            HdfsConstants.LEASE_SOFTLIMIT_PERIOD, TimeUnit.MILLISECONDS);
+
+    Map<Class<? extends Exception>, RetryPolicy> remoteExceptionToPolicyMap = 
+        new HashMap<Class<? extends Exception>, RetryPolicy>();
+    remoteExceptionToPolicyMap.put(AlreadyBeingCreatedException.class,
+        createPolicy);
+
+    Map<Class<? extends Exception>, RetryPolicy> exceptionToPolicyMap = 
+        new HashMap<Class<? extends Exception>, RetryPolicy>();
+    exceptionToPolicyMap.put(RemoteException.class, RetryPolicies
+        .retryByRemoteException(RetryPolicies.TRY_ONCE_THEN_FAIL,
+            remoteExceptionToPolicyMap));
+    RetryPolicy methodPolicy = RetryPolicies.retryByException(
+        RetryPolicies.TRY_ONCE_THEN_FAIL, exceptionToPolicyMap);
+    Map<String, RetryPolicy> methodNameToPolicyMap = new HashMap<String, RetryPolicy>();
+
+    methodNameToPolicyMap.put("create", methodPolicy);
+
+    return (NamenodeWireProtocol) RetryProxy.create(
+        NamenodeWireProtocol.class, rpcNamenode, methodNameToPolicyMap);
+  }
+
+  public NamenodeProtocolTranslatorR23(InetSocketAddress nameNodeAddr,
+      Configuration conf, UserGroupInformation ugi) throws IOException {
+    rpcProxyWithoutRetry = createNamenode(nameNodeAddr, conf, ugi);
+    rpcProxy = createNamenodeWithRetry(rpcProxyWithoutRetry);
+  }
+
+  public Object getProxyWithoutRetry() {
+    return rpcProxyWithoutRetry;
+  }
+
+  public void close() {
+    RPC.stopProxy(rpcProxyWithoutRetry);
+  }
+
+  @Override
+  public ProtocolSignature getProtocolSignature(String protocolName,
+      long clientVersion, int clientMethodHash)
+      throws IOException {
+    return ProtocolSignatureWritable.convert(rpcProxy.getProtocolSignature2(
+        protocolName, clientVersion, clientMethodHash));
+  }
+
+  @Override
+  public long getProtocolVersion(String protocolName, long clientVersion) throws IOException {
+    return rpcProxy.getProtocolVersion(protocolName, clientVersion);
+  }
+
+  @Override
+  public BlocksWithLocations getBlocks(DatanodeInfo datanode, long size)
+      throws IOException {
+    return rpcProxy.getBlocks(
+        DatanodeInfoWritable.convertDatanodeInfo(datanode), size).convert();
+  }
+
+  @Override
+  public ExportedBlockKeys getBlockKeys() throws IOException {
+    return rpcProxy.getBlockKeys().convert();
+  }
+
+  @Override
+  public long getTransactionID() throws IOException {
+    return rpcProxy.getTransactionID();
+  }
+
+  @Override
+  @SuppressWarnings("deprecation")
+  public CheckpointSignature rollEditLog() throws IOException {
+    return rpcProxy.rollEditLog().convert();
+  }
+
+  @Override
+  public NamespaceInfo versionRequest() throws IOException {
+    return rpcProxy.versionRequest().convert();
+  }
+
+  @Override
+  public void errorReport(NamenodeRegistration registration, int errorCode,
+      String msg) throws IOException {
+    rpcProxy.errorReport(NamenodeRegistrationWritable.convert(registration),
+        errorCode, msg);
+  }
+
+  @Override
+  public NamenodeRegistration register(NamenodeRegistration registration)
+      throws IOException {
+    return rpcProxy
+        .register(NamenodeRegistrationWritable.convert(registration)).convert();
+  }
+
+  @Override
+  public NamenodeCommand startCheckpoint(NamenodeRegistration registration)
+      throws IOException {
+    return rpcProxy.startCheckpoint(
+        NamenodeRegistrationWritable.convert(registration)).convert();
+  }
+
+  @Override
+  public void endCheckpoint(NamenodeRegistration registration,
+      CheckpointSignature sig) throws IOException {
+    rpcProxy.endCheckpoint(NamenodeRegistrationWritable.convert(registration),
+        CheckpointSignatureWritable.convert(sig));
+  }
+
+  @Override
+  public RemoteEditLogManifest getEditLogManifest(long sinceTxId)
+      throws IOException {
+    return rpcProxy.getEditLogManifest(sinceTxId).convert();
+  }
+}

+ 169 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/NamenodeWireProtocol.java

@@ -0,0 +1,169 @@
+/**
+ * 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.protocolR23Compatible;
+
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.server.protocol.CheckpointCommand;
+import org.apache.hadoop.hdfs.server.protocol.NamenodeRegistration;
+import org.apache.hadoop.ipc.RemoteException;
+import org.apache.hadoop.ipc.VersionedProtocol;
+import org.apache.hadoop.security.KerberosInfo;
+
+/*****************************************************************************
+ * Protocol that a secondary NameNode uses to communicate with the NameNode.
+ * It's used to get part of the name node state
+ *****************************************************************************/
+/** 
+ * This class defines the actual protocol used to communicate between namenodes.
+ * The parameters in the methods which are specified in the
+ * package are separate from those used internally in the DN and DFSClient
+ * and hence need to be converted using {@link NamenodeProtocolTranslatorR23}
+ * and {@link NamenodeProtocolServerSideTranslatorR23}.
+ */
+@KerberosInfo(
+    serverPrincipal = DFSConfigKeys.DFS_NAMENODE_USER_NAME_KEY,
+    clientPrincipal = DFSConfigKeys.DFS_NAMENODE_USER_NAME_KEY)
+@InterfaceAudience.Private
+public interface NamenodeWireProtocol extends VersionedProtocol {
+  /**
+   * The  rules for changing this protocol are the same as that for
+   * {@link ClientNamenodeWireProtocol} - see that java file for details.
+   */
+  public static final long versionID = 6L;
+
+  /**
+   * Get a list of blocks belonging to <code>datanode</code>
+   * whose total size equals <code>size</code>.
+   * 
+   * @see org.apache.hadoop.hdfs.server.balancer.Balancer
+   * @param datanode  a data node
+   * @param size      requested size
+   * @return          a list of blocks & their locations
+   * @throws RemoteException if size is less than or equal to 0 or
+   *                               datanode does not exist
+   */
+  public BlocksWithLocationsWritable getBlocks(DatanodeInfoWritable datanode,
+      long size) throws IOException;
+
+  /**
+   * Get the current block keys
+   * 
+   * @return ExportedBlockKeys containing current block keys
+   * @throws IOException 
+   */
+  public ExportedBlockKeysWritable getBlockKeys() throws IOException;
+
+  /**
+   * @return The most recent transaction ID that has been synced to
+   * persistent storage.
+   * @throws IOException
+   */
+  public long getTransactionID() throws IOException;
+
+  /**
+   * Closes the current edit log and opens a new one. The 
+   * call fails if the file system is in SafeMode.
+   * @throws IOException
+   * @return a unique token to identify this transaction.
+   * @deprecated 
+   *    See {@link org.apache.hadoop.hdfs.server.namenode.SecondaryNameNode}
+   */
+  @Deprecated
+  public CheckpointSignatureWritable rollEditLog() throws IOException;
+
+  /**
+   * Request name-node version and storage information.
+   * @throws IOException
+   */
+  public NamespaceInfoWritable versionRequest() throws IOException;
+
+  /**
+   * Report to the active name-node an error occurred on a subordinate node.
+   * Depending on the error code the active node may decide to unregister the
+   * reporting node.
+   * 
+   * @param registration requesting node.
+   * @param errorCode indicates the error
+   * @param msg free text description of the error
+   * @throws IOException
+   */
+  public void errorReport(NamenodeRegistrationWritable registration,
+                          int errorCode, 
+                          String msg) throws IOException;
+
+  /** 
+   * Register a subordinate name-node like backup node.
+   *
+   * @return  {@link NamenodeRegistration} of the node,
+   *          which this node has just registered with.
+   */
+  public NamenodeRegistrationWritable register(
+      NamenodeRegistrationWritable registration) throws IOException;
+
+  /**
+   * A request to the active name-node to start a checkpoint.
+   * The name-node should decide whether to admit it or reject.
+   * The name-node also decides what should be done with the backup node
+   * image before and after the checkpoint.
+   * 
+   * @see CheckpointCommand
+   * @see NamenodeCommandWritable
+   * @see #ACT_SHUTDOWN
+   * 
+   * @param registration the requesting node
+   * @return {@link CheckpointCommand} if checkpoint is allowed.
+   * @throws IOException
+   */
+  public NamenodeCommandWritable startCheckpoint(
+      NamenodeRegistrationWritable registration) throws IOException;
+
+  /**
+   * A request to the active name-node to finalize
+   * previously started checkpoint.
+   * 
+   * @param registration the requesting node
+   * @param sig {@code CheckpointSignature} which identifies the checkpoint.
+   * @throws IOException
+   */
+  public void endCheckpoint(NamenodeRegistrationWritable registration,
+                            CheckpointSignatureWritable sig) throws IOException;
+  
+  
+  /**
+   * Return a structure containing details about all edit logs
+   * available to be fetched from the NameNode.
+   * @param sinceTxId return only logs that contain transactions >= sinceTxId
+   */
+  public RemoteEditLogManifestWritable getEditLogManifest(long sinceTxId)
+    throws IOException;
+  
+  /**
+   * This method is defined to get the protocol signature using 
+   * the R23 protocol - hence we have added the suffix of 2 the method name
+   * to avoid conflict.
+   */
+  public org.apache.hadoop.hdfs.protocolR23Compatible.ProtocolSignatureWritable
+           getProtocolSignature2(String protocol, 
+      long clientVersion,
+      int clientMethodsHash) throws IOException;
+}
+

+ 100 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/NamespaceInfoWritable.java

@@ -0,0 +1,100 @@
+/**
+ * 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.protocolR23Compatible;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hdfs.DeprecatedUTF8;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.server.common.Storage;
+import org.apache.hadoop.hdfs.server.common.StorageInfo;
+import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableFactories;
+import org.apache.hadoop.io.WritableFactory;
+import org.apache.hadoop.io.WritableUtils;
+
+/**
+ * NamespaceInfoWritable is returned by the name-node in reply 
+ * to a data-node handshake.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class NamespaceInfoWritable extends StorageInfo {
+  private String  buildVersion;
+  private int distributedUpgradeVersion;
+  private String blockPoolID = "";
+  private StorageInfoWritable storageInfo;
+
+  public NamespaceInfoWritable() {
+    super();
+    buildVersion = null;
+  }
+  
+  public NamespaceInfoWritable(int nsID, String clusterID, String bpID, 
+      long cT, int duVersion) {
+    this.blockPoolID = bpID;
+    this.buildVersion = Storage.getBuildVersion();
+    this.distributedUpgradeVersion = duVersion;
+    storageInfo = new StorageInfoWritable(HdfsConstants.LAYOUT_VERSION, nsID,
+        clusterID, cT);
+  }
+  
+  /////////////////////////////////////////////////
+  // Writable
+  /////////////////////////////////////////////////
+  static {  // register a ctor
+    WritableFactories.setFactory
+      (NamespaceInfoWritable.class,
+       new WritableFactory() {
+         public Writable newInstance() { return new NamespaceInfoWritable(); }
+       });
+  }
+
+  @Override
+  public void write(DataOutput out) throws IOException {
+    DeprecatedUTF8.writeString(out, buildVersion);
+    storageInfo.write(out);
+    out.writeInt(distributedUpgradeVersion);
+    WritableUtils.writeString(out, blockPoolID);
+  }
+
+  @Override
+  public void readFields(DataInput in) throws IOException {
+    buildVersion = DeprecatedUTF8.readString(in);
+    storageInfo.readFields(in);
+    distributedUpgradeVersion = in.readInt();
+    blockPoolID = WritableUtils.readString(in);
+  }
+
+  public static NamespaceInfoWritable convert(NamespaceInfo info) {
+    return new NamespaceInfoWritable(info.getNamespaceID(), info.getClusterID(),
+        info.getBlockPoolID(), info.getCTime(),
+        info.getDistributedUpgradeVersion());
+  }
+  
+  public NamespaceInfo convert() {
+    return new NamespaceInfo(namespaceID, clusterID, blockPoolID, cTime,
+        distributedUpgradeVersion);
+  }
+}

+ 90 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/RemoteEditLogManifestWritable.java

@@ -0,0 +1,90 @@
+/**
+ * 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.protocolR23Compatible;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.hadoop.hdfs.server.protocol.RemoteEditLog;
+import org.apache.hadoop.hdfs.server.protocol.RemoteEditLogManifest;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableFactories;
+import org.apache.hadoop.io.WritableFactory;
+
+import com.google.common.collect.Lists;
+
+/**
+ * An enumeration of logs available on a remote NameNode.
+ */
+public class RemoteEditLogManifestWritable implements Writable {
+  private List<RemoteEditLogWritable> logs;
+  
+  static { // register a ctor
+    WritableFactories.setFactory(RemoteEditLogManifestWritable.class,
+        new WritableFactory() {
+          public Writable newInstance() {
+            return new RemoteEditLogManifestWritable();
+          }
+        });
+  }
+  
+  public RemoteEditLogManifestWritable() {
+  }
+  
+  public RemoteEditLogManifestWritable(List<RemoteEditLogWritable> logs) {
+    this.logs = logs;
+  }
+  
+  @Override
+  public void write(DataOutput out) throws IOException {
+    out.writeInt(logs.size());
+    for (RemoteEditLogWritable log : logs) {
+      log.write(out);
+    }
+  }
+
+  @Override
+  public void readFields(DataInput in) throws IOException {
+    int numLogs = in.readInt();
+    logs = Lists.newArrayList();
+    for (int i = 0; i < numLogs; i++) {
+      RemoteEditLogWritable log = new RemoteEditLogWritable();
+      log.readFields(in);
+      logs.add(log);
+    }
+  }
+
+  public static RemoteEditLogManifestWritable convert(
+      RemoteEditLogManifest editLogManifest) {
+    List<RemoteEditLogWritable> list = Lists.newArrayList();
+    for (RemoteEditLog log : editLogManifest.getLogs()) {
+      list.add(RemoteEditLogWritable.convert(log));
+    }
+    return new RemoteEditLogManifestWritable(list);
+  }
+
+  public RemoteEditLogManifest convert() {
+    List<RemoteEditLog> list = Lists.newArrayList();
+    for (RemoteEditLogWritable log : logs) {
+      list.add(log.convert());
+    }
+    return new RemoteEditLogManifest(list);
+  }
+}

+ 69 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/RemoteEditLogWritable.java

@@ -0,0 +1,69 @@
+/**
+ * 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.protocolR23Compatible;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.hadoop.hdfs.server.protocol.RemoteEditLog;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableFactories;
+import org.apache.hadoop.io.WritableFactory;
+
+public class RemoteEditLogWritable implements Writable  {
+  private long startTxId;
+  private long endTxId;
+  
+  static { // register a ctor
+    WritableFactories.setFactory(RemoteEditLogWritable.class,
+        new WritableFactory() {
+          public Writable newInstance() {
+            return new RemoteEditLogWritable();
+          }
+        });
+  }
+  
+  public RemoteEditLogWritable() {
+  }
+
+  public RemoteEditLogWritable(long startTxId, long endTxId) {
+    this.startTxId = startTxId;
+    this.endTxId = endTxId;
+  }
+
+  @Override
+  public void write(DataOutput out) throws IOException {
+    out.writeLong(startTxId);
+    out.writeLong(endTxId);
+  }
+
+  @Override
+  public void readFields(DataInput in) throws IOException {
+    startTxId = in.readLong();
+    endTxId = in.readLong();
+  }
+
+  public static RemoteEditLogWritable convert(RemoteEditLog log) {
+    return new RemoteEditLogWritable(log.getStartTxId(), log.getEndTxId());
+  }
+
+  public RemoteEditLog convert() {
+    return new RemoteEditLog(startTxId, endTxId);
+  }
+}

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/security/token/block/ExportedBlockKeys.java

@@ -43,7 +43,7 @@ public class ExportedBlockKeys implements Writable {
     this(false, 0, 0, new BlockKey(), new BlockKey[0]);
   }
 
-  ExportedBlockKeys(boolean isBlockTokenEnabled, long keyUpdateInterval,
+  public ExportedBlockKeys(boolean isBlockTokenEnabled, long keyUpdateInterval,
       long tokenLifetime, BlockKey currentKey, BlockKey[] allKeys) {
     this.isBlockTokenEnabled = isBlockTokenEnabled;
     this.keyUpdateInterval = keyUpdateInterval;

+ 16 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/CheckpointSignature.java

@@ -37,9 +37,7 @@ public class CheckpointSignature extends StorageInfo
                       implements WritableComparable<CheckpointSignature> {
   private static final String FIELD_SEPARATOR = ":";
   private static final int NUM_FIELDS = 7;
-
   String blockpoolID = "";
-  
   long mostRecentCheckpointTxId;
   long curSegmentTxId;
 
@@ -67,6 +65,14 @@ public class CheckpointSignature extends StorageInfo
     blockpoolID = fields[i++];
   }
 
+  public CheckpointSignature(StorageInfo info, String blockpoolID,
+      long mostRecentCheckpointTxId, long curSegmentTxId) {
+    super(info);
+    this.blockpoolID = blockpoolID;
+    this.mostRecentCheckpointTxId = mostRecentCheckpointTxId;
+    this.curSegmentTxId = curSegmentTxId;
+  }
+
   /**
    * Get the cluster id from CheckpointSignature
    * @return the cluster id
@@ -83,6 +89,14 @@ public class CheckpointSignature extends StorageInfo
     return blockpoolID;
   }
 
+  public long getMostRecentCheckpointTxId() {
+    return mostRecentCheckpointTxId;
+  }
+
+  public long getCurSegmentTxId() {
+    return curSegmentTxId;
+  }
+
   /**
    * Set the block pool id of CheckpointSignature.
    * 

+ 16 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/NamenodeProtocol.java

@@ -38,9 +38,21 @@ import org.apache.hadoop.security.KerberosInfo;
 @InterfaceAudience.Private
 public interface NamenodeProtocol extends VersionedProtocol {
   /**
-   * Compared to the previous version the following changes have been introduced:
-   * (Only the latest change is reflected.
-   * The log of historical changes can be retrieved from the svn).
+   * Until version 6L, this class served as both
+   * the client interface to the NN AND the RPC protocol used to 
+   * communicate with the NN.
+   * 
+   * Post version 70 (release 23 of Hadoop), the protocol is implemented in
+   * {@literal ../protocolR23Compatible/ClientNamenodeWireProtocol}
+   * 
+   * This class is used by both the DFSClient and the 
+   * NN server side to insulate from the protocol serialization.
+   * 
+   * If you are adding/changing NN's interface then you need to 
+   * change both this class and ALSO
+   * {@link org.apache.hadoop.hdfs.protocolR23Compatible.NamenodeWireProtocol}.
+   * These changes need to be done in a compatible fashion as described in 
+   * {@link org.apache.hadoop.hdfs.protocolR23Compatible.ClientNamenodeWireProtocol}
    * 
    * 6: Switch to txid-based file naming for image and edits
    */
@@ -62,7 +74,7 @@ public interface NamenodeProtocol extends VersionedProtocol {
    * @param datanode  a data node
    * @param size      requested size
    * @return          a list of blocks & their locations
-   * @throws RemoteException if size is less than or equal to 0 or
+   * @throws IOException if size is less than or equal to 0 or
                                    datanode does not exist
    */
   public BlocksWithLocations getBlocks(DatanodeInfo datanode, long size)