Browse Source

HDS-2895. Remove Writable wire protocol types and translators to complete transition to protocol buffers. Contributed by Suresh Srinivas.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1241007 13f79535-47bb-0310-9956-ffa450edef68
Suresh Srinivas 13 years ago
parent
commit
2a9e430ff9
66 changed files with 35 additions and 8304 deletions
  1. 3 0
      hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
  2. 5 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientDatanodeProtocol.java
  3. 6 5
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
  4. 0 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
  5. 1 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolClientSideTranslatorPB.java
  6. 0 111
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/BlockWritable.java
  7. 0 129
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/BlocksWithLocationsWritable.java
  8. 0 92
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/CheckpointSignatureWritable.java
  9. 0 129
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/ClientDatanodeProtocolServerSideTranslatorR23.java
  10. 0 162
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/ClientDatanodeProtocolTranslatorR23.java
  11. 0 100
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/ClientDatanodeWireProtocol.java
  12. 0 463
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/ClientNamenodeProtocolServerSideTranslatorR23.java
  13. 0 480
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/ClientNamenodeProtocolTranslatorR23.java
  14. 0 478
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/ClientNamenodeWireProtocol.java
  15. 0 184
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/ContentSummaryWritable.java
  16. 0 88
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/CorruptFileBlocksWritable.java
  17. 0 209
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/DatanodeIDWritable.java
  18. 0 334
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/DatanodeInfoWritable.java
  19. 0 157
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/DirectoryListingWritable.java
  20. 0 104
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/ExportedBlockKeysWritable.java
  21. 0 121
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/ExtendedBlockWritable.java
  22. 0 80
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/FsPermissionWritable.java
  23. 0 122
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/FsServerDefaultsWritable.java
  24. 0 348
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/HdfsFileStatusWritable.java
  25. 0 116
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/HdfsLocatedFileStatusWritable.java
  26. 0 114
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/JournalProtocolServerSideTranslatorR23.java
  27. 0 80
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/JournalProtocolTranslatorR23.java
  28. 0 81
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/JournalWireProtocol.java
  29. 0 253
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/LocatedBlockWritable.java
  30. 0 197
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/LocatedBlocksWritable.java
  31. 0 70
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/NamenodeCommandWritable.java
  32. 0 163
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/NamenodeProtocolServerSideTranslatorR23.java
  33. 0 180
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/NamenodeProtocolTranslatorR23.java
  34. 0 98
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/NamenodeRegistrationWritable.java
  35. 0 169
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/NamenodeWireProtocol.java
  36. 0 100
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/NamespaceInfoWritable.java
  37. 0 90
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/RemoteEditLogManifestWritable.java
  38. 0 69
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/RemoteEditLogWritable.java
  39. 0 86
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/StorageInfoWritable.java
  40. 0 208
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/TokenWritable.java
  41. 0 140
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/UpgradeStatusReportWritable.java
  42. 5 6
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeProtocol.java
  43. 5 6
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/InterDatanodeProtocol.java
  44. 5 6
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/JournalProtocol.java
  45. 5 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/NamenodeProtocol.java
  46. 0 107
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocolR23Compatible/BalancerBandwidthCommandWritable.java
  47. 0 142
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocolR23Compatible/BlockCommandWritable.java
  48. 0 118
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocolR23Compatible/BlockRecoveryCommandWritable.java
  49. 0 73
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocolR23Compatible/DatanodeCommandHelper.java
  50. 0 58
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocolR23Compatible/DatanodeCommandWritable.java
  51. 0 170
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocolR23Compatible/DatanodeProtocolServerSideTranslatorR23.java
  52. 0 193
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocolR23Compatible/DatanodeProtocolTranslatorR23.java
  53. 0 113
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocolR23Compatible/DatanodeRegistrationWritable.java
  54. 0 181
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocolR23Compatible/DatanodeWireProtocol.java
  55. 0 88
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocolR23Compatible/FinalizeCommandWritable.java
  56. 0 116
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocolR23Compatible/InterDatanodeProtocolServerSideTranslatorR23.java
  57. 0 96
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocolR23Compatible/InterDatanodeProtocolTranslatorR23.java
  58. 0 73
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocolR23Compatible/InterDatanodeWireProtocol.java
  59. 0 87
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocolR23Compatible/KeyUpdateCommandWritable.java
  60. 0 95
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocolR23Compatible/ReceivedDeletedBlockInfoWritable.java
  61. 0 104
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocolR23Compatible/RecoveringBlockWritable.java
  62. 0 69
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocolR23Compatible/RegisterCommandWritable.java
  63. 0 87
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocolR23Compatible/ReplicaRecoveryInfoWritable.java
  64. 0 75
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocolR23Compatible/ServerCommandWritable.java
  65. 0 106
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocolR23Compatible/UpgradeCommandWritable.java
  66. 0 13
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java

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

@@ -119,6 +119,9 @@ Trunk (unreleased changes)
     HDFS-2785. Update webhdfs and httpfs for host-based token support.
     (Robert Joseph Evans via jitendra)
 
+    HDS-2895. Remove Writable wire protocol types and translators to
+    complete transition to protocol buffers. (suresh)
+
   OPTIMIZATIONS
     HDFS-2477. Optimize computing the diff between a block report and the
     namenode state. (Tomasz Nykiel via hairong)

+ 5 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientDatanodeProtocol.java

@@ -49,10 +49,11 @@ public interface ClientDatanodeProtocol extends VersionedProtocol {
    * DN server side to insulate from the protocol serialization.
    * 
    * If you are adding/changing DN's interface then you need to 
-   * change both this class and ALSO
-   * {@link org.apache.hadoop.hdfs.protocolR23Compatible.ClientDatanodeWireProtocol}.
-   * These changes need to be done in a compatible fashion as described in 
-   * {@link org.apache.hadoop.hdfs.protocolR23Compatible.ClientNamenodeWireProtocol}
+   * change both this class and ALSO related protocol buffer
+   * wire protocol definition in ClientDatanodeProtocol.proto.
+   * 
+   * For more details on protocol buffer wire protocol, please see 
+   * .../org/apache/hadoop/hdfs/protocolPB/overview.html
    * 
    * The log of historical changes can be retrieved from the svn).
    * 9: Added deleteBlockPool method

+ 6 - 5
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java

@@ -72,11 +72,12 @@ public interface ClientProtocol extends VersionedProtocol {
    * 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.ClientNamenodeWireProtocol}.
-   * These changes need to be done in a compatible fashion as described in 
-   * {@link org.apache.hadoop.hdfs.protocolR23Compatible.ClientNamenodeWireProtocol}
+   * If you are adding/changing this interface then you need to 
+   * change both this class and ALSO related protocol buffer
+   * wire protocol definition in ClientNamenodeProtocol.proto.
+   * 
+   * For more details on protocol buffer wire protocol, please see 
+   * .../org/apache/hadoop/hdfs/protocolPB/overview.html
    * 
    * The log of historical changes can be retrieved from the svn).
    * 69: Eliminate overloaded method names.

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

@@ -127,8 +127,6 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetSaf
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetTimesRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.UpdateBlockForPipelineRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.UpdatePipelineRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DirectoryListingProto;
-import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.HdfsFileStatusProto;
 
 import com.google.protobuf.ByteString;
 import com.google.protobuf.ServiceException;

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

@@ -58,7 +58,6 @@ import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
 import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
 import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo;
 import org.apache.hadoop.hdfs.server.protocol.UpgradeCommand;
-import org.apache.hadoop.hdfs.server.protocolR23Compatible.DatanodeWireProtocol;
 import org.apache.hadoop.io.retry.RetryPolicies;
 import org.apache.hadoop.io.retry.RetryPolicy;
 import org.apache.hadoop.io.retry.RetryProxy;
@@ -105,7 +104,7 @@ public class DatanodeProtocolClientSideTranslatorPB implements
       UserGroupInformation ugi) throws IOException {
     return RPC.getProxy(DatanodeProtocolPB.class,
         RPC.getProtocolVersion(DatanodeProtocolPB.class), nameNodeAddr, ugi,
-        conf, NetUtils.getSocketFactory(conf, DatanodeWireProtocol.class));
+        conf, NetUtils.getSocketFactory(conf, DatanodeProtocolPB.class));
   }
 
   /** Create a {@link NameNode} proxy */

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

@@ -1,111 +0,0 @@
-/**
- * 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);
-  }
-  
-  public long getBlockId() {
-    return blockId;
-  }
-
-  public long getNumBytes() {
-    return numBytes;
-  }
-
-  public long getGenerationStamp() {
-    return generationStamp;
-  }
-  
-  public static Block[] convert(BlockWritable[] blocks) {
-    Block[] ret = new Block[blocks.length];
-    for (int i = 0; i < blocks.length; i++) {
-      ret[i] = blocks[i].convert();
-    }
-    return ret;
-  }
-  
-  public static BlockWritable[] convert(Block[] blocks) {
-    BlockWritable[] ret = new BlockWritable[blocks.length];
-    for (int i = 0; i < blocks.length; i++) {
-      ret[i] = BlockWritable.convert(blocks[i]);
-    }
-    return ret;
-  }
-}

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

@@ -1,129 +0,0 @@
-/**
- * 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);
-  }
-}

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

@@ -1,92 +0,0 @@
-/**
- * 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);
-  }
-}

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

@@ -1,129 +0,0 @@
-/**
- * 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.protocol.BlockLocalPathInfo;
-import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
-import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
-import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
-import org.apache.hadoop.ipc.ProtocolSignature;
-import org.apache.hadoop.ipc.RPC;
-import org.apache.hadoop.security.token.Token;
-
-/**
- * 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 ClientDatanodeProtocol.
- */
-@InterfaceAudience.Private
-@InterfaceStability.Stable
-public class ClientDatanodeProtocolServerSideTranslatorR23 implements
-  ClientDatanodeWireProtocol {
-  
-  final private ClientDatanodeProtocol server;
-
-  /**
-   * 
-   * @param server - the NN server
-   * @throws IOException
-   */
-  public ClientDatanodeProtocolServerSideTranslatorR23(
-      ClientDatanodeProtocol 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 ClientDatanodeProtocol}
-     */
-    if (!protocol.equals(RPC.getProtocolName(
-        ClientDatanodeWireProtocol.class))) {
-      throw new IOException("Datanode Serverside implements " + 
-          ClientDatanodeWireProtocol.class + 
-          ". The following requested protocol is unknown: " + protocol);
-    }
-    
-    return ProtocolSignature.getProtocolSignature(clientMethodsHash, 
-        ClientDatanodeWireProtocol.versionID, 
-        ClientDatanodeWireProtocol.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(
-        ClientDatanodeWireProtocol.class))) {
-      return ClientDatanodeWireProtocol.versionID; 
-    }
-    throw new IOException("Datanode Serverside implements " + 
-        ClientDatanodeWireProtocol.class + 
-        ". The following requested protocol is unknown: " + protocol);
-  }
-
-  @Override
-  public long getReplicaVisibleLength(ExtendedBlockWritable b) throws IOException {
-    return 
-        server.getReplicaVisibleLength(ExtendedBlockWritable.convertExtendedBlock(b));
-  }
-
-  @Override
-  public void refreshNamenodes() throws IOException {
-    server.refreshNamenodes();
-  }
-
-  @Override
-  public void deleteBlockPool(String bpid, boolean force) throws IOException {
-    server.deleteBlockPool(bpid, force);
-  }
-
-  @Override
-  public BlockLocalPathInfo getBlockLocalPathInfo(ExtendedBlock block,
-      Token<BlockTokenIdentifier> token) throws IOException {
-    return server.getBlockLocalPathInfo(block, token);
-  }
-}

+ 0 - 162
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/ClientDatanodeProtocolTranslatorR23.java

@@ -1,162 +0,0 @@
-/**
- * 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 java.net.InetSocketAddress;
-
-import javax.net.SocketFactory;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
-import org.apache.hadoop.hdfs.protocol.BlockLocalPathInfo;
-import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
-import org.apache.hadoop.hdfs.protocol.DatanodeID;
-import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
-import org.apache.hadoop.hdfs.protocol.LocatedBlock;
-import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
-import org.apache.hadoop.ipc.ProtocolSignature;
-import org.apache.hadoop.ipc.RPC;
-import org.apache.hadoop.net.NetUtils;
-import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.security.token.Token;
-
-
-/**
- * This class forwards ClientDatanodeProtocol calls as RPC to the DN server
- * while translating from the parameter types used in ClientDatanodeProtocol to
- * those used in protocolR23Compatile.*.
- */
-@InterfaceAudience.Private
-@InterfaceStability.Stable
-public class ClientDatanodeProtocolTranslatorR23 implements 
-  ClientDatanodeProtocol {
-  
-  final private ClientDatanodeWireProtocol rpcProxy;
-  
-  public ClientDatanodeProtocolTranslatorR23(DatanodeID datanodeid,
-      Configuration conf, int socketTimeout, LocatedBlock locatedBlock)
-      throws IOException {
-    rpcProxy = createClientDatanodeProtocolProxy( datanodeid, conf, 
-                  socketTimeout, locatedBlock);
-  }
-  
-  /** used for testing */
-  public ClientDatanodeProtocolTranslatorR23(InetSocketAddress addr,
-      UserGroupInformation ticket,
-      Configuration conf,
-      SocketFactory factory) throws IOException {
-    rpcProxy = createClientDatanodeProtocolProxy(addr, ticket, conf, factory);
-  }
-  
-  /**
-   * Constructor.
-   * @param datanodeid Datanode to connect to.
-   * @param conf Configuration.
-   * @param socketTimeout Socket timeout to use.
-   * @throws IOException
-   */
-  public ClientDatanodeProtocolTranslatorR23(DatanodeID datanodeid,
-      Configuration conf, int socketTimeout) throws IOException {
-    InetSocketAddress addr = NetUtils.createSocketAddr(datanodeid.getHost()
-        + ":" + datanodeid.getIpcPort());
-    rpcProxy = RPC.getProxy(ClientDatanodeWireProtocol.class,
-        ClientDatanodeWireProtocol.versionID, addr,
-        UserGroupInformation.getCurrentUser(), conf,
-        NetUtils.getDefaultSocketFactory(conf), socketTimeout);
-  }
-
-  static ClientDatanodeWireProtocol createClientDatanodeProtocolProxy(
-      DatanodeID datanodeid, Configuration conf, int socketTimeout,
-      LocatedBlock locatedBlock)
-      throws IOException {
-    InetSocketAddress addr = NetUtils.createSocketAddr(
-      datanodeid.getHost() + ":" + datanodeid.getIpcPort());
-    if (ClientDatanodeWireProtocol.LOG.isDebugEnabled()) {
-      ClientDatanodeWireProtocol.LOG.debug(
-          "ClientDatanodeProtocol addr=" + addr);
-    }
-    
-    // Since we're creating a new UserGroupInformation here, we know that no
-    // future RPC proxies will be able to re-use the same connection. And
-    // usages of this proxy tend to be one-off calls.
-    //
-    // This is a temporary fix: callers should really achieve this by using
-    // RPC.stopProxy() on the resulting object, but this is currently not
-    // working in trunk. See the discussion on HDFS-1965.
-    Configuration confWithNoIpcIdle = new Configuration(conf);
-    confWithNoIpcIdle.setInt(CommonConfigurationKeysPublic
-        .IPC_CLIENT_CONNECTION_MAXIDLETIME_KEY, 0);
-
-    UserGroupInformation ticket = UserGroupInformation
-        .createRemoteUser(locatedBlock.getBlock().getLocalBlock().toString());
-    ticket.addToken(locatedBlock.getBlockToken());
-    return RPC.getProxy(ClientDatanodeWireProtocol.class,
-      ClientDatanodeWireProtocol.versionID, addr, ticket, confWithNoIpcIdle,
-        NetUtils.getDefaultSocketFactory(conf), socketTimeout);
-  }
-  
-  static ClientDatanodeWireProtocol createClientDatanodeProtocolProxy(
-      InetSocketAddress addr, UserGroupInformation ticket, Configuration conf,
-      SocketFactory factory) throws IOException {
-    return RPC.getProxy(ClientDatanodeWireProtocol.class,
-        ClientDatanodeWireProtocol.versionID, addr, ticket, conf,
-        factory);
-  }
-
-  @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 long getReplicaVisibleLength(ExtendedBlock b) throws IOException {
-    return rpcProxy.getReplicaVisibleLength(
-        ExtendedBlockWritable.convertExtendedBlock(b));
-  }
-
-  @Override
-  public void refreshNamenodes() throws IOException {
-    rpcProxy.refreshNamenodes();
-
-  }
-
-  @Override
-  public void deleteBlockPool(String bpid, boolean force) throws IOException {
-    rpcProxy.deleteBlockPool(bpid, force);
-
-  }
-
-  @Override
-  public BlockLocalPathInfo getBlockLocalPathInfo(ExtendedBlock block,
-      Token<BlockTokenIdentifier> token) throws IOException {
-    return rpcProxy.getBlockLocalPathInfo(block, token);
-  }
-}

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

@@ -1,100 +0,0 @@
-/**
- * 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.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.hdfs.DFSConfigKeys;
-import org.apache.hadoop.hdfs.protocol.BlockLocalPathInfo;
-import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
-import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
-import org.apache.hadoop.hdfs.security.token.block.BlockTokenSelector;
-import org.apache.hadoop.ipc.ProtocolInfo;
-import org.apache.hadoop.ipc.VersionedProtocol;
-import org.apache.hadoop.security.KerberosInfo;
-import org.apache.hadoop.security.token.Token;
-import org.apache.hadoop.security.token.TokenInfo;
-
-/** 
- * This class defines the actual protocol used to communicate with the
- * DN via RPC using writable types.
- * 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 ClientDatanodeProtocolTranslatorR23}
- * and {@link ClientDatanodeProtocolServerSideTranslatorR23}.
- */
-@InterfaceAudience.Private
-@InterfaceStability.Stable
-@KerberosInfo(
-    serverPrincipal = DFSConfigKeys.DFS_DATANODE_USER_NAME_KEY)
-@TokenInfo(BlockTokenSelector.class)
-@ProtocolInfo(protocolName = HdfsConstants.CLIENT_DATANODE_PROTOCOL_NAME)
-public interface ClientDatanodeWireProtocol extends VersionedProtocol {
-  public static final Log LOG = 
-      LogFactory.getLog(ClientDatanodeWireProtocol.class);
-
-  /**
-   * The  rules for changing this protocol are the same as that for
-   * {@link ClientNamenodeWireProtocol} - see that java file for details.
-   * 9: Added deleteBlockPool method
-   * 10 Moved the R23 protocol
-   */
-  public static final long versionID = 10L;
-
-  /**
-   * The specification of this method matches that of
-   * 
-   * {@link org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol
-   * #getReplicaVisibleLength(org.apache.hadoop.hdfs.protocol.ExtendedBlock)}
-   */
-  long getReplicaVisibleLength(ExtendedBlockWritable b) throws IOException;
-  
-  /**
-   * The specification of this method matches that of
-   * {@link org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol#refreshNamenodes()}
-   */
-  void refreshNamenodes() throws IOException;
-
-  /**
-   * The specification of this method matches that of
-   * {@link org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol#deleteBlockPool(String, boolean)}
-   */
-  void deleteBlockPool(String bpid, boolean force) throws IOException; 
-  
-  /**
-   * The specification of this method matches that of
-   * {@link org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol#getBlockLocalPathInfo(ExtendedBlock, Token)}
-   */
-  BlockLocalPathInfo getBlockLocalPathInfo(ExtendedBlock block,
-      Token<BlockTokenIdentifier> token) throws IOException;
-  
-  /**
-   * This method is defined to get the protocol signature using 
-   * the R23 protocol - hence we have added the suffix of 2 to the method name
-   * to avoid conflict.
-   */
-  public org.apache.hadoop.hdfs.protocolR23Compatible.ProtocolSignatureWritable
-           getProtocolSignature2(String protocol, 
-      long clientVersion,
-      int clientMethodsHash) throws IOException;
-}

+ 0 - 463
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/ClientNamenodeProtocolServerSideTranslatorR23.java

