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 sz...@apache.org on 2013/01/24 03:45:57 UTC

svn commit: r1437843 [2/2] - in /hadoop/common/branches/HDFS-2802/hadoop-hdfs-project: ./ hadoop-hdfs-httpfs/ hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/servlet/ hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/lib/servlet/ hadoop-hdfs/ h...

Modified: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/RefreshAuthorizationPolicyProtocolServerSideTranslatorPB.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/RefreshAuthorizationPolicyProtocolServerSideTranslatorPB.java?rev=1437843&r1=1437842&r2=1437843&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/RefreshAuthorizationPolicyProtocolServerSideTranslatorPB.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/RefreshAuthorizationPolicyProtocolServerSideTranslatorPB.java Thu Jan 24 02:45:45 2013
@@ -32,6 +32,10 @@ public class RefreshAuthorizationPolicyP
 
   private final RefreshAuthorizationPolicyProtocol impl;
 
+  private final static RefreshServiceAclResponseProto
+  VOID_REFRESH_SERVICE_ACL_RESPONSE = RefreshServiceAclResponseProto
+      .newBuilder().build();
+
   public RefreshAuthorizationPolicyProtocolServerSideTranslatorPB(
       RefreshAuthorizationPolicyProtocol impl) {
     this.impl = impl;
@@ -46,6 +50,6 @@ public class RefreshAuthorizationPolicyP
     } catch (IOException e) {
       throw new ServiceException(e);
     }
-    return RefreshServiceAclResponseProto.newBuilder().build();
+    return VOID_REFRESH_SERVICE_ACL_RESPONSE;
   }
 }

Modified: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/RefreshUserMappingsProtocolClientSideTranslatorPB.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/RefreshUserMappingsProtocolClientSideTranslatorPB.java?rev=1437843&r1=1437842&r2=1437843&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/RefreshUserMappingsProtocolClientSideTranslatorPB.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/RefreshUserMappingsProtocolClientSideTranslatorPB.java Thu Jan 24 02:45:45 2013
@@ -39,6 +39,14 @@ public class RefreshUserMappingsProtocol
   private final static RpcController NULL_CONTROLLER = null;
   private final RefreshUserMappingsProtocolPB rpcProxy;
   
+  private final static RefreshUserToGroupsMappingsRequestProto 
+  VOID_REFRESH_USER_TO_GROUPS_MAPPING_REQUEST = 
+      RefreshUserToGroupsMappingsRequestProto.newBuilder().build();
+
+  private final static RefreshSuperUserGroupsConfigurationRequestProto
+  VOID_REFRESH_SUPERUSER_GROUPS_CONFIGURATION_REQUEST = 
+      RefreshSuperUserGroupsConfigurationRequestProto.newBuilder().build();
+
   public RefreshUserMappingsProtocolClientSideTranslatorPB(
       RefreshUserMappingsProtocolPB rpcProxy) {
     this.rpcProxy = rpcProxy;
@@ -51,10 +59,9 @@ public class RefreshUserMappingsProtocol
 
   @Override
   public void refreshUserToGroupsMappings() throws IOException {
-    RefreshUserToGroupsMappingsRequestProto request = 
-        RefreshUserToGroupsMappingsRequestProto.newBuilder().build();
     try {
-      rpcProxy.refreshUserToGroupsMappings(NULL_CONTROLLER, request);
+      rpcProxy.refreshUserToGroupsMappings(NULL_CONTROLLER,
+          VOID_REFRESH_USER_TO_GROUPS_MAPPING_REQUEST);
     } catch (ServiceException se) {
       throw ProtobufHelper.getRemoteException(se);
     }
@@ -62,10 +69,9 @@ public class RefreshUserMappingsProtocol
 
   @Override
   public void refreshSuperUserGroupsConfiguration() throws IOException {
-    RefreshSuperUserGroupsConfigurationRequestProto request = 
-        RefreshSuperUserGroupsConfigurationRequestProto.newBuilder().build();
     try {
-      rpcProxy.refreshSuperUserGroupsConfiguration(NULL_CONTROLLER, request);
+      rpcProxy.refreshSuperUserGroupsConfiguration(NULL_CONTROLLER,
+          VOID_REFRESH_SUPERUSER_GROUPS_CONFIGURATION_REQUEST);
     } catch (ServiceException se) {
       throw ProtobufHelper.getRemoteException(se);
     }

Modified: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/RefreshUserMappingsProtocolServerSideTranslatorPB.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/RefreshUserMappingsProtocolServerSideTranslatorPB.java?rev=1437843&r1=1437842&r2=1437843&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/RefreshUserMappingsProtocolServerSideTranslatorPB.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/RefreshUserMappingsProtocolServerSideTranslatorPB.java Thu Jan 24 02:45:45 2013
@@ -33,6 +33,15 @@ public class RefreshUserMappingsProtocol
 
   private final RefreshUserMappingsProtocol impl;
   
+  private final static RefreshUserToGroupsMappingsResponseProto 
+  VOID_REFRESH_USER_GROUPS_MAPPING_RESPONSE =
+      RefreshUserToGroupsMappingsResponseProto.newBuilder().build();
+
+  private final static RefreshSuperUserGroupsConfigurationResponseProto
+  VOID_REFRESH_SUPERUSER_GROUPS_CONFIGURATION_RESPONSE = 
+      RefreshSuperUserGroupsConfigurationResponseProto.newBuilder()
+      .build();
+
   public RefreshUserMappingsProtocolServerSideTranslatorPB(RefreshUserMappingsProtocol impl) {
     this.impl = impl;
   }
@@ -47,7 +56,7 @@ public class RefreshUserMappingsProtocol
     } catch (IOException e) {
       throw new ServiceException(e);
     }
-    return RefreshUserToGroupsMappingsResponseProto.newBuilder().build();
+    return VOID_REFRESH_USER_GROUPS_MAPPING_RESPONSE;
   }
 
   @Override
@@ -60,7 +69,6 @@ public class RefreshUserMappingsProtocol
     } catch (IOException e) {
       throw new ServiceException(e);
     }
