Browse Source

HDFS-3238. ServerCommand and friends don't need to be writables. Contributed by Eli Collins

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1311774 13f79535-47bb-0310-9956-ffa450edef68
Eli Collins 13 years ago
parent
commit
926f0a5ae0

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

@@ -353,6 +353,8 @@ Release 2.0.0 - UNRELEASED
     HDFS-3240. Drop log level of "heartbeat: ..." in BPServiceActor to DEBUG
     (todd)
 
+    HDFS-3238. ServerCommand and friends don't need to be writables. (eli)
+
   OPTIMIZATIONS
 
     HDFS-3024. Improve performance of stringification in addStoredBlock (todd)

+ 0 - 39
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BalancerBandwidthCommand.java

@@ -25,14 +25,6 @@ package org.apache.hadoop.hdfs.server.protocol;
  * each datanode.
  */
 
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-
-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
@@ -71,35 +63,4 @@ public class BalancerBandwidthCommand extends DatanodeCommand {
   public long getBalancerBandwidthValue() {
     return this.bandwidth;
   }
-
-  // ///////////////////////////////////////////////
-  // Writable
-  // ///////////////////////////////////////////////
-  static { // register a ctor
-    WritableFactories.setFactory(BalancerBandwidthCommand.class, new WritableFactory() {
-      public Writable newInstance() {
-        return new BalancerBandwidthCommand();
-      }
-    });
-  }
-
-  /**
-   * 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();
-  }
 }

+ 0 - 56
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlockCommand.java

@@ -17,9 +17,6 @@
  */
 package org.apache.hadoop.hdfs.server.protocol;
 
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
 import java.util.List;
 
 import org.apache.hadoop.classification.InterfaceAudience;
@@ -27,11 +24,6 @@ import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor.BlockTargetPair;
-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 
@@ -58,8 +50,6 @@ public class BlockCommand extends DatanodeCommand {
   Block blocks[];
   DatanodeInfo targets[][];
 
-  public BlockCommand() {}
-
   /**
    * Create BlockCommand for transferring blocks to another datanode
    * @param blocktargetlist    blocks to be transferred 
@@ -110,50 +100,4 @@ public class BlockCommand extends DatanodeCommand {
   public DatanodeInfo[][] getTargets() {
     return targets;
   }
-
-  ///////////////////////////////////////////
-  // Writable
-  ///////////////////////////////////////////
-  static {                                      // register a ctor
-    WritableFactories.setFactory
-      (BlockCommand.class,
-       new WritableFactory() {
-         public Writable newInstance() { return new BlockCommand(); }
-       });
-  }
-
-  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);
-      }
-    }
-  }
-
-  public void readFields(DataInput in) throws IOException {
-    super.readFields(in);
-    this.poolId = Text.readString(in);
-    this.blocks = new Block[in.readInt()];
-    for (int i = 0; i < blocks.length; i++) {
-      blocks[i] = new Block();
-      blocks[i].readFields(in);
-    }
-
-    this.targets = new DatanodeInfo[in.readInt()][];
-    for (int i = 0; i < targets.length; i++) {
-      this.targets[i] = new DatanodeInfo[in.readInt()];
-      for (int j = 0; j < targets[i].length; j++) {
-        targets[i][j] = new DatanodeInfo();
-        targets[i][j].readFields(in);
-      }
-    }
-  }
 }

+ 0 - 54
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlockRecoveryCommand.java

@@ -28,9 +28,6 @@ import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.io.WritableFactories;
-import org.apache.hadoop.io.WritableFactory;
 
 import com.google.common.base.Joiner;
 
@@ -84,27 +81,6 @@ public class BlockRecoveryCommand extends DatanodeCommand {
     public long getNewGenerationStamp() {
       return newGenerationStamp;
     }
-
-    ///////////////////////////////////////////
-    // Writable
-    ///////////////////////////////////////////
-    static {                                      // register a ctor
-      WritableFactories.setFactory
-        (RecoveringBlock.class,
-         new WritableFactory() {
-           public Writable newInstance() { return new RecoveringBlock(); }
-         });
-    }
-
-    public void write(DataOutput out) throws IOException {
-      super.write(out);
-      out.writeLong(newGenerationStamp);
-    }
-
-    public void readFields(DataInput in) throws IOException {
-      super.readFields(in);
-      newGenerationStamp = in.readLong();
-    }
   }
 
   /**
@@ -149,34 +125,4 @@ public class BlockRecoveryCommand extends DatanodeCommand {
     sb.append("\n)");
     return sb.toString();
   }
-
-  ///////////////////////////////////////////
-  // Writable
-  ///////////////////////////////////////////
-  static {                                      // register a ctor
-    WritableFactories.setFactory
-      (BlockRecoveryCommand.class,
-       new WritableFactory() {
-         public Writable newInstance() { return new BlockRecoveryCommand(); }
-       });
-  }
-
-  public void write(DataOutput out) throws IOException {
-    super.write(out);
-    out.writeInt(recoveringBlocks.size());
-    for(RecoveringBlock block : recoveringBlocks) {
-      block.write(out);
-    }
-  }
-
-  public void readFields(DataInput in) throws IOException {
-    super.readFields(in);
-    int numBlocks = in.readInt();
-    recoveringBlocks = new ArrayList<RecoveringBlock>(numBlocks);
-    for(int i = 0; i < numBlocks; i++) {
-      RecoveringBlock b = new RecoveringBlock();
-      b.readFields(in);
-      add(b);
-    }
-  }
 }

+ 0 - 30
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/CheckpointCommand.java

@@ -17,13 +17,6 @@
  */
 package org.apache.hadoop.hdfs.server.protocol;
 
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-
-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.namenode.CheckpointSignature;
@@ -77,27 +70,4 @@ public class CheckpointCommand extends NamenodeCommand {
   public boolean needToReturnImage() {
     return needToReturnImage;
   }
-
-  ///////////////////////////////////////////
-  // Writable
-  ///////////////////////////////////////////
-  static {
-    WritableFactories.setFactory(CheckpointCommand.class,
-        new WritableFactory() {
-          public Writable newInstance() {return new CheckpointCommand();}
-        });
-  }
-
-  public void write(DataOutput out) throws IOException {
-    super.write(out);
-    cSig.write(out);
-    out.writeBoolean(needToReturnImage);
-  }
-  
-  public void readFields(DataInput in) throws IOException {
-    super.readFields(in);
-    cSig = new CheckpointSignature();
-    cSig.readFields(in);
-    needToReturnImage = in.readBoolean();
-  }
 }

+ 1 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeCommand.java

@@ -27,10 +27,7 @@ import org.apache.hadoop.classification.InterfaceStability;
 @InterfaceAudience.Private
 @InterfaceStability.Evolving
 public abstract class DatanodeCommand extends ServerCommand {
-  public DatanodeCommand() {
-    super();
-  }
-  
+
   DatanodeCommand(int action) {
     super(action);
   }

+ 0 - 26
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/FinalizeCommand.java

@@ -17,16 +17,8 @@
  */
 package org.apache.hadoop.hdfs.server.protocol;
 
-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;
-import org.apache.hadoop.io.WritableUtils;
 
 /**
  * A BlockCommand is an instruction to a datanode to register with the namenode.
@@ -34,17 +26,6 @@ import org.apache.hadoop.io.WritableUtils;
 @InterfaceAudience.Private
 @InterfaceStability.Evolving
 public class FinalizeCommand extends DatanodeCommand {
-  // /////////////////////////////////////////
-  // Writable
-  // /////////////////////////////////////////
-  static { // register a ctor
-    WritableFactories.setFactory(FinalizeCommand.class, new WritableFactory() {
-      public Writable newInstance() {
-        return new FinalizeCommand();
-      }
-    });
-  }
-  
   String blockPoolId;
   private FinalizeCommand() {
     super(DatanodeProtocol.DNA_FINALIZE);
@@ -58,11 +39,4 @@ public class FinalizeCommand extends DatanodeCommand {
   public String getBlockPoolId() {
     return blockPoolId;
   }
-  
-  public void readFields(DataInput in) throws IOException {
-    blockPoolId = WritableUtils.readString(in);
-  }
-  public void write(DataOutput out) throws IOException {
-    WritableUtils.writeString(out, blockPoolId);
-  }
 }

+ 0 - 32
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/KeyUpdateCommand.java

@@ -17,16 +17,9 @@
  */
 package org.apache.hadoop.hdfs.server.protocol;
 
-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.security.token.block.ExportedBlockKeys;
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.io.WritableFactories;
-import org.apache.hadoop.io.WritableFactory;
 
 @InterfaceAudience.Private
 @InterfaceStability.Evolving
@@ -45,29 +38,4 @@ public class KeyUpdateCommand extends DatanodeCommand {
   public ExportedBlockKeys getExportedKeys() {
     return this.keys;
   }
-
-  // ///////////////////////////////////////////////
-  // Writable
-  // ///////////////////////////////////////////////
-  static { // register a ctor
-    WritableFactories.setFactory(KeyUpdateCommand.class, new WritableFactory() {
-      public Writable newInstance() {
-        return new KeyUpdateCommand();
-      }
-    });
-  }
-
-  /**
-   */
-  public void write(DataOutput out) throws IOException {
-    super.write(out);
-    keys.write(out);
-  }
-
-  /**
-   */
-  public void readFields(DataInput in) throws IOException {
-    super.readFields(in);
-    keys.readFields(in);
-  }
 }

+ 0 - 14
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/NamenodeCommand.java

@@ -19,9 +19,6 @@ package org.apache.hadoop.hdfs.server.protocol;
 
 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 class for name-node command.
@@ -30,17 +27,6 @@ import org.apache.hadoop.io.WritableFactory;
 @InterfaceAudience.Private
 @InterfaceStability.Evolving
 public class NamenodeCommand extends ServerCommand {
-  static {
-    WritableFactories.setFactory(NamenodeCommand.class,
-        new WritableFactory() {
-          public Writable newInstance() {return new NamenodeCommand();}
-        });
-  }
-
-  public NamenodeCommand() {
-    super();
-  }
-
   public NamenodeCommand(int action) {
     super(action);
   }

+ 0 - 22
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/RegisterCommand.java

@@ -17,14 +17,8 @@
  */
 package org.apache.hadoop.hdfs.server.protocol;
 
-import java.io.DataInput;
-import java.io.DataOutput;
-
 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;
 
 /**
  * A BlockCommand is an instruction to a datanode to register with the namenode.
@@ -32,26 +26,10 @@ import org.apache.hadoop.io.WritableFactory;
 @InterfaceAudience.Private
 @InterfaceStability.Evolving
 public class RegisterCommand extends DatanodeCommand {
-  // /////////////////////////////////////////
-  // Writable
-  // /////////////////////////////////////////
-  static { // register a ctor
-    WritableFactories.setFactory(RegisterCommand.class, new WritableFactory() {
-      public Writable newInstance() {
-        return new RegisterCommand();
-      }
-    });
-  }
   
   public static final DatanodeCommand REGISTER = new RegisterCommand();
 
   public RegisterCommand() {
     super(DatanodeProtocol.DNA_REGISTER);
   }
-
-  @Override
-  public void readFields(DataInput in) { }
- 
-  @Override
-  public void write(DataOutput out) { }
 }

+ 1 - 26
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/ServerCommand.java

@@ -17,11 +17,8 @@
  */
 package org.apache.hadoop.hdfs.server.protocol;
 
-import java.io.*;
-
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.io.Writable;
 
 /**
  * Base class for a server command.
@@ -33,20 +30,9 @@ import org.apache.hadoop.io.Writable;
  */
 @InterfaceAudience.Private
 @InterfaceStability.Evolving
-public abstract class ServerCommand implements Writable {
+public abstract class ServerCommand {
   private int action;
 
-  /**
-   * Unknown server command constructor.
-   * Creates a command with action 0.
-   * 
-   * @see NamenodeProtocol#ACT_UNKNOWN
-   * @see DatanodeProtocol#DNA_UNKNOWN
-   */
-  public ServerCommand() {
-    this(0);
-  }
-
   /**
    * Create a command for the specified action.
    * Actions are protocol specific.
@@ -66,15 +52,4 @@ public abstract class ServerCommand implements Writable {
   public int getAction() {
     return this.action;
   }
-
-  ///////////////////////////////////////////
-  // Writable
-  ///////////////////////////////////////////
-  public void write(DataOutput out) throws IOException {
-    out.writeInt(this.action);
-  }
-
-  public void readFields(DataInput in) throws IOException {
-    this.action = in.readInt();
-  }
 }

+ 0 - 34
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/UpgradeCommand.java

@@ -17,15 +17,8 @@
  */
 package org.apache.hadoop.hdfs.server.protocol;
 
-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 as a generic distributed upgrade command.
@@ -68,31 +61,4 @@ public class UpgradeCommand extends DatanodeCommand {
   public short getCurrentStatus() {
     return this.upgradeStatus;
   }
-
-  /////////////////////////////////////////////////
-  // Writable
-  /////////////////////////////////////////////////
-  static {                                      // register a ctor
-    WritableFactories.setFactory
-      (UpgradeCommand.class,
-       new WritableFactory() {
-         public Writable newInstance() { return new UpgradeCommand(); }
-       });
-  }
-
-  /**
-   */
-  public void write(DataOutput out) throws IOException {
-    super.write(out);
-    out.writeInt(this.version);
-    out.writeShort(this.upgradeStatus);
-  }
-
-  /**
-   */
-  public void readFields(DataInput in) throws IOException {
-    super.readFields(in);
-    this.version = in.readInt();
-    this.upgradeStatus = in.readShort();
-  }
 }