@@ -1,463 +0,0 @@
-/**
- * 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.FileNotFoundException;
-import java.io.IOException;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.fs.CreateFlag;
-import org.apache.hadoop.fs.FileAlreadyExistsException;
-import org.apache.hadoop.fs.Options.Rename;
-import org.apache.hadoop.fs.ParentNotDirectoryException;
-import org.apache.hadoop.fs.UnresolvedLinkException;
-import org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException;
-import org.apache.hadoop.hdfs.protocol.ClientProtocol;
-import org.apache.hadoop.hdfs.protocol.DSQuotaExceededException;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants.UpgradeAction;
-import org.apache.hadoop.hdfs.protocol.NSQuotaExceededException;
-import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
-import org.apache.hadoop.hdfs.server.namenode.NotReplicatedYetException;
-import org.apache.hadoop.hdfs.server.namenode.SafeModeException;
-import org.apache.hadoop.io.EnumSetWritable;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.ipc.ProtocolSignature;
-import org.apache.hadoop.ipc.RPC;
-import org.apache.hadoop.security.AccessControlException;
-import org.apache.hadoop.security.token.Token;
-
-/**
- * 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 native data types used inside the NN as specified in the generic
- * ClientProtocol.
- */
-@InterfaceAudience.Private
-@InterfaceStability.Stable
-public class ClientNamenodeProtocolServerSideTranslatorR23 implements
-    ClientNamenodeWireProtocol {
-  final private ClientProtocol server;
-
-  /**
-   * Constructor
-   * 
-   * @param server - the NN server
-   * @throws IOException
-   */
-  public ClientNamenodeProtocolServerSideTranslatorR23(ClientProtocol 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 ClientNamenodeProtocol}
-     * 
-     */
-    if (!protocol.equals(RPC.getProtocolName(
-        ClientNamenodeWireProtocol.class))) {
-      throw new IOException("Namenode Serverside implements " +
-          RPC.getProtocolName(ClientNamenodeWireProtocol.class) +
-          ". The following requested protocol is unknown: " + protocol);
-    }
-
-    return ProtocolSignature.getProtocolSignature(clientMethodsHash,
-        ClientNamenodeWireProtocol.versionID,
-        ClientNamenodeWireProtocol.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(
-        ClientNamenodeWireProtocol.class))) {
-      return ClientNamenodeWireProtocol.versionID;
-    }
-    throw new IOException("Namenode Serverside implements " +
-        RPC.getProtocolName(ClientNamenodeWireProtocol.class) +
-        ". The following requested protocol is unknown: " + protocol);
-  }
-
-  @Override
-  public LocatedBlocksWritable getBlockLocations(
-      String src, long offset, long length)
-      throws AccessControlException, FileNotFoundException,
-      UnresolvedLinkException, IOException {
-    return LocatedBlocksWritable.convertLocatedBlocks(
-        server.getBlockLocations(src, offset, length));
-  }
-
-  @Override
-  public FsServerDefaultsWritable getServerDefaults() throws IOException {
-    return FsServerDefaultsWritable.convert(server.getServerDefaults());
-  }
-
-  @Override
-  public void create(String src, FsPermissionWritable masked, String clientName,
-      EnumSetWritable<CreateFlag> flag, boolean createParent,
-      short replication, long blockSize) throws AccessControlException,
-      AlreadyBeingCreatedException, DSQuotaExceededException,
-      FileAlreadyExistsException, FileNotFoundException,
-      NSQuotaExceededException, ParentNotDirectoryException, SafeModeException,
-      UnresolvedLinkException, IOException {
-    server.create(src, FsPermissionWritable.convertPermission(masked),
-        clientName, flag, createParent, replication, blockSize);
-
-  }
-
-  @Override
-  public LocatedBlockWritable append(String src, String clientName)
-      throws AccessControlException, DSQuotaExceededException,
-      FileNotFoundException, SafeModeException, UnresolvedLinkException,
-      IOException {
-    return LocatedBlockWritable.convertLocatedBlock(
-        server.append(src, clientName));
-  }
-
-  @Override
-  public boolean setReplication(String src, short replication)
-      throws AccessControlException, DSQuotaExceededException,
-      FileNotFoundException, SafeModeException, UnresolvedLinkException,
-      IOException {
-    return server.setReplication(src, replication);
-  }
-
-  @Override
-  public void setPermission(String src, FsPermissionWritable permission)
-      throws AccessControlException, FileNotFoundException, SafeModeException,
-      UnresolvedLinkException, IOException {
-    server.setPermission(src, 
-        FsPermissionWritable.convertPermission(permission));
-
-  }
-
-  @Override
-  public void setOwner(String src, String username, String groupname)
-      throws AccessControlException, FileNotFoundException, SafeModeException,
-      UnresolvedLinkException, IOException {
-    server.setOwner(src, username, groupname);
-
-  }
-
-  @Override
-  public void abandonBlock(ExtendedBlockWritable b, String src, String holder)
-      throws AccessControlException, FileNotFoundException,
-      UnresolvedLinkException, IOException {
-    server.abandonBlock(
-        ExtendedBlockWritable.convertExtendedBlock(b), src, holder);
-
-  }
-
-  @Override
-  public LocatedBlockWritable addBlock(String src, String clientName,
-      ExtendedBlockWritable previous, DatanodeInfoWritable[] excludeNodes)
-      throws AccessControlException, FileNotFoundException,
-      NotReplicatedYetException, SafeModeException, UnresolvedLinkException,
-      IOException {
-    return LocatedBlockWritable.convertLocatedBlock(
-        server.addBlock(src, clientName,
-        ExtendedBlockWritable.convertExtendedBlock(previous),
-        DatanodeInfoWritable.convertDatanodeInfo(excludeNodes)));
-  }
-
-  @Override
-  public LocatedBlockWritable getAdditionalDatanode(String src, ExtendedBlockWritable blk,
-      DatanodeInfoWritable[] existings, DatanodeInfoWritable[] excludes,
-      int numAdditionalNodes, String clientName) throws AccessControlException,
-      FileNotFoundException, SafeModeException, UnresolvedLinkException,
-      IOException {
-    return LocatedBlockWritable.convertLocatedBlock(
-        server.getAdditionalDatanode(src,
-              ExtendedBlockWritable.convertExtendedBlock(blk),
-              DatanodeInfoWritable.convertDatanodeInfo(existings),
-              DatanodeInfoWritable.convertDatanodeInfo(excludes),
-              numAdditionalNodes, clientName));
-  }
-
-  @Override
-  public boolean complete(String src, String clientName, ExtendedBlockWritable last)
-      throws AccessControlException, FileNotFoundException, SafeModeException,
-      UnresolvedLinkException, IOException {
-    return server.complete(src, clientName,
-        ExtendedBlockWritable.convertExtendedBlock(last));
-  }
-
-  @Override
-  public void reportBadBlocks(LocatedBlockWritable[] blocks) throws IOException {
-    server.reportBadBlocks(LocatedBlockWritable.convertLocatedBlock(blocks));
-
-  }
-
-  @Override
-  public boolean rename(String src, String dst) throws UnresolvedLinkException,
-      IOException {
-    return server.rename(src, dst);
-  }
-
-  @Override
-  public void concat(String trg, String[] srcs) throws IOException,
-      UnresolvedLinkException {
-    server.concat(trg, srcs);
-
-  }
-
-  @Override
-  public void rename2(String src, String dst, Rename... options)
-      throws AccessControlException, DSQuotaExceededException,
-      FileAlreadyExistsException, FileNotFoundException,
-      NSQuotaExceededException, ParentNotDirectoryException, SafeModeException,
-      UnresolvedLinkException, IOException {
-    server.rename2(src, dst, options);
-  }
-
-  @Override
-  public boolean delete(String src, boolean recursive)
-      throws AccessControlException, FileNotFoundException, SafeModeException,
-      UnresolvedLinkException, IOException {
-    return server.delete(src, recursive);
-  }
-
-  @Override
-  public boolean mkdirs(String src, FsPermissionWritable masked, boolean createParent)
-      throws AccessControlException, FileAlreadyExistsException,
-      FileNotFoundException, NSQuotaExceededException,
-      ParentNotDirectoryException, SafeModeException, UnresolvedLinkException,
-      IOException {
-
-    return server.mkdirs(src, FsPermissionWritable.convertPermission(masked),
-        createParent);
-  }
-
-  @Override
-  public DirectoryListingWritable getListing(String src, byte[] startAfter,
-      boolean needLocation) throws AccessControlException,
-      FileNotFoundException, UnresolvedLinkException, IOException {
-    return DirectoryListingWritable.convertDirectoryListing(
-        server.getListing(src, startAfter, needLocation));
-  }
-
-  @Override
-  public void renewLease(String clientName) throws AccessControlException,
-      IOException {
-    server.renewLease(clientName);
-
-  }
-
-  @Override
-  public boolean recoverLease(String src, String clientName) throws IOException {
-    return server.recoverLease(src, clientName);
-  }
-
-  @Override
-  public long[] getStats() throws IOException {
-    return server.getStats();
-  }
-
-  @Override
-  public DatanodeInfoWritable[] getDatanodeReport(DatanodeReportType type)
-      throws IOException {
-    return DatanodeInfoWritable
-        .convertDatanodeInfo(server.getDatanodeReport(type));
-  }
-
-  @Override
-  public long getPreferredBlockSize(String filename) throws IOException,
-      UnresolvedLinkException {
-    return server.getPreferredBlockSize(filename);
-  }
-
-  @Override
-  public boolean setSafeMode(SafeModeAction action) throws IOException {
-    return server.setSafeMode(action);
-  }
-
-  @Override
-  public void saveNamespace() throws AccessControlException, IOException {
-    server.saveNamespace();
-
-  }
-
-  @Override
-  public boolean restoreFailedStorage(String arg)
-      throws AccessControlException, IOException {
-    return server.restoreFailedStorage(arg);
-  }
-
-  @Override
-  public void refreshNodes() throws IOException {
-    server.refreshNodes();
-
-  }
-
-  @Override
-  public void finalizeUpgrade() throws IOException {
-    server.finalizeUpgrade();
-
-  }
-
-  @Override
-  public UpgradeStatusReportWritable distributedUpgradeProgress(UpgradeAction action)
-      throws IOException {
-    return UpgradeStatusReportWritable.convert(
-        server.distributedUpgradeProgress(action));
-  }
-
-  @Override
-  public CorruptFileBlocksWritable listCorruptFileBlocks(String path, String cookie)
-      throws IOException {
-    return CorruptFileBlocksWritable.convertCorruptFilesBlocks(
-        server.listCorruptFileBlocks(path, cookie));
-  }
-
-  @Override
-  public void metaSave(String filename) throws IOException {
-    server.metaSave(filename);
-
-  }
-
-  @Override
-  public HdfsFileStatusWritable getFileInfo(String src) throws AccessControlException,
-      FileNotFoundException, UnresolvedLinkException, IOException {
-    return HdfsFileStatusWritable.convertHdfsFileStatus(
-        server.getFileInfo(src));
-  }
-
-  @Override
-  public HdfsFileStatusWritable getFileLinkInfo(String src)
-      throws AccessControlException, UnresolvedLinkException, IOException {
-    return HdfsFileStatusWritable.convertHdfsFileStatus(
-        server.getFileLinkInfo(src));
-  }
-
-  @Override
-  public ContentSummaryWritable getContentSummary(String path)
-      throws AccessControlException, FileNotFoundException,
-      UnresolvedLinkException, IOException {
-    return ContentSummaryWritable.convert(server.getContentSummary(path));
-  }
-
-  @Override
-  public void setQuota(String path, long namespaceQuota, long diskspaceQuota)
-      throws AccessControlException, FileNotFoundException,
-      UnresolvedLinkException, IOException {
-    server.setQuota(path, namespaceQuota, diskspaceQuota);
-
-  }
-
-  @Override
-  public void fsync(String src, String client) throws AccessControlException,
-      FileNotFoundException, UnresolvedLinkException, IOException {
-    server.fsync(src, client);
-
-  }
-
-  @Override
-  public void setTimes(String src, long mtime, long atime)
-      throws AccessControlException, FileNotFoundException,
-      UnresolvedLinkException, IOException {
-    server.setTimes(src, mtime, atime);
-
-  }
-
-  @Override
-  public void createSymlink(String target, String link, FsPermissionWritable dirPerm,
-      boolean createParent) throws AccessControlException,
-      FileAlreadyExistsException, FileNotFoundException,
-      ParentNotDirectoryException, SafeModeException, UnresolvedLinkException,
-      IOException {
-    server.createSymlink(target, link, FsPermissionWritable.convertPermission(dirPerm),
-        createParent);
-
-  }
-
-  @Override
-  public String getLinkTarget(String path) throws AccessControlException,
-      FileNotFoundException, IOException {
-    return server.getLinkTarget(path);
-  }
-
-  @Override
-  public LocatedBlockWritable updateBlockForPipeline(ExtendedBlockWritable block,
-      String clientName) throws IOException {
-    return LocatedBlockWritable.convertLocatedBlock(
-        server.updateBlockForPipeline(
-        ExtendedBlockWritable.convertExtendedBlock(block), clientName));
-  }
-
-  @Override
-  public void updatePipeline(String clientName, ExtendedBlockWritable oldBlock,
-      ExtendedBlockWritable newBlock, DatanodeIDWritable[] newNodes)
-    throws IOException {
-    server.updatePipeline(clientName, 
-              ExtendedBlockWritable.convertExtendedBlock(oldBlock), 
-              ExtendedBlockWritable.convertExtendedBlock(newBlock),
-              DatanodeIDWritable.convertDatanodeID(newNodes));
-  }
-
-  @Override
-  public Token<DelegationTokenIdentifier> getDelegationToken(Text renewer)
-      throws IOException {
-    return server.getDelegationToken(renewer);
-  }
-
-  @Override
-  public long renewDelegationToken(Token<DelegationTokenIdentifier> token)
-      throws IOException {
-    return server.renewDelegationToken(token);
-  }
-
-  @Override
-  public void cancelDelegationToken(Token<DelegationTokenIdentifier> token)
-      throws IOException {
-    server.cancelDelegationToken(token);
-  }
-
-  @Override
-  public void setBalancerBandwidth(long bandwidth) throws IOException {
-    server.setBalancerBandwidth(bandwidth);
-  }
-}

+ 0 - 480
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/ClientNamenodeProtocolTranslatorR23.java

@@ -1,480 +0,0 @@
-/**
- * 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.FileNotFoundException;
-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.fs.ContentSummary;
-import org.apache.hadoop.fs.CreateFlag;
-import org.apache.hadoop.fs.FileAlreadyExistsException;
-import org.apache.hadoop.fs.FsServerDefaults;
-import org.apache.hadoop.fs.ParentNotDirectoryException;
-import org.apache.hadoop.fs.UnresolvedLinkException;
-import org.apache.hadoop.fs.Options.Rename;
-import org.apache.hadoop.fs.permission.FsPermission;
-import org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException;
-import org.apache.hadoop.hdfs.protocol.ClientProtocol;
-import org.apache.hadoop.hdfs.protocol.LocatedBlock;
-import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
-import org.apache.hadoop.hdfs.protocol.CorruptFileBlocks;
-import org.apache.hadoop.hdfs.protocol.DSQuotaExceededException;
-import org.apache.hadoop.hdfs.protocol.DatanodeID;
-import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
-import org.apache.hadoop.hdfs.protocol.DirectoryListing;
-import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
-import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
-import org.apache.hadoop.hdfs.protocol.NSQuotaExceededException;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants.UpgradeAction;
-import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
-import org.apache.hadoop.hdfs.server.common.UpgradeStatusReport;
-import org.apache.hadoop.hdfs.server.namenode.NameNode;
-import org.apache.hadoop.hdfs.server.namenode.NotReplicatedYetException;
-import org.apache.hadoop.hdfs.server.namenode.SafeModeException;
-import org.apache.hadoop.io.EnumSetWritable;
-import org.apache.hadoop.io.Text;
-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.AccessControlException;
-import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.security.token.Token;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
-
-/**
- * This class forwards NN's ClientProtocol calls as RPC calls to the NN server
- * while translating from the parameter types used in ClientProtocol to those
- * used in protocolR23Compatile.*.
- */
-@InterfaceAudience.Private
-@InterfaceStability.Stable
-public class ClientNamenodeProtocolTranslatorR23 implements
-    ClientProtocol, Closeable {
-  final private ClientNamenodeWireProtocol rpcProxy;
-
-  private static ClientNamenodeWireProtocol createNamenode(
-      InetSocketAddress nameNodeAddr, Configuration conf,
-      UserGroupInformation ugi) throws IOException {
-    return RPC.getProxy(ClientNamenodeWireProtocol.class,
-        ClientNamenodeWireProtocol.versionID, nameNodeAddr, ugi, conf,
-        NetUtils.getSocketFactory(conf, ClientNamenodeWireProtocol.class));
-  }
-
-  /** Create a {@link NameNode} proxy */
-  static ClientNamenodeWireProtocol createNamenodeWithRetry(
-      ClientNamenodeWireProtocol 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 (ClientNamenodeWireProtocol) RetryProxy.create(
-        ClientNamenodeWireProtocol.class, rpcNamenode, methodNameToPolicyMap);
-  }
-
-  public ClientNamenodeProtocolTranslatorR23(InetSocketAddress nameNodeAddr,
-      Configuration conf, UserGroupInformation ugi) throws IOException {
-    rpcProxy = createNamenodeWithRetry(createNamenode(nameNodeAddr, conf, ugi));
-  }
-
-  public void close() {
-    RPC.stopProxy(rpcProxy);
-  }
-
-  @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 LocatedBlocks getBlockLocations(String src, long offset, long length)
-      throws AccessControlException, FileNotFoundException,
-      UnresolvedLinkException, IOException {
-    return LocatedBlocksWritable
-        .convertLocatedBlocks(rpcProxy.getBlockLocations(src, offset, length));
-  }
-
-  @Override
-  public FsServerDefaults getServerDefaults() throws IOException {
-    return FsServerDefaultsWritable
-        .convert(rpcProxy.getServerDefaults());
-  }
-
-  @Override
-  public void create(String src, FsPermission masked, String clientName,
-      EnumSetWritable<CreateFlag> flag, boolean createParent,
-      short replication, long blockSize) throws AccessControlException,
-      AlreadyBeingCreatedException, DSQuotaExceededException,
-      FileAlreadyExistsException, FileNotFoundException,
-      NSQuotaExceededException, ParentNotDirectoryException, SafeModeException,
-      UnresolvedLinkException, IOException {
-    rpcProxy.create(src, FsPermissionWritable.convertPermission(masked),
-        clientName, flag, createParent, replication, blockSize);
-
-  }
-
-  @Override
-  public LocatedBlock append(String src, String clientName)
-      throws AccessControlException, DSQuotaExceededException,
-      FileNotFoundException, SafeModeException, UnresolvedLinkException,
-      IOException {
-    return LocatedBlockWritable
-        .convertLocatedBlock(rpcProxy.append(src, clientName));
-  }
-
-  @Override
-  public boolean setReplication(String src, short replication)
-      throws AccessControlException, DSQuotaExceededException,
-      FileNotFoundException, SafeModeException, UnresolvedLinkException,
-      IOException {
-    return rpcProxy.setReplication(src, replication);
-  }
-
-  @Override
-  public void setPermission(String src, FsPermission permission)
-      throws AccessControlException, FileNotFoundException, SafeModeException,
-      UnresolvedLinkException, IOException {
-    rpcProxy.setPermission(src,
-        FsPermissionWritable.convertPermission(permission));
-
-  }
-
-  @Override
-  public void setOwner(String src, String username, String groupname)
-      throws AccessControlException, FileNotFoundException, SafeModeException,
-      UnresolvedLinkException, IOException {
-    rpcProxy.setOwner(src, username, groupname);
-
-  }
-
-  @Override
-  public void abandonBlock(ExtendedBlock b, String src, String holder)
-      throws AccessControlException, FileNotFoundException,
-      UnresolvedLinkException, IOException {
-    rpcProxy.abandonBlock(
-        ExtendedBlockWritable.convertExtendedBlock(b), src, holder);
-
-  }
-
-  @Override
-  public LocatedBlock addBlock(String src, String clientName,
-      ExtendedBlock previous, DatanodeInfo[] excludeNodes)
-      throws AccessControlException, FileNotFoundException,
-      NotReplicatedYetException, SafeModeException, UnresolvedLinkException,
-      IOException {
-    return LocatedBlockWritable
-        .convertLocatedBlock(rpcProxy.addBlock(src, clientName,
-            ExtendedBlockWritable.convertExtendedBlock(previous),
-            DatanodeInfoWritable.convertDatanodeInfo(excludeNodes)));
-  }
-
-  @Override
-  public LocatedBlock getAdditionalDatanode(String src, ExtendedBlock blk,
-      DatanodeInfo[] existings, DatanodeInfo[] excludes,
-      int numAdditionalNodes, String clientName) throws AccessControlException,
-      FileNotFoundException, SafeModeException, UnresolvedLinkException,
-      IOException {
-    return LocatedBlockWritable
-        .convertLocatedBlock(rpcProxy.getAdditionalDatanode(src,
-            ExtendedBlockWritable.convertExtendedBlock(blk),
-            DatanodeInfoWritable.convertDatanodeInfo(existings),
-            DatanodeInfoWritable.convertDatanodeInfo(excludes),
-            numAdditionalNodes, clientName));
-  }
-
-  @Override
-  public boolean complete(String src, String clientName, ExtendedBlock last)
-      throws AccessControlException, FileNotFoundException, SafeModeException,
-      UnresolvedLinkException, IOException {
-    return rpcProxy.complete(src, clientName,
-        ExtendedBlockWritable.convertExtendedBlock(last));
-  }
-
-  @Override
-  public void reportBadBlocks(LocatedBlock[] blocks) throws IOException {
-    rpcProxy.reportBadBlocks(LocatedBlockWritable.convertLocatedBlock(blocks));
-
-  }
-
-  @Override
-  public boolean rename(String src, String dst) throws UnresolvedLinkException,
-      IOException {
-    return rpcProxy.rename(src, dst);
-  }
-
-  @Override
-  public void concat(String trg, String[] srcs) throws IOException,
-      UnresolvedLinkException {
-    rpcProxy.concat(trg, srcs);
-
-  }
-
-  @Override
-  public void rename2(String src, String dst, Rename... options)
-      throws AccessControlException, DSQuotaExceededException,
-      FileAlreadyExistsException, FileNotFoundException,
-      NSQuotaExceededException, ParentNotDirectoryException, SafeModeException,
-      UnresolvedLinkException, IOException {
-    rpcProxy.rename2(src, dst, options);
-
-  }
-
-  @Override
-  public boolean delete(String src, boolean recursive)
-      throws AccessControlException, FileNotFoundException, SafeModeException,
-      UnresolvedLinkException, IOException {
-    return rpcProxy.delete(src, recursive);
-  }
-
-  @Override
-  public boolean mkdirs(String src, FsPermission masked, boolean createParent)
-      throws AccessControlException, FileAlreadyExistsException,
-      FileNotFoundException, NSQuotaExceededException,
-      ParentNotDirectoryException, SafeModeException, UnresolvedLinkException,
-      IOException {
-
-    return rpcProxy.mkdirs(src,
-        FsPermissionWritable.convertPermission(masked), createParent);
-  }
-
-  @Override
-  public DirectoryListing getListing(String src, byte[] startAfter,
-      boolean needLocation) throws AccessControlException,
-      FileNotFoundException, UnresolvedLinkException, IOException {
-    return DirectoryListingWritable.convertDirectoryListing(
-        rpcProxy.getListing(src, startAfter, needLocation));
-  }
-
-  @Override
-  public void renewLease(String clientName) throws AccessControlException,
-      IOException {
-    rpcProxy.renewLease(clientName);
-
-  }
-
-  @Override
-  public boolean recoverLease(String src, String clientName) throws IOException {
-    return rpcProxy.recoverLease(src, clientName);
-  }
-
-  @Override
-  public long[] getStats() throws IOException {
-    return rpcProxy.getStats();
-  }
-
-  @Override
-  public DatanodeInfo[] getDatanodeReport(DatanodeReportType type)
-      throws IOException {
-    return DatanodeInfoWritable.convertDatanodeInfo(
-        rpcProxy.getDatanodeReport(type));
-  }
-
-  @Override
-  public long getPreferredBlockSize(String filename) throws IOException,
-      UnresolvedLinkException {
-    return rpcProxy.getPreferredBlockSize(filename);
-  }
-
-  @Override
-  public boolean setSafeMode(SafeModeAction action) throws IOException {
-    return rpcProxy.setSafeMode(action);
-  }
-
-  @Override
-  public void saveNamespace() throws AccessControlException, IOException {
-    rpcProxy.saveNamespace();
-
-  }
-
-  @Override
-  public boolean restoreFailedStorage(String arg)
-      throws AccessControlException, IOException{
-    return rpcProxy.restoreFailedStorage(arg);
-  }
-
-  @Override
-  public void refreshNodes() throws IOException {
-    rpcProxy.refreshNodes();
-
-  }
-
-  @Override
-  public void finalizeUpgrade() throws IOException {
-    rpcProxy.finalizeUpgrade();
-
-  }
-
-  @Override
-  public UpgradeStatusReport distributedUpgradeProgress(UpgradeAction action)
-      throws IOException {
-    return UpgradeStatusReportWritable.convert(
-        rpcProxy.distributedUpgradeProgress(action));
-  }
-
-  @Override
-  public CorruptFileBlocks listCorruptFileBlocks(String path, String cookie)
-      throws IOException {
-    return CorruptFileBlocksWritable.convertCorruptFileBlocks(
-        rpcProxy.listCorruptFileBlocks(path, cookie));
-  }
-
-  @Override
-  public void metaSave(String filename) throws IOException {
-    rpcProxy.metaSave(filename);
-
-  }
-
-  @Override
-  public HdfsFileStatus getFileInfo(String src) throws AccessControlException,
-      FileNotFoundException, UnresolvedLinkException, IOException {
-    return HdfsFileStatusWritable.convertHdfsFileStatus(
-        rpcProxy.getFileInfo(src));
-  }
-
-  @Override
-  public HdfsFileStatus getFileLinkInfo(String src)
-      throws AccessControlException, UnresolvedLinkException, IOException {
-    return HdfsFileStatusWritable
-        .convertHdfsFileStatus(rpcProxy.getFileLinkInfo(src));
-  }
-
-  @Override
-  public ContentSummary getContentSummary(String path)
-      throws AccessControlException, FileNotFoundException,
-      UnresolvedLinkException, IOException {
-    return ContentSummaryWritable
-        .convert(rpcProxy.getContentSummary(path));
-  }
-
-  @Override
-  public void setQuota(String path, long namespaceQuota, long diskspaceQuota)
-      throws AccessControlException, FileNotFoundException,
-      UnresolvedLinkException, IOException {
-    rpcProxy.setQuota(path, namespaceQuota, diskspaceQuota);
-
-  }
-
-  @Override
-  public void fsync(String src, String client) throws AccessControlException,
-      FileNotFoundException, UnresolvedLinkException, IOException {
-    rpcProxy.fsync(src, client);
-
-  }
-
-  @Override
-  public void setTimes(String src, long mtime, long atime)
-      throws AccessControlException, FileNotFoundException,
-      UnresolvedLinkException, IOException {
-    rpcProxy.setTimes(src, mtime, atime);
-
-  }
-
-  @Override
-  public void createSymlink(String target, String link, FsPermission dirPerm,
-      boolean createParent) throws AccessControlException,
-      FileAlreadyExistsException, FileNotFoundException,
-      ParentNotDirectoryException, SafeModeException, UnresolvedLinkException,
-      IOException {
-    rpcProxy.createSymlink(target, link,
-        FsPermissionWritable.convertPermission(dirPerm), createParent);
-
-  }
-
-  @Override
-  public String getLinkTarget(String path) throws AccessControlException,
-      FileNotFoundException, IOException {
-    return rpcProxy.getLinkTarget(path);
-  }
-
-  @Override
-  public LocatedBlock updateBlockForPipeline(ExtendedBlock block,
-      String clientName) throws IOException {
-    return LocatedBlockWritable.convertLocatedBlock(
-        rpcProxy.updateBlockForPipeline(
-            ExtendedBlockWritable.convertExtendedBlock(block), clientName));
-  }
-
-  @Override
-  public void updatePipeline(String clientName, ExtendedBlock oldBlock,
-      ExtendedBlock newBlock, DatanodeID[] newNodes) throws IOException {
-    rpcProxy.updatePipeline(clientName,
-        ExtendedBlockWritable.convertExtendedBlock(oldBlock),
-        ExtendedBlockWritable.convertExtendedBlock(newBlock),
-        DatanodeIDWritable.convertDatanodeID(newNodes));
-
-  }
-
-  @Override
-  public Token<DelegationTokenIdentifier> getDelegationToken(Text renewer)
-      throws IOException {
-    return rpcProxy.getDelegationToken(renewer);
-  }
-
-  @Override
-  public long renewDelegationToken(Token<DelegationTokenIdentifier> token)
-      throws IOException {
-    return rpcProxy.renewDelegationToken(token);
-  }
-
-  @Override
-  public void cancelDelegationToken(Token<DelegationTokenIdentifier> token)
-      throws IOException {
-    rpcProxy.cancelDelegationToken(token);
-  }
-
-  @Override
-  public void setBalancerBandwidth(long bandwidth) throws IOException {
-    rpcProxy.setBalancerBandwidth(bandwidth);
-  }
-}

+ 0 - 478
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/ClientNamenodeWireProtocol.java