-    return RefreshSuperUserGroupsConfigurationResponseProto.newBuilder()
-        .build();
+    return VOID_REFRESH_SUPERUSER_GROUPS_CONFIGURATION_RESPONSE;
   }
 }

Modified: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/protocolPB/QJournalProtocolServerSideTranslatorPB.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/protocolPB/QJournalProtocolServerSideTranslatorPB.java?rev=1437843&r1=1437842&r2=1437843&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/protocolPB/QJournalProtocolServerSideTranslatorPB.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/protocolPB/QJournalProtocolServerSideTranslatorPB.java Thu Jan 24 02:45:45 2013
@@ -65,6 +65,13 @@ public class QJournalProtocolServerSideT
   /** Server side implementation to delegate the requests to */
   private final QJournalProtocol impl;
 
+  private final static JournalResponseProto VOID_JOURNAL_RESPONSE =
+  JournalResponseProto.newBuilder().build();
+
+  private final static StartLogSegmentResponseProto
+  VOID_START_LOG_SEGMENT_RESPONSE =
+      StartLogSegmentResponseProto.newBuilder().build();
+
   public QJournalProtocolServerSideTranslatorPB(QJournalProtocol impl) {
     this.impl = impl;
   }
@@ -135,7 +142,7 @@ public class QJournalProtocolServerSideT
     } catch (IOException e) {
       throw new ServiceException(e);
     }
-    return JournalResponseProto.newBuilder().build();
+    return VOID_JOURNAL_RESPONSE;
   }
 
   /** @see JournalProtocol#heartbeat */
@@ -160,7 +167,7 @@ public class QJournalProtocolServerSideT
     } catch (IOException e) {
       throw new ServiceException(e);
     }
-    return StartLogSegmentResponseProto.newBuilder().build();
+    return VOID_START_LOG_SEGMENT_RESPONSE;
   }
   
   @Override

Modified: hadoop/common/branches/HDFS-2802/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/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPOfferService.java?rev=1437843&r1=1437842&r2=1437843&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPOfferService.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPOfferService.java Thu Jan 24 02:45:45 2013
@@ -395,7 +395,7 @@ class BPOfferService {
   }
 
   @VisibleForTesting
