You are viewing a plain text version of this content. The canonical link for it is here.
Posted to hdfs-commits@hadoop.apache.org by su...@apache.org on 2012/08/22 19:05:39 UTC

svn commit: r1376145 - in /hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs: ./ src/main/java/org/apache/hadoop/hdfs/ src/main/java/org/apache/hadoop/hdfs/protocol/ src/main/java/org/apache/hadoop/hdfs/protocolPB/ src/main/java/org/apach...

Author: suresh
Date: Wed Aug 22 17:05:38 2012
New Revision: 1376145

URL: http://svn.apache.org/viewvc?rev=1376145&view=rev
Log:
HDFS-3832. Merging change 1376139 from trunk to branch-2. Contributed by Suresh Srinivas.

Removed:
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/UpgradeCommand.java
Modified:
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolClientSideTranslatorPB.java
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolServerSideTranslatorPB.java
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPOfferService.java
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeJspHelper.java
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeProtocol.java
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto

Modified: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt?rev=1376145&r1=1376144&r2=1376145&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt Wed Aug 22 17:05:38 2012
@@ -222,6 +222,8 @@ Release 2.0.1-alpha - UNRELEASED
     HDFS-2727. libhdfs should get the default block size from the server.
     (Colin Patrick McCabe via eli)
 
+    HDFS-3832. Remove protocol methods related to DistributedUpgrade. (suresh)
+
   OPTIMIZATIONS
 
     HDFS-2982. Startup performance suffers when there are many edit log

Modified: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java?rev=1376145&r1=1376144&r2=1376145&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java Wed Aug 22 17:05:38 2012
@@ -111,7 +111,6 @@ import org.apache.hadoop.hdfs.protocol.H
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 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.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.HdfsProtoUtil;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
@@ -131,7 +130,6 @@ import org.apache.hadoop.hdfs.security.t
 import org.apache.hadoop.hdfs.security.token.block.InvalidBlockTokenException;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
-import org.apache.hadoop.hdfs.server.common.UpgradeStatusReport;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.SafeModeException;
 import org.apache.hadoop.io.DataOutputBuffer;
@@ -1238,7 +1236,7 @@ public class DFSClient implements java.i
    * @param blockSize maximum block size
    * @param progress interface for reporting client progress
    * @param buffersize underlying buffer size 
-   * @param checksumOpts checksum options
+   * @param checksumOpt checksum options
    * 
    * @return output stream
    * 
@@ -1936,14 +1934,6 @@ public class DFSClient implements java.i
   }
 
   /**
-   * @see ClientProtocol#distributedUpgradeProgress(HdfsConstants.UpgradeAction)
-   */
-  public UpgradeStatusReport distributedUpgradeProgress(UpgradeAction action)
-      throws IOException {
-    return namenode.distributedUpgradeProgress(action);
-  }
-
-  /**
    */
   @Deprecated
   public boolean mkdirs(String src) throws IOException {

Modified: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java?rev=1376145&r1=1376144&r2=1376145&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java Wed Aug 22 17:05:38 2012
@@ -55,13 +55,11 @@ import org.apache.hadoop.hdfs.protocol.E
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 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.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.HdfsLocatedFileStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.security.token.block.InvalidBlockTokenException;
 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.io.Text;
 import org.apache.hadoop.security.AccessControlException;
@@ -698,11 +696,6 @@ public class DistributedFileSystem exten
     dfs.finalizeUpgrade();
   }
 
