瀏覽代碼

commit 423684b08fca3426c2adee9487ce31673712d168
Author: Jitendra Nath Pandey <jitendra@yahoo-inc.com>
Date: Tue Dec 22 12:26:46 2009 -0800

HADOOP-5824 from https://issues.apache.org/jira/secure/attachment/12428759/HADOOP-5824-0_20.1.patch

+++ b/YAHOO-CHANGES.txt
+ HADOOP-5824. remove OP_READ_METADATA functionality from Datanode.
+ (Jitendra Nath Pandey)
+


git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-0.20-security-patches@1077087 13f79535-47bb-0310-9956-ffa450edef68

Owen O'Malley 14 年之前
父節點
當前提交
0ac9b09c54

+ 4 - 1
src/hdfs/org/apache/hadoop/hdfs/protocol/DataTransferProtocol.java

@@ -40,7 +40,10 @@ public interface DataTransferProtocol {
   // Processed at datanode stream-handler
   public static final byte OP_WRITE_BLOCK = (byte) 80;
   public static final byte OP_READ_BLOCK = (byte) 81;
-  public static final byte OP_READ_METADATA = (byte) 82;
+  /**
+   * @deprecated As of version 15, OP_READ_METADATA is no longer supported
+   */
+  @Deprecated public static final byte OP_READ_METADATA = (byte) 82;
   public static final byte OP_REPLACE_BLOCK = (byte) 83;
   public static final byte OP_COPY_BLOCK = (byte) 84;
   public static final byte OP_BLOCK_CHECKSUM = (byte) 85;

+ 0 - 42
src/hdfs/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java

@@ -109,10 +109,6 @@ class DataXceiver implements Runnable, FSConstants {
         else
           datanode.myMetrics.writesFromRemoteClient.inc();
         break;
-      case DataTransferProtocol.OP_READ_METADATA:
-        readMetadata( in );
-        datanode.myMetrics.readMetadataOp.inc(DataNode.now() - startTime);
-        break;
       case DataTransferProtocol.OP_REPLACE_BLOCK: // for balancing purpose; send to a destination
         replaceBlock(in);
         datanode.myMetrics.replaceBlockOp.inc(DataNode.now() - startTime);
@@ -416,44 +412,6 @@ class DataXceiver implements Runnable, FSConstants {
     }
   }
 
-  /**
-   * Reads the metadata and sends the data in one 'DATA_CHUNK'.
-   * @param in
-   */
-  void readMetadata(DataInputStream in) throws IOException {
-    Block block = new Block( in.readLong(), 0 , in.readLong());
-    MetaDataInputStream checksumIn = null;
-    DataOutputStream out = null;
-    
-    try {
-
-      checksumIn = datanode.data.getMetaDataInputStream(block);
-      
-      long fileSize = checksumIn.getLength();
-
-      if (fileSize >= 1L<<31 || fileSize <= 0) {
-          throw new IOException("Unexpected size for checksumFile of block" +
-                  block);
-      }
-
-      byte [] buf = new byte[(int)fileSize];
-      IOUtils.readFully(checksumIn, buf, 0, buf.length);
-      
-      out = new DataOutputStream(
-                NetUtils.getOutputStream(s, datanode.socketWriteTimeout));
-      
-      out.writeByte(DataTransferProtocol.OP_STATUS_SUCCESS);
-      out.writeInt(buf.length);
-      out.write(buf);
-      
-      //last DATA_CHUNK
-      out.writeInt(0);
-    } finally {
-      IOUtils.closeStream(out);
-      IOUtils.closeStream(checksumIn);
-    }
-  }
-  
   /**
    * Get block checksum (MD5 of CRC32).
    * @param in

+ 0 - 3
src/hdfs/org/apache/hadoop/hdfs/server/datanode/metrics/DataNodeMetrics.java

@@ -78,8 +78,6 @@ public class DataNodeMetrics implements Updater {
                 new MetricsTimeVaryingRate("readBlockOp", registry);
   public MetricsTimeVaryingRate writeBlockOp = 
                 new MetricsTimeVaryingRate("writeBlockOp", registry);
-  public MetricsTimeVaryingRate readMetadataOp = 
-                new MetricsTimeVaryingRate("readMetadataOp", registry);
   public MetricsTimeVaryingRate blockChecksumOp = 
                 new MetricsTimeVaryingRate("blockChecksumOp", registry);
   public MetricsTimeVaryingRate copyBlockOp = 
@@ -128,7 +126,6 @@ public class DataNodeMetrics implements Updater {
   public void resetAllMinMax() {
     readBlockOp.resetMinMax();
     writeBlockOp.resetMinMax();
-    readMetadataOp.resetMinMax();
     blockChecksumOp.resetMinMax();
     copyBlockOp.resetMinMax();
     replaceBlockOp.resetMinMax();