-  synchronized List<BPServiceActor> getBPServiceActors() {
+  List<BPServiceActor> getBPServiceActors() {
     return Lists.newArrayList(bpServices);
   }
   

Modified: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockPoolSliceScanner.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockPoolSliceScanner.java?rev=1437843&r1=1437842&r2=1437843&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockPoolSliceScanner.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockPoolSliceScanner.java Thu Jan 24 02:45:45 2013
@@ -388,8 +388,8 @@ class BlockPoolSliceScanner {
       try {
         adjustThrottler();
         
-        blockSender = new BlockSender(block, 0, -1, false, true, datanode,
-            null);
+        blockSender = new BlockSender(block, 0, -1, false, true, true, 
+            datanode, null);
 
         DataOutputStream out = 
                 new DataOutputStream(new IOUtils.NullOutputStream());

Modified: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockSender.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockSender.java?rev=1437843&r1=1437842&r2=1437843&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockSender.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockSender.java Thu Jan 24 02:45:45 2013
@@ -45,6 +45,8 @@ import org.apache.hadoop.io.nativeio.Nat
 import org.apache.hadoop.net.SocketOutputStream;
 import org.apache.hadoop.util.DataChecksum;
 
+import com.google.common.base.Preconditions;
+
 /**
  * Reads a block from the disk and sends it to a recipient.
  * 
@@ -158,12 +160,14 @@ class BlockSender implements java.io.Clo
    * @param length length of data to read
    * @param corruptChecksumOk
    * @param verifyChecksum verify checksum while reading the data
+   * @param sendChecksum send checksum to client.
    * @param datanode datanode from which the block is being read
    * @param clientTraceFmt format string used to print client trace logs
    * @throws IOException
    */
   BlockSender(ExtendedBlock block, long startOffset, long length,
               boolean corruptChecksumOk, boolean verifyChecksum,
+              boolean sendChecksum,
               DataNode datanode, String clientTraceFmt)
       throws IOException {
     try {
@@ -175,6 +179,13 @@ class BlockSender implements java.io.Clo
       this.shouldDropCacheBehindRead = datanode.getDnConf().dropCacheBehindReads;
       this.datanode = datanode;
       
+      if (verifyChecksum) {
+        // To simplify implementation, callers may not specify verification
+        // without sending.
+        Preconditions.checkArgument(sendChecksum,
+            "If verifying checksum, currently must also send it.");
+      }
+      
       final Replica replica;
       final long replicaVisibleLength;
       synchronized(datanode.data) { 
@@ -213,29 +224,37 @@ class BlockSender implements java.io.Clo
        * False,  True: will verify checksum
        * False, False: throws IOException file not found
        */
-      DataChecksum csum;
-      final InputStream metaIn = datanode.data.getMetaDataInputStream(block);
-      if (!corruptChecksumOk || metaIn != null) {
-      	if (metaIn == null) {
-          //need checksum but meta-data not found
-          throw new FileNotFoundException("Meta-data not found for " + block);
-        } 
-      	
-        checksumIn = new DataInputStream(
-            new BufferedInputStream(metaIn, HdfsConstants.IO_FILE_BUFFER_SIZE));
-
-        // read and handle the common header here. For now just a version
-        BlockMetadataHeader header = BlockMetadataHeader.readHeader(checksumIn);
-        short version = header.getVersion();
-        if (version != BlockMetadataHeader.VERSION) {
-          LOG.warn("Wrong version (" + version + ") for metadata file for "
-              + block + " ignoring ...");
+      DataChecksum csum = null;
+      if (verifyChecksum || sendChecksum) {
+        final InputStream metaIn = datanode.data.getMetaDataInputStream(block);
+        if (!corruptChecksumOk || metaIn != null) {
+          if (metaIn == null) {
+            //need checksum but meta-data not found
+            throw new FileNotFoundException("Meta-data not found for " + block);
+          }
+
+          checksumIn = new DataInputStream(
+              new BufferedInputStream(metaIn, HdfsConstants.IO_FILE_BUFFER_SIZE));
+  
+          // read and handle the common header here. For now just a version
+          BlockMetadataHeader header = BlockMetadataHeader.readHeader(checksumIn);
+          short version = header.getVersion();
+          if (version != BlockMetadataHeader.VERSION) {
+            LOG.warn("Wrong version (" + version + ") for metadata file for "
+                + block + " ignoring ...");
+          }
+          csum = header.getChecksum();
+        } else {
+          LOG.warn("Could not find metadata file for " + block);
         }
-        csum = header.getChecksum();
-      } else {
-        LOG.warn("Could not find metadata file for " + block);
-        // This only decides the buffer size. Use BUFFER_SIZE?
-        csum = DataChecksum.newDataChecksum(DataChecksum.Type.NULL, 16 * 1024);
+      }
+      if (csum == null) {
+        // The number of bytes per checksum here determines the alignment
+        // of reads: we always start reading at a checksum chunk boundary,
+        // even if the checksum type is NULL. So, choosing too big of a value
+        // would risk sending too much unnecessary data. 512 (1 disk sector)
+        // is likely to result in minimal extra IO.
+        csum = DataChecksum.newDataChecksum(DataChecksum.Type.NULL, 512);
       }
 
       /*

Modified: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java?rev=1437843&r1=1437842&r2=1437843&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java Thu Jan 24 02:45:45 2013
@@ -1441,7 +1441,7 @@ public class DataNode extends Configured
             HdfsConstants.SMALL_BUFFER_SIZE));
         in = new DataInputStream(unbufIn);
         blockSender = new BlockSender(b, 0, b.getNumBytes(), 
-            false, false, DataNode.this, null);
+            false, false, true, DataNode.this, null);
         DatanodeInfo srcNode = new DatanodeInfo(bpReg);
 
         //

Modified: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java?rev=1437843&r1=1437842&r2=1437843&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java Thu Jan 24 02:45:45 2013
@@ -241,7 +241,8 @@ class DataXceiver extends Receiver imple
       final Token<BlockTokenIdentifier> blockToken,
       final String clientName,
       final long blockOffset,
-      final long length) throws IOException {
+      final long length,
+      final boolean sendChecksum) throws IOException {
     previousOpClientName = clientName;
 
     OutputStream baseStream = getOutputStream();
@@ -266,7 +267,7 @@ class DataXceiver extends Receiver imple
     try {
       try {
         blockSender = new BlockSender(block, blockOffset, length,
-            true, false, datanode, clientTraceFmt);
+            true, false, sendChecksum, datanode, clientTraceFmt);
       } catch(IOException e) {
         String msg = "opReadBlock " + block + " received exception " + e; 
         LOG.info(msg);
@@ -654,7 +655,7 @@ class DataXceiver extends Receiver imple
 
     try {
       // check if the block exists or not
-      blockSender = new BlockSender(block, 0, -1, false, false, datanode, 
+      blockSender = new BlockSender(block, 0, -1, false, false, true, datanode, 
           null);
 
       // set up response stream

Modified: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupState.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupState.java?rev=1437843&r1=1437842&r2=1437843&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupState.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupState.java Thu Jan 24 02:45:45 2013
@@ -1,3 +1,20 @@
+/**
+* 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.namenode;
 
 import java.io.IOException;

Modified: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileChecksumServlets.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileChecksumServlets.java?rev=1437843&r1=1437842&r2=1437843&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileChecksumServlets.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileChecksumServlets.java Thu Jan 24 02:45:45 2013
@@ -21,7 +21,6 @@ import java.io.IOException;
 import java.io.PrintWriter;
 import java.net.URL;
 
-import javax.net.SocketFactory;
 import javax.servlet.ServletContext;
 import javax.servlet.ServletException;
 import javax.servlet.http.HttpServletRequest;
@@ -33,14 +32,11 @@ import org.apache.hadoop.fs.MD5MD5CRC32F
 import org.apache.hadoop.hdfs.DFSClient;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
-import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
-import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.common.JspHelper;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.DatanodeJspHelper;
-import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.util.ServletUtil;
 import org.znerd.xmlenc.XMLOutputter;
@@ -116,18 +112,11 @@ public class FileChecksumServlets {
       final DataNode datanode = (DataNode) context.getAttribute("datanode");
       final Configuration conf = 
         new HdfsConfiguration(datanode.getConf());
-      final int socketTimeout = conf.getInt(
-          DFSConfigKeys.DFS_CLIENT_SOCKET_TIMEOUT_KEY,
-          HdfsServerConstants.READ_TIMEOUT);
-      final SocketFactory socketFactory = NetUtils.getSocketFactory(conf,
-          ClientProtocol.class);
       
       try {
         final DFSClient dfs = DatanodeJspHelper.getDFSClient(request, 
             datanode, conf, getUGI(request, conf));
-        final ClientProtocol nnproxy = dfs.getNamenode();
-        final MD5MD5CRC32FileChecksum checksum = DFSClient.getFileChecksum(
-            path, nnproxy, socketFactory, socketTimeout, dfs.getDataEncryptionKey(), false);
+        final MD5MD5CRC32FileChecksum checksum = dfs.getFileChecksum(path);
         MD5MD5CRC32FileChecksum.write(xml, checksum);
       } catch(IOException ioe) {
         writeXml(ioe, path, xml);

Modified: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java?rev=1437843&r1=1437842&r2=1437843&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java Thu Jan 24 02:45:45 2013
@@ -282,6 +282,17 @@ public class SecondaryNameNode implement
   }
 
   /**
+   * Wait for the service to finish.
+   * (Normally, it runs forever.)
+   */
+  private void join() {
+    try {
+      infoServer.join();
+    } catch (InterruptedException ie) {
+    }
+  }
+
+  /**
    * Shut down this instance of the datanode.
    * Returns only after shutdown is complete.
    */
@@ -607,6 +618,7 @@ public class SecondaryNameNode implement
 
     if (secondary != null) {
       secondary.startCheckpointThread();
+      secondary.join();
     }
   }
   

Propchange: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/native/
------------------------------------------------------------------------------
  Merged /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/native:r1432789-1437840

Modified: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto?rev=1437843&r1=1437842&r2=1437843&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto Thu Jan 24 02:45:45 2013
@@ -168,7 +168,7 @@ message RenameRequestProto {
   required string dst = 2;
 }
 
-message RenameResponseProto { // void response
+message RenameResponseProto {
   required bool result = 1;
 }
 
@@ -393,7 +393,7 @@ message GetLinkTargetRequestProto {
   required string path = 1;
 }
 message GetLinkTargetResponseProto {
-  required string targetPath = 1;
+  optional string targetPath = 1;
 }
 
 message UpdateBlockForPipelineRequestProto {
@@ -415,29 +415,6 @@ message UpdatePipelineRequestProto {
 message UpdatePipelineResponseProto { // void response
 }
 
-message GetDelegationTokenRequestProto {
-  required string renewer = 1;
-}
-
-message GetDelegationTokenResponseProto {
-  optional hadoop.common.TokenProto token = 1;
-}
-
-message RenewDelegationTokenRequestProto {
-  required hadoop.common.TokenProto token = 1;
-}
-
-message RenewDelegationTokenResponseProto {
-  required uint64 newExireTime = 1;
-}
-
-message CancelDelegationTokenRequestProto {
-  required hadoop.common.TokenProto token = 1;
-}
-
-message CancelDelegationTokenResponseProto { // void response
-}
-
 message SetBalancerBandwidthRequestProto {
   required int64 bandwidth = 1;
 }
@@ -554,12 +531,12 @@ service ClientNamenodeProtocol {
       returns(UpdateBlockForPipelineResponseProto);
   rpc updatePipeline(UpdatePipelineRequestProto)
       returns(UpdatePipelineResponseProto);
-  rpc getDelegationToken(GetDelegationTokenRequestProto)
-      returns(GetDelegationTokenResponseProto);
-  rpc renewDelegationToken(RenewDelegationTokenRequestProto)
-      returns(RenewDelegationTokenResponseProto);
-  rpc cancelDelegationToken(CancelDelegationTokenRequestProto)
-      returns(CancelDelegationTokenResponseProto);
+  rpc getDelegationToken(hadoop.common.GetDelegationTokenRequestProto)
+      returns(hadoop.common.GetDelegationTokenResponseProto);
+  rpc renewDelegationToken(hadoop.common.RenewDelegationTokenRequestProto)
+      returns(hadoop.common.RenewDelegationTokenResponseProto);
+  rpc cancelDelegationToken(hadoop.common.CancelDelegationTokenRequestProto)
+      returns(hadoop.common.CancelDelegationTokenResponseProto);
   rpc setBalancerBandwidth(SetBalancerBandwidthRequestProto)
       returns(SetBalancerBandwidthResponseProto);
   rpc getDataEncryptionKey(GetDataEncryptionKeyRequestProto)

Modified: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/NamenodeProtocol.proto
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/NamenodeProtocol.proto?rev=1437843&r1=1437842&r2=1437843&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/NamenodeProtocol.proto (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/NamenodeProtocol.proto Thu Jan 24 02:45:45 2013
@@ -56,7 +56,7 @@ message GetBlockKeysRequestProto {
  * keys - Information about block keys at the active namenode
  */
 message GetBlockKeysResponseProto {
-  required ExportedBlockKeysProto keys = 1;
+  optional ExportedBlockKeysProto keys = 1;
 }
 
 /**

Modified: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/datatransfer.proto
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/datatransfer.proto?rev=1437843&r1=1437842&r2=1437843&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/datatransfer.proto (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/datatransfer.proto Thu Jan 24 02:45:45 2013
@@ -52,6 +52,7 @@ message OpReadBlockProto {
   required ClientOperationHeaderProto header = 1;
   required uint64 offset = 2;
   required uint64 len = 3;
+  optional bool sendChecksums = 4 [default = true];
 }
 
 
@@ -182,5 +183,5 @@ message OpBlockChecksumResponseProto {
   required uint32 bytesPerCrc = 1;
   required uint64 crcPerBlock = 2;
   required bytes md5 = 3;
-  optional ChecksumTypeProto crcType = 4 [default = CHECKSUM_CRC32];
+  optional ChecksumTypeProto crcType = 4;
 }

Modified: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/META-INF/services/org.apache.hadoop.security.token.TokenIdentifier
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/META-INF/services/org.apache.hadoop.security.token.TokenIdentifier?rev=1437843&r1=1437842&r2=1437843&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/META-INF/services/org.apache.hadoop.security.token.TokenIdentifier (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/META-INF/services/org.apache.hadoop.security.token.TokenIdentifier Thu Jan 24 02:45:45 2013
@@ -1,2 +1,15 @@
+#
+#   Licensed 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.
+#
 org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier
 org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier

Modified: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/META-INF/services/org.apache.hadoop.security.token.TokenRenewer
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/META-INF/services/org.apache.hadoop.security.token.TokenRenewer?rev=1437843&r1=1437842&r2=1437843&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/META-INF/services/org.apache.hadoop.security.token.TokenRenewer (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/META-INF/services/org.apache.hadoop.security.token.TokenRenewer Thu Jan 24 02:45:45 2013
@@ -1,3 +1,16 @@
+#
+#   Licensed 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.
+#
 org.apache.hadoop.hdfs.DFSClient$Renewer
 org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier$Renewer
 org.apache.hadoop.hdfs.HftpFileSystem$TokenManager

Propchange: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/datanode/
------------------------------------------------------------------------------
  Merged /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/datanode:r1432789-1437840

Propchange: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/
------------------------------------------------------------------------------
  Merged /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs:r1432789-1437840

Propchange: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/secondary/
------------------------------------------------------------------------------
  Merged /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/secondary:r1432789-1437840

Propchange: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/hdfs/
------------------------------------------------------------------------------
  Merged /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/hdfs:r1432789-1437840

Modified: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java?rev=1437843&r1=1437842&r2=1437843&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java Thu Jan 24 02:45:45 2013
@@ -48,7 +48,6 @@ import java.io.IOException;
 import java.io.PrintWriter;
 import java.io.RandomAccessFile;
 import java.net.InetSocketAddress;
-import java.net.ServerSocket;
 import java.net.URI;
 import java.net.URISyntaxException;
 import java.nio.channels.FileChannel;
@@ -2290,19 +2289,6 @@ public class MiniDFSCluster {
     return nameNodes[nnIndex].nameNode;
   }
   
-  private int getFreeSocketPort() {
-    int port = 0;
-    try {
-      ServerSocket s = new ServerSocket(0);
-      port = s.getLocalPort();
-      s.close();
-      return port;
-    } catch (IOException e) {
-      // Could not get a free port. Return default port 0.
-    }
-    return port;
-  }
-  
   protected void setupDatanodeAddress(Configuration conf, boolean setupHostsFile,
                            boolean checkDataNodeAddrConfig) throws IOException {
     if (setupHostsFile) {
@@ -2311,7 +2297,7 @@ public class MiniDFSCluster {
         throw new IOException("Parameter dfs.hosts is not setup in conf");
       }
       // Setup datanode in the include file, if it is defined in the conf
-      String address = "127.0.0.1:" + getFreeSocketPort();
+      String address = "127.0.0.1:" + NetUtils.getFreeSocketPort();
       if (checkDataNodeAddrConfig) {
         conf.setIfUnset(DFS_DATANODE_ADDRESS_KEY, address);
       } else {

Modified: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDataTransferProtocol.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDataTransferProtocol.java?rev=1437843&r1=1437842&r2=1437843&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDataTransferProtocol.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDataTransferProtocol.java Thu Jan 24 02:45:45 2013
@@ -444,21 +444,21 @@ public class TestDataTransferProtocol {
     recvBuf.reset();
     blk.setBlockId(blkid-1);
     sender.readBlock(blk, BlockTokenSecretManager.DUMMY_TOKEN, "cl",
-        0L, fileLen);
+        0L, fileLen, true);
     sendRecvData("Wrong block ID " + newBlockId + " for read", false); 
 
     // negative block start offset -1L
     sendBuf.reset();
     blk.setBlockId(blkid);
     sender.readBlock(blk, BlockTokenSecretManager.DUMMY_TOKEN, "cl",
-        -1L, fileLen);
+        -1L, fileLen, true);
     sendRecvData("Negative start-offset for read for block " + 
                  firstBlock.getBlockId(), false);
 
     // bad block start offset
     sendBuf.reset();
     sender.readBlock(blk, BlockTokenSecretManager.DUMMY_TOKEN, "cl",
-        fileLen, fileLen);
+        fileLen, fileLen, true);
     sendRecvData("Wrong start-offset for reading block " +
                  firstBlock.getBlockId(), false);
     
@@ -475,7 +475,7 @@ public class TestDataTransferProtocol {
     
     sendBuf.reset();
     sender.readBlock(blk, BlockTokenSecretManager.DUMMY_TOKEN, "cl",
-        0L, -1L-random.nextInt(oneMil));
+        0L, -1L-random.nextInt(oneMil), true);
     sendRecvData("Negative length for reading block " +
                  firstBlock.getBlockId(), false);
     
@@ -488,14 +488,14 @@ public class TestDataTransferProtocol {
         recvOut);
     sendBuf.reset();
     sender.readBlock(blk, BlockTokenSecretManager.DUMMY_TOKEN, "cl",
-        0L, fileLen+1);
+        0L, fileLen+1, true);
     sendRecvData("Wrong length for reading block " +
                  firstBlock.getBlockId(), false);
     
     //At the end of all this, read the file to make sure that succeeds finally.
     sendBuf.reset();
     sender.readBlock(blk, BlockTokenSecretManager.DUMMY_TOKEN, "cl",
-        0L, fileLen);
+        0L, fileLen, true);
     readFile(fileSys, file, fileLen);
     } finally {
       cluster.shutdown();

Modified: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestParallelRead.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestParallelRead.java?rev=1437843&r1=1437842&r2=1437843&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestParallelRead.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestParallelRead.java Thu Jan 24 02:45:45 2013
@@ -19,6 +19,9 @@ package org.apache.hadoop.hdfs;
 
 import java.io.IOException;
 
+import org.apache.commons.logging.impl.Log4JLogger;
+import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtocol;
+import org.apache.log4j.Level;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -56,4 +59,11 @@ public class TestParallelRead extends Te
   public void testParallelReadMixed() throws IOException {
     runTestWorkload(new MixedWorkloadHelper());
   }
+  
+  @Test
+  public void testParallelNoChecksums() throws IOException {
+    verifyChecksums = false;
+    runTestWorkload(new MixedWorkloadHelper());
+  }
+
 }

Modified: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestParallelReadUtil.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestParallelReadUtil.java?rev=1437843&r1=1437842&r2=1437843&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestParallelReadUtil.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestParallelReadUtil.java Thu Jan 24 02:45:45 2013
@@ -46,6 +46,7 @@ public class TestParallelReadUtil {
   static final int FILE_SIZE_K = 256;
   static Random rand = null;
   static final int DEFAULT_REPLICATION_FACTOR = 2;
+  protected boolean verifyChecksums = true;
 
   static {
     // The client-trace log ends up causing a lot of blocking threads
@@ -317,7 +318,8 @@ public class TestParallelReadUtil {
 
       testInfo.filepath = new Path("/TestParallelRead.dat." + i);
       testInfo.authenticData = util.writeFile(testInfo.filepath, FILE_SIZE_K);
-      testInfo.dis = dfsClient.open(testInfo.filepath.toString());
+      testInfo.dis = dfsClient.open(testInfo.filepath.toString(),
+          dfsClient.dfsClientConf.ioBufferSize, verifyChecksums);
 
       for (int j = 0; j < nWorkerEach; ++j) {
         workers[nWorkers++] = new ReadWorker(testInfo, nWorkers, helper);

Modified: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPread.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPread.java?rev=1437843&r1=1437842&r2=1437843&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPread.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPread.java Thu Jan 24 02:45:45 2013
@@ -24,11 +24,14 @@ import java.io.DataOutputStream;
 import java.io.IOException;
 import java.util.Random;
 
+import org.apache.commons.logging.impl.Log4JLogger;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtocol;
 import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset;
+import org.apache.log4j.Level;
 import org.junit.Test;
 
 /**
@@ -194,11 +197,19 @@ public class TestPread {
    */
   @Test
   public void testPreadDFS() throws IOException {
-    dfsPreadTest(false); //normal pread
-    dfsPreadTest(true); //trigger read code path without transferTo.
+    dfsPreadTest(false, true); //normal pread
+    dfsPreadTest(true, true); //trigger read code path without transferTo.
   }
   
-  private void dfsPreadTest(boolean disableTransferTo) throws IOException {
+  @Test
+  public void testPreadDFSNoChecksum() throws IOException {
+    ((Log4JLogger)DataTransferProtocol.LOG).getLogger().setLevel(Level.ALL);
+    dfsPreadTest(false, false);
+    dfsPreadTest(true, false);
+  }
+  
+  private void dfsPreadTest(boolean disableTransferTo, boolean verifyChecksum)
+      throws IOException {
     Configuration conf = new HdfsConfiguration();
     conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, 4096);
     conf.setLong(DFSConfigKeys.DFS_CLIENT_READ_PREFETCH_SIZE_KEY, 4096);
@@ -210,6 +221,7 @@ public class TestPread {
     }
     MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(3).build();
     FileSystem fileSys = cluster.getFileSystem();
+    fileSys.setVerifyChecksum(verifyChecksum);
     try {
       Path file1 = new Path("preadtest.dat");
       writeFile(fileSys, file1);

Modified: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tools/FakeRenewer.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tools/FakeRenewer.java?rev=1437843&r1=1437842&r2=1437843&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tools/FakeRenewer.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tools/FakeRenewer.java Thu Jan 24 02:45:45 2013
@@ -1,3 +1,20 @@
+/**
+* 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.tools;
 
 import java.io.IOException;
@@ -37,4 +54,4 @@ public class FakeRenewer extends TokenRe
     lastRenewed = null;
     lastCanceled = null;
   }
-}
\ No newline at end of file
+}

Modified: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/META-INF/services/org.apache.hadoop.security.token.TokenRenewer
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/META-INF/services/org.apache.hadoop.security.token.TokenRenewer?rev=1437843&r1=1437842&r2=1437843&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/META-INF/services/org.apache.hadoop.security.token.TokenRenewer (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/META-INF/services/org.apache.hadoop.security.token.TokenRenewer Thu Jan 24 02:45:45 2013
@@ -1 +1,14 @@
+#
+#   Licensed 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.
+#
 org.apache.hadoop.tools.FakeRenewer

Modified: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/pom.xml
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/pom.xml?rev=1437843&r1=1437842&r2=1437843&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/pom.xml (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/pom.xml Thu Jan 24 02:45:45 2013
@@ -48,9 +48,6 @@ http://maven.apache.org/xsd/maven-4.0.0.
         <groupId>org.apache.rat</groupId>
         <artifactId>apache-rat-plugin</artifactId>
         <configuration>
-          <includes>
-            <include>pom.xml</include>
-          </includes>
         </configuration>
       </plugin>
     </plugins>