@@ -1,478 +0,0 @@
-/**
- * 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.FileNotFoundException;
-import java.io.IOException;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.fs.CreateFlag;
-import org.apache.hadoop.fs.Options;
-import org.apache.hadoop.fs.FileAlreadyExistsException;
-import org.apache.hadoop.fs.ParentNotDirectoryException;
-import org.apache.hadoop.fs.UnresolvedLinkException;
-import org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException;
-import org.apache.hadoop.hdfs.protocol.DSQuotaExceededException;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
-import org.apache.hadoop.hdfs.DFSConfigKeys;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants.UpgradeAction;
-import org.apache.hadoop.hdfs.server.namenode.NotReplicatedYetException;
-import org.apache.hadoop.hdfs.server.namenode.SafeModeException;
-import org.apache.hadoop.io.EnumSetWritable;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.ipc.VersionedProtocol;
-import org.apache.hadoop.security.AccessControlException;
-import org.apache.hadoop.security.KerberosInfo;
-import org.apache.hadoop.security.token.Token;
-import org.apache.hadoop.security.token.TokenInfo;
-import org.apache.hadoop.hdfs.protocol.NSQuotaExceededException;
-import org.apache.hadoop.ipc.ProtocolInfo;
-
-import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
-import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSelector;
-
-/**********************************************************************
- * This class defines the actual protocol used to communicate with the
- * NN via RPC using writable types.
- * The parameters in the methods which are specified in the
- * package are separate from those used internally in the NN and DFSClient
- * and hence need to be converted using {@link ClientNamenodeProtocolTranslatorR23}
- * and {@link ClientNamenodeProtocolServerSideTranslatorR23}.
- *
- **********************************************************************/
-@InterfaceAudience.Private
-@InterfaceStability.Stable
-@KerberosInfo(
-    serverPrincipal = DFSConfigKeys.DFS_NAMENODE_USER_NAME_KEY)
-@TokenInfo(DelegationTokenSelector.class)
-@ProtocolInfo(protocolName = HdfsConstants.CLIENT_NAMENODE_PROTOCOL_NAME)
-public interface ClientNamenodeWireProtocol extends VersionedProtocol {
-
-  /**
-   * Changes to the protocol:
-   * 
-   * Do NOT change a method's signature (ie name, parameters, parameter types
-   * or exceptions thrown). If you need to make changes then ADD new methods and
-   * new data types.
-   * Hence if you maintain compatibility you will NOT have to change
-   * the version number below. The version number is changed ONLY
-   * if you break compatibility (which is a big deal).
-   * Hence the version number is really a Major Version Number.
-   *
-   * The log of historical changes prior to 69 can be retrieved from the svn.
-   * ALL changes since version 69L are recorded.
-   * Version number is changed ONLY for Incompatible changes.
-   *  (note previously we used to change version number for both
-   *  compatible and incompatible changes).
-   * 69: Eliminate overloaded method names. (Compatible)
-   * 70: Separation of Datatypes - the client namenode protocol is implemented
-   *     in this class instead of in 
-   *           {@link org.apache.hadoop.hdfs.protocol.ClientProtocol}
-   *     as was done prior to version 70.
-   */
-  public static final long versionID = 70L;
-  
-  ///////////////////////////////////////
-  // File contents
-  ///////////////////////////////////////
-  /**
-   * The specification of this method matches that of
-   * {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#getBlockLocations}
-   */
-  public LocatedBlocksWritable getBlockLocations(String src,
-                                         long offset,
-                                         long length) 
-      throws AccessControlException, FileNotFoundException,
-      UnresolvedLinkException, IOException;
-
-  /**
-   * The specification of this method matches that of
-   * {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#getServerDefaults()}
-   */
-  public FsServerDefaultsWritable getServerDefaults() throws IOException;
-
-  /**
-   * The specification of this method matches that of
-   * {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#create(String, 
-   * org.apache.hadoop.fs.permission.FsPermission, String, 
-   * EnumSetWritable, boolean, short, long)}
-   */
-  public void create(String src, FsPermissionWritable masked, String clientName,
-      EnumSetWritable<CreateFlag> flag, boolean createParent,
-      short replication, long blockSize) throws AccessControlException,
-      AlreadyBeingCreatedException, DSQuotaExceededException,
-      FileAlreadyExistsException, FileNotFoundException,
-      NSQuotaExceededException, ParentNotDirectoryException, SafeModeException,
-      UnresolvedLinkException, IOException;
-
-  /**
-   * The specification of this method matches that of
-   * {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#append(String, String)}
-   */
-  public LocatedBlockWritable append(String src, String clientName)
-      throws AccessControlException, DSQuotaExceededException,
-      FileNotFoundException, SafeModeException, UnresolvedLinkException,
-      IOException;
-
-  /**
-   * The specification of this method matches that of
-   * {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#setReplication(String, short)}
-   */
-  public boolean setReplication(String src, short replication)
-      throws AccessControlException, DSQuotaExceededException,
-      FileNotFoundException, SafeModeException, UnresolvedLinkException,
-      IOException;
-
-  /**
-   * The specification of this method matches that of
-   * {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#setPermission(String,
-   * org.apache.hadoop.fs.permission.FsPermission)}
-   */
-  public void setPermission(String src, FsPermissionWritable permission)
-      throws AccessControlException, FileNotFoundException, SafeModeException,
-      UnresolvedLinkException, IOException;
-
-  /**
-   * The specification of this method matches that of
-   * {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#setOwner(String, String, String)}
-   */
-  public void setOwner(String src, String username, String groupname)
-      throws AccessControlException, FileNotFoundException, SafeModeException,
-      UnresolvedLinkException, IOException;
-
-  /**
-   * The specification of this method matches that of
-   * {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#abandonBlock(
-   * org.apache.hadoop.hdfs.protocol.ExtendedBlock, String, String)}
-   */
-  public void abandonBlock(ExtendedBlockWritable b, String src, String holder)
-      throws AccessControlException, FileNotFoundException,
-      UnresolvedLinkException, IOException;
-
-  /**
-   * The specification of this method matches that of
-   * {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#addBlock(String, 
-   * String, org.apache.hadoop.hdfs.protocol.ExtendedBlock, 
-   * org.apache.hadoop.hdfs.protocol.DatanodeInfo[])}
-   */
-  public LocatedBlockWritable addBlock(String src, String clientName,
-      ExtendedBlockWritable previous, DatanodeInfoWritable[] excludeNodes)
-      throws AccessControlException, FileNotFoundException,
-      NotReplicatedYetException, SafeModeException, UnresolvedLinkException,
-      IOException;
-
-  /**
-   * The specification of this method matches that of
-   * {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#getAdditionalDatanode}
-   */
-  public LocatedBlockWritable getAdditionalDatanode(
-      final String src, final ExtendedBlockWritable blk,
-      final DatanodeInfoWritable[] existings,
-      final DatanodeInfoWritable[] excludes,
-      final int numAdditionalNodes, final String clientName
-      ) throws AccessControlException, FileNotFoundException,
-          SafeModeException, UnresolvedLinkException, IOException;
-
-  /**
-   * The specification of this method matches that of
-   * {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#complete}
-   */
-  public boolean complete(
-      String src, String clientName, ExtendedBlockWritable last)
-      throws AccessControlException, FileNotFoundException, SafeModeException,
-      UnresolvedLinkException, IOException;
-
-  /**
-   * The specification of this method matches that of
-   * {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#reportBadBlocks}
-   */
-  public void reportBadBlocks(LocatedBlockWritable[] blocks) throws IOException;
-
-  ///////////////////////////////////////
-  // Namespace management
-  ///////////////////////////////////////
-  /**
-   * The specification of this method matches that of
-   * {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#rename(String, String)}
-   */
-  public boolean rename(String src, String dst) 
-      throws UnresolvedLinkException, IOException;
-
-  /**
-   * The specification of this method matches that of
-   * {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#concat(String, String[])}
-   */
-  public void concat(String trg, String[] srcs) 
-      throws IOException, UnresolvedLinkException;
-
-  /**
-   * The specification of this method matches that of
-   * {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#rename2}
-   */
-  public void rename2(String src, String dst, Options.Rename... options)
-      throws AccessControlException, DSQuotaExceededException,
-      FileAlreadyExistsException, FileNotFoundException,
-      NSQuotaExceededException, ParentNotDirectoryException, SafeModeException,
-      UnresolvedLinkException, IOException;
-
-  /**
-   * The specification of this method matches that of
-   * {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#delete(String, boolean)}
-   */
-  public boolean delete(String src, boolean recursive)
-      throws AccessControlException, FileNotFoundException, SafeModeException,
-      UnresolvedLinkException, IOException;
-  
-  /**
-   * The specification of this method matches that of
-   * {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#mkdirs}
-   */
-  public boolean mkdirs(
-      String src, FsPermissionWritable masked, boolean createParent)
-      throws AccessControlException, FileAlreadyExistsException,
-      FileNotFoundException, NSQuotaExceededException,
-      ParentNotDirectoryException, SafeModeException, UnresolvedLinkException,
-      IOException;
-
-  /**
-   * The specification of this method matches that of
-   * {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#getListing}
-   */
-  public DirectoryListingWritable getListing(String src,
-                                     byte[] startAfter,
-                                     boolean needLocation)
-      throws AccessControlException, FileNotFoundException,
-      UnresolvedLinkException, IOException;
-
-  ///////////////////////////////////////
-  // System issues and management
-  ///////////////////////////////////////
-
-  /**
-   * The specification of this method matches that of
-   * {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#renewLease(String)}
-   */
-  public void renewLease(String clientName) throws AccessControlException,
-      IOException;
-
-  /**
-   * The specification of this method matches that of
-   * {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#recoverLease(String, String)}
-   */
-  public boolean recoverLease(String src, String clientName) throws IOException;
-
-  public int GET_STATS_CAPACITY_IDX = 0;
-  public int GET_STATS_USED_IDX = 1;
-  public int GET_STATS_REMAINING_IDX = 2;
-  public int GET_STATS_UNDER_REPLICATED_IDX = 3;
-  public int GET_STATS_CORRUPT_BLOCKS_IDX = 4;
-  public int GET_STATS_MISSING_BLOCKS_IDX = 5;
-  
-  /**
-   * The specification of this method matches that of
-   * {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#getStats()}
-   */
-  public long[] getStats() throws IOException;
-
-  /**
-   * The specification of this method matches that of
-   * {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#getDatanodeReport}
-   */
-  public DatanodeInfoWritable[] getDatanodeReport(
-      HdfsConstants.DatanodeReportType type)
-      throws IOException;
-
-  /**
-   * The specification of this method matches that of
-   * {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#getPreferredBlockSize}
-   */
-  public long getPreferredBlockSize(String filename) 
-      throws IOException, UnresolvedLinkException;
-
-  /**
-   * The specification of this method matches that of
-   * {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#setSafeMode(org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction)}
-   */
-  public boolean setSafeMode(HdfsConstants.SafeModeAction action) 
-      throws IOException;
-
-  /**
-   * The specification of this method matches that of
-   * {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#saveNamespace()}
-   */
-  public void saveNamespace() throws AccessControlException, IOException;
-
-  /**
-   * The specification of this method matches that of
-   * {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#restoreFailedStorage(String)}
-   */
-  public boolean restoreFailedStorage(String arg) 
-      throws AccessControlException, IOException;
-
-  /**
-   * The specification of this method matches that of
-   * {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#refreshNodes()}
-   */
-  public void refreshNodes() throws IOException;
-
-  /**
-   * The specification of this method matches that of
-   * {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#finalizeUpgrade()}
-   */
-  public void finalizeUpgrade() throws IOException;
-
-  /**
-   * The specification of this method matches that of
-   * {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#distributedUpgradeProgress}
-   */
-  public UpgradeStatusReportWritable distributedUpgradeProgress(
-      UpgradeAction action) 
-      throws IOException;
-
-  /**
-   * The specification of this method matches that of
-   * {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#listCorruptFileBlocks(String, String)}
-   */
-  public CorruptFileBlocksWritable
-    listCorruptFileBlocks(String path, String cookie)
-    throws IOException;
-  
-  /**
-   * The specification of this method matches that of
-   * {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#metaSave(String)}
-   */
-  public void metaSave(String filename) throws IOException;
-  
-  /**
-   * The specification of this method matches that of
-   * {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#setBalancerBandwidth(long)}
-   */
-  public void setBalancerBandwidth(long bandwidth) throws IOException;
-  
-  /**
-   * The specification of this method matches that of
-   * {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#getFileInfo(String)}
-   */
-  public HdfsFileStatusWritable getFileInfo(String src)
-      throws AccessControlException,
-      FileNotFoundException, UnresolvedLinkException, IOException;
-
-  /**
-   * The specification of this method matches that of
-   * {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#getFileLinkInfo(String)}
-   */
-  public HdfsFileStatusWritable getFileLinkInfo(String src)
-      throws AccessControlException, UnresolvedLinkException, IOException;
-  
-  /**
-   * The specification of this method matches that of
-   * {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#getContentSummary(String)}
-   */
-  public ContentSummaryWritable getContentSummary(String path)
-      throws AccessControlException, FileNotFoundException,
-      UnresolvedLinkException, IOException;
-
-  /**
-   * The specification of this method matches that of
-   * {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#setQuota(String, long, long)}
-   */
-  public void setQuota(String path, long namespaceQuota, long diskspaceQuota)
-      throws AccessControlException, FileNotFoundException,
-      UnresolvedLinkException, IOException;
-
-  /**
-   * The specification of this method matches that of
-   * {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#fsync(String, String)}
-   */
-  public void fsync(String src, String client) 
-      throws AccessControlException, FileNotFoundException, 
-      UnresolvedLinkException, IOException;
-
-  /**
-   * The specification of this method matches that of
-   * {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#setTimes(String, long, long)}
-   */
-  public void setTimes(String src, long mtime, long atime)
-      throws AccessControlException, FileNotFoundException, 
-      UnresolvedLinkException, IOException;
-
-  /**
-   * The specification of this method matches that of
-   * {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#createSymlink}
-   */
-  public void createSymlink(
-      String target, String link, FsPermissionWritable dirPerm,
-      boolean createParent) throws AccessControlException,
-      FileAlreadyExistsException, FileNotFoundException,
-      ParentNotDirectoryException, SafeModeException, UnresolvedLinkException,
-      IOException;
-
-  /**
-   * The specification of this method matches that of
-   * {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#getLinkTarget(String)}
-   */
-  public String getLinkTarget(String path) throws AccessControlException,
-      FileNotFoundException, IOException; 
-  
-  /**
-   * The specification of this method matches that of
-   * {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#updateBlockForPipeline}
-   */
-  public LocatedBlockWritable updateBlockForPipeline(
-      ExtendedBlockWritable block, String clientName) throws IOException;
-
-  /**
-   * The specification of this method matches that of
-   * {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#updatePipeline}
-   */
-  public void updatePipeline(String clientName, ExtendedBlockWritable oldBlock, 
-      ExtendedBlockWritable newBlock, DatanodeIDWritable[] newNodes)
-      throws IOException;
-
-  /**
-   * The specification of this method matches that of
-   * {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#getDelegationToken(Text)}
-   */
-  public Token<DelegationTokenIdentifier> getDelegationToken(Text renewer) 
-      throws IOException;
-
-  /**
-   * The specification of this method matches that of
-   * {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#renewDelegationToken(Token)}
-   */
-  public long renewDelegationToken(Token<DelegationTokenIdentifier> token)
-      throws IOException;
-  
-  /**
-   * The specification of this method matches that of
-   * {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#cancelDelegationToken(Token)}
-   */
-  public void cancelDelegationToken(Token<DelegationTokenIdentifier> token)
-      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;
-}

+ 0 - 184
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/ContentSummaryWritable.java

@@ -1,184 +0,0 @@
-/**
- * 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.io.Writable;
-
-/** Store the summary of a content (a directory or a file). */
-@InterfaceAudience.Public
-@InterfaceStability.Stable
-public class ContentSummaryWritable implements Writable{
-  private long length;
-  private long fileCount;
-  private long directoryCount;
-  private long quota;
-  private long spaceConsumed;
-  private long spaceQuota;
-  
-  
-  public static org.apache.hadoop.fs.ContentSummary convert(ContentSummaryWritable cs) {
-    if (cs == null) return null;
-    return new org.apache.hadoop.fs.ContentSummary(
-      cs.getLength(), cs.getFileCount(), cs.getDirectoryCount(), cs.getQuota(),
-      cs.getSpaceConsumed(), cs.getSpaceQuota());
-  }
-  
-  public static ContentSummaryWritable convert(org.apache.hadoop.fs.ContentSummary cs) {
-    if (cs == null) return null;
-    return new  ContentSummaryWritable(
-      cs.getLength(), cs.getFileCount(), cs.getDirectoryCount(), cs.getQuota(),
-      cs.getSpaceConsumed(), cs.getSpaceQuota());
-  }
-
-  /** Constructor */
-  public ContentSummaryWritable() {}
-  
-  /** Constructor */
-  public ContentSummaryWritable(long length, long fileCount, long directoryCount) {
-    this(length, fileCount, directoryCount, -1L, length, -1L);
-  }
-
-  /** Constructor */
-  public ContentSummaryWritable(
-      long length, long fileCount, long directoryCount, long quota,
-      long spaceConsumed, long spaceQuota) {
-    this.length = length;
-    this.fileCount = fileCount;
-    this.directoryCount = directoryCount;
-    this.quota = quota;
-    this.spaceConsumed = spaceConsumed;
-    this.spaceQuota = spaceQuota;
-  }
-
-  /** @return the length */
-  public long getLength() {return length;}
-
-  /** @return the directory count */
-  public long getDirectoryCount() {return directoryCount;}
-
-  /** @return the file count */
-  public long getFileCount() {return fileCount;}
-  
-  /** Return the directory quota */
-  public long getQuota() {return quota;}
-  
-  /** Retuns (disk) space consumed */ 
-  public long getSpaceConsumed() {return spaceConsumed;}
-
-  /** Returns (disk) space quota */
-  public long getSpaceQuota() {return spaceQuota;}
-  
-  @InterfaceAudience.Private
-  @Override
-  public void write(DataOutput out) throws IOException {
-    out.writeLong(length);
-    out.writeLong(fileCount);
-    out.writeLong(directoryCount);
-    out.writeLong(quota);
-    out.writeLong(spaceConsumed);
-    out.writeLong(spaceQuota);
-  }
-
-  @InterfaceAudience.Private
-  @Override
-  public void readFields(DataInput in) throws IOException {
-    this.length = in.readLong();
-    this.fileCount = in.readLong();
-    this.directoryCount = in.readLong();
-    this.quota = in.readLong();
-    this.spaceConsumed = in.readLong();
-    this.spaceQuota = in.readLong();
-  }
-  
-  /** 
-   * Output format:
-   * <----12----> <----12----> <-------18------->
-   *    DIR_COUNT   FILE_COUNT       CONTENT_SIZE FILE_NAME    
-   */
-  private static final String STRING_FORMAT = "%12d %12d %18d ";
-  /** 
-   * Output format:
-   * <----12----> <----15----> <----15----> <----15----> <----12----> <----12----> <-------18------->
-   *    QUOTA   REMAINING_QUATA SPACE_QUOTA SPACE_QUOTA_REM DIR_COUNT   FILE_COUNT   CONTENT_SIZE     FILE_NAME    
-   */
-  private static final String QUOTA_STRING_FORMAT = "%12s %15s ";
-  private static final String SPACE_QUOTA_STRING_FORMAT = "%15s %15s ";
-  
-  /** The header string */
-  private static final String HEADER = String.format(
-      STRING_FORMAT.replace('d', 's'), "directories", "files", "bytes");
-
-  private static final String QUOTA_HEADER = String.format(
-      QUOTA_STRING_FORMAT + SPACE_QUOTA_STRING_FORMAT, 
-      "quota", "remaining quota", "space quota", "reamaining quota") +
-      HEADER;
-  
-  /** Return the header of the output.
-   * if qOption is false, output directory count, file count, and content size;
-   * if qOption is true, output quota and remaining quota as well.
-   * 
-   * @param qOption a flag indicating if quota needs to be printed or not
-   * @return the header of the output
-   */
-  public static String getHeader(boolean qOption) {
-    return qOption ? QUOTA_HEADER : HEADER;
-  }
-  
-  @Override
-  public String toString() {
-    return toString(true);
-  }
-
-  /** Return the string representation of the object in the output format.
-   * if qOption is false, output directory count, file count, and content size;
-   * if qOption is true, output quota and remaining quota as well.
-   * 
-   * @param qOption a flag indicating if quota needs to be printed or not
-   * @return the string representation of the object
-   */
-  public String toString(boolean qOption) {
-    String prefix = "";
-    if (qOption) {
-      String quotaStr = "none";
-      String quotaRem = "inf";
-      String spaceQuotaStr = "none";
-      String spaceQuotaRem = "inf";
-      
-      if (quota>0) {
-        quotaStr = Long.toString(quota);
-        quotaRem = Long.toString(quota-(directoryCount+fileCount));
-      }
-      if (spaceQuota>0) {
-        spaceQuotaStr = Long.toString(spaceQuota);
-        spaceQuotaRem = Long.toString(spaceQuota - spaceConsumed);        
-      }
-      
-      prefix = String.format(QUOTA_STRING_FORMAT + SPACE_QUOTA_STRING_FORMAT, 
-                             quotaStr, quotaRem, spaceQuotaStr, spaceQuotaRem);
-    }
-    
-    return prefix + String.format(STRING_FORMAT, directoryCount, 
-                                  fileCount, length);
-  }
-}

+ 0 - 88
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/CorruptFileBlocksWritable.java

@@ -1,88 +0,0 @@
-/**
- * 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 org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.io.Text;
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-
-/**
- * Contains a list of paths corresponding to corrupt files and a cookie
- * used for iterative calls to NameNode.listCorruptFileBlocks.
- *
- */
-@InterfaceAudience.Private
-@InterfaceStability.Stable
-public class CorruptFileBlocksWritable implements Writable {
-
-  private String[] files;
-  private String cookie;
-
-  static public org.apache.hadoop.hdfs.protocol.CorruptFileBlocks 
-    convertCorruptFileBlocks(CorruptFileBlocksWritable c) {
-    if (c == null) return null;
-    return new org.apache.hadoop.hdfs.protocol.CorruptFileBlocks(
-        c.getFiles(), c.getCookie());
-  }
-  
-  public static CorruptFileBlocksWritable convertCorruptFilesBlocks(
-      org.apache.hadoop.hdfs.protocol.CorruptFileBlocks c) {
-    if (c == null) return null;
-    return new CorruptFileBlocksWritable(c.getFiles(), c.getCookie());
-  }
- 
-  public CorruptFileBlocksWritable() {
-    this(new String[0], "");
-  }
-
-  public CorruptFileBlocksWritable(String[] files, String cookie) {
-    this.files = files;
-    this.cookie = cookie;
-  }
- 
-  public String[] getFiles() {
-    return files;
-  }
-
-  public String getCookie() {
-    return cookie;
-  }
-
-  @Override
-  public void readFields(DataInput in) throws IOException {
-    int fileCount = in.readInt();
-    files = new String[fileCount];
-    for (int i = 0; i < fileCount; i++) {
-      files[i] = Text.readString(in);
-    }
-    cookie = Text.readString(in);
-  }
-
-  @Override
-  public void write(DataOutput out) throws IOException {
-    out.writeInt(files.length);
-    for (int i = 0; i < files.length; i++) {
-      Text.writeString(out, files[i]);
-    }
-    Text.writeString(out, cookie);
-  }
-}

+ 0 - 209
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/DatanodeIDWritable.java

@@ -1,209 +0,0 @@
-/**
- * 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.io.Writable;
-
-/**
- * DatanodeID is composed of the data node 
- * name (hostname:portNumber) and the data storage ID, 
- * which it currently represents.
- * 
- */
-@InterfaceAudience.Private
-@InterfaceStability.Stable
-public class DatanodeIDWritable implements Writable {
-  public static final DatanodeIDWritable[] EMPTY_ARRAY = {}; 
-
-  public String name;      /// hostname:portNumber
-  public String storageID; /// unique per cluster storageID
-  protected int infoPort;     /// the port where the infoserver is running
-  public int ipcPort;     /// the port where the ipc server is running
-
-  
-  static public DatanodeIDWritable[] 
-      convertDatanodeID(org.apache.hadoop.hdfs.protocol.DatanodeID[] did) {
-    if (did == null) return null;
-    final int len = did.length;
-    DatanodeIDWritable[] result = new DatanodeIDWritable[len];
-    for (int i = 0; i < len; ++i) {
-      result[i] = convertDatanodeID(did[i]);
-    }
-    return result;
-  }
-  
-  static public org.apache.hadoop.hdfs.protocol.DatanodeID[] 
-      convertDatanodeID(DatanodeIDWritable[] did) {
-    if (did == null) return null;
-    final int len = did.length;
-    org.apache.hadoop.hdfs.protocol.DatanodeID[] result = new org.apache.hadoop.hdfs.protocol.DatanodeID[len];
-    for (int i = 0; i < len; ++i) {
-      result[i] = convertDatanodeID(did[i]);
-    }
-    return result;
-  }
-  
-  static public org.apache.hadoop.hdfs.protocol.DatanodeID convertDatanodeID(
-      DatanodeIDWritable did) {
-    if (did == null) return null;
-    return new org.apache.hadoop.hdfs.protocol.DatanodeID(
-        did.getName(), did.getStorageID(), did.getInfoPort(), did.getIpcPort());
-    
-  }
-  
-  public static DatanodeIDWritable convertDatanodeID(org.apache.hadoop.hdfs.protocol.DatanodeID from) {
-    return new DatanodeIDWritable(from.getName(),
-        from.getStorageID(),
-        from.getInfoPort(),
-        from.getIpcPort());
-  }
-  
-  /** Equivalent to DatanodeID(""). */
-  public DatanodeIDWritable() {this("");}
-
-  /** Equivalent to DatanodeID(nodeName, "", -1, -1). */
-  public DatanodeIDWritable(String nodeName) {this(nodeName, "", -1, -1);}
-
-  /**
-   * DatanodeID copy constructor
-   * 
-   * @param from
-   */
-  public DatanodeIDWritable(DatanodeIDWritable from) {
-    this(from.getName(),
-        from.getStorageID(),
-        from.getInfoPort(),
-        from.getIpcPort());
-  }
-  
-  /**
-   * Create DatanodeID
-   * @param nodeName (hostname:portNumber) 
-   * @param storageID data storage ID
-   * @param infoPort info server port 
-   * @param ipcPort ipc server port
-   */
-  public DatanodeIDWritable(String nodeName, String storageID,
-      int infoPort, int ipcPort) {
-    this.name = nodeName;
-    this.storageID = storageID;
-    this.infoPort = infoPort;
-    this.ipcPort = ipcPort;
-  }
-  
-  public void setName(String name) {
-    this.name = name;
-  }
-
-  public void setInfoPort(int infoPort) {
-    this.infoPort = infoPort;
-  }
-  
-  public void setIpcPort(int ipcPort) {
-    this.ipcPort = ipcPort;
-  }
-  
-  /**
-   * @return hostname:portNumber.
-   */
-  public String getName() {
-    return name;
-  }
-  
-  /**
-   * @return data storage ID.
-   */
-  public String getStorageID() {
-    return this.storageID;
-  }
-
-  /**
-   * @return infoPort (the port at which the HTTP server bound to)
-   */
-  public int getInfoPort() {
-    return infoPort;
-  }
-
-  /**
-   * @return ipcPort (the port at which the IPC server bound to)
-   */
-  public int getIpcPort() {
-    return ipcPort;
-  }
-
-  /**
-   * sets the data storage ID.
-   */
-  public void setStorageID(String storageID) {
-    this.storageID = storageID;
-  }
-
-  /**
-   * @return hostname and no :portNumber.
-   */
-  public String getHost() {
-    int colon = name.indexOf(":");
-    if (colon < 0) {
-      return name;
-    } else {
-      return name.substring(0, colon);
-    }
-  }
-  
-  public int getPort() {
-    int colon = name.indexOf(":");
-    if (colon < 0) {
-      return 50010; // default port.
-    }
-    return Integer.parseInt(name.substring(colon+1));
-  }
-
-  
-  public String toString() {
-    return name;
-  }    
-
-  /////////////////////////////////////////////////
-  // Writable
-  /////////////////////////////////////////////////
-  @Override
-  public void write(DataOutput out) throws IOException {
-    DeprecatedUTF8.writeString(out, name);
-    DeprecatedUTF8.writeString(out, storageID);
-    out.writeShort(infoPort);
-  }
-
-  @Override
-  public void readFields(DataInput in) throws IOException {
-    name = DeprecatedUTF8.readString(in);
-    storageID = DeprecatedUTF8.readString(in);
-    // the infoPort read could be negative, if the port is a large number (more
-    // than 15 bits in storage size (but less than 16 bits).
-    // So chop off the first two bytes (and hence the signed bits) before 
-    // setting the field.
-    this.infoPort = in.readShort() & 0x0000ffff;
-  }
-}

+ 0 - 334
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/DatanodeInfoWritable.java