-  public UpgradeStatusReport distributedUpgradeProgress(UpgradeAction action
-  ) throws IOException {
-    return dfs.distributedUpgradeProgress(action);
-  }
-
   /*
    * Requests the namenode to dump data strcutures into specified 
    * file.

Modified: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java?rev=1376145&r1=1376144&r2=1376145&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java Wed Aug 22 17:05:38 2012
@@ -33,8 +33,6 @@ import org.apache.hadoop.fs.UnresolvedLi
 import org.apache.hadoop.fs.Options.Rename;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants.UpgradeAction;
-import org.apache.hadoop.hdfs.server.common.UpgradeStatusReport;
 import org.apache.hadoop.hdfs.server.namenode.NotReplicatedYetException;
 import org.apache.hadoop.hdfs.server.namenode.SafeModeException;
 import org.apache.hadoop.io.EnumSetWritable;
@@ -695,17 +693,6 @@ public interface ClientProtocol {
   public void finalizeUpgrade() throws IOException;
 
   /**
-   * <em>Method no longer used - retained only for backward compatibility</em>
-   * 
-   * Report distributed upgrade progress or force current upgrade to proceed.
-   * @param action {@link HdfsConstants.UpgradeAction} to perform
-   * @return upgrade status information or null if no upgrades are in progress
-   * @throws IOException
-   */
-  public UpgradeStatusReport distributedUpgradeProgress(UpgradeAction action) 
-      throws IOException;
-
-  /**
    * @return CorruptFileBlocks, containing a list of corrupt files (with
    *         duplicates if there is more than one corrupt block in a file)
    *         and a cookie

Modified: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java?rev=1376145&r1=1376144&r2=1376145&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java Wed Aug 22 17:05:38 2012
@@ -60,7 +60,7 @@ public class HdfsConstants {
   public static int MAX_PATH_LENGTH = 8000;
   public static int MAX_PATH_DEPTH = 1000;
 
-  // TODO mb@media-style.com: should be conf injected?
+  // TODO should be conf injected?
   public static final int DEFAULT_DATA_SOCKET_SIZE = 128 * 1024;
   public static final int IO_FILE_BUFFER_SIZE = new HdfsConfiguration().getInt(
       DFSConfigKeys.IO_FILE_BUFFER_SIZE_KEY,
@@ -85,16 +85,6 @@ public class HdfsConstants {
   public static final long INVALID_TXID = -12345;
 
   /**
-   * Distributed upgrade actions:
-   * 
-   * 1. Get upgrade status. 2. Get detailed upgrade status. 3. Proceed with the
-   * upgrade if it is stuck, no matter what the status is.
-   */
-  public static enum UpgradeAction {
-    GET_STATUS, DETAILED_STATUS, FORCE_PROCEED;
-  }
-
-  /**
    * URI Scheme for hdfs://namenode/ URIs.
    */
   public static final String HDFS_URI_SCHEME = "hdfs";

Modified: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java?rev=1376145&r1=1376144&r2=1376145&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java Wed Aug 22 17:05:38 2012
@@ -50,8 +50,6 @@ import org.apache.hadoop.hdfs.protocol.p
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CreateSymlinkResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.DeleteRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.DeleteResponseProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.DistributedUpgradeProgressRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.DistributedUpgradeProgressResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.FinalizeUpgradeRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.FinalizeUpgradeResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.FsyncRequestProto;
@@ -130,7 +128,6 @@ import org.apache.hadoop.hdfs.protocol.p
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.LocatedBlockProto;
 import org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey;
-import org.apache.hadoop.hdfs.server.common.UpgradeStatusReport;
 import org.apache.hadoop.io.Text;
 
 import com.google.protobuf.RpcController;
@@ -571,24 +568,6 @@ public class ClientNamenodeProtocolServe
   }
 
   @Override