@@ -1,334 +0,0 @@
-/**
- * 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.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;
-import org.apache.hadoop.io.WritableFactory;
-import org.apache.hadoop.io.WritableUtils;
-import org.apache.hadoop.net.NetworkTopology;
-import org.apache.hadoop.net.NodeBase;
-import org.apache.hadoop.HadoopIllegalArgumentException;
-
-/** 
- * DatanodeInfo represents the status of a DataNode.
- * This object is used for communication in the
- * Datanode Protocol and the Client Protocol.
- */
-@InterfaceAudience.Private
-@InterfaceStability.Stable
-public class DatanodeInfoWritable extends DatanodeIDWritable  {
-  protected long capacity;
-  protected long dfsUsed;
-  protected long remaining;
-  protected long blockPoolUsed;
-  protected long lastUpdate;
-  protected int xceiverCount;
-  protected String location = NetworkTopology.DEFAULT_RACK;
-
-  /** HostName as supplied by the datanode during registration as its 
-   * name. Namenode uses datanode IP address as the name.
-   */
-  protected String hostName = null;
-  
-  // administrative states of a datanode
-  public enum AdminStates {
-    NORMAL(DatanodeInfo.AdminStates.NORMAL.toString()), 
-    DECOMMISSION_INPROGRESS(DatanodeInfo.AdminStates.DECOMMISSION_INPROGRESS.toString()), 
-    DECOMMISSIONED(DatanodeInfo.AdminStates.DECOMMISSIONED.toString());
-
-    final String value;
-
-    AdminStates(final String v) {
-      this.value = v;
-    }
-
-    public String toString() {
-      return value;
-    }
-    
-    public static AdminStates fromValue(final String value) {
-      for (AdminStates as : AdminStates.values()) {
-        if (as.value.equals(value)) return as;
-      }
-      throw new HadoopIllegalArgumentException("Unknown Admin State" + value);
-    }
-  }
-
-  protected AdminStates adminState;
-  
-  static public DatanodeInfo convertDatanodeInfo(DatanodeInfoWritable di) {
-    if (di == null) return null;
-    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() ,
-        DatanodeInfo.AdminStates.fromValue(di.getAdminState().value)); 
-  }
-  
-  
-  static public DatanodeInfo[] convertDatanodeInfo(DatanodeInfoWritable di[]) {
-    if (di == null) return null;
-    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(DatanodeInfo[] di) {
-    if (di == null) return null;
-    DatanodeInfoWritable[] result = new DatanodeInfoWritable[di.length];
-    for (int i = 0; i < di.length; i++) {
-      result[i] = new DatanodeInfoWritable(new DatanodeIDWritable(di[i].getName(), di[i].getStorageID(), di[i].getInfoPort(), di[i].getIpcPort()),
-          di[i].getNetworkLocation(), di[i].getHostName(),
-          di[i].getCapacity(),  di[i].getDfsUsed(),  di[i].getRemaining(),
-          di[i].getBlockPoolUsed()  ,  di[i].getLastUpdate() , di[i].getXceiverCount() ,
-          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() {
-    super();
-    adminState = null;
-  }
-  
-  public DatanodeInfoWritable(DatanodeInfoWritable from) {
-    super(from);
-    this.capacity = from.getCapacity();
-    this.dfsUsed = from.getDfsUsed();
-    this.remaining = from.getRemaining();
-    this.blockPoolUsed = from.getBlockPoolUsed();
-    this.lastUpdate = from.getLastUpdate();
-    this.xceiverCount = from.getXceiverCount();
-    this.location = from.getNetworkLocation();
-    this.adminState = from.adminState;
-    this.hostName = from.hostName;
-  }
-
-  public DatanodeInfoWritable(DatanodeIDWritable nodeID) {
-    super(nodeID);
-    this.capacity = 0L;
-    this.dfsUsed = 0L;
-    this.remaining = 0L;
-    this.blockPoolUsed = 0L;
-    this.lastUpdate = 0L;
-    this.xceiverCount = 0;
-    this.adminState = null;    
-  }
-  
-  protected DatanodeInfoWritable(DatanodeIDWritable nodeID, String location, String hostName) {
-    this(nodeID);
-    this.location = location;
-    this.hostName = hostName;
-  }
-  
-  public DatanodeInfoWritable(DatanodeIDWritable nodeID, String location, String hostName,
-      final long capacity, final long dfsUsed, final long remaining,
-      final long blockPoolUsed, final long lastUpdate, final int xceiverCount,
-      final AdminStates adminState) {
-    this(nodeID, location, hostName);
-    this.capacity = capacity;
-    this.dfsUsed = dfsUsed;
-    this.remaining = remaining;
-    this.blockPoolUsed = blockPoolUsed;
-    this.lastUpdate = lastUpdate;
-    this.xceiverCount = xceiverCount;
-    this.adminState = adminState;
-  }
-  
-  /** The raw capacity. */
-  public long getCapacity() { return capacity; }
-  
-  /** The used space by the data node. */
-  public long getDfsUsed() { return dfsUsed; }
-
-  /** The used space by the block pool on data node. */
-  public long getBlockPoolUsed() { return blockPoolUsed; }
-
-  /** The used space by the data node. */
-  public long getNonDfsUsed() { 
-    long nonDFSUsed = capacity - dfsUsed - remaining;
-    return nonDFSUsed < 0 ? 0 : nonDFSUsed;
-  }
-
-  /** The used space by the data node as percentage of present capacity */
-  public float getDfsUsedPercent() { 
-    return DFSUtil.getPercentUsed(dfsUsed, capacity);
-  }
-
-  /** The raw free space. */
-  public long getRemaining() { return remaining; }
-
-  /** Used space by the block pool as percentage of present capacity */
-  public float getBlockPoolUsedPercent() {
-    return DFSUtil.getPercentUsed(blockPoolUsed, capacity);
-  }
-  
-  /** The remaining space as percentage of configured capacity. */
-  public float getRemainingPercent() { 
-    return DFSUtil.getPercentRemaining(remaining, capacity);
-  }
-
-  /** The time when this information was accurate. */
-  public long getLastUpdate() { return lastUpdate; }
-
-  /** number of active connections */
-  public int getXceiverCount() { return xceiverCount; }
-
-  /** Sets raw capacity. */
-  public void setCapacity(long capacity) { 
-    this.capacity = capacity; 
-  }
-  
-  /** Sets the used space for the datanode. */
-  public void setDfsUsed(long dfsUsed) {
-    this.dfsUsed = dfsUsed;
-  }
-
-  /** Sets raw free space. */
-  public void setRemaining(long remaining) { 
-    this.remaining = remaining; 
-  }
-
-  /** Sets block pool used space */
-  public void setBlockPoolUsed(long bpUsed) { 
-    this.blockPoolUsed = bpUsed; 
-  }
-
-  /** Sets time when this information was accurate. */
-  public void setLastUpdate(long lastUpdate) { 
-    this.lastUpdate = lastUpdate; 
-  }
-
-  /** Sets number of active connections */
-  public void setXceiverCount(int xceiverCount) { 
-    this.xceiverCount = xceiverCount; 
-  }
-
-  /** rack name */
-  public String getNetworkLocation() {return location;}
-    
-  /** Sets the rack name */
-  public void setNetworkLocation(String location) {
-    this.location = NodeBase.normalize(location);
-  }
-  
-  public String getHostName() {
-    return (hostName == null || hostName.length()==0) ? getHost() : hostName;
-  }
-  
-  public void setHostName(String host) {
-    hostName = host;
-  }
-
-  /**
-   * Retrieves the admin state of this node.
-   */
-  public AdminStates getAdminState() {
-    if (adminState == null) {
-      return AdminStates.NORMAL;
-    }
-    return adminState;
-  }
-
-  /**
-   * Sets the admin state of this node.
-   */
-  protected void setAdminState(AdminStates newState) {
-    if (newState == AdminStates.NORMAL) {
-      adminState = null;
-    }
-    else {
-      adminState = newState;
-    }
-  }
-
-  /////////////////////////////////////////////////
-  // Writable
-  /////////////////////////////////////////////////
-  static {                                      // register a ctor
-    WritableFactories.setFactory
-      (DatanodeInfoWritable.class,
-       new WritableFactory() {
-         public Writable newInstance() { return new DatanodeInfoWritable(); }
-       });
-  }
-
-  @Override
-  public void write(DataOutput out) throws IOException {
-    super.write(out);
-
-    out.writeShort(ipcPort);
-
-    out.writeLong(capacity);
-    out.writeLong(dfsUsed);
-    out.writeLong(remaining);
-    out.writeLong(blockPoolUsed);
-    out.writeLong(lastUpdate);
-    out.writeInt(xceiverCount);
-    Text.writeString(out, location);
-    Text.writeString(out, hostName == null? "" : hostName);
-    WritableUtils.writeEnum(out, getAdminState());
-  }
-
-  @Override
-  public void readFields(DataInput in) throws IOException {
-    super.readFields(in);
-
-    this.ipcPort = in.readShort() & 0x0000ffff;
-
-    this.capacity = in.readLong();
-    this.dfsUsed = in.readLong();
-    this.remaining = in.readLong();
-    this.blockPoolUsed = in.readLong();
-    this.lastUpdate = in.readLong();
-    this.xceiverCount = in.readInt();
-    this.location = Text.readString(in);
-    this.hostName = Text.readString(in);
-    setAdminState(WritableUtils.readEnum(in, AdminStates.class));
-  }
-
-  /** Read a DatanodeInfo */
-  public static DatanodeInfoWritable read(DataInput in) throws IOException {
-    final DatanodeInfoWritable d = new DatanodeInfoWritable();
-    d.readFields(in);
-    return d;
-  }
-}

+ 0 - 157
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/DirectoryListingWritable.java

@@ -1,157 +0,0 @@
-/* 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.io.Writable;
-import org.apache.hadoop.io.WritableFactories;
-import org.apache.hadoop.io.WritableFactory;
-
-/**
- * This class defines a partial listing of a directory to support
- * iterative directory listing.
- */
-@InterfaceAudience.Private
-@InterfaceStability.Stable
-public class DirectoryListingWritable implements Writable {
-  static {                                      // register a ctor
-    WritableFactories.setFactory
-      (DirectoryListingWritable.class,
-       new WritableFactory() {
-         public Writable newInstance() { return new DirectoryListingWritable(); }
-       });
-  }
-
-  private HdfsFileStatusWritable[] partialListing;
-  private int remainingEntries;
-  
-  public static org.apache.hadoop.hdfs.protocol.DirectoryListing 
-    convertDirectoryListing(DirectoryListingWritable dl) {
-    if (dl == null) return null;
-    return new org.apache.hadoop.hdfs.protocol.DirectoryListing(
-        HdfsFileStatusWritable.convertHdfsFileStatus(
-            dl.getPartialListing()), dl.getRemainingEntries());
-  }
-
-  public static DirectoryListingWritable convertDirectoryListing(
-      org.apache.hadoop.hdfs.protocol.DirectoryListing d) {
-    if (d == null) return null;
-    return new DirectoryListingWritable(
-        org.apache.hadoop.hdfs.protocolR23Compatible.HdfsFileStatusWritable.
-        convertHdfsFileStatus(d.getPartialListing()), d.getRemainingEntries());
-  } 
-  
-  /**
-   * default constructor
-   */
-  public DirectoryListingWritable() {
-  }
-  
-  /**
-   * constructor
-   * @param partialListing a partial listing of a directory
-   * @param remainingEntries number of entries that are left to be listed
-   */
-  public DirectoryListingWritable(HdfsFileStatusWritable[] partialListing, 
-      int remainingEntries) {
-    if (partialListing == null) {
-      throw new IllegalArgumentException("partial listing should not be null");
-    }
-    if (partialListing.length == 0 && remainingEntries != 0) {
-      throw new IllegalArgumentException("Partial listing is empty but " +
-          "the number of remaining entries is not zero");
-    }
-    this.partialListing = partialListing;
-    this.remainingEntries = remainingEntries;
-  }
-
-  /**
-   * Get the partial listing of file status
-   * @return the partial listing of file status
-   */
-  public HdfsFileStatusWritable[] getPartialListing() {
-    return partialListing;
-  }
-  
-  /**
-   * Get the number of remaining entries that are left to be listed
-   * @return the number of remaining entries that are left to be listed
-   */
-  public int getRemainingEntries() {
-    return remainingEntries;
-  }
-  
-  /**
-   * Check if there are more entries that are left to be listed
-   * @return true if there are more entries that are left to be listed;
-   *         return false otherwise.
-   */
-  public boolean hasMore() {
-    return remainingEntries != 0;
-  }
-  
-  /**
-   * Get the last name in this list
-   * @return the last name in the list if it is not empty; otherwise return null
-   */
-  public byte[] getLastName() {
-    if (partialListing.length == 0) {
-      return null;
-    }
-    return partialListing[partialListing.length-1].getLocalNameInBytes();
-  }
-
-  // Writable interface
-  @Override
-  public void readFields(DataInput in) throws IOException {
-    int numEntries = in.readInt();
-    partialListing = new HdfsFileStatusWritable[numEntries];
-    if (numEntries !=0 ) {
-      boolean hasLocation = in.readBoolean();
-      for (int i=0; i<numEntries; i++) {
-        if (hasLocation) {
-          partialListing[i] = new HdfsLocatedFileStatusWritable();
-        } else {
-          partialListing[i] = new HdfsFileStatusWritable();
-        }
-        partialListing[i].readFields(in);
-      }
-    }
-    remainingEntries = in.readInt();
-  }
-
-  @Override
-  public void write(DataOutput out) throws IOException {
-    out.writeInt(partialListing.length);
-    if (partialListing.length != 0) { 
-       if (partialListing[0] instanceof HdfsLocatedFileStatusWritable) {
-         out.writeBoolean(true);
-       } else {
-         out.writeBoolean(false);
-       }
-       for (HdfsFileStatusWritable fileStatus : partialListing) {
-         fileStatus.write(out);
-       }
-    }
-    out.writeInt(remainingEntries);
-  }
-}

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

@@ -1,104 +0,0 @@
-/**
- * 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);
-  }
-}

+ 0 - 121
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/ExtendedBlockWritable.java

@@ -1,121 +0,0 @@
-/**
- * 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.io.Writable;
-import org.apache.hadoop.io.WritableFactories;
-import org.apache.hadoop.io.WritableFactory;
-
-/**
- * Identifies a Block uniquely across the block pools
- */
-@InterfaceAudience.Private
-@InterfaceStability.Stable
-public class ExtendedBlockWritable implements Writable {
-  private String poolId;
-  private long blockId;
-  private long numBytes;
-  private long generationStamp;
-
-  static { // register a ctor
-    WritableFactories.setFactory(ExtendedBlockWritable.class, new WritableFactory() {
-      public Writable newInstance() {
-        return new ExtendedBlockWritable();
-      }
-    });
-  }
-
-  static public org.apache.hadoop.hdfs.protocol.ExtendedBlock convertExtendedBlock(ExtendedBlockWritable eb) {
-    if (eb == null) return null;
-    return new org.apache.hadoop.hdfs.protocol.ExtendedBlock( eb.getBlockPoolId(),  eb.getBlockId(),   eb.getNumBytes(),
-       eb.getGenerationStamp());
-  }
-  
-  public static ExtendedBlockWritable convertExtendedBlock(final org.apache.hadoop.hdfs.protocol.ExtendedBlock b) {
-    if (b == null) return null;
-    return new ExtendedBlockWritable(b.getBlockPoolId(), 
-        b.getBlockId(), b.getNumBytes(), b.getGenerationStamp());
-  }
-  
-  public ExtendedBlockWritable() {
-    this(null, 0, 0, 0);
-  }
-
-  public ExtendedBlockWritable(final ExtendedBlockWritable b) {
-    this(b.poolId, b.blockId, b.numBytes, b.generationStamp);
-  }
-  
-  public ExtendedBlockWritable(final String poolId, final long blockId) {
-    this(poolId, blockId, 0, 0);
-  }
-
-  public ExtendedBlockWritable(final String poolId, final long blkid, final long len,
-      final long genstamp) {
-    this.poolId = poolId;
-    this.blockId = blkid;
-    this.numBytes = len;
-    this.generationStamp = genstamp;
-  }
-
-  @Override
-  public void write(DataOutput out) throws IOException {
-    DeprecatedUTF8.writeString(out, poolId);
-    out.writeLong(blockId);
-    out.writeLong(numBytes);
-    out.writeLong(generationStamp);
-  }
-
-  @Override
-  public void readFields(DataInput in) throws IOException {
-    this.poolId = DeprecatedUTF8.readString(in);
-    this.blockId = in.readLong();
-    this.numBytes = in.readLong();
-    this.generationStamp = in.readLong();
-    if (numBytes < 0) {
-      throw new IOException("Unexpected block size: " + numBytes);
-    }
-  }
-
-  public String getBlockPoolId() {
-    return poolId;
-  }
-
-  public long getNumBytes() {
-    return numBytes;
-  }
-
-  public long getBlockId() {
-    return blockId;
-  }
-
-  public long getGenerationStamp() {
-    return generationStamp;
-  }
-  
-  @Override // Object
-  public String toString() {
-    return poolId + ":" + (new org.apache.hadoop.hdfs.protocol.Block(blockId, numBytes, generationStamp));
-  }
-}

+ 0 - 80
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/FsPermissionWritable.java

@@ -1,80 +0,0 @@
-/**
- * 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.io.Writable;
-import org.apache.hadoop.io.WritableFactories;
-import org.apache.hadoop.io.WritableFactory;
-
-@InterfaceAudience.Private
-@InterfaceStability.Stable
-public class FsPermissionWritable  implements Writable {
-  static final WritableFactory FACTORY = new WritableFactory() {
-	public Writable newInstance() { return new FsPermissionWritable(); }
-  };
-  static {                                      // register a ctor
-    WritableFactories.setFactory(FsPermissionWritable.class, FACTORY);
-  }
-  //POSIX permission style
-  private short thePermissions = 0;
-  
-  public static FsPermissionWritable convertPermission(org.apache.hadoop.fs.permission.FsPermission p) {
-    if (p == null) return null;
-    return new FsPermissionWritable(p.toShort());
-  }
-  
-  public static org.apache.hadoop.fs.permission.FsPermission convertPermission(FsPermissionWritable p) {
-    if (p == null) return null;
-    return new org.apache.hadoop.fs.permission.FsPermission(p.thePermissions);
-  }
-  
-  public static FsPermissionWritable getDefault() {
-    return new FsPermissionWritable((short)00777);
-  }
-  
-  FsPermissionWritable() {
-  }
-	FsPermissionWritable(short p) {
-	  thePermissions = p;
-	}
-  
-  @Override
-  public void write(DataOutput out) throws IOException {
-    out.writeShort(thePermissions);
-  }
-
-  @Override
-  public void readFields(DataInput in) throws IOException {
-    thePermissions = in.readShort();
-  }
-
-  /**
-   * Create and initialize a {@link FsPermissionWritable} from {@link DataInput}.
-   */
-  public static FsPermissionWritable read(DataInput in) throws IOException {
-    FsPermissionWritable p = new FsPermissionWritable();
-    p.readFields(in);
-    return p;
-  }
-}

+ 0 - 122
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/FsServerDefaultsWritable.java

@@ -1,122 +0,0 @@
-/**
- * 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.io.Writable;
-import org.apache.hadoop.io.WritableFactories;
-import org.apache.hadoop.io.WritableFactory;
-
-/****************************************************
- * Provides server default configuration values to clients.
- * 
- ****************************************************/
-@InterfaceAudience.Public
-@InterfaceStability.Stable
-public class FsServerDefaultsWritable implements Writable {
-
-  static { // register a ctor
-    WritableFactories.setFactory(FsServerDefaultsWritable.class, new WritableFactory() {
-      public Writable newInstance() {
-        return new FsServerDefaultsWritable();
-      }
-    });
-  }
-
-  private long blockSize;
-  private int bytesPerChecksum;
-  private int writePacketSize;
-  private short replication;
-  private int fileBufferSize;
-  
-  public static org.apache.hadoop.fs.FsServerDefaults convert(
-      FsServerDefaultsWritable fs) {
-    if (fs == null) return null;
-    return new org.apache.hadoop.fs.FsServerDefaults(
-        fs.getBlockSize(), fs.getBytesPerChecksum(), 
-        fs.getWritePacketSize(), fs.getReplication(), fs.getFileBufferSize());
-  }
-  
-  public static FsServerDefaultsWritable convert(
-      org.apache.hadoop.fs.FsServerDefaults fs) {
-    if (fs == null) return null;
-    return new FsServerDefaultsWritable(
-        fs.getBlockSize(), fs.getBytesPerChecksum(), 
-        fs.getWritePacketSize(), fs.getReplication(), fs.getFileBufferSize());
-  }
-  
-  public FsServerDefaultsWritable() {
-  }
-
-  public FsServerDefaultsWritable(long blockSize, int bytesPerChecksum,
-      int writePacketSize, short replication, int fileBufferSize) {
-    this.blockSize = blockSize;
-    this.bytesPerChecksum = bytesPerChecksum;
-    this.writePacketSize = writePacketSize;
-    this.replication = replication;
-    this.fileBufferSize = fileBufferSize;
-  }
-
-  public long getBlockSize() {
-    return blockSize;
-  }
-
-  public int getBytesPerChecksum() {
-    return bytesPerChecksum;
-  }
-
-  public int getWritePacketSize() {
-    return writePacketSize;
-  }
-
-  public short getReplication() {
-    return replication;
-  }
-
-  public int getFileBufferSize() {
-    return fileBufferSize;
-  }
-
-  // /////////////////////////////////////////
-  // Writable
-  // /////////////////////////////////////////
-  @Override
-  @InterfaceAudience.Private
-  public void write(DataOutput out) throws IOException {
-    out.writeLong(blockSize);
-    out.writeInt(bytesPerChecksum);
-    out.writeInt(writePacketSize);
-    out.writeShort(replication);
-    out.writeInt(fileBufferSize);
-  }
-
-  @Override
-  @InterfaceAudience.Private
-  public void readFields(DataInput in) throws IOException {
-    blockSize = in.readLong();
-    bytesPerChecksum = in.readInt();
-    writePacketSize = in.readInt();
-    replication = in.readShort();
-    fileBufferSize = in.readInt();
-  }
-}

+ 0 - 348
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/HdfsFileStatusWritable.java

@@ -1,348 +0,0 @@
-/**
- * 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.fs.Path;
-import org.apache.hadoop.hdfs.DFSUtil;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.io.WritableFactories;
-import org.apache.hadoop.io.WritableFactory;
-
-/** Interface that represents the over the wire information for a file.
- */
-@InterfaceAudience.Private
-@InterfaceStability.Stable
-public class HdfsFileStatusWritable implements Writable {
-  static {                                      // register a ctor
-    WritableFactories.setFactory
-      (HdfsFileStatusWritable.class,
-       new WritableFactory() {
-         public Writable newInstance() { return new HdfsFileStatusWritable(); }
-       });
-  }
-
-  private byte[] path;  // local name of the inode that's encoded in java UTF8
-  private byte[] symlink; // symlink target encoded in java UTF8 or null
-  private long length;
-  private boolean isdir;
-  private short block_replication;
-  private long blocksize;
-  private long modification_time;
-  private long access_time;
-  private FsPermissionWritable permission;
-  private String owner;
-  private String group;
-  
-  public static final byte[] EMPTY_NAME = new byte[0];
-
-  static public org.apache.hadoop.hdfs.protocol.HdfsFileStatus
-    convertHdfsFileStatus(HdfsFileStatusWritable fs) {
-    if (fs == null) return null;
-    return new org.apache.hadoop.hdfs.protocol.HdfsFileStatus(fs.getLen(),
-        fs.isDir(), fs.getReplication(), fs.getBlockSize(),
-        fs.getModificationTime(), fs.getAccessTime(), 
-        FsPermissionWritable.convertPermission(fs.getPermission()),
-        fs.getOwner(), fs.getGroup(), fs.getSymlinkInBytes(),
-        fs.getLocalNameInBytes());
-  }
-  
-  static public HdfsFileStatusWritable[] convertHdfsFileStatus(org.apache.hadoop.hdfs.protocol.HdfsFileStatus[] fs) {
-    if (fs == null) return null;
-    final int len = fs.length;
-    HdfsFileStatusWritable[] result = new HdfsFileStatusWritable[len];
-    for (int i = 0; i < len; ++i) {
-      if (fs[i] instanceof org.apache.hadoop.hdfs.protocol.HdfsLocatedFileStatus) {
-        result[i] = 
-            HdfsLocatedFileStatusWritable.convertLocatedHdfsFileStatus(
-                (org.apache.hadoop.hdfs.protocol.HdfsLocatedFileStatus)fs[i]);
-      } else {
-        result[i] = HdfsFileStatusWritable.convertHdfsFileStatus(fs[i]);
-      }
-    }
-    return result;
-  }
-  
-
-  public static org.apache.hadoop.hdfs.protocol.HdfsFileStatus[] convertHdfsFileStatus(
-      HdfsFileStatusWritable[] fs) {
-    if (fs == null) return null;
-    final int len = fs.length;
-    org.apache.hadoop.hdfs.protocol.HdfsFileStatus[] result = 
-        new org.apache.hadoop.hdfs.protocol.HdfsFileStatus[len];
-    for (int i = 0; i < len; ++i) {
-      if (fs[i] instanceof HdfsLocatedFileStatusWritable) {
-        result[i] = 
-            HdfsLocatedFileStatusWritable.convertLocatedHdfsFileStatus((HdfsLocatedFileStatusWritable)fs[i]);
-      } else {
-        result[i] = convertHdfsFileStatus(fs[i]);
-      }
-    }
-    return result;
-  }
- 
-  public static HdfsFileStatusWritable convertHdfsFileStatus(org.apache.hadoop.hdfs.protocol.HdfsFileStatus fs) {
-    if (fs == null) return null;
-    return new HdfsFileStatusWritable(fs.getLen(), fs.isDir(), fs.getReplication(),
-       fs.getBlockSize(),  fs.getModificationTime(),  fs.getAccessTime(),
-       org.apache.hadoop.hdfs.protocolR23Compatible.FsPermissionWritable.
-         convertPermission(fs.getPermission()),
-       fs.getOwner(),  fs.getGroup(), 
-       fs.getSymlinkInBytes(), fs.getLocalNameInBytes());
-  }
-  
-  /**
-   * default constructor
-   */
-  public HdfsFileStatusWritable() { 
-    this(0, false, 0, 0, 0, 0, null, null, null, null, null); 
-  }
-  
-  /**
-   * Constructor
-   * @param length the number of bytes the file has
-   * @param isdir if the path is a directory
-   * @param block_replication the replication factor
-   * @param blocksize the block size
-   * @param modification_time modification time
-   * @param access_time access time
-   * @param permission permission
-   * @param owner the owner of the path
-   * @param group the group of the path
-   * @param path the local name in java UTF8 encoding the same as that in-memory
-   */
-  public HdfsFileStatusWritable(long length, boolean isdir, int block_replication,
-                    long blocksize, long modification_time, long access_time,
-                    FsPermissionWritable permission, String owner, String group, 
-                    byte[] symlink, byte[] path) {
-    this.length = length;
-    this.isdir = isdir;
-    this.block_replication = (short)block_replication;
-    this.blocksize = blocksize;
-    this.modification_time = modification_time;
-    this.access_time = access_time;
-    this.permission = (permission == null) ? FsPermissionWritable.getDefault() : permission;
-    this.owner = (owner == null) ? "" : owner;
-    this.group = (group == null) ? "" : group;
-    this.symlink = symlink;
-    this.path = path;
-  }
-
-  /**
-   * Get the length of this file, in bytes.
-   * @return the length of this file, in bytes.
-   */
-  final public long getLen() {
-    return length;
-  }
-
-  /**
-   * Is this a directory?
-   * @return true if this is a directory
-   */
-  final public boolean isDir() {
-    return isdir;
-  }
-
-  /**
-   * Is this a symbolic link?
-   * @return true if this is a symbolic link
-   */
-  public boolean isSymlink() {
-    return symlink != null;
-  }
-   
-  /**
-   * Get the block size of the file.
-   * @return the number of bytes
-   */
-  final public long getBlockSize() {
-    return blocksize;
-  }
-
-  /**
-   * Get the replication factor of a file.
-   * @return the replication factor of a file.
-   */
-  final public short getReplication() {
-    return block_replication;
-  }
-
-  /**
-   * Get the modification time of the file.
-   * @return the modification time of file in milliseconds since January 1, 1970 UTC.
-   */
-  final public long getModificationTime() {
-    return modification_time;
-  }
-
-  /**
-   * Get the access time of the file.
-   * @return the access time of file in milliseconds since January 1, 1970 UTC.
-   */
-  final public long getAccessTime() {
-    return access_time;
-  }
-
-  /**
-   * Get FsPermission associated with the file.
-   * @return permssion
-   */
-  final public FsPermissionWritable getPermission() {
-    return permission;
-  }
-  
-  /**
-   * Get the owner of the file.
-   * @return owner of the file
-   */
-  final public String getOwner() {
-    return owner;
-  }
-  
-  /**
-   * Get the group associated with the file.
-   * @return group for the file. 
-   */
-  final public String getGroup() {
-    return group;
-  }
-  
-  /**
-   * Check if the local name is empty
-   * @return true if the name is empty
-   */
-  final public boolean isEmptyLocalName() {
-    return path.length == 0;
-  }
-
-  /**
-   * Get the string representation of the local name
-   * @return the local name in string
-   */
-  final public String getLocalName() {
-    return DFSUtil.bytes2String(path);
-  }
-  
-  /**
-   * Get the Java UTF8 representation of the local name
-   * @return the local name in java UTF8
-   */
-  final public byte[] getLocalNameInBytes() {
-    return path;
-  }
-
-  /**
-   * Get the string representation of the full path name
-   * @param parent the parent path
-   * @return the full path in string
-   */
-  final public String getFullName(final String parent) {
-    if (isEmptyLocalName()) {
-      return parent;
-    }
-    
-    StringBuilder fullName = new StringBuilder(parent);
-    if (!parent.endsWith(Path.SEPARATOR)) {
-      fullName.append(Path.SEPARATOR);
-    }
-    fullName.append(getLocalName());
-    return fullName.toString();
-  }
-
-  /**
-   * Get the full path
-   * @param parent the parent path
-   * @return the full path
-   */
-  final public Path getFullPath(final Path parent) {
-    if (isEmptyLocalName()) {
-      return parent;
-    }
-    
-    return new Path(parent, getLocalName());
-  }
-
-  /**
-   * Get the string representation of the symlink.
-   * @return the symlink as a string.
-   */
-  final public String getSymlink() {
-    return DFSUtil.bytes2String(symlink);
-  }
-  
-  final public byte[] getSymlinkInBytes() {
-    return symlink;
-  }
-  
-
-  //////////////////////////////////////////////////
-  // Writable
-  //////////////////////////////////////////////////
-  @Override
-  public void write(DataOutput out) throws IOException {
-    out.writeInt(path.length);
-    out.write(path);
-    out.writeLong(length);
-    out.writeBoolean(isdir);
-    out.writeShort(block_replication);
-    out.writeLong(blocksize);
-    out.writeLong(modification_time);
-    out.writeLong(access_time);
-    permission.write(out);
-    Text.writeString(out, owner);
-    Text.writeString(out, group);
-    out.writeBoolean(isSymlink());
-    if (isSymlink()) {
-      out.writeInt(symlink.length);
-      out.write(symlink);
-    }
-  }
-
-  @Override
-  public void readFields(DataInput in) throws IOException {
-    int numOfBytes = in.readInt();
-    if (numOfBytes == 0) {
-      this.path = EMPTY_NAME;
-    } else {
-      this.path = new byte[numOfBytes];
-      in.readFully(path);
-    }
-    this.length = in.readLong();
-    this.isdir = in.readBoolean();
-    this.block_replication = in.readShort();
-    blocksize = in.readLong();
-    modification_time = in.readLong();
-    access_time = in.readLong();
-    permission.readFields(in);
-    owner = Text.readString(in);
-    group = Text.readString(in);
-    if (in.readBoolean()) {
-      numOfBytes = in.readInt();
-      this.symlink = new byte[numOfBytes];
-      in.readFully(symlink);
-    }
-  }
-
-}

+ 0 - 116
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/HdfsLocatedFileStatusWritable.java

@@ -1,116 +0,0 @@
-/**
- * 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;
-
-/** 
- * Interface that represents the over the wire information
- * including block locations for a file.
- */
-@InterfaceAudience.Private
-@InterfaceStability.Stable
-public class HdfsLocatedFileStatusWritable extends HdfsFileStatusWritable {
-  private LocatedBlocksWritable locations;
-  
-  /**
-   * Default constructor
-   */
-  public HdfsLocatedFileStatusWritable() {
-  }
-  
-  /**
-   * Constructor
-   * 
-   * @param length size
-   * @param isdir if this is directory
-   * @param block_replication the file's replication factor
-   * @param blocksize the file's block size
-   * @param modification_time most recent modification time
-   * @param access_time most recent access time
-   * @param permission permission
-   * @param owner owner
-   * @param group group
-   * @param symlink symbolic link
-   * @param path local path name in java UTF8 format 
-   * @param locations block locations
-   */
-  public HdfsLocatedFileStatusWritable(long length, boolean isdir,
-      int block_replication,
-	    long blocksize, long modification_time, long access_time,
-	    FsPermissionWritable permission, String owner, String group, 
-	    byte[] symlink, byte[] path, LocatedBlocksWritable locations) {
-	  super(length, isdir, block_replication, blocksize, modification_time,
-		  access_time, permission, owner, group, symlink, path);
-    this.locations = locations;
-	}
-  
-  static public org.apache.hadoop.hdfs.protocol.HdfsLocatedFileStatus
-  convertLocatedHdfsFileStatus(HdfsLocatedFileStatusWritable fs) {
-  if (fs == null) return null;
-  return new org.apache.hadoop.hdfs.protocol.HdfsLocatedFileStatus(fs.getLen(),
-      fs.isDir(), fs.getReplication(), fs.getBlockSize(),
-      fs.getModificationTime(), fs.getAccessTime(), 
-      FsPermissionWritable.convertPermission(fs.getPermission()),
-      fs.getOwner(), fs.getGroup(), fs.getSymlinkInBytes(),
-      fs.getLocalNameInBytes(),
-      LocatedBlocksWritable.convertLocatedBlocks(fs.getBlockLocations()));
-}
-  
-  static public HdfsLocatedFileStatusWritable convertLocatedHdfsFileStatus(
-      org.apache.hadoop.hdfs.protocol.HdfsLocatedFileStatus fs) {
-  if (fs == null) return null;
-  return new HdfsLocatedFileStatusWritable(fs.getLen(),
-      fs.isDir(), fs.getReplication(), fs.getBlockSize(),
-      fs.getModificationTime(), fs.getAccessTime(), 
-      org.apache.hadoop.hdfs.protocolR23Compatible.FsPermissionWritable.
-        convertPermission(fs.getPermission()),
-      fs.getOwner(), fs.getGroup(), fs.getSymlinkInBytes(),
-      fs.getLocalNameInBytes(), 
-      LocatedBlocksWritable.convertLocatedBlocks(fs.getBlockLocations()));
-}
-	
-	public LocatedBlocksWritable getBlockLocations() {
-		return locations;
-	}
-	
-  //////////////////////////////////////////////////
-  // Writable
-  //////////////////////////////////////////////////
-  @Override
-  public void write(DataOutput out) throws IOException {
-    super.write(out);
-    if (!isDir() && !isSymlink()) {
-      locations.write(out);
-    }
-  }
-
-  @Override
-  public void readFields(DataInput in) throws IOException {
-    super.readFields(in);
-    if (!isDir() && !isSymlink()) {
-      locations = new LocatedBlocksWritable();
-      locations.readFields(in);
-    }
-  }
-}

+ 0 - 114
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/JournalProtocolServerSideTranslatorR23.java

@@ -1,114 +0,0 @@
-/**
- * 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.JournalProtocol;
-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 native data types used inside the NN as specified in the generic
- * JournalProtocol.
- */
-@InterfaceAudience.Private
-@InterfaceStability.Stable
-public class JournalProtocolServerSideTranslatorR23 implements
-    JournalWireProtocol {
-  final private JournalProtocol server;
-
-  /**
-   * Constructor
-   * 
-   * @param server - the NN server
-   * @throws IOException
-   */
-  public JournalProtocolServerSideTranslatorR23(JournalProtocol 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 JournalProtocol}
-     * 
-     */
-    if (!protocol.equals(RPC.getProtocolName(JournalWireProtocol.class))) {
-      throw new IOException("Namenode Serverside implements " +
-          RPC.getProtocolName(JournalWireProtocol.class) +
-          ". The following requested protocol is unknown: " + protocol);
-    }
-
-    return ProtocolSignature.getProtocolSignature(clientMethodsHash,
-        JournalWireProtocol.versionID, JournalWireProtocol.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(JournalWireProtocol.class))) {
-      return JournalWireProtocol.versionID;
-    }
-    throw new IOException("Namenode Serverside implements " +
-        RPC.getProtocolName(JournalWireProtocol.class) +
-        ". The following requested protocol is unknown: " + protocol);
-  }
-
-  @Override
-  public void journal(NamenodeRegistrationWritable registration,
-      long firstTxnId, int numTxns, byte[] records) throws IOException {
-    server.journal(registration.convert(), firstTxnId, numTxns, records);
-  }
-
-  @Override
-  public void startLogSegment(NamenodeRegistrationWritable registration,
-      long txid) throws IOException {
-    server.startLogSegment(registration.convert(), txid);
-  }
-}

+ 0 - 80
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/JournalProtocolTranslatorR23.java

@@ -1,80 +0,0 @@
-/**
- * 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 org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hdfs.server.protocol.JournalProtocol;
-import org.apache.hadoop.hdfs.server.protocol.NamenodeRegistration;
-import org.apache.hadoop.ipc.ProtocolSignature;
-import org.apache.hadoop.ipc.RPC;
-
-/**
- * This class forwards NN's ClientProtocol calls as RPC calls to the NN server
- * while translating from the parameter types used in ClientProtocol to those
- * used in protocolR23Compatile.*.
- */
-@InterfaceAudience.Private
-@InterfaceStability.Stable
-public class JournalProtocolTranslatorR23 implements
-    JournalProtocol, Closeable {
-  private final JournalWireProtocol rpcProxy;
-
-  public JournalProtocolTranslatorR23(InetSocketAddress nameNodeAddr,
-      Configuration conf) throws IOException {
-    rpcProxy = RPC.getProxy(JournalWireProtocol.class,
-        JournalWireProtocol.versionID, nameNodeAddr, conf);
-  }
-
-  @Override
-  public void close() {
-    RPC.stopProxy(rpcProxy);
-  }
-
-  @Override
-  public long getProtocolVersion(String protocolName, long clientVersion)
-      throws IOException {
-    return rpcProxy.getProtocolVersion(protocolName, clientVersion);
-  }
-
-  @Override
-  public ProtocolSignature getProtocolSignature(String protocol,
-      long clientVersion, int clientMethodsHash) throws IOException {
-    return ProtocolSignatureWritable.convert(rpcProxy.getProtocolSignature2(
-        protocol, clientVersion, clientMethodsHash));
-  }
-
-  @Override
-  public void journal(NamenodeRegistration registration, long firstTxnId,
-      int numTxns, byte[] records) throws IOException {
-    rpcProxy.journal(NamenodeRegistrationWritable.convert(registration),
-        firstTxnId, numTxns, records);
-  }
-
-  @Override
-  public void startLogSegment(NamenodeRegistration registration, long txid)
-      throws IOException {
-    rpcProxy.startLogSegment(NamenodeRegistrationWritable.convert(registration),
-        txid);
-  }
-}

+ 0 - 81
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/JournalWireProtocol.java

@@ -1,81 +0,0 @@
-/**
- * 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.ipc.VersionedProtocol;
-import org.apache.hadoop.security.KerberosInfo;
-
-/**
- * This class defines the actual protocol used to communicate with the
- * NN via RPC using writable types.
- * The parameters in the methods which are specified in the
- * package are separate from those used internally in the NN and DFSClient
- * and hence need to be converted using {@link JournalProtocolTranslatorR23}
- * and {@link JournalProtocolServerSideTranslatorR23}.
- *
- */
-@KerberosInfo(
-    serverPrincipal = DFSConfigKeys.DFS_NAMENODE_USER_NAME_KEY,
-    clientPrincipal = DFSConfigKeys.DFS_NAMENODE_USER_NAME_KEY)
-@InterfaceAudience.Private
-public interface JournalWireProtocol 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 = 1L;
-
-  /**
-   * Journal edit records.
-   * This message is sent by the active name-node to the backup node
-   * via {@code EditLogBackupOutputStream} in order to synchronize meta-data
-   * changes with the backup namespace image.
-   * 
-   * @param registration active node registration
-   * @param firstTxnId the first transaction of this batch
-   * @param numTxns number of transactions
-   * @param records byte array containing serialized journal records
-   */
-  public void journal(NamenodeRegistrationWritable registration,
-                      long firstTxnId,
-                      int numTxns,
-                      byte[] records) throws IOException;
-
-  /**
-   * Notify the BackupNode that the NameNode has rolled its edit logs
-   * and is now writing a new log segment.
-   * @param registration the registration of the active NameNode
-   * @param txid the first txid in the new log
-   */
-  public void startLogSegment(NamenodeRegistrationWritable registration,
-      long txid) 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;
-}

+ 0 - 253
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/LocatedBlockWritable.java

@@ -1,253 +0,0 @@
-/**
- * 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 org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
-import org.apache.hadoop.io.*;
-
-import java.io.*;
-import java.util.ArrayList;
-import java.util.List;
-
-/****************************************************
- * A LocatedBlock is a pair of Block, DatanodeInfo[]
- * objects.  It tells where to find a Block.
- * 
- ****************************************************/
-@InterfaceAudience.Private
-@InterfaceStability.Evolving
-public class LocatedBlockWritable implements Writable {
-
-  static {                                      // register a ctor
-    WritableFactories.setFactory
-      (LocatedBlockWritable.class,
-       new WritableFactory() {
-         public Writable newInstance() { return new LocatedBlockWritable(); }
-       });
-  }
-
-  private ExtendedBlockWritable b;
-  private long offset;  // offset of the first byte of the block in the file
-  private DatanodeInfoWritable[] locs;
-  // corrupt flag is true if all of the 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
-  private boolean corrupt;
-  private TokenWritable blockToken = new TokenWritable();
-
-  
-  static public org.apache.hadoop.hdfs.protocol.LocatedBlock
-    convertLocatedBlock(LocatedBlockWritable lb) {
-    if (lb == null) return null;
-    org.apache.hadoop.hdfs.protocol.LocatedBlock result =  
-        new org.apache.hadoop.hdfs.protocol.LocatedBlock(ExtendedBlockWritable.
-            convertExtendedBlock(lb.getBlock()),
-        DatanodeInfoWritable.convertDatanodeInfo(
-            lb.getLocations()), lb.getStartOffset(), lb.isCorrupt());
-    
-    // Fill in the token
-    TokenWritable tok = lb.getBlockToken();
-    result.setBlockToken(
-        new org.apache.hadoop.security.token.Token<BlockTokenIdentifier>(
-            tok.getIdentifier(), tok.getPassword(), tok.getKind(),
-            tok.getService()));
-    return result;
-  }
-  
-  public static LocatedBlockWritable 
-    convertLocatedBlock(org.apache.hadoop.hdfs.protocol.LocatedBlock lb) {
-    if (lb == null) return null;
-    LocatedBlockWritable result =  
-        new LocatedBlockWritable(ExtendedBlockWritable.convertExtendedBlock(lb.getBlock()), 
-        DatanodeInfoWritable.convertDatanodeInfo(lb.getLocations()),
-        lb.getStartOffset(), lb.isCorrupt());
-    
-    // Fill in the token
-    org.apache.hadoop.security.token.Token<BlockTokenIdentifier> tok = 
-        lb.getBlockToken();
-    result.setBlockToken(new TokenWritable(tok.getIdentifier(), tok.getPassword(), 
-        tok.getKind(), tok.getService()));
-    return result;
-  }
-  
-  static public LocatedBlockWritable[] 
-      convertLocatedBlock(org.apache.hadoop.hdfs.protocol.LocatedBlock[] lb) {
-    if (lb == null) return null;
-    final int len = lb.length;
-    LocatedBlockWritable[] result = new LocatedBlockWritable[len];
-    for (int i = 0; i < len; ++i) {
-      result[i] = new LocatedBlockWritable(
-          ExtendedBlockWritable.convertExtendedBlock(lb[i].getBlock()),
-          DatanodeInfoWritable.convertDatanodeInfo(lb[i].getLocations()), 
-          lb[i].getStartOffset(), lb[i].isCorrupt());
-    }
-    return result;
-  }
-  
-  static public org.apache.hadoop.hdfs.protocol.LocatedBlock[] 
-      convertLocatedBlock(LocatedBlockWritable[] lb) {
-    if (lb == null) return null;
-    final int len = lb.length;
-    org.apache.hadoop.hdfs.protocol.LocatedBlock[] result = 
-        new org.apache.hadoop.hdfs.protocol.LocatedBlock[len];
-    for (int i = 0; i < len; ++i) {
-      result[i] = new org.apache.hadoop.hdfs.protocol.LocatedBlock(
-          ExtendedBlockWritable.convertExtendedBlock(lb[i].getBlock()),
-          DatanodeInfoWritable.convertDatanodeInfo(lb[i].getLocations()), 
-          lb[i].getStartOffset(), lb[i].isCorrupt());
-    }
-    return result;
-  }
-  
-  static public List<org.apache.hadoop.hdfs.protocol.LocatedBlock> 
-    convertLocatedBlock(
-        List<org.apache.hadoop.hdfs.protocolR23Compatible.LocatedBlockWritable> lb) {
-    if (lb == null) return null;
-    final int len = lb.size();
-    List<org.apache.hadoop.hdfs.protocol.LocatedBlock> result = 
-        new ArrayList<org.apache.hadoop.hdfs.protocol.LocatedBlock>(len);
-    for (int i = 0; i < len; ++i) {
-      result.add(LocatedBlockWritable.convertLocatedBlock(lb.get(i)));
-    }
-    return result;
-  }
-  
-  static public List<LocatedBlockWritable> 
-  convertLocatedBlock2(List<org.apache.hadoop.hdfs.protocol.LocatedBlock> lb) {
-    if (lb == null) return null;
-    final int len = lb.size();
-    List<LocatedBlockWritable> result = new ArrayList<LocatedBlockWritable>(len);
-    for (int i = 0; i < len; ++i) {
-      result.add(LocatedBlockWritable.convertLocatedBlock(lb.get(i)));
-    }
-    return result;
-  }
-  
-  public LocatedBlockWritable() {
-    this(new ExtendedBlockWritable(), new DatanodeInfoWritable[0], 0L, false);
-  }
-
-  public LocatedBlockWritable(ExtendedBlockWritable eb) {
-    this(eb, new DatanodeInfoWritable[0], 0L, false);
-  }
-  
-  public LocatedBlockWritable(ExtendedBlockWritable b, DatanodeInfoWritable[] locs) {
-    this(b, locs, -1, false); // startOffset is unknown
-  }
-
-  public LocatedBlockWritable(ExtendedBlockWritable b, DatanodeInfoWritable[] locs, long startOffset) {
-    this(b, locs, startOffset, false);
-  }
-
-  public LocatedBlockWritable(ExtendedBlockWritable b, DatanodeInfoWritable[] locs, long startOffset, 
-                      boolean corrupt) {
-    this.b = b;
-    this.offset = startOffset;
-    this.corrupt = corrupt;
-    if (locs==null) {
-      this.locs = new DatanodeInfoWritable[0];
-    } else {
-      this.locs = locs;
-    }
-  }
-
-  public TokenWritable getBlockToken() {
-    return blockToken;
-  }
-
-  public void setBlockToken(TokenWritable token) {
-    this.blockToken = token;
-  }
-
-  public ExtendedBlockWritable getBlock() {
-    return b;
-  }
-
-  public DatanodeInfoWritable[] getLocations() {
-    return locs;
-  }
-  
-  public long getStartOffset() {
-    return offset;
-  }
-  
-  public long getBlockSize() {
-    return b.getNumBytes();
-  }
-
-  void setStartOffset(long value) {
-    this.offset = value;
-  }
-
-  void setCorrupt(boolean corrupt) {
-    this.corrupt = corrupt;
-  }
-  
-  public boolean isCorrupt() {
-    return this.corrupt;
-  }
-
-  ///////////////////////////////////////////
-  // Writable
-  ///////////////////////////////////////////
-  @Override
-  public void write(DataOutput out) throws IOException {
-    blockToken.write(out);
-    out.writeBoolean(corrupt);
-    out.writeLong(offset);
-    b.write(out);
-    out.writeInt(locs.length);
-    for (int i = 0; i < locs.length; i++) {
-      locs[i].write(out);
-    }
-  }
-
-  @Override
-  public void readFields(DataInput in) throws IOException {
-    blockToken.readFields(in);
-    this.corrupt = in.readBoolean();
-    offset = in.readLong();
-    this.b = new ExtendedBlockWritable();
-    b.readFields(in);
-    int count = in.readInt();
-    this.locs = new DatanodeInfoWritable[count];
-    for (int i = 0; i < locs.length; i++) {
-      locs[i] = new DatanodeInfoWritable();
-      locs[i].readFields(in);
-    }
-  }
-
-  /** Read LocatedBlock from in. */
-  public static LocatedBlockWritable read(DataInput in) throws IOException {
-    final LocatedBlockWritable lb = new LocatedBlockWritable();
-    lb.readFields(in);
-    return lb;
-  }
-
-  @Override
-  public String toString() {
-    return getClass().getSimpleName() + "{" + b
-        + "; getBlockSize()=" + getBlockSize()
-        + "; corrupt=" + corrupt
-        + "; offset=" + offset
-        + "; locs=" + java.util.Arrays.asList(locs)
-        + "}";
-  }
-}

+ 0 - 197
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/LocatedBlocksWritable.java

@@ -1,197 +0,0 @@
-/**
- * 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 java.util.ArrayList;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.io.WritableFactories;
-import org.apache.hadoop.io.WritableFactory;
-
-/**
- * Collection of blocks with their locations and the file length.
- */
-@InterfaceAudience.Private
-@InterfaceStability.Stable
-public class LocatedBlocksWritable implements Writable {
-  private long fileLength;
-  private List<LocatedBlockWritable> blocks; // array of blocks with prioritized locations
-  private boolean underConstruction;
-  private LocatedBlockWritable lastLocatedBlock = null;
-  private boolean isLastBlockComplete = false;
-
-  public static org.apache.hadoop.hdfs.protocol.LocatedBlocks convertLocatedBlocks(
-      LocatedBlocksWritable lb) {
-    if (lb == null) {
-      return null;
-    }
-    return new org.apache.hadoop.hdfs.protocol.LocatedBlocks(
-        lb.getFileLength(), lb.isUnderConstruction(),
-        LocatedBlockWritable.convertLocatedBlock(lb.getLocatedBlocks()),
-        LocatedBlockWritable.convertLocatedBlock(lb.getLastLocatedBlock()),
-        lb.isLastBlockComplete());
-  }
-  
-  public static LocatedBlocksWritable convertLocatedBlocks(
-      org.apache.hadoop.hdfs.protocol.LocatedBlocks lb) {
-    if (lb == null) {
-      return null;
-    }
-    return new LocatedBlocksWritable(lb.getFileLength(), lb.isUnderConstruction(),
-        LocatedBlockWritable.convertLocatedBlock2(lb.getLocatedBlocks()),
-        LocatedBlockWritable.convertLocatedBlock(lb.getLastLocatedBlock()),
-        lb.isLastBlockComplete());
-  }
-  
-  public LocatedBlocksWritable() {
-    this(0, false, null, null, false);
-  }
-  
-  /** public Constructor */
-  public LocatedBlocksWritable(long flength, boolean isUnderConstuction,
-      List<LocatedBlockWritable> blks, 
-      LocatedBlockWritable lastBlock, boolean isLastBlockCompleted) {
-    fileLength = flength;
-    blocks = blks;
-    underConstruction = isUnderConstuction;
-    this.lastLocatedBlock = lastBlock;
-    this.isLastBlockComplete = isLastBlockCompleted;
-  }
-  
-  /**
-   * Get located blocks.
-   */
-  public List<LocatedBlockWritable> getLocatedBlocks() {
-    return blocks;
-  }
-  
-  /** Get the last located block. */
-  public LocatedBlockWritable getLastLocatedBlock() {
-    return lastLocatedBlock;
-  }
-  
-  /** Is the last block completed? */
-  public boolean isLastBlockComplete() {
-    return isLastBlockComplete;
-  }
-
-  /**
-   * Get located block.
-   */
-  public LocatedBlockWritable get(int index) {
-    return blocks.get(index);
-  }
-  
-  /**
-   * Get number of located blocks.
-   */
-  public int locatedBlockCount() {
-    return blocks == null ? 0 : blocks.size();
-  }
-
-  /**
-   * Get file length
-   */
-  public long getFileLength() {
-    return this.fileLength;
-  }
-
-  /**
-   * Return ture if file was under construction when 
-   * this LocatedBlocks was constructed, false otherwise.
-   */
-  public boolean isUnderConstruction() {
-    return underConstruction;
-  }
-
-  //////////////////////////////////////////////////
-  // Writable
-  //////////////////////////////////////////////////
-  static {                                      // register a ctor
-    WritableFactories.setFactory
-      (LocatedBlocksWritable.class,
-       new WritableFactory() {
-         public Writable newInstance() { return new LocatedBlocksWritable(); }
-       });
-  }
-
-  @Override
-  public void write(DataOutput out) throws IOException {
-    out.writeLong(this.fileLength);
-    out.writeBoolean(underConstruction);
-
-    //write the last located block
-    final boolean isNull = lastLocatedBlock == null;
-    out.writeBoolean(isNull);
-    if (!isNull) {
-      lastLocatedBlock.write(out);
-    }
-    out.writeBoolean(isLastBlockComplete);
-
-    // write located blocks
-    int nrBlocks = locatedBlockCount();
-    out.writeInt(nrBlocks);
-    if (nrBlocks == 0) {
-      return;
-    }
-    for (LocatedBlockWritable blk : this.blocks) {
-      blk.write(out);
-    }
-  }
-  
-  @Override
-  public void readFields(DataInput in) throws IOException {
-    this.fileLength = in.readLong();
-    underConstruction = in.readBoolean();
-
-    //read the last located block
-    final boolean isNull = in.readBoolean();
-    if (!isNull) {
-      lastLocatedBlock = LocatedBlockWritable.read(in);
-    }
-    isLastBlockComplete = in.readBoolean();
-
-    // read located blocks
-    int nrBlocks = in.readInt();
-    this.blocks = new ArrayList<LocatedBlockWritable>(nrBlocks);
-    for (int idx = 0; idx < nrBlocks; idx++) {
-      LocatedBlockWritable blk = new LocatedBlockWritable();
-      blk.readFields(in);
-      this.blocks.add(blk);
-    }
-  }
-
-  @Override
-  public String toString() {
-    final StringBuilder b = new StringBuilder(getClass().getSimpleName());
-    b.append("{")
-     .append("\n  fileLength=").append(fileLength)
-     .append("\n  underConstruction=").append(underConstruction)
-     .append("\n  blocks=").append(blocks)
-     .append("\n  lastLocatedBlock=").append(lastLocatedBlock)
-     .append("\n  isLastBlockComplete=").append(isLastBlockComplete)
-     .append("}");
-    return b.toString();
-  }
-}

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