-  public DistributedUpgradeProgressResponseProto distributedUpgradeProgress(
-      RpcController controller, DistributedUpgradeProgressRequestProto req)
-      throws ServiceException {
-    try {
-      UpgradeStatusReport result = server.distributedUpgradeProgress(PBHelper
-          .convert(req.getAction()));
-      DistributedUpgradeProgressResponseProto.Builder builder = 
-          DistributedUpgradeProgressResponseProto.newBuilder();
-      if (result != null) {
-        builder.setReport(PBHelper.convert(result));
-      }
-      return builder.build();
-    } catch (IOException e) {
-      throw new ServiceException(e);
-    }
-  }
-
-  @Override
   public ListCorruptFileBlocksResponseProto listCorruptFileBlocks(
       RpcController controller, ListCorruptFileBlocksRequestProto req)
       throws ServiceException {

Modified: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java?rev=1376145&r1=1376144&r2=1376145&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java Wed Aug 22 17:05:38 2012
@@ -42,7 +42,6 @@ import org.apache.hadoop.hdfs.protocol.D
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 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.ipc.ProtocolTranslator;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
@@ -58,8 +57,6 @@ import org.apache.hadoop.hdfs.protocol.p
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CreateRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CreateSymlinkRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.DeleteRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.DistributedUpgradeProgressRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.DistributedUpgradeProgressResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.FinalizeUpgradeRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.FsyncRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetAdditionalDatanodeRequestProto;
@@ -102,7 +99,6 @@ import org.apache.hadoop.hdfs.protocol.p
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.UpdatePipelineRequestProto;
 import org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
-import org.apache.hadoop.hdfs.server.common.UpgradeStatusReport;
 import org.apache.hadoop.hdfs.server.namenode.NotReplicatedYetException;
 import org.apache.hadoop.hdfs.server.namenode.SafeModeException;
 import org.apache.hadoop.io.EnumSetWritable;
@@ -128,8 +124,7 @@ public class ClientNamenodeProtocolTrans
     ProtocolMetaInterface, ClientProtocol, Closeable, ProtocolTranslator {
   final private ClientNamenodeProtocolPB rpcProxy;
 
-  public ClientNamenodeProtocolTranslatorPB(ClientNamenodeProtocolPB proxy)
-      throws IOException {
+  public ClientNamenodeProtocolTranslatorPB(ClientNamenodeProtocolPB proxy) {
     rpcProxy = proxy;
   }
   
@@ -565,21 +560,6 @@ public class ClientNamenodeProtocolTrans
   }
 
   @Override
-  public UpgradeStatusReport distributedUpgradeProgress(UpgradeAction action)
-      throws IOException {
-    DistributedUpgradeProgressRequestProto req = 
-        DistributedUpgradeProgressRequestProto.newBuilder().
-        setAction(PBHelper.convert(action)).build();
-    try {
-      DistributedUpgradeProgressResponseProto res = rpcProxy
-          .distributedUpgradeProgress(null, req);
-      return res.hasReport() ? PBHelper.convert(res.getReport()) : null;
-    } catch (ServiceException e) {
-      throw ProtobufHelper.getRemoteException(e);
-    }
-  }
-
-  @Override
   public CorruptFileBlocks listCorruptFileBlocks(String path, String cookie)
       throws IOException {
     ListCorruptFileBlocksRequestProto.Builder req = 

Modified: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolClientSideTranslatorPB.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolClientSideTranslatorPB.java?rev=1376145&r1=1376144&r2=1376145&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolClientSideTranslatorPB.java (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolClientSideTranslatorPB.java Wed Aug 22 17:05:38 2012
@@ -41,8 +41,6 @@ import org.apache.hadoop.hdfs.protocol.p
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ErrorReportRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.HeartbeatRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.HeartbeatResponseProto;
-import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ProcessUpgradeRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ProcessUpgradeResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterDatanodeRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterDatanodeResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReportBadBlocksRequestProto;
@@ -59,7 +57,6 @@ import org.apache.hadoop.hdfs.server.pro
 import org.apache.hadoop.hdfs.server.protocol.StorageBlockReport;
 import org.apache.hadoop.hdfs.server.protocol.StorageReceivedDeletedBlocks;
 import org.apache.hadoop.hdfs.server.protocol.StorageReport;
-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;
@@ -253,20 +250,6 @@ public class DatanodeProtocolClientSideT
   }
 
   @Override
-  public UpgradeCommand processUpgradeCommand(UpgradeCommand comm)
-      throws IOException {
-    ProcessUpgradeRequestProto req = ProcessUpgradeRequestProto.newBuilder()
-        .setCmd(PBHelper.convert(comm)).build();
-    ProcessUpgradeResponseProto resp;
-    try {
-      resp = rpcProxy.processUpgrade(NULL_CONTROLLER, req);
-    } catch (ServiceException se) {
-      throw ProtobufHelper.getRemoteException(se);
-    }
-    return resp.hasCmd() ? PBHelper.convert(resp.getCmd()) : null;
-  }
-
-  @Override
   public void reportBadBlocks(LocatedBlock[] blocks) throws IOException {
     ReportBadBlocksRequestProto.Builder builder = ReportBadBlocksRequestProto
         .newBuilder();

Modified: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolServerSideTranslatorPB.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolServerSideTranslatorPB.java?rev=1376145&r1=1376144&r2=1376145&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolServerSideTranslatorPB.java (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolServerSideTranslatorPB.java Wed Aug 22 17:05:38 2012
@@ -33,8 +33,6 @@ import org.apache.hadoop.hdfs.protocol.p
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ErrorReportResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.HeartbeatRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.HeartbeatResponseProto;
-import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ProcessUpgradeRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ProcessUpgradeResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReceivedDeletedBlockInfoProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterDatanodeRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterDatanodeResponseProto;
@@ -56,7 +54,6 @@ import org.apache.hadoop.hdfs.server.pro
 import org.apache.hadoop.hdfs.server.protocol.StorageBlockReport;
 import org.apache.hadoop.hdfs.server.protocol.StorageReceivedDeletedBlocks;
 import org.apache.hadoop.hdfs.server.protocol.StorageReport;
-import org.apache.hadoop.hdfs.server.protocol.UpgradeCommand;
 
 import com.google.protobuf.RpcController;
 import com.google.protobuf.ServiceException;
@@ -212,25 +209,6 @@ public class DatanodeProtocolServerSideT
   }
 
   @Override
-  public ProcessUpgradeResponseProto processUpgrade(RpcController controller,
-      ProcessUpgradeRequestProto request) throws ServiceException {
-    UpgradeCommand ret;
-    try {
-      UpgradeCommand cmd = request.hasCmd() ? PBHelper
-          .convert(request.getCmd()) : null;
-      ret = impl.processUpgradeCommand(cmd);
-    } catch (IOException e) {
-      throw new ServiceException(e);
-    }
-    ProcessUpgradeResponseProto.Builder builder = 
-        ProcessUpgradeResponseProto.newBuilder();
-    if (ret != null) {
-      builder.setCmd(PBHelper.convert(ret));
-    }
-    return builder.build();
-  }
-
-  @Override
   public ReportBadBlocksResponseProto reportBadBlocks(RpcController controller,
       ReportBadBlocksRequestProto request) throws ServiceException {
     List<LocatedBlockProto> lbps = request.getBlocksList();

Modified: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java?rev=1376145&r1=1376144&r2=1376145&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java Wed Aug 22 17:05:38 2012
@@ -37,7 +37,6 @@ import org.apache.hadoop.hdfs.protocol.D
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 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.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.HdfsLocatedFileStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
@@ -47,7 +46,6 @@ import org.apache.hadoop.hdfs.protocol.p
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.DatanodeReportTypeProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFsStatsResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SafeModeActionProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.UpgradeActionProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BalancerBandwidthCommandProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockCommandProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockRecoveryCommandProto;
@@ -61,7 +59,6 @@ import org.apache.hadoop.hdfs.protocol.p
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReceivedDeletedBlockInfoProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterCommandProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.StorageReportProto;
-import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.UpgradeCommandProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DataEncryptionKeyProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockKeyProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto;
@@ -96,7 +93,6 @@ import org.apache.hadoop.hdfs.protocol.p
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto.NamenodeRoleProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ReplicaStateProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StorageInfoProto;
-import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.UpgradeStatusReportProto;
 import org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.JournalInfoProto;
 import org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey;
 import org.apache.hadoop.hdfs.security.token.block.BlockKey;
@@ -106,7 +102,6 @@ import org.apache.hadoop.hdfs.security.t
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
 import org.apache.hadoop.hdfs.server.common.StorageInfo;
-import org.apache.hadoop.hdfs.server.common.UpgradeStatusReport;
 import org.apache.hadoop.hdfs.server.namenode.CheckpointSignature;
 import org.apache.hadoop.hdfs.server.protocol.BalancerBandwidthCommand;
 import org.apache.hadoop.hdfs.server.protocol.BlockCommand;
@@ -132,7 +127,6 @@ import org.apache.hadoop.hdfs.server.pro
 import org.apache.hadoop.hdfs.server.protocol.RegisterCommand;
 import org.apache.hadoop.hdfs.server.protocol.RemoteEditLog;
 import org.apache.hadoop.hdfs.server.protocol.RemoteEditLogManifest;
-import org.apache.hadoop.hdfs.server.protocol.UpgradeCommand;
 import org.apache.hadoop.io.EnumSetWritable;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.util.DataChecksum;
@@ -640,8 +634,6 @@ public class PBHelper {
       return PBHelper.convert(proto.getKeyUpdateCmd());
     case RegisterCommand:
       return REG_CMD;
-    case UpgradeCommand:
-      return PBHelper.convert(proto.getUpgradeCmd());
     }
     return null;
   }
@@ -738,11 +730,6 @@ public class PBHelper {
       builder.setCmdType(DatanodeCommandProto.Type.BlockCommand).setBlkCmd(
           PBHelper.convert((BlockCommand) datanodeCommand));
       break;
-    case DatanodeProtocol.DNA_UC_ACTION_REPORT_STATUS:
-    case DatanodeProtocol.DNA_UC_ACTION_START_UPGRADE:
-      builder.setCmdType(DatanodeCommandProto.Type.UpgradeCommand)
-          .setUpgradeCmd(PBHelper.convert((UpgradeCommand) datanodeCommand));
-      break;
     case DatanodeProtocol.DNA_UNKNOWN: //Not expected
     default:
       builder.setCmdType(DatanodeCommandProto.Type.NullDatanodeCommand);
@@ -750,19 +737,6 @@ public class PBHelper {
     return builder.build();
   }
 
-  public static UpgradeCommand convert(UpgradeCommandProto upgradeCmd) {
-    int action = UpgradeCommand.UC_ACTION_UNKNOWN;
-    switch (upgradeCmd.getAction()) {
-    case REPORT_STATUS:
-      action = UpgradeCommand.UC_ACTION_REPORT_STATUS;
-      break;
-    case START_UPGRADE:
-      action = UpgradeCommand.UC_ACTION_START_UPGRADE;
-    }
-    return new UpgradeCommand(action, upgradeCmd.getVersion(),
-        (short) upgradeCmd.getUpgradeStatus());
-  }
-
   public static KeyUpdateCommand convert(KeyUpdateCommandProto keyUpdateCmd) {
     return new KeyUpdateCommand(PBHelper.convert(keyUpdateCmd.getKeys()));
   }
@@ -852,28 +826,6 @@ public class PBHelper {
         .build();
   }
 
-  public static UpgradeCommandProto convert(UpgradeCommand comm) {
-    UpgradeCommandProto.Builder builder = UpgradeCommandProto.newBuilder();
-    if (comm == null) {
-      return builder.setAction(UpgradeCommandProto.Action.UNKNOWN)
-          .setVersion(0).setUpgradeStatus(0).build();
-    }
-    builder.setVersion(comm.getVersion()).setUpgradeStatus(
-        comm.getCurrentStatus());
-    switch (comm.getAction()) {
-    case UpgradeCommand.UC_ACTION_REPORT_STATUS:
-      builder.setAction(UpgradeCommandProto.Action.REPORT_STATUS);
-      break;
-    case UpgradeCommand.UC_ACTION_START_UPGRADE:
-      builder.setAction(UpgradeCommandProto.Action.START_UPGRADE);
-      break;
-    default:
-      builder.setAction(UpgradeCommandProto.Action.UNKNOWN);
-      break;
-    }
-    return builder.build();
-  }
-
   public static ReceivedDeletedBlockInfo convert(
       ReceivedDeletedBlockInfoProto proto) {
     ReceivedDeletedBlockInfo.BlockStatus status = null;
@@ -1238,51 +1190,6 @@ public class PBHelper {
     }
   }
   
-  public static UpgradeActionProto convert(
-      UpgradeAction a) {
-    switch (a) {
-    case GET_STATUS:
-      return UpgradeActionProto.GET_STATUS;
-    case DETAILED_STATUS:
-      return UpgradeActionProto.DETAILED_STATUS;
-    case FORCE_PROCEED:
-      return UpgradeActionProto.FORCE_PROCEED;
-    default:
-      throw new IllegalArgumentException("Unexpected UpgradeAction :" + a);
-    }
-  }
-  
-  
-  public static UpgradeAction convert(
-      UpgradeActionProto a) {
-    switch (a) {
-    case GET_STATUS:
-      return UpgradeAction.GET_STATUS;
-    case DETAILED_STATUS:
-      return UpgradeAction.DETAILED_STATUS;
-    case FORCE_PROCEED:
-      return UpgradeAction.FORCE_PROCEED;
-    default:
-      throw new IllegalArgumentException("Unexpected UpgradeAction :" + a);
-    }
-  }
-
-  public static UpgradeStatusReportProto convert(UpgradeStatusReport r) {
-    if (r == null)
-      return null;
-    return UpgradeStatusReportProto.newBuilder()
-        .setVersion(r.getVersion())
-        .setUpgradeStatus(r.getUpgradeStatus())
-        .setFinalized(r.isFinalized())
-        .build();
-  }
-  
-  public static UpgradeStatusReport convert(UpgradeStatusReportProto r) {
-    if (r == null) return null;
-    return new UpgradeStatusReport(r.getVersion(),
-        (short) r.getUpgradeStatus(), r.getFinalized());
-  }
-  
   public static CorruptFileBlocks convert(CorruptFileBlocksProto c) {
     if (c == null)
       return null;

Modified: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPOfferService.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPOfferService.java?rev=1376145&r1=1376144&r2=1376145&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPOfferService.java (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPOfferService.java Wed Aug 22 17:05:38 2012
@@ -42,7 +42,6 @@ import org.apache.hadoop.hdfs.server.pro
 import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
 import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo;
 import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo.BlockStatus;
-import org.apache.hadoop.hdfs.server.protocol.UpgradeCommand;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
@@ -561,10 +560,6 @@ class BPOfferService {
 
       dn.finalizeUpgradeForPool(bp);
       break;
-    case UpgradeCommand.UC_ACTION_START_UPGRADE:
-      // start distributed upgrade here
-      LOG.warn("Distibuted upgrade is no longer supported");
-      break;
     case DatanodeProtocol.DNA_RECOVERBLOCK:
       String who = "NameNode at " + actor.getNNSocketAddress();
       dn.recoverBlocks(who, ((BlockRecoveryCommand)cmd).getRecoveringBlocks());

Modified: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java?rev=1376145&r1=1376144&r2=1376145&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java Wed Aug 22 17:05:38 2012
@@ -60,7 +60,6 @@ import org.apache.hadoop.hdfs.protocol.E
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 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.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
@@ -89,7 +88,6 @@ import org.apache.hadoop.hdfs.security.t
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
 import org.apache.hadoop.hdfs.server.common.IncorrectVersionException;
-import org.apache.hadoop.hdfs.server.common.UpgradeStatusReport;
 import org.apache.hadoop.hdfs.server.namenode.NameNode.OperationCategory;
 import org.apache.hadoop.hdfs.server.namenode.metrics.NameNodeMetrics;
 import org.apache.hadoop.hdfs.server.namenode.web.resources.NamenodeWebHdfsMethods;
@@ -108,7 +106,6 @@ import org.apache.hadoop.hdfs.server.pro
 import org.apache.hadoop.hdfs.server.protocol.StorageBlockReport;
 import org.apache.hadoop.hdfs.server.protocol.StorageReceivedDeletedBlocks;
 import org.apache.hadoop.hdfs.server.protocol.StorageReport;
-import org.apache.hadoop.hdfs.server.protocol.UpgradeCommand;
 import org.apache.hadoop.io.EnumSetWritable;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.ipc.ProtobufRpcEngine;
@@ -741,13 +738,6 @@ class NameNodeRpcServer implements Namen
   }
 
   @Override // ClientProtocol
-  public UpgradeStatusReport distributedUpgradeProgress(UpgradeAction action)
-      throws IOException {
-    throw new UnsupportedActionException(
-        "Deprecated method. No longer supported");
-  }
-
-  @Override // ClientProtocol
   public void metaSave(String filename) throws IOException {
     namesystem.checkOperation(OperationCategory.UNCHECKED);
     namesystem.metaSave(filename);
@@ -917,13 +907,6 @@ class NameNodeRpcServer implements Namen
     return namesystem.getNamespaceInfo();
   }
 
-  @Override // DatanodeProtocol
-  public UpgradeCommand processUpgradeCommand(UpgradeCommand comm)
-      throws IOException {
-    throw new UnsupportedActionException(
-        "Deprecated method, no longer supported");
-  }
-
   /** 
    * Verifies the given registration.
    * 

Modified: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeJspHelper.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeJspHelper.java?rev=1376145&r1=1376144&r2=1376145&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeJspHelper.java (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeJspHelper.java Wed Aug 22 17:05:38 2012
@@ -41,7 +41,6 @@ import org.apache.hadoop.hdfs.DFSConfigK
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants.UpgradeAction;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
@@ -49,7 +48,6 @@ import org.apache.hadoop.hdfs.server.blo
 import org.apache.hadoop.hdfs.server.common.JspHelper;
 import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
-import org.apache.hadoop.hdfs.server.common.UpgradeStatusReport;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
 import org.apache.hadoop.http.HttpConfig;
 import org.apache.hadoop.io.Text;

Modified: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeProtocol.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeProtocol.java?rev=1376145&r1=1376144&r2=1376145&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeProtocol.java (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeProtocol.java Wed Aug 22 17:05:38 2012
@@ -72,8 +72,6 @@ public interface DatanodeProtocol {
   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
-  final static int DNA_UC_ACTION_REPORT_STATUS = 100; // Report upgrade status
-  final static int DNA_UC_ACTION_START_UPGRADE = 101; // start upgrade
 
   /** 
    * Register Datanode.
@@ -151,18 +149,6 @@ public interface DatanodeProtocol {
   public NamespaceInfo 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
-   */
-  UpgradeCommand processUpgradeCommand(UpgradeCommand comm) throws IOException;
-  
-  /**
    * same as {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#reportBadBlocks(LocatedBlock[])}
    * }
    */

Modified: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto?rev=1376145&r1=1376144&r2=1376145&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto Wed Aug 22 17:05:38 2012
@@ -296,19 +296,6 @@ message FinalizeUpgradeRequestProto { //
 message FinalizeUpgradeResponseProto { // void response
 }
 
-enum UpgradeActionProto {
-  GET_STATUS = 1;
-  DETAILED_STATUS = 2;
-  FORCE_PROCEED = 3;
-}
-
-message DistributedUpgradeProgressRequestProto {
-  required UpgradeActionProto action = 1;
-}
-message DistributedUpgradeProgressResponseProto {
-  optional UpgradeStatusReportProto report = 1;
-}
-
 message ListCorruptFileBlocksRequestProto {
   required string path = 1;
   optional string cookie = 2;
@@ -490,8 +477,6 @@ service ClientNamenodeProtocol {
   rpc refreshNodes(RefreshNodesRequestProto) returns(RefreshNodesResponseProto);
   rpc finalizeUpgrade(FinalizeUpgradeRequestProto)
       returns(FinalizeUpgradeResponseProto);
-  rpc distributedUpgradeProgress(DistributedUpgradeProgressRequestProto)
-      returns(DistributedUpgradeProgressResponseProto);
   rpc listCorruptFileBlocks(ListCorruptFileBlocksRequestProto)
       returns(ListCorruptFileBlocksResponseProto);
   rpc metaSave(MetaSaveRequestProto) returns(MetaSaveResponseProto);

Modified: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto?rev=1376145&r1=1376144&r2=1376145&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto Wed Aug 22 17:05:38 2012
@@ -60,7 +60,7 @@ message DatanodeCommandProto {
     FinalizeCommand = 3;
     KeyUpdateCommand = 4;
     RegisterCommand = 5;
-    UpgradeCommand = 6;
+    UnusedUpgradeCommand = 6;
     NullDatanodeCommand = 7;
   }
 
@@ -74,7 +74,6 @@ message DatanodeCommandProto {
   optional FinalizeCommandProto finalizeCmd = 5;
   optional KeyUpdateCommandProto keyUpdateCmd = 6;
   optional RegisterCommandProto registerCmd = 7;
-  optional UpgradeCommandProto upgradeCmd = 8;
 }
 
 /**
@@ -132,20 +131,6 @@ message RegisterCommandProto {
 }
 
 /**
- * Generic distributed upgrade Command
- */
-message UpgradeCommandProto {
-  enum Action {
-    UNKNOWN = 0;          // Unknown action
-    REPORT_STATUS = 100;  // Report upgrade status
-    START_UPGRADE = 101;  // Start upgrade
-  }
-  required Action action = 1;  // Upgrade action
-  required uint32 version = 2; // Version of the upgrade
-  required uint32 upgradeStatus = 3; // % completed in range 0 & 100
-}
-
-/**
  * registration - Information of the datanode registering with the namenode
  */
 message RegisterDatanodeRequestProto {
@@ -303,20 +288,6 @@ message ErrorReportResponseProto {
 }
 
 /**
- * cmd - Upgrade command sent from datanode to namenode
- */
-message ProcessUpgradeRequestProto {
-  optional UpgradeCommandProto cmd = 1;
-}
-
-/**
- * cmd - Upgrade command sent from namenode to datanode
- */
-message ProcessUpgradeResponseProto {
-  optional UpgradeCommandProto cmd = 1;
-}
-
-/**
  * blocks - list of blocks that are reported as corrupt
  */
 message ReportBadBlocksRequestProto {
@@ -389,12 +360,6 @@ service DatanodeProtocolService {
   rpc versionRequest(VersionRequestProto) returns(VersionResponseProto);
 
   /**
-   * Generic way to send commands from datanode to namenode during
-   * distributed upgrade process.
-   */
-  rpc processUpgrade(ProcessUpgradeRequestProto) returns(ProcessUpgradeResponseProto);
-
-  /**
    * Report corrupt blocks at the specified location
    */
   rpc reportBadBlocks(ReportBadBlocksRequestProto) returns(ReportBadBlocksResponseProto);

Modified: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto?rev=1376145&r1=1376144&r2=1376145&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto Wed Aug 22 17:05:38 2012
@@ -211,15 +211,6 @@ message DirectoryListingProto {
 }
 
 /**
- * Status of current cluster upgrade from one version to another
- */
-message UpgradeStatusReportProto {
-  required uint32 version = 1;;
-  required uint32 upgradeStatus = 2; // % completed in range 0 & 100
-	required bool finalized = 3;
-}
-
-/**
  * Common node information shared by all the nodes in the cluster
  */
 message StorageInfoProto {