@@ -1,70 +0,0 @@
-/**
- * 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);
-  }
-}

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

@@ -1,163 +0,0 @@
-/**
- * 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));
-  }
-}

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

@@ -1,180 +0,0 @@
-/**
- * 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 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 {
-    rpcProxy = createNamenodeWithRetry(createNamenode(nameNodeAddr, conf, ugi));
-  }
-
-  public void close() {
-    RPC.stopProxy(rpcProxy);
-  }
-
-  @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();
-  }
-}

+ 0 - 98
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/NamenodeRegistrationWritable.java

@@ -1,98 +0,0 @@
-/**
- * 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.io.Text;
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.io.WritableFactories;
-import org.apache.hadoop.io.WritableFactory;
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
-import org.apache.hadoop.hdfs.server.common.StorageInfo;
-import org.apache.hadoop.hdfs.server.protocol.NamenodeRegistration;
-
-/**
- * Information sent by a subordinate name-node to the active name-node
- * during the registration process. 
- */
-@InterfaceAudience.Private
-@InterfaceStability.Evolving
-public class NamenodeRegistrationWritable implements Writable {
-  private String rpcAddress;    // RPC address of the node
-  private String httpAddress;   // HTTP address of the node
-  private NamenodeRole role;    // node role
-  private StorageInfoWritable storageInfo;
-
-  public NamenodeRegistrationWritable() { }
-
-  public NamenodeRegistrationWritable(String address,
-                              String httpAddress,
-                              NamenodeRole role,
-                              StorageInfo storageInfo) {
-    this.rpcAddress = address;
-    this.httpAddress = httpAddress;
-    this.role = role;
-    this.storageInfo = StorageInfoWritable.convert(storageInfo);
-  }
-
-  /////////////////////////////////////////////////
-  // Writable
-  /////////////////////////////////////////////////
-  static {
-    WritableFactories.setFactory
-      (NamenodeRegistrationWritable.class,
-       new WritableFactory() {
-          public Writable newInstance() {
-            return new NamenodeRegistrationWritable();
-          }
-       });
-  }
-
-  @Override // Writable
-  public void write(DataOutput out) throws IOException {
-    Text.writeString(out, rpcAddress);
-    Text.writeString(out, httpAddress);
-    Text.writeString(out, role.name());
-    storageInfo.write(out);
-  }
-
-  @Override // Writable
-  public void readFields(DataInput in) throws IOException {
-    rpcAddress = Text.readString(in);
-    httpAddress = Text.readString(in);
-    role = NamenodeRole.valueOf(Text.readString(in));
-    storageInfo = new StorageInfoWritable();
-    storageInfo.readFields(in);
-  }
-
-  public static NamenodeRegistrationWritable convert(NamenodeRegistration reg) {
-    return new NamenodeRegistrationWritable(reg.getAddress(),
-        reg.getHttpAddress(), reg.getRole(), reg);
-  }
-
-  public NamenodeRegistration convert() {
-    return new NamenodeRegistration(rpcAddress, httpAddress,
-        storageInfo.convert(), role);
-  }
-}

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

@@ -1,169 +0,0 @@
-/**
- * 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;
-}
-

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

@@ -1,100 +0,0 @@
-/**
- * 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);
-  }
-}

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

@@ -1,90 +0,0 @@
-/**
- * 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);
-  }
-}

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

@@ -1,69 +0,0 @@
-/**
- * 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);
-  }
-}

+ 0 - 86
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/StorageInfoWritable.java

@@ -1,86 +0,0 @@
-/**
- * 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.common.StorageInfo;
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.io.WritableFactories;
-import org.apache.hadoop.io.WritableFactory;
-import org.apache.hadoop.io.WritableUtils;
-
-/**
- * Common writable class for storage information.
- */
-@InterfaceAudience.Private
-public class StorageInfoWritable implements Writable {
-  private int layoutVersion;
-  private int namespaceID;
-  private String clusterID;
-  private long cTime;
-  
-  public StorageInfoWritable () {
-    this(0, 0, "", 0L);
-  }
-  
-  public StorageInfoWritable(int layoutV, int nsID, String cid, long cT) {
-    layoutVersion = layoutV;
-    clusterID = cid;
-    namespaceID = nsID;
-    cTime = cT;
-  }
-  
-  /////////////////////////////////////////////////
-  // Writable
-  /////////////////////////////////////////////////
-  static {
-    WritableFactories.setFactory(StorageInfoWritable.class,
-        new WritableFactory() {
-          public Writable newInstance() {
-            return new StorageInfoWritable();
-          }
-        });
-  }
-  
-  public void write(DataOutput out) throws IOException {
-    out.writeInt(layoutVersion);
-    out.writeInt(namespaceID);
-    WritableUtils.writeString(out, clusterID);
-    out.writeLong(cTime);
-  }
-
-  public void readFields(DataInput in) throws IOException {
-    layoutVersion = in.readInt();
-    namespaceID = in.readInt();
-    clusterID = WritableUtils.readString(in);
-    cTime = in.readLong();
-  }
-
-  public StorageInfo convert() {
-    return new StorageInfo(layoutVersion, namespaceID, clusterID, cTime);
-  }
-  
-  public static StorageInfoWritable convert(StorageInfo from) {
-    return new StorageInfoWritable(from.getLayoutVersion(),
-        from.getNamespaceID(), from.getClusterID(), from.getCTime());
-  }
-}

+ 0 - 208
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/TokenWritable.java

@@ -1,208 +0,0 @@
-/**
- * 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.commons.codec.binary.Base64;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.io.DataInputBuffer;
-import org.apache.hadoop.io.DataOutputBuffer;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.io.WritableUtils;
-
-/**
- * The client-side form of the token.
- */
-@InterfaceAudience.LimitedPrivate({"HDFS", "MapReduce"})
-@InterfaceStability.Stable
-public class TokenWritable implements Writable {
-  private byte[] identifier;
-  private byte[] password;
-  private Text kind;
-  private Text service;
- 
-  /**
-   * Construct a token from the components.
-   * @param identifier the token identifier
-   * @param password the token's password
-   * @param kind the kind of token
-   * @param service the service for this token
-   */
-  public TokenWritable(byte[] identifier, byte[] password, Text kind, Text service) {
-    this.identifier = identifier;
-    this.password = password;
-    this.kind = kind;
-    this.service = service;
-  }
-
-  /**
-   * Default constructor
-   */
-  public TokenWritable() {
-    this(new byte[0], new byte[0], new Text(), new Text());
-  }
-
-  /**
-   * Get the token identifier
-   * @return the token identifier
-   */
-  public byte[] getIdentifier() {
-    return identifier;
-  }
-  
-  /**
-   * Get the token password/secret
-   * @return the token password/secret
-   */
-  public byte[] getPassword() {
-    return password;
-  }
-  
-  /**
-   * Get the token kind
-   * @return the kind of the token
-   */
-  public Text getKind() {
-    return kind;
-  }
-
-  /**
-   * Get the service on which the token is supposed to be used
-   * @return the service name
-   */
-  public Text getService() {
-    return service;
-  }
-  
-  /**
-   * Set the service on which the token is supposed to be used
-   * @param newService the service name
-   */
-  public void setService(Text newService) {
-    service = newService;
-  }
-
-  @Override
-  public void readFields(DataInput in) throws IOException {
-    int len = WritableUtils.readVInt(in);
-    if (identifier == null || identifier.length != len) {
-      identifier = new byte[len];
-    }
-    in.readFully(identifier);
-    len = WritableUtils.readVInt(in);
-    if (password == null || password.length != len) {
-      password = new byte[len];
-    }
-    in.readFully(password);
-    kind.readFields(in);
-    service.readFields(in);
-  }
-
-  @Override
-  public void write(DataOutput out) throws IOException {
-    WritableUtils.writeVInt(out, identifier.length);
-    out.write(identifier);
-    WritableUtils.writeVInt(out, password.length);
-    out.write(password);
-    kind.write(out);
-    service.write(out);
-  }
-
-  /**
-   * Generate a string with the url-quoted base64 encoded serialized form
-   * of the Writable.
-   * @param obj the object to serialize
-   * @return the encoded string
-   * @throws IOException
-   */
-  private static String encodeWritable(Writable obj) throws IOException {
-    DataOutputBuffer buf = new DataOutputBuffer();
-    obj.write(buf);
-    Base64 encoder = new Base64(0, null, true);
-    byte[] raw = new byte[buf.getLength()];
-    System.arraycopy(buf.getData(), 0, raw, 0, buf.getLength());
-    return encoder.encodeToString(raw);
-  }
-  
-  /**
-   * Modify the writable to the value from the newValue
-   * @param obj the object to read into
-   * @param newValue the string with the url-safe base64 encoded bytes
-   * @throws IOException
-   */
-  private static void decodeWritable(Writable obj, 
-                                     String newValue) throws IOException {
-    Base64 decoder = new Base64(0, null, true);
-    DataInputBuffer buf = new DataInputBuffer();
-    byte[] decoded = decoder.decode(newValue);
-    buf.reset(decoded, decoded.length);
-    obj.readFields(buf);
-  }
-
-  /**
-   * Encode this token as a url safe string
-   * @return the encoded string
-   * @throws IOException
-   */
-  public String encodeToUrlString() throws IOException {
-    return encodeWritable(this);
-  }
-  
-  /**
-   * Decode the given url safe string into this token.
-   * @param newValue the encoded string
-   * @throws IOException
-   */
-  public void decodeFromUrlString(String newValue) throws IOException {
-    decodeWritable(this, newValue);
-  }
-  
-  private static void addBinaryBuffer(StringBuilder buffer, byte[] bytes) {
-    for (int idx = 0; idx < bytes.length; idx++) {
-      // if not the first, put a blank separator in
-      if (idx != 0) {
-        buffer.append(' ');
-      }
-      String num = Integer.toHexString(0xff & bytes[idx]);
-      // if it is only one digit, add a leading 0.
-      if (num.length() < 2) {
-        buffer.append('0');
-      }
-      buffer.append(num);
-    }
-  }
-
-  @Override
-  public String toString() {
-    StringBuilder buffer = new StringBuilder();
-    buffer.append("Ident: ");
-    addBinaryBuffer(buffer, identifier);
-    buffer.append(", Kind: ");
-    buffer.append(kind.toString());
-    buffer.append(", Service: ");
-    buffer.append(service.toString());
-    return buffer.toString();
-  }
-}

+ 0 - 140
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/UpgradeStatusReportWritable.java

@@ -1,140 +0,0 @@
-/**
- * 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.io.Writable;
-import org.apache.hadoop.io.WritableFactories;
-import org.apache.hadoop.io.WritableFactory;
-
-/**
- * Base upgrade upgradeStatus class.
- * 
- * Describes status of current upgrade.
- */
-@InterfaceAudience.Private
-@InterfaceStability.Stable
-public class UpgradeStatusReportWritable implements Writable {
-  protected int version;
-  protected short upgradeStatus;
-  protected boolean finalized;
-  
-  public static UpgradeStatusReportWritable convert(
-      org.apache.hadoop.hdfs.server.common.UpgradeStatusReport r) {
-    if (r == null) return null;
-    return new UpgradeStatusReportWritable(
-        r.getVersion(), r.getUpgradeStatus(), r.isFinalized());
-  }
-  
-  public static org.apache.hadoop.hdfs.server.common.UpgradeStatusReport 
-    convert(UpgradeStatusReportWritable r) {
-    if (r == null) return null;
-    return new org.apache.hadoop.hdfs.server.common.UpgradeStatusReport(
-        r.getVersion(), r.getUpgradeStatus(), r.isFinalized());
-  }
-  
-  public UpgradeStatusReportWritable() {
-    this(0, (short)0, false);
-  }
-
-  public UpgradeStatusReportWritable(int version, short status, boolean isFinalized) {
-    this.version = version;
-    this.upgradeStatus = status;
-    this.finalized = isFinalized;
-  }
-
-  /**
-   * Get the layout version of the currently running upgrade.
-   * @return layout version
-   */
-  public int getVersion() {
-    return this.version;
-  }
-
-  /**
-   * Get upgrade upgradeStatus as a percentage of the total upgrade done.
-   */ 
-  public short getUpgradeStatus() {
-    return upgradeStatus;
-  }
-
-  /**
-   * Is current upgrade finalized.
-   * @return true if finalized or false otherwise.
-   */
-  public boolean isFinalized() {
-    return this.finalized;
-  }
-
-  /**
-   * Get upgradeStatus data as a text for reporting.
-   * Should be overloaded for a particular upgrade specific upgradeStatus data.
-   * 
-   * @param details true if upgradeStatus details need to be included, 
-   *                false otherwise
-   * @return text
-   */
-  public String getStatusText(boolean details) {
-    return "Upgrade for version " + getVersion() 
-            + (upgradeStatus<100 ? 
-              " is in progress. Status = " + upgradeStatus + "%" : 
-              " has been completed."
-              + "\nUpgrade is " + (finalized ? "" : "not ")
-              + "finalized.");
-  }
-
-  /**
-   * Print basic upgradeStatus details.
-   */
-  @Override
-  public String toString() {
-    return getStatusText(false);
-  }
-
-  /////////////////////////////////////////////////
-  // Writable
-  /////////////////////////////////////////////////
-  static {                                      // register a ctor
-    WritableFactories.setFactory
-      (UpgradeStatusReportWritable.class,
-       new WritableFactory() {
-         public Writable newInstance() { return new UpgradeStatusReportWritable(); }
-       });
-  }
-
-  
-  // Note when upgrade has been finalized then the NN always
-  // returns a null as the report. 
-  // hence the isFinalized is serialized (ugly)
-  @Override
-  public void write(DataOutput out) throws IOException {
-    out.writeInt(this.version);
-    out.writeShort(this.upgradeStatus);
-  }
-
-  @Override
-  public void readFields(DataInput in) throws IOException {
-    this.version = in.readInt();
-    this.upgradeStatus = in.readShort();
-  }
-}

+ 5 - 6
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeProtocol.java

@@ -25,8 +25,6 @@ import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
-import org.apache.hadoop.hdfs.protocolR23Compatible.ClientNamenodeWireProtocol;
-import org.apache.hadoop.hdfs.server.protocolR23Compatible.DatanodeWireProtocol;
 import org.apache.hadoop.ipc.VersionedProtocol;
 import org.apache.hadoop.security.KerberosInfo;
 
@@ -48,10 +46,11 @@ public interface DatanodeProtocol extends VersionedProtocol {
    * to insulate from the protocol serialization.
    * 
    * If you are adding/changing DN's interface then you need to 
-   * change both this class and ALSO
-   * {@link DatanodeWireProtocol}.
-   * These changes need to be done in a compatible fashion as described in 
-   * {@link ClientNamenodeWireProtocol}
+   * change both this class and ALSO related protocol buffer
+   * wire protocol definition in DatanodeProtocol.proto.
+   * 
+   * For more details on protocol buffer wire protocol, please see 
+   * .../org/apache/hadoop/hdfs/protocolPB/overview.html
    */
   public static final long versionID = 28L;
   

+ 5 - 6
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/InterDatanodeProtocol.java

@@ -25,9 +25,7 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
-import org.apache.hadoop.hdfs.protocolR23Compatible.ClientNamenodeWireProtocol;
 import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringBlock;
-import org.apache.hadoop.hdfs.server.protocolR23Compatible.InterDatanodeWireProtocol;
 import org.apache.hadoop.ipc.VersionedProtocol;
 import org.apache.hadoop.security.KerberosInfo;
 
@@ -52,10 +50,11 @@ public interface InterDatanodeProtocol extends VersionedProtocol {
    * serialization.
    * 
    * If you are adding/changing DN's interface then you need to 
-   * change both this class and ALSO
-   * {@link InterDatanodeWireProtocol}
-   * These changes need to be done in a compatible fashion as described in 
-   * {@link ClientNamenodeWireProtocol}
+   * change both this class and ALSO related protocol buffer
+   * wire protocol definition in InterDatanodeProtocol.proto.
+   * 
+   * For more details on protocol buffer wire protocol, please see 
+   * .../org/apache/hadoop/hdfs/protocolPB/overview.html
    * 
    * The log of historical changes can be retrieved from the svn).
    * 6: Add block pool ID to Block

+ 5 - 6
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/JournalProtocol.java

@@ -21,8 +21,6 @@ import java.io.IOException;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
-import org.apache.hadoop.hdfs.protocolR23Compatible.ClientNamenodeWireProtocol;
-import org.apache.hadoop.hdfs.protocolR23Compatible.JournalWireProtocol;
 import org.apache.hadoop.ipc.VersionedProtocol;
 import org.apache.hadoop.security.KerberosInfo;
 
@@ -41,10 +39,11 @@ public interface JournalProtocol extends VersionedProtocol {
    * to insulate from the protocol serialization.
    * 
    * If you are adding/changing DN's interface then you need to 
-   * change both this class and ALSO
-   * {@link JournalWireProtocol}.
-   * These changes need to be done in a compatible fashion as described in 
-   * {@link ClientNamenodeWireProtocol}
+   * change both this class and ALSO related protocol buffer
+   * wire protocol definition in JournalProtocol.proto.
+   * 
+   * For more details on protocol buffer wire protocol, please see 
+   * .../org/apache/hadoop/hdfs/protocolPB/overview.html
    */
   public static final long versionID = 1L;
 

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

@@ -49,10 +49,11 @@ public interface NamenodeProtocol extends VersionedProtocol {
    * 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}
+   * change both this class and ALSO related protocol buffer
+   * wire protocol definition in NamenodeProtocol.proto.
+   * 
+   * For more details on protocol buffer wire protocol, please see 
+   * .../org/apache/hadoop/hdfs/protocolPB/overview.html
    * 
    * 6: Switch to txid-based file naming for image and edits
    */

+ 0 - 107
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocolR23Compatible/BalancerBandwidthCommandWritable.java

@@ -1,107 +0,0 @@
-/**
- * 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.server.protocolR23Compatible;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-
-import org.apache.hadoop.hdfs.server.protocol.BalancerBandwidthCommand;
-import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.io.WritableFactories;
-import org.apache.hadoop.io.WritableFactory;
-
-/**
- * Balancer bandwidth command instructs each datanode to change its value for
- * the max amount of network bandwidth it may use during the block balancing
- * operation.
- * 
- * The Balancer Bandwidth Command contains the new bandwidth value as its
- * payload. The bandwidth value is in bytes per second.
- */
-public class BalancerBandwidthCommandWritable extends DatanodeCommandWritable {
-  private final static long BBC_DEFAULTBANDWIDTH = 0L;
-
-  private long bandwidth;
-
-  /**
-   * Balancer Bandwidth Command constructor. Sets bandwidth to 0.
-   */
-  BalancerBandwidthCommandWritable() {
-    this(BBC_DEFAULTBANDWIDTH);
-  }
-
-  /**
-   * Balancer Bandwidth Command constructor.
-   * @param bandwidth Blanacer bandwidth in bytes per second.
-   */
-  public BalancerBandwidthCommandWritable(long bandwidth) {
-    super(DatanodeWireProtocol.DNA_BALANCERBANDWIDTHUPDATE);
-    this.bandwidth = bandwidth;
-  }
-
-  /**
-   * Get current value of the max balancer bandwidth in bytes per second.
-   * @return bandwidth Blanacer bandwidth in bytes per second for this datanode.
-   */
-  public long getBalancerBandwidthValue() {
-    return this.bandwidth;
-  }
-
-  // ///////////////////////////////////////////////
-  // Writable
-  // ///////////////////////////////////////////////
-  static { // register a ctor
-    WritableFactories.setFactory(BalancerBandwidthCommandWritable.class,
-        new WritableFactory() {
-          public Writable newInstance() {
-            return new BalancerBandwidthCommandWritable();
-          }
-        });
-  }
-
-  /**
-   * Writes the bandwidth payload to the Balancer Bandwidth Command packet.
-   * @param out DataOutput stream used for writing commands to the datanode.
-   * @throws IOException
-   */
-  public void write(DataOutput out) throws IOException {
-    super.write(out);
-    out.writeLong(this.bandwidth);
-  }
-
-  /**
-   * Reads the bandwidth payload from the Balancer Bandwidth Command packet.
-   * @param in DataInput stream used for reading commands to the datanode.
-   * @throws IOException
-   */
-  public void readFields(DataInput in) throws IOException {
-    super.readFields(in);
-    this.bandwidth = in.readLong();
-  }
-
-  @Override
-  public DatanodeCommand convert() {
-    return new BalancerBandwidthCommand(bandwidth);
-  }
-
-  public static DatanodeCommandWritable convert(BalancerBandwidthCommand cmd) {
-    return new BalancerBandwidthCommandWritable(cmd.getBalancerBandwidthValue());
-  }
-}

+ 0 - 142
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocolR23Compatible/BlockCommandWritable.java

@@ -1,142 +0,0 @@
-/**
- * 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.server.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.DatanodeInfo;
-import org.apache.hadoop.hdfs.protocolR23Compatible.BlockWritable;
-import org.apache.hadoop.hdfs.protocolR23Compatible.DatanodeInfoWritable;
-import org.apache.hadoop.hdfs.server.protocol.BlockCommand;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.io.WritableFactories;
-import org.apache.hadoop.io.WritableFactory;
-
-/****************************************************
- * A BlockCommand is an instruction to a datanode regarding some blocks under
- * its control. It tells the DataNode to either invalidate a set of indicated
- * blocks, or to copy a set of indicated blocks to another DataNode.
- * 
- ****************************************************/
-@InterfaceAudience.Private
-@InterfaceStability.Evolving
-public class BlockCommandWritable extends DatanodeCommandWritable {
-
-  /**
-   * This constant is used to indicate that the block deletion does not need
-   * explicit ACK from the datanode. When a block is put into the list of blocks
-   * to be deleted, it's size is set to this constant. We assume that no block
-   * would actually have this size. Otherwise, we would miss ACKs for blocks
-   * with such size. Positive number is used for compatibility reasons.
-   */
-  public static final long NO_ACK = Long.MAX_VALUE;
-
-  String poolId;
-  BlockWritable blocks[];
-  DatanodeInfoWritable targets[][];
-
-  public BlockCommandWritable() {
-  }
-
-  /**
-   * Create BlockCommand for the given action
-   * 
-   * @param blocks blocks related to the action
-   */
-  public BlockCommandWritable(int action, String poolId, BlockWritable[] blocks,
-      DatanodeInfoWritable[][] targets) {
-    super(action);
-    this.poolId = poolId;
-    this.blocks = blocks;
-    this.targets = targets;
-  }
-
-  // /////////////////////////////////////////
-  // Writable
-  // /////////////////////////////////////////
-  static { // register a ctor
-    WritableFactories.setFactory(BlockCommandWritable.class,
-        new WritableFactory() {
-          public Writable newInstance() {
-            return new BlockCommandWritable();
-          }
-        });
-  }
-
-  @Override
-  public void write(DataOutput out) throws IOException {
-    super.write(out);
-    Text.writeString(out, poolId);
-    out.writeInt(blocks.length);
-    for (int i = 0; i < blocks.length; i++) {
-      blocks[i].write(out);
-    }
-    out.writeInt(targets.length);
-    for (int i = 0; i < targets.length; i++) {
-      out.writeInt(targets[i].length);
-      for (int j = 0; j < targets[i].length; j++) {
-        targets[i][j].write(out);
-      }
-    }
-  }
-
-  @Override
-  public void readFields(DataInput in) throws IOException {
-    super.readFields(in);
-    this.poolId = Text.readString(in);
-    this.blocks = new BlockWritable[in.readInt()];
-    for (int i = 0; i < blocks.length; i++) {
-      blocks[i] = new BlockWritable();
-      blocks[i].readFields(in);
-    }
-
-    this.targets = new DatanodeInfoWritable[in.readInt()][];
-    for (int i = 0; i < targets.length; i++) {
-      this.targets[i] = new DatanodeInfoWritable[in.readInt()];
-      for (int j = 0; j < targets[i].length; j++) {
-        targets[i][j] = new DatanodeInfoWritable();
-        targets[i][j].readFields(in);
-      }
-    }
-  }
-
-  @Override
-  public BlockCommand convert() {
-    DatanodeInfo[][] dinfo = new DatanodeInfo[targets.length][];
-    for (int i = 0; i < targets.length; i++) {
-      dinfo[i] = DatanodeInfoWritable.convertDatanodeInfo(targets[i]);
-    }
-    return new BlockCommand(getAction(), poolId, BlockWritable.convert(blocks),
-        dinfo);
-  }
-
-  public static BlockCommandWritable convert(BlockCommand cmd) {
-    if (cmd == null) return null;
-    DatanodeInfo[][] targets = cmd.getTargets();
-    DatanodeInfoWritable[][] dinfo = new DatanodeInfoWritable[targets.length][];
-    for (int i = 0; i < targets.length; i++) {
-      dinfo[i] = DatanodeInfoWritable.convertDatanodeInfo(targets[i]);
-    }
-    return new BlockCommandWritable(cmd.getAction(), cmd.getBlockPoolId(),
-        BlockWritable.convert(cmd.getBlocks()), dinfo);
-  }
-}

+ 0 - 118
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocolR23Compatible/BlockRecoveryCommandWritable.java

@@ -1,118 +0,0 @@
-/**
- * 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.server.protocolR23Compatible;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collection;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand;
-import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringBlock;
-import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.io.WritableFactories;
-import org.apache.hadoop.io.WritableFactory;
-
-/**
- * BlockRecoveryCommand is an instruction to a data-node to recover the
- * specified blocks.
- * 
- * The data-node that receives this command treats itself as a primary data-node
- * in the recover process.
- * 
- * Block recovery is identified by a recoveryId, which is also the new
- * generation stamp, which the block will have after the recovery succeeds.
- */
-@InterfaceAudience.Private
-@InterfaceStability.Evolving
-public class BlockRecoveryCommandWritable extends DatanodeCommandWritable {
-  Collection<RecoveringBlockWritable> recoveringBlocks;
-
-  /**
-   * Create empty BlockRecoveryCommand.
-   */
-  public BlockRecoveryCommandWritable() { }
-
-  /**
-   * Create BlockRecoveryCommand with the specified capacity for recovering
-   * blocks.
-   */
-  public BlockRecoveryCommandWritable(int capacity) {
-    this(new ArrayList<RecoveringBlockWritable>(capacity));
-  }
-  
-  public BlockRecoveryCommandWritable(Collection<RecoveringBlockWritable> blocks) {
-    super(DatanodeWireProtocol.DNA_RECOVERBLOCK);
-    recoveringBlocks = blocks;
-  }
-
-  // /////////////////////////////////////////
-  // Writable
-  // /////////////////////////////////////////
-  static { // register a ctor
-    WritableFactories.setFactory(BlockRecoveryCommandWritable.class,
-        new WritableFactory() {
-          public Writable newInstance() {
-            return new BlockRecoveryCommandWritable();
-          }
-        });
-  }
-
-  public void write(DataOutput out) throws IOException {
-    super.write(out);
-    out.writeInt(recoveringBlocks.size());
-    for (RecoveringBlockWritable block : recoveringBlocks) {
-      block.write(out);
-    }
-  }
-
-  public void readFields(DataInput in) throws IOException {
-    super.readFields(in);
-    int numBlocks = in.readInt();
-    recoveringBlocks = new ArrayList<RecoveringBlockWritable>(numBlocks);
-    for (int i = 0; i < numBlocks; i++) {
-      RecoveringBlockWritable b = new RecoveringBlockWritable();
-      b.readFields(in);
-      recoveringBlocks.add(b);
-    }
-  }
-
-  @Override
-  public DatanodeCommand convert() {
-    Collection<RecoveringBlock> blks = 
-        new ArrayList<RecoveringBlock>(recoveringBlocks.size());
-    for (RecoveringBlockWritable b : recoveringBlocks) {
-      blks.add(b.convert());
-    }
-    return new BlockRecoveryCommand(blks);
-  }
-
-  public static BlockRecoveryCommandWritable convert(BlockRecoveryCommand cmd) {
-    if (cmd == null) return null;
-    Collection<RecoveringBlockWritable> blks = 
-        new ArrayList<RecoveringBlockWritable>(cmd.getRecoveringBlocks().size());
-    for (RecoveringBlock b : cmd.getRecoveringBlocks()) {
-      blks.add(RecoveringBlockWritable.convert(b));
-    }
-    return new BlockRecoveryCommandWritable(blks);
-  }
-}

+ 0 - 73
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocolR23Compatible/DatanodeCommandHelper.java

@@ -1,73 +0,0 @@
-/**
- * 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.server.protocolR23Compatible;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hdfs.server.protocol.BalancerBandwidthCommand;
-import org.apache.hadoop.hdfs.server.protocol.BlockCommand;
-import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand;
-import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
-import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
-import org.apache.hadoop.hdfs.server.protocol.FinalizeCommand;
-import org.apache.hadoop.hdfs.server.protocol.KeyUpdateCommand;
-import org.apache.hadoop.hdfs.server.protocol.UpgradeCommand;
-
-/**
- * Class for translating DatanodeCommandWritable to and from DatanodeCommand.
- */
-class DatanodeCommandHelper {
-  public static final Log LOG = LogFactory.getLog(DatanodeCommandHelper.class);
-  
-  private DatanodeCommandHelper() {
-    /* Private constructor to prevent instantiation */
-  }
-  
-  static DatanodeCommand convert(DatanodeCommandWritable cmd) {
-    return cmd.convert();
-  }
-  
-  /**
-   * Given a subclass of {@link DatanodeCommand} return the corresponding
-   * writable type.
-   */
-  static DatanodeCommandWritable convert(DatanodeCommand cmd) {
-    switch (cmd.getAction()) {
-    case DatanodeProtocol.DNA_BALANCERBANDWIDTHUPDATE:
-      return BalancerBandwidthCommandWritable
-          .convert((BalancerBandwidthCommand) cmd);
-    
-    case DatanodeProtocol.DNA_FINALIZE:
-      return FinalizeCommandWritable.convert((FinalizeCommand)cmd);
-    case DatanodeProtocol.DNA_ACCESSKEYUPDATE:
-      return KeyUpdateCommandWritable.convert((KeyUpdateCommand)cmd);
-    case DatanodeProtocol.DNA_REGISTER:
-      return RegisterCommandWritable.REGISTER;
-    case DatanodeProtocol.DNA_TRANSFER:
-    case DatanodeProtocol.DNA_INVALIDATE:
-      return BlockCommandWritable.convert((BlockCommand)cmd);
-    case UpgradeCommand.UC_ACTION_START_UPGRADE:
-      return UpgradeCommandWritable.convert((UpgradeCommand)cmd);
-    case DatanodeProtocol.DNA_RECOVERBLOCK:
-      return BlockRecoveryCommandWritable.convert((BlockRecoveryCommand)cmd);
-    default:
-      LOG.warn("Unknown DatanodeCommand action - " + cmd.getAction());
-      return null;
-    }
-  }
-}

+ 0 - 58
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocolR23Compatible/DatanodeCommandWritable.java

@@ -1,58 +0,0 @@
-/**
- * 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.server.protocolR23Compatible;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
-
-/**
- * Base class for data-node command.
- * Issued by the name-node to notify data-nodes what should be done.
- */
-@InterfaceAudience.Private
-@InterfaceStability.Evolving
-public abstract class DatanodeCommandWritable extends ServerCommandWritable {
-  public DatanodeCommandWritable() {
-    super();
-  }
-  
-  DatanodeCommandWritable(int action) {
-    super(action);
-  }
-
-  /** Method to convert from writable type to internal type */
-  public abstract DatanodeCommand convert();
-
-  public static DatanodeCommandWritable[] convert(DatanodeCommand[] cmds) {
-    DatanodeCommandWritable[] ret = new DatanodeCommandWritable[cmds.length];
-    for (int i = 0; i < cmds.length; i++) {
-      ret[i] = DatanodeCommandHelper.convert(cmds[i]);
-    }
-    return ret;
-  }
-
-  public static DatanodeCommand[] convert(DatanodeCommandWritable[] cmds) {
-    if (cmds == null) return null;
-    DatanodeCommand[] ret = new DatanodeCommand[cmds.length];
-    for (int i = 0; i < cmds.length; i++) {
-      ret[i] = cmds[i].convert();
-    }
-    return ret;
-  }
-}

+ 0 - 170
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocolR23Compatible/DatanodeProtocolServerSideTranslatorR23.java

@@ -1,170 +0,0 @@
-/**
- * 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.server.protocolR23Compatible;
-
-import java.io.IOException;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.hdfs.protocolR23Compatible.DatanodeIDWritable;
-import org.apache.hadoop.hdfs.protocolR23Compatible.ExtendedBlockWritable;
-import org.apache.hadoop.hdfs.protocolR23Compatible.LocatedBlockWritable;
-import org.apache.hadoop.hdfs.protocolR23Compatible.NamespaceInfoWritable;
-import org.apache.hadoop.hdfs.protocolR23Compatible.ProtocolSignatureWritable;
-import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
-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 native data types used inside the NN as specified in the generic
- * DatanodeProtocol.
- */
-@InterfaceAudience.Private
-@InterfaceStability.Stable
-public class DatanodeProtocolServerSideTranslatorR23 implements
-    DatanodeWireProtocol {
-  final private DatanodeProtocol server;
-
-  /**
-   * Constructor
-   * @param server - the NN server
-   * @throws IOException
-   */
-  public DatanodeProtocolServerSideTranslatorR23(DatanodeProtocol 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 DatanodeProtocol}
-     * 
-     */
-    if (!protocol.equals(RPC.getProtocolName(DatanodeWireProtocol.class))) {
-      throw new IOException("Namenode Serverside implements " +
-          RPC.getProtocolName(DatanodeWireProtocol.class) +
-          ". The following requested protocol is unknown: " + protocol);
-    }
-
-    return ProtocolSignature.getProtocolSignature(clientMethodsHash,
-        DatanodeWireProtocol.versionID, DatanodeWireProtocol.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 DatanodeProtocol}
-     */
-    return ProtocolSignatureWritable.convert(
-        this.getProtocolSignature(protocol, clientVersion, clientMethodsHash));
-  }
-
-  @Override
-  public long getProtocolVersion(String protocol, long clientVersion)
-      throws IOException {
-    if (protocol.equals(RPC.getProtocolName(DatanodeWireProtocol.class))) {
-      return DatanodeWireProtocol.versionID;
-    }
-    throw new IOException("Namenode Serverside implements " +
-        RPC.getProtocolName(DatanodeWireProtocol.class) +
-        ". The following requested protocol is unknown: " + protocol);
-  }
-
-  @Override
-  public DatanodeRegistrationWritable registerDatanode(
-      DatanodeRegistrationWritable registration) throws IOException {
-    return DatanodeRegistrationWritable.convert(server
-        .registerDatanode(registration.convert()));
-  }
-
-  @Override
-  public DatanodeCommandWritable[] sendHeartbeat(
-      DatanodeRegistrationWritable registration, long capacity, long dfsUsed,
-      long remaining, long blockPoolUsed, int xmitsInProgress,
-      int xceiverCount, int failedVolumes) throws IOException {
-    return DatanodeCommandWritable.convert(server.sendHeartbeat(
-        registration.convert(), capacity, dfsUsed, remaining, blockPoolUsed,
-        xmitsInProgress, xceiverCount, failedVolumes));
-  }
-
-  @Override
-  public DatanodeCommandWritable blockReport(
-      DatanodeRegistrationWritable registration, String poolId, long[] blocks)
-      throws IOException {
-    return DatanodeCommandHelper.convert(server.blockReport(
-        registration.convert(), poolId, blocks));
-  }
-
-  @Override
-  public void blockReceivedAndDeleted(
-      DatanodeRegistrationWritable registration, String poolId,
-      ReceivedDeletedBlockInfoWritable[] receivedAndDeletedBlocks)
-      throws IOException {
-    server.blockReceivedAndDeleted(registration.convert(), poolId,
-        ReceivedDeletedBlockInfoWritable.convert(receivedAndDeletedBlocks));
-  }
-
-  @Override
-  public void errorReport(DatanodeRegistrationWritable registration,
-      int errorCode, String msg) throws IOException {
-    server.errorReport(registration.convert(), errorCode, msg);
-  }
-
-  @Override
-  public NamespaceInfoWritable versionRequest() throws IOException {
-    return NamespaceInfoWritable.convert(server.versionRequest());
-  }
-
-  @Override
-  public UpgradeCommandWritable processUpgradeCommand(
-      UpgradeCommandWritable comm) throws IOException {
-    return UpgradeCommandWritable.convert(server.processUpgradeCommand(comm.convert()));
-  }
-
-  @Override
-  public void reportBadBlocks(LocatedBlockWritable[] blocks) throws IOException {
-    server.reportBadBlocks(LocatedBlockWritable.convertLocatedBlock(blocks));
-  }
-
-  @Override
-  public void commitBlockSynchronization(ExtendedBlockWritable block,
-      long newgenerationstamp, long newlength, boolean closeFile,
-      boolean deleteblock, DatanodeIDWritable[] newtargets) throws IOException {
-    server.commitBlockSynchronization(
-        ExtendedBlockWritable.convertExtendedBlock(block), newgenerationstamp,
-        newlength, closeFile, deleteblock,
-        DatanodeIDWritable.convertDatanodeID(newtargets));
-  }
-}

+ 0 - 193
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocolR23Compatible/DatanodeProtocolTranslatorR23.java

@@ -1,193 +0,0 @@
-/**
- * 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.server.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.DatanodeID;
-import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
-import org.apache.hadoop.hdfs.protocol.LocatedBlock;
-import org.apache.hadoop.hdfs.protocolR23Compatible.DatanodeIDWritable;
-import org.apache.hadoop.hdfs.protocolR23Compatible.ExtendedBlockWritable;
-import org.apache.hadoop.hdfs.protocolR23Compatible.LocatedBlockWritable;
-import org.apache.hadoop.hdfs.protocolR23Compatible.ProtocolSignatureWritable;
-import org.apache.hadoop.hdfs.server.namenode.NameNode;
-import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
-import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
-import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
-import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
-import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo;
-import org.apache.hadoop.hdfs.server.protocol.UpgradeCommand;
-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 ClientProtocol calls as RPC calls to the NN server
- * while translating from the parameter types used in ClientProtocol to those
- * used in protocolR23Compatile.*.
- */
-@InterfaceAudience.Private
-@InterfaceStability.Stable
-public class DatanodeProtocolTranslatorR23 implements
-    DatanodeProtocol, Closeable {
-  final private DatanodeWireProtocol rpcProxy;
-
-  private static DatanodeWireProtocol createNamenode(
-      InetSocketAddress nameNodeAddr, Configuration conf,
-      UserGroupInformation ugi) throws IOException {
-    return RPC.getProxy(DatanodeWireProtocol.class,
-        DatanodeWireProtocol.versionID, nameNodeAddr, ugi, conf,
-        NetUtils.getSocketFactory(conf, DatanodeWireProtocol.class));
-  }
-
-  /** Create a {@link NameNode} proxy */
-  static DatanodeWireProtocol createNamenodeWithRetry(
-      DatanodeWireProtocol 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 (DatanodeWireProtocol) RetryProxy.create(
-        DatanodeWireProtocol.class, rpcNamenode, methodNameToPolicyMap);
-  }
-
-  public DatanodeProtocolTranslatorR23(InetSocketAddress nameNodeAddr,
-      Configuration conf) throws IOException {
-    UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
-    rpcProxy = createNamenodeWithRetry(createNamenode(nameNodeAddr, conf, ugi));
-  }
-
-  @Override
-  public void close() {
-    RPC.stopProxy(rpcProxy);
-  }
-
-  @Override
-  public long getProtocolVersion(String protocolName, long clientVersion)
-      throws IOException {
-    return rpcProxy.getProtocolVersion(protocolName, clientVersion);
-  }
-
-  @Override
-  public ProtocolSignature getProtocolSignature(String protocol,
-      long clientVersion, int clientMethodsHash) throws IOException {
-    return ProtocolSignatureWritable.convert(rpcProxy.getProtocolSignature2(
-        protocol, clientVersion, clientMethodsHash));
-  }
-
-  @Override
-  public DatanodeRegistration registerDatanode(DatanodeRegistration registration)
-      throws IOException {
-    return rpcProxy.registerDatanode(
-        DatanodeRegistrationWritable.convert(registration)).convert();
-  }
-
-  @Override
-  public DatanodeCommand[] sendHeartbeat(DatanodeRegistration registration,
-      long capacity, long dfsUsed, long remaining, long blockPoolUsed,
-      int xmitsInProgress, int xceiverCount, int failedVolumes)
-      throws IOException {
-    return DatanodeCommandWritable.convert(rpcProxy.sendHeartbeat(
-            DatanodeRegistrationWritable.convert(registration), capacity,
-            dfsUsed, remaining, blockPoolUsed, xmitsInProgress, xceiverCount,
-            failedVolumes));
-  }
-
-  @Override
-  public DatanodeCommand blockReport(DatanodeRegistration registration,
-      String poolId, long[] blocks) throws IOException {
-    return rpcProxy.blockReport(
-        DatanodeRegistrationWritable.convert(registration), poolId, blocks)
-        .convert();
-  }
-
-  @Override
-  public void blockReceivedAndDeleted(DatanodeRegistration registration,
-      String poolId, ReceivedDeletedBlockInfo[] receivedAndDeletedBlocks)
-      throws IOException {
-    rpcProxy.blockReceivedAndDeleted(
-        DatanodeRegistrationWritable.convert(registration), poolId,
-        ReceivedDeletedBlockInfoWritable.convert(receivedAndDeletedBlocks));
-  }
-
-  @Override
-  public void errorReport(DatanodeRegistration registration, int errorCode,
-      String msg) throws IOException {
-    rpcProxy.errorReport(DatanodeRegistrationWritable.convert(registration),
-        errorCode, msg);
-  }
-
-  @Override
-  public NamespaceInfo versionRequest() throws IOException {
-    return rpcProxy.versionRequest().convert();
-  }
-
-  @Override
-  public UpgradeCommand processUpgradeCommand(UpgradeCommand cmd)
-      throws IOException {
-    return rpcProxy.processUpgradeCommand(UpgradeCommandWritable.convert(cmd))
-        .convert();
-  }
-
-  @Override
-  public void reportBadBlocks(LocatedBlock[] blocks) throws IOException {
-    rpcProxy.reportBadBlocks(LocatedBlockWritable.convertLocatedBlock(blocks));
-  }
-
-  @Override
-  public void commitBlockSynchronization(ExtendedBlock block,
-      long newgenerationstamp, long newlength, boolean closeFile,
-      boolean deleteblock, DatanodeID[] newtargets) throws IOException {
-    rpcProxy.commitBlockSynchronization(
-        ExtendedBlockWritable.convertExtendedBlock(block), newgenerationstamp,
-        newlength, closeFile, deleteblock,
-        DatanodeIDWritable.convertDatanodeID(newtargets));
-  }
-}

+ 0 - 113
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocolR23Compatible/DatanodeRegistrationWritable.java

@@ -1,113 +0,0 @@
-/**
- * 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.server.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.protocolR23Compatible.DatanodeIDWritable;
-import org.apache.hadoop.hdfs.protocolR23Compatible.ExportedBlockKeysWritable;
-import org.apache.hadoop.hdfs.protocolR23Compatible.StorageInfoWritable;
-import org.apache.hadoop.hdfs.security.token.block.ExportedBlockKeys;
-import org.apache.hadoop.hdfs.server.common.StorageInfo;
-import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.io.WritableFactories;
-import org.apache.hadoop.io.WritableFactory;
-
-/** 
- * DatanodeRegistration class contains all information the name-node needs
- * to identify and verify a data-node when it contacts the name-node.
- * This information is sent by data-node with each communication request.
- */
-@InterfaceAudience.Private
-@InterfaceStability.Evolving
-public class DatanodeRegistrationWritable implements Writable {
-  static {                                      // register a ctor
-    WritableFactories.setFactory
-      (DatanodeRegistrationWritable.class,
-       new WritableFactory() {
-         public Writable newInstance() { return new DatanodeRegistrationWritable(); }
-       });
-  }
-
-  private DatanodeIDWritable datanodeId;
-  private StorageInfoWritable storageInfo;
-  private ExportedBlockKeysWritable exportedKeys;
-
-  /**
-   * Default constructor.
-   */
-  public DatanodeRegistrationWritable() {
-    this("", new StorageInfo(), new ExportedBlockKeys());
-  }
-  
-  /**
-   * Create DatanodeRegistration
-   */
-  public DatanodeRegistrationWritable(String nodeName, StorageInfo info,
-      ExportedBlockKeys keys) {
-    this.datanodeId = new DatanodeIDWritable(nodeName);
-    this.storageInfo = StorageInfoWritable.convert(info);
-    this.exportedKeys = ExportedBlockKeysWritable.convert(keys);
-  }
-  
-  /////////////////////////////////////////////////
-  // Writable
-  /////////////////////////////////////////////////
-  @Override
-  public void write(DataOutput out) throws IOException {
-    datanodeId.write(out);
-
-    //TODO: move it to DatanodeID once HADOOP-2797 has been committed
-    out.writeShort(datanodeId.ipcPort);
-
-    storageInfo.write(out);
-    exportedKeys.write(out);
-  }
-
-  @Override
-  public void readFields(DataInput in) throws IOException {
-    datanodeId.readFields(in);
-
-    //TODO: move it to DatanodeID once HADOOP-2797 has been committed
-    datanodeId.ipcPort = in.readShort() & 0x0000ffff;
-
-    storageInfo.readFields(in);
-    exportedKeys.readFields(in);
-  }
-
-  public DatanodeRegistration convert() {
-    DatanodeRegistration dnReg = new DatanodeRegistration(datanodeId.name,
-        storageInfo.convert(), exportedKeys.convert());
-    dnReg.setIpcPort(datanodeId.ipcPort);
-    return dnReg;
-  }
-
-  public static DatanodeRegistrationWritable convert(DatanodeRegistration dnReg) {
-    if (dnReg == null) return null;
-    DatanodeRegistrationWritable ret = new DatanodeRegistrationWritable(
-        dnReg.getName(), dnReg.storageInfo, dnReg.exportedKeys);
-    ret.datanodeId.ipcPort = dnReg.ipcPort;
-    return ret;
-  }
-}

+ 0 - 181
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocolR23Compatible/DatanodeWireProtocol.java

@@ -1,181 +0,0 @@
-/**
- * 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.server.protocolR23Compatible;
-
-import java.io.IOException;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.hdfs.DFSConfigKeys;
-import org.apache.hadoop.hdfs.protocol.ClientProtocol;
-import org.apache.hadoop.hdfs.protocol.LocatedBlock;
-import org.apache.hadoop.hdfs.protocolR23Compatible.ClientNamenodeWireProtocol;
-import org.apache.hadoop.hdfs.protocolR23Compatible.DatanodeIDWritable;
-import org.apache.hadoop.hdfs.protocolR23Compatible.ExtendedBlockWritable;
-import org.apache.hadoop.hdfs.protocolR23Compatible.LocatedBlockWritable;
-import org.apache.hadoop.hdfs.protocolR23Compatible.NamespaceInfoWritable;
-import org.apache.hadoop.hdfs.protocolR23Compatible.ProtocolSignatureWritable;
-import org.apache.hadoop.ipc.VersionedProtocol;
-import org.apache.hadoop.security.KerberosInfo;
-
-/**********************************************************************
- * Protocol that a DFS datanode uses to communicate with the NameNode.
- * It's used to upload current load information and block reports.
- *
- * The only way a NameNode can communicate with a DataNode is by
- * returning values from these functions.
- *
- **********************************************************************/
-@KerberosInfo(
-    serverPrincipal = DFSConfigKeys.DFS_NAMENODE_USER_NAME_KEY, 
-    clientPrincipal = DFSConfigKeys.DFS_DATANODE_USER_NAME_KEY)
-@InterfaceAudience.Private
-public interface DatanodeWireProtocol 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 = 28L;
-  
-  // error code
-  final static int NOTIFY = 0;
-  final static int DISK_ERROR = 1; // there are still valid volumes on DN
-  final static int INVALID_BLOCK = 2;
-  final static int FATAL_DISK_ERROR = 3; // no valid volumes left on DN
-
-  /**
-   * Determines actions that data node should perform 
-   * when receiving a datanode command. 
-   */
-  final static int DNA_UNKNOWN = 0;    // unknown action   
-  final static int DNA_TRANSFER = 1;   // transfer blocks to another datanode
-  final static int DNA_INVALIDATE = 2; // invalidate blocks
-  final static int DNA_SHUTDOWN = 3;   // shutdown node
-  final static int DNA_REGISTER = 4;   // re-register
-  final static int DNA_FINALIZE = 5;   // finalize previous upgrade
-  final static int DNA_RECOVERBLOCK = 6;  // request a block recovery
-  final static int DNA_ACCESSKEYUPDATE = 7;  // update access key
-  final static int DNA_BALANCERBANDWIDTHUPDATE = 8; // update balancer bandwidth
-  
-  /** 
-   * Register Datanode.
-   * @return updated {@link DatanodeRegistrationWritable}, which contains 
-   * new storageID if the datanode did not have one and
-   * registration ID for further communication.
-   */
-  public DatanodeRegistrationWritable registerDatanode(
-      DatanodeRegistrationWritable registration) throws IOException;
-  /**
-   * sendHeartbeat() tells the NameNode that the DataNode is still
-   * alive and well.  Includes some status info, too. 
-   * It also gives the NameNode a chance to return 
-   * an array of "DatanodeCommand" objects.
-   * A DatanodeCommand tells the DataNode to invalidate local block(s), 
-   * or to copy them to other DataNodes, etc.
-   * @param registration datanode registration information
-   * @param capacity total storage capacity available at the datanode
-   * @param dfsUsed storage used by HDFS
-   * @param remaining remaining storage available for HDFS
-   * @param blockPoolUsed storage used by the block pool
-   * @param xmitsInProgress number of transfers from this datanode to others
-   * @param xceiverCount number of active transceiver threads
-   * @param failedVolumes number of failed volumes
-   * @throws IOException on error
-   */
-  public DatanodeCommandWritable[] sendHeartbeat(
-      DatanodeRegistrationWritable registration, long capacity, long dfsUsed,
-      long remaining, long blockPoolUsed, int xmitsInProgress,
-      int xceiverCount, int failedVolumes) throws IOException;
-
-  /**
-   * blockReport() tells the NameNode about all the locally-stored blocks.
-   * The NameNode returns an array of Blocks that have become obsolete
-   * and should be deleted.  This function is meant to upload *all*
-   * the locally-stored blocks.  It's invoked upon startup and then
-   * infrequently afterwards.
-   * @param registration
-   * @param poolId - the block pool ID for the blocks
-   * @param blocks - the block list as an array of longs.
-   *     Each block is represented as 2 longs.
-   *     This is done instead of Block[] to reduce memory used by block reports.
-   *     
-   * @return - the next command for DN to process.
-   * @throws IOException
-   */
-  public DatanodeCommandWritable blockReport(
-      DatanodeRegistrationWritable registration, String poolId, long[] blocks)
-      throws IOException;
-    
-  /**
-   * blockReceivedAndDeleted() allows the DataNode to tell the NameNode about
-   * recently-received and -deleted block data. 
-   * 
-   * For the case of received blocks, a hint for preferred replica to be 
-   * deleted when there is any excessive blocks is provided.
-   * For example, whenever client code
-   * writes a new Block here, or another DataNode copies a Block to
-   * this DataNode, it will call blockReceived().
-   */
-  public void blockReceivedAndDeleted(
-      DatanodeRegistrationWritable registration, String poolId,
-      ReceivedDeletedBlockInfoWritable[] receivedAndDeletedBlocks)
-      throws IOException;
-
-  /**
-   * errorReport() tells the NameNode about something that has gone
-   * awry.  Useful for debugging.
-   */
-  public void errorReport(DatanodeRegistrationWritable registration,
-      int errorCode, String msg) throws IOException;
-    
-  public NamespaceInfoWritable versionRequest() throws IOException;
-
-  /**
-   * This is a very general way to send a command to the name-node during
-   * distributed upgrade process.
-   * 
-   * The generosity is because the variety of upgrade commands is unpredictable.
-   * The reply from the name-node is also received in the form of an upgrade 
-   * command. 
-   * 
-   * @return a reply in the form of an upgrade command
-   */
-  UpgradeCommandWritable processUpgradeCommand(UpgradeCommandWritable comm)
-      throws IOException;
-  
-  /**
-   * same as {@link ClientProtocol#reportBadBlocks(LocatedBlock[])}
-   * }
-   */
-  public void reportBadBlocks(LocatedBlockWritable[] blocks) throws IOException;
-  
-  /**
-   * Commit block synchronization in lease recovery
-   */
-  public void commitBlockSynchronization(ExtendedBlockWritable block,
-      long newgenerationstamp, long newlength, boolean closeFile,
-      boolean deleteblock, DatanodeIDWritable[] newtargets) 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 ProtocolSignatureWritable getProtocolSignature2(String protocol,
-      long clientVersion, int clientMethodsHash) throws IOException;
-}

+ 0 - 88
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocolR23Compatible/FinalizeCommandWritable.java

@@ -1,88 +0,0 @@
-/**
- * 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.server.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.DatanodeCommand;
-import org.apache.hadoop.hdfs.server.protocol.FinalizeCommand;
-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 FinalizeCommand is an instruction from namenode to finalize the previous
- * upgrade to a datanode
- */
-@InterfaceAudience.Private
-@InterfaceStability.Evolving
-public class FinalizeCommandWritable extends DatanodeCommandWritable {
-  // /////////////////////////////////////////
-  // Writable
-  // /////////////////////////////////////////
-  static { // register a ctor
-    WritableFactories.setFactory(FinalizeCommandWritable.class,
-        new WritableFactory() {
-          public Writable newInstance() {
-            return new FinalizeCommandWritable();
-          }
-        });
-  }
-
-  String blockPoolId;
-
-  private FinalizeCommandWritable() {
-    this(null);
-  }
-
-  public FinalizeCommandWritable(String bpid) {
-    super(DatanodeWireProtocol.DNA_FINALIZE);
-    blockPoolId = bpid;
-  }
-
-  public String getBlockPoolId() {
-    return blockPoolId;
-  }
-
-  @Override
-  public void readFields(DataInput in) throws IOException {
-    blockPoolId = WritableUtils.readString(in);
-  }
-
-  @Override
-  public void write(DataOutput out) throws IOException {
-    WritableUtils.writeString(out, blockPoolId);
-  }
-
-  @Override
-  public DatanodeCommand convert() {
-    return new FinalizeCommand(blockPoolId);
-  }
-
-  public static FinalizeCommandWritable convert(FinalizeCommand cmd) {
-    if (cmd == null) {
-      return null;
-    }
-    return new FinalizeCommandWritable(cmd.getBlockPoolId());
-  }
-}

+ 0 - 116
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocolR23Compatible/InterDatanodeProtocolServerSideTranslatorR23.java

@@ -1,116 +0,0 @@
-/**
- * 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.server.protocolR23Compatible;
-
-import java.io.IOException;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
-import org.apache.hadoop.hdfs.protocolR23Compatible.ExtendedBlockWritable;
-import org.apache.hadoop.hdfs.protocolR23Compatible.ProtocolSignatureWritable;
-import org.apache.hadoop.hdfs.server.protocol.InterDatanodeProtocol;
-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 InterDatanodeProtocol.
- */
-@InterfaceAudience.Private
-@InterfaceStability.Stable
-public class InterDatanodeProtocolServerSideTranslatorR23 implements
-    InterDatanodeWireProtocol {
-  final private InterDatanodeProtocol server;
-
-  /**
-   * 
-   * @param server - datanode server
-   * @throws IOException
-   */
-  public InterDatanodeProtocolServerSideTranslatorR23(
-      InterDatanodeProtocol 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 InterDatanodeProtocol}
-     */
-    if (!protocol.equals(RPC.getProtocolName(InterDatanodeWireProtocol.class))) {
-      throw new IOException("Datanode Serverside implements "
-          + InterDatanodeWireProtocol.class
-          + ". The following requested protocol is unknown: " + protocol);
-    }
-
-    return ProtocolSignature.getProtocolSignature(clientMethodsHash,
-        InterDatanodeWireProtocol.versionID, InterDatanodeWireProtocol.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(InterDatanodeWireProtocol.class))) {
-      return InterDatanodeWireProtocol.versionID;
-    }
-    throw new IOException("Datanode Serverside implements "
-        + InterDatanodeWireProtocol.class
-        + ". The following requested protocol is unknown: " + protocol);
-  }
-
-  @Override
-  public ReplicaRecoveryInfoWritable initReplicaRecovery(
-      RecoveringBlockWritable rBlock) throws IOException {
-    return ReplicaRecoveryInfoWritable.convert(server
-        .initReplicaRecovery(rBlock.convert()));
-  }
-
-  @Override
-  public ExtendedBlockWritable updateReplicaUnderRecovery(
-      ExtendedBlockWritable oldBlock, long recoveryId, long newLength)
-      throws IOException {
-    ExtendedBlock b = ExtendedBlockWritable.convertExtendedBlock(oldBlock);
-    return ExtendedBlockWritable.convertExtendedBlock(server
-        .updateReplicaUnderRecovery(b, recoveryId, newLength));
-  }
-}

+ 0 - 96
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocolR23Compatible/InterDatanodeProtocolTranslatorR23.java

@@ -1,96 +0,0 @@
-/**
- * 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.server.protocolR23Compatible;
-
-import java.io.IOException;
-import java.net.InetSocketAddress;
-
-import javax.net.SocketFactory;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
-import org.apache.hadoop.hdfs.protocolR23Compatible.ExtendedBlockWritable;
-import org.apache.hadoop.hdfs.protocolR23Compatible.ProtocolSignatureWritable;
-import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringBlock;
-import org.apache.hadoop.hdfs.server.protocol.InterDatanodeProtocol;
-import org.apache.hadoop.hdfs.server.protocol.ReplicaRecoveryInfo;
-import org.apache.hadoop.ipc.ProtocolSignature;
-import org.apache.hadoop.ipc.RPC;
-import org.apache.hadoop.security.UserGroupInformation;
-
-/**
- * This class forwards InterDatanodeProtocol calls as RPC to the DN server while
- * translating from the parameter types used in InterDatanodeProtocol to those
- * used in protocolR23Compatile.*.
- */
-@InterfaceAudience.Private
-@InterfaceStability.Stable
-public class InterDatanodeProtocolTranslatorR23 implements
-    InterDatanodeProtocol {
-
-  final private InterDatanodeWireProtocol rpcProxy;
-
-  /** used for testing */
-  public InterDatanodeProtocolTranslatorR23(InetSocketAddress addr,
-      UserGroupInformation ugi, Configuration conf, SocketFactory factory,
-      int socketTimeout)
-      throws IOException {
-    rpcProxy = createInterDatanodeProtocolProxy(addr, ugi, conf, factory,
-        socketTimeout);
-  }
-
-  static InterDatanodeWireProtocol createInterDatanodeProtocolProxy(
-      InetSocketAddress addr, UserGroupInformation ugi, Configuration conf,
-      SocketFactory factory, int socketTimeout) throws IOException {
-    return RPC.getProxy(InterDatanodeWireProtocol.class,
-        InterDatanodeWireProtocol.versionID, addr, ugi, conf, factory,
-        socketTimeout);
-  }
-
-  @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 ReplicaRecoveryInfo initReplicaRecovery(RecoveringBlock rBlock)
-      throws IOException {
-    return rpcProxy
-        .initReplicaRecovery(RecoveringBlockWritable.convert(rBlock)).convert();
-  }
-
-  @Override
-  public ExtendedBlock updateReplicaUnderRecovery(ExtendedBlock oldBlock,
-      long recoveryId, long newLength) throws IOException {
-    ExtendedBlockWritable eb = ExtendedBlockWritable
-        .convertExtendedBlock(oldBlock);
-    ExtendedBlockWritable b = rpcProxy.updateReplicaUnderRecovery(eb,
-        recoveryId, newLength);
-    return ExtendedBlockWritable.convertExtendedBlock(b);
-  }
-}

+ 0 - 73
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocolR23Compatible/InterDatanodeWireProtocol.java

@@ -1,73 +0,0 @@
-/**
- * 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.server.protocolR23Compatible;
-
-import java.io.IOException;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.hdfs.DFSConfigKeys;
-import org.apache.hadoop.hdfs.protocolR23Compatible.ClientNamenodeWireProtocol;
-import org.apache.hadoop.hdfs.protocolR23Compatible.ExtendedBlockWritable;
-import org.apache.hadoop.hdfs.protocolR23Compatible.ProtocolSignatureWritable;
-import org.apache.hadoop.ipc.VersionedProtocol;
-import org.apache.hadoop.security.KerberosInfo;
-
-/** An inter-datanode protocol for updating generation stamp
- */
-@KerberosInfo(
-    serverPrincipal = DFSConfigKeys.DFS_DATANODE_USER_NAME_KEY,
-    clientPrincipal = DFSConfigKeys.DFS_DATANODE_USER_NAME_KEY)
-@InterfaceAudience.Private
-public interface InterDatanodeWireProtocol extends VersionedProtocol {
-  public static final Log LOG = 
-      LogFactory.getLog(InterDatanodeWireProtocol.class);
-  /**
-   * The  rules for changing this protocol are the same as that for
-   * {@link ClientNamenodeWireProtocol} - see that java file for details.
-   * 6: Add block pool ID to Block
-   */
-  public static final long versionID = 6L;
-
-  /**
-   * Initialize a replica recovery.
-   * 
-   * @return actual state of the replica on this data-node or 
-   * null if data-node does not have the replica.
-   */
-  ReplicaRecoveryInfoWritable initReplicaRecovery(RecoveringBlockWritable rBlock)
-      throws IOException;
-
-  /**
-   * Update replica with the new generation stamp and length.  
-   */
-  ExtendedBlockWritable updateReplicaUnderRecovery(
-      ExtendedBlockWritable oldBlock, long recoveryId, long newLength)
-      throws IOException;
-  
-  /**
-   * This method is defined to get the protocol signature using 
-   * the R23 protocol - hence we have added the suffix of 2 to the method name
-   * to avoid conflict.
-   */
-  public ProtocolSignatureWritable getProtocolSignature2(
-      String protocol, long clientVersion, int clientMethodsHash)
-      throws IOException;
-}

+ 0 - 87
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocolR23Compatible/KeyUpdateCommandWritable.java

@@ -1,87 +0,0 @@
-/**
- * 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.server.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.protocolR23Compatible.ExportedBlockKeysWritable;
-import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
-import org.apache.hadoop.hdfs.server.protocol.KeyUpdateCommand;
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.io.WritableFactories;
-import org.apache.hadoop.io.WritableFactory;
-
-@InterfaceAudience.Private
-@InterfaceStability.Evolving
-public class KeyUpdateCommandWritable extends DatanodeCommandWritable {
-  private ExportedBlockKeysWritable keys;
-
-  KeyUpdateCommandWritable() {
-    this(new ExportedBlockKeysWritable());
-  }
-
-  public KeyUpdateCommandWritable(ExportedBlockKeysWritable keys) {
-    super(DatanodeWireProtocol.DNA_ACCESSKEYUPDATE);
-    this.keys = keys;
-  }
-
-  public ExportedBlockKeysWritable getExportedKeys() {
-    return this.keys;
-  }
-
-  // ///////////////////////////////////////////////
-  // Writable
-  // ///////////////////////////////////////////////
-  static { // register a ctor
-    WritableFactories.setFactory(KeyUpdateCommandWritable.class,
-        new WritableFactory() {
-          public Writable newInstance() {
-            return new KeyUpdateCommandWritable();
-          }
-        });
-  }
-
-  @Override
-  public void write(DataOutput out) throws IOException {
-    super.write(out);
-    keys.write(out);
-  }
-
-  @Override
-  public void readFields(DataInput in) throws IOException {
-    super.readFields(in);
-    keys.readFields(in);
-  }
-
-  @Override
-  public DatanodeCommand convert() {
-    return new KeyUpdateCommand(keys.convert());
-  }
-
-  public static KeyUpdateCommandWritable convert(KeyUpdateCommand cmd) {
-    if (cmd == null) {
-      return null;
-    }
-    return new KeyUpdateCommandWritable(ExportedBlockKeysWritable.convert(cmd
-        .getExportedKeys()));
-  }
-}

+ 0 - 95
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocolR23Compatible/ReceivedDeletedBlockInfoWritable.java

@@ -1,95 +0,0 @@
-/**
- * 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.server.protocolR23Compatible;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-
-import org.apache.hadoop.hdfs.protocolR23Compatible.BlockWritable;
-import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.Writable;
-
-/**
- * A data structure to store Block and delHints together, used to send
- * received/deleted ACKs.
- */
-public class ReceivedDeletedBlockInfoWritable implements Writable {
-  BlockWritable block;
-  String delHints;
-
-  public final static String TODELETE_HINT = "-";
-
-  public ReceivedDeletedBlockInfoWritable() {
-  }
-
-  public ReceivedDeletedBlockInfoWritable(BlockWritable blk, String delHints) {
-    this.block = blk;
-    this.delHints = delHints;
-  }
-
-  @Override
-  public void write(DataOutput out) throws IOException {
-    this.block.write(out);
-    Text.writeString(out, this.delHints);
-  }
-
-  @Override
-  public void readFields(DataInput in) throws IOException {
-    this.block = new BlockWritable();
-    this.block.readFields(in);
-    this.delHints = Text.readString(in);
-  }
-
-  public String toString() {
-    return block.toString() + ", delHint: " + delHints;
-  }
-
-  public static ReceivedDeletedBlockInfo[] convert(
-      ReceivedDeletedBlockInfoWritable[] rdBlocks) {
-    ReceivedDeletedBlockInfo[] ret = 
-        new ReceivedDeletedBlockInfo[rdBlocks.length];
-    for (int i = 0; i < rdBlocks.length; i++) {
-      ret[i] = rdBlocks[i].convert();
-    }
-    return ret;
-  }
-  
-  public static ReceivedDeletedBlockInfoWritable[] convert(
-      ReceivedDeletedBlockInfo[] blocks) {
-    ReceivedDeletedBlockInfoWritable[] ret = 
-        new ReceivedDeletedBlockInfoWritable[blocks.length];
-    for (int i = 0; i < blocks.length; i++) {
-      ret[i] = convert(blocks[i]);
-    }
-    return ret;
-  }
-
-  public ReceivedDeletedBlockInfo convert() {
-    return new ReceivedDeletedBlockInfo(block.convert(), delHints);
-  }
-
-  public static ReceivedDeletedBlockInfoWritable convert(
-      ReceivedDeletedBlockInfo b) {
-    if (b == null) return null;
-    return new ReceivedDeletedBlockInfoWritable(BlockWritable.convert(b
-        .getBlock()), b.getDelHints());
-  }
-}

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

@@ -1,104 +0,0 @@
-/**
- * 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.server.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.protocolR23Compatible.DatanodeInfoWritable;
-import org.apache.hadoop.hdfs.protocolR23Compatible.ExtendedBlockWritable;
-import org.apache.hadoop.hdfs.protocolR23Compatible.LocatedBlockWritable;
-import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringBlock;
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.io.WritableFactories;
-import org.apache.hadoop.io.WritableFactory;
-
-/**
- * This is a block with locations from which it should be recovered and the new
- * generation stamp, which the block will have after successful recovery.
- * 
- * The new generation stamp of the block, also plays role of the recovery id.
- */
-@InterfaceAudience.Private
-@InterfaceStability.Evolving
-public class RecoveringBlockWritable implements Writable {
-  private long newGenerationStamp;
-  private LocatedBlockWritable locatedBlock;
-
-  /**
-   * Create empty RecoveringBlock.
-   */
-  public RecoveringBlockWritable() {
-    locatedBlock = new LocatedBlockWritable();
-    newGenerationStamp = -1L;
-  }
-
-  /**
-   * Create RecoveringBlock.
-   */
-  public RecoveringBlockWritable(ExtendedBlockWritable b,
-      DatanodeInfoWritable[] locs, long newGS) {
-    locatedBlock = new LocatedBlockWritable(b, locs, -1, false);
-    this.newGenerationStamp = newGS;
-  }
-
-  // /////////////////////////////////////////
-  // Writable
-  // /////////////////////////////////////////
-  static { // register a ctor
-    WritableFactories.setFactory(RecoveringBlockWritable.class,
-        new WritableFactory() {
-          public Writable newInstance() {
-            return new RecoveringBlockWritable();
-          }
-        });
-  }
-
-  public void write(DataOutput out) throws IOException {
-    locatedBlock.write(out);
-    out.writeLong(newGenerationStamp);
-  }
-
-  public void readFields(DataInput in) throws IOException {
-    locatedBlock = new LocatedBlockWritable();
-    locatedBlock.readFields(in);
-    newGenerationStamp = in.readLong();
-  }
-
-  public RecoveringBlock convert() {
-    ExtendedBlockWritable eb = locatedBlock.getBlock();
-    DatanodeInfoWritable[] dnInfo = locatedBlock.getLocations();
-    return new RecoveringBlock(ExtendedBlockWritable.convertExtendedBlock(eb),
-        DatanodeInfoWritable.convertDatanodeInfo(dnInfo), newGenerationStamp);
-  }
-
-  public static RecoveringBlockWritable convert(RecoveringBlock rBlock) {
-    if (rBlock == null) {
-      return null;
-    }
-    ExtendedBlockWritable eb = ExtendedBlockWritable
-        .convertExtendedBlock(rBlock.getBlock());
-    DatanodeInfoWritable[] dnInfo = DatanodeInfoWritable
-        .convertDatanodeInfo(rBlock.getLocations());
-    return new RecoveringBlockWritable(eb, dnInfo,
-        rBlock.getNewGenerationStamp());
-  }
-}

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

@@ -1,69 +0,0 @@
-/**
- * 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.server.protocolR23Compatible;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
-import org.apache.hadoop.hdfs.server.protocol.RegisterCommand;
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.io.WritableFactories;
-import org.apache.hadoop.io.WritableFactory;
-
-/**
- * A RegisterCommand is an instruction to a datanode to register with the
- * namenode.
- */
-@InterfaceAudience.Private
-@InterfaceStability.Evolving
-public class RegisterCommandWritable extends DatanodeCommandWritable {
-  public static final RegisterCommandWritable REGISTER = 
-      new RegisterCommandWritable();
-  
-  // /////////////////////////////////////////
-  // Writable
-  // /////////////////////////////////////////
-  static { // register a ctor
-    WritableFactories.setFactory(RegisterCommandWritable.class,
-        new WritableFactory() {
-          public Writable newInstance() {
-            return new RegisterCommandWritable();
-          }
-        });
-  }
-
-  public RegisterCommandWritable() {
-    super(DatanodeWireProtocol.DNA_REGISTER);
-  }
-
-  @Override
-  public void readFields(DataInput in) { /* Nothing to read */
-  }
-
-  @Override
-  public void write(DataOutput out) { /* Nothing to write */
-  }
-
-  @Override
-  public DatanodeCommand convert() {
-    return RegisterCommand.REGISTER;
-  }
-}

+ 0 - 87
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocolR23Compatible/ReplicaRecoveryInfoWritable.java

@@ -1,87 +0,0 @@
-/**
- * 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.server.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.protocolR23Compatible.BlockWritable;
-import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
-import org.apache.hadoop.hdfs.server.protocol.ReplicaRecoveryInfo;
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.io.WritableFactories;
-import org.apache.hadoop.io.WritableFactory;
-
-/**
- * Replica recovery information.
- */
-@InterfaceAudience.Private
-@InterfaceStability.Evolving
-public class ReplicaRecoveryInfoWritable implements Writable {
-  private int originalState;
-  private BlockWritable block;
-
-  public ReplicaRecoveryInfoWritable() {
-  }
-
-  public ReplicaRecoveryInfoWritable(long blockId, long diskLen, long gs,
-      ReplicaState rState) {
-    block = new BlockWritable(blockId, diskLen, gs);
-    originalState = rState.getValue();
-  }
-
-  // /////////////////////////////////////////
-  // Writable
-  // /////////////////////////////////////////
-  static { // register a ctor
-    WritableFactories.setFactory(ReplicaRecoveryInfoWritable.class,
-        new WritableFactory() {
-          public Writable newInstance() {
-            return new ReplicaRecoveryInfoWritable();
-          }
-        });
-  }
-
-  @Override
-  public void readFields(DataInput in) throws IOException {
-    block = new BlockWritable();
-    block.readFields(in);
-    originalState = in.readInt();
-  }
-
-  @Override
-  public void write(DataOutput out) throws IOException {
-    block.write(out);
-    out.writeInt(originalState);
-  }
-
-  public static ReplicaRecoveryInfoWritable convert(ReplicaRecoveryInfo rrInfo) {
-    return new ReplicaRecoveryInfoWritable(rrInfo.getBlockId(),
-        rrInfo.getNumBytes(), rrInfo.getGenerationStamp(),
-        rrInfo.getOriginalReplicaState());
-  }
-
-  public ReplicaRecoveryInfo convert() {
-    return new ReplicaRecoveryInfo(block.getBlockId(), block.getNumBytes(),
-        block.getGenerationStamp(), ReplicaState.getState(originalState));
-  }
-}

+ 0 - 75
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocolR23Compatible/ServerCommandWritable.java

@@ -1,75 +0,0 @@
-/**
- * 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.server.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.io.Writable;
-
-/**
- * Base class for a server command.
- * Issued by the name-node to notify other servers what should be done.
- * Commands are defined by actions defined in respective protocols.
- */
-@InterfaceAudience.Private
-@InterfaceStability.Evolving
-public abstract class ServerCommandWritable implements Writable {
-  private int action;
-
-  /**
-   * Unknown server command constructor.
-   * Creates a command with action 0.
-   */
-  public ServerCommandWritable() {
-    this(0);
-  }
-
-  /**
-   * Create a command for the specified action.
-   * Actions are protocol specific.
-   * @param action
-   */
-  public ServerCommandWritable(int action) {
-    this.action = action;
-  }
-
-  /**
-   * Get server command action.
-   * @return action code.
-   */
-  public int getAction() {
-    return this.action;
-  }
-
-  ///////////////////////////////////////////
-  // Writable
-  ///////////////////////////////////////////
-  @Override
-  public void write(DataOutput out) throws IOException {
-    out.writeInt(this.action);
-  }
-
-  @Override
-  public void readFields(DataInput in) throws IOException {
-    this.action = in.readInt();
-  }
-}

+ 0 - 106
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocolR23Compatible/UpgradeCommandWritable.java

@@ -1,106 +0,0 @@
-/**
- * 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.server.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.UpgradeCommand;
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.io.WritableFactories;
-import org.apache.hadoop.io.WritableFactory;
-
-/**
- * This as a generic distributed upgrade command.
- * 
- * During the upgrade cluster components send upgrade commands to each other
- * in order to obtain or share information with them.
- * It is supposed that each upgrade defines specific upgrade command by
- * deriving them from this class.
- * The upgrade command contains version of the upgrade, which is verified 
- * on the receiving side and current status of the upgrade.
- */
-@InterfaceAudience.Private
-@InterfaceStability.Evolving
-public class UpgradeCommandWritable extends DatanodeCommandWritable {
-  final static int UC_ACTION_UNKNOWN = DatanodeWireProtocol.DNA_UNKNOWN;
-  public final static int UC_ACTION_REPORT_STATUS = 100; // report upgrade status
-  public final static int UC_ACTION_START_UPGRADE = 101; // start upgrade
-
-  private int version;
-  private short upgradeStatus;
-
-  public UpgradeCommandWritable() {
-    super(UC_ACTION_UNKNOWN);
-    this.version = 0;
-    this.upgradeStatus = 0;
-  }
-
-  public UpgradeCommandWritable(int action, int version, short status) {
-    super(action);
-    this.version = version;
-    this.upgradeStatus = status;
-  }
-
-  public int getVersion() {
-    return this.version;
-  }
-
-  public short getCurrentStatus() {
-    return this.upgradeStatus;
-  }
-
-  /////////////////////////////////////////////////
-  // Writable
-  /////////////////////////////////////////////////
-  static {                                      // register a ctor
-    WritableFactories.setFactory
-      (UpgradeCommandWritable.class,
-       new WritableFactory() {
-         public Writable newInstance() { return new UpgradeCommandWritable(); }
-       });
-  }
-
-  @Override
-  public void write(DataOutput out) throws IOException {
-    super.write(out);
-    out.writeInt(this.version);
-    out.writeShort(this.upgradeStatus);
-  }
-
-  @Override
-  public void readFields(DataInput in) throws IOException {
-    super.readFields(in);
-    this.version = in.readInt();
-    this.upgradeStatus = in.readShort();
-  }
-
-  @Override
-  public UpgradeCommand convert() {
-    return new UpgradeCommand(getAction(), version, upgradeStatus);
-  }
-
-  public static UpgradeCommandWritable convert(UpgradeCommand cmd) {
-    if (cmd == null) return null;
-    return new UpgradeCommandWritable(cmd.getAction(), cmd.getVersion(),
-        cmd.getCurrentStatus());
-  }
-}

+ 0 - 13
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java

@@ -44,14 +44,9 @@ import org.apache.hadoop.fs.FileUtil;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.*;
 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.protocolPB.ClientDatanodeProtocolPB;
-import org.apache.hadoop.hdfs.protocolPB.DatanodeProtocolPB;
-import org.apache.hadoop.hdfs.protocolPB.NamenodeProtocolPB;
-import org.apache.hadoop.hdfs.protocolR23Compatible.ClientNamenodeWireProtocol;
 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;
@@ -62,7 +57,6 @@ import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset;
 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.NamenodeProtocols;
 import org.apache.hadoop.hdfs.tools.DFSAdmin;
@@ -70,11 +64,8 @@ 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;
 
@@ -639,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
    */