You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by ha...@apache.org on 2009/02/12 22:35:31 UTC

svn commit: r743892 - in /hadoop/core/trunk: ./ src/hdfs/org/apache/hadoop/hdfs/server/datanode/ src/test/org/apache/hadoop/hdfs/server/datanode/

Author: hairong
Date: Thu Feb 12 21:35:30 2009
New Revision: 743892

URL: http://svn.apache.org/viewvc?rev=743892&view=rev
Log:
HADOOP-5192. When replication fails, a block receiver should not remove a block that's created or being written by other threads. Contributed by Hairong Kuang.

Added:
    hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/server/datanode/BlockAlreadyExistsException.java
Modified:
    hadoop/core/trunk/CHANGES.txt
    hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java
    hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/server/datanode/FSDataset.java
    hadoop/core/trunk/src/test/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java

Modified: hadoop/core/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/CHANGES.txt?rev=743892&r1=743891&r2=743892&view=diff
==============================================================================
--- hadoop/core/trunk/CHANGES.txt (original)
+++ hadoop/core/trunk/CHANGES.txt Thu Feb 12 21:35:30 2009
@@ -1826,6 +1826,9 @@
     HADOOP-5114. Remove timeout for accept() in DataNode. This makes accept() 
     fail in JDK on Windows and causes many tests to fail. (Raghu Angadi)
 
+    HADOOP-5192. Block receiver should not remove a block that's created or
+    being written by other threads. (hairong)
+ 
 Release 0.18.3 - 2009-01-27
 
   IMPROVEMENTS

Added: hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/server/datanode/BlockAlreadyExistsException.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/server/datanode/BlockAlreadyExistsException.java?rev=743892&view=auto
==============================================================================
--- hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/server/datanode/BlockAlreadyExistsException.java (added)
+++ hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/server/datanode/BlockAlreadyExistsException.java Thu Feb 12 21:35:30 2009
@@ -0,0 +1,38 @@
+/**
+ * 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.datanode;
+
+import java.io.IOException;
+
+/**
+ * Exception indicating that the target block already exists 
+ * and is not set to be recovered/overwritten.  
+ */
+class BlockAlreadyExistsException extends IOException {
+  private static final long serialVersionUID = 1L;
+
+  public BlockAlreadyExistsException() {
+    super();
+  }
+
+  public BlockAlreadyExistsException(String msg) {
+    super(msg);
+  }
+}

Modified: hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java?rev=743892&r1=743891&r2=743892&view=diff
==============================================================================
--- hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java (original)
+++ hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java Thu Feb 12 21:35:30 2009
@@ -107,9 +107,11 @@
           datanode.blockScanner.deleteBlock(block);
         }
       }
+    } catch (BlockAlreadyExistsException bae) {
+      throw bae;
     } catch(IOException ioe) {
       IOUtils.closeStream(this);
-      removeBlock();
+      cleanupBlock();
       
       // check if there is a disk error
       IOException cause = FSDataset.getCauseIfDiskError(ioe);
@@ -557,7 +559,7 @@
       if (responder != null) {
         responder.interrupt();
       }
-      removeBlock();
+      cleanupBlock();
       throw ioe;
     } finally {
       if (responder != null) {
@@ -571,10 +573,10 @@
     }
   }
 
-  /** Remove a partial block 
+  /** Cleanup a partial block 
    * if this write is for a replication request (and not from a client)
    */
-  private void removeBlock() throws IOException {
+  private void cleanupBlock() throws IOException {
     if (clientName.length() == 0) { // not client write
       datanode.data.unfinalizeBlock(block);
     }

Modified: hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/server/datanode/FSDataset.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/server/datanode/FSDataset.java?rev=743892&r1=743891&r2=743892&view=diff
==============================================================================
--- hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/server/datanode/FSDataset.java (original)
+++ hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/server/datanode/FSDataset.java Thu Feb 12 21:35:30 2009
@@ -975,7 +975,7 @@
     //
     if (isValidBlock(b)) {
       if (!isRecovery) {
-        throw new IOException("Block " + b + " is valid, and cannot be written to.");
+        throw new BlockAlreadyExistsException("Block " + b + " is valid, and cannot be written to.");
       }
       // If the block was successfully finalized because all packets
       // were successfully processed at the Datanode but the ack for
@@ -1001,7 +1001,7 @@
         threads = activeFile.threads;
         
         if (!isRecovery) {
-          throw new IOException("Block " + b +
+          throw new BlockAlreadyExistsException("Block " + b +
                                   " has already been started (though not completed), and thus cannot be created.");
         } else {
           for (Thread thread:threads) {

Modified: hadoop/core/trunk/src/test/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/test/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java?rev=743892&r1=743891&r2=743892&view=diff
==============================================================================
--- hadoop/core/trunk/src/test/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java (original)
+++ hadoop/core/trunk/src/test/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java Thu Feb 12 21:35:30 2009
@@ -373,7 +373,7 @@
                                             boolean isRecovery)
                                             throws IOException {
     if (isValidBlock(b)) {
-          throw new IOException("Block " + b + 
+          throw new BlockAlreadyExistsException("Block " + b + 
               " is valid, and cannot be written to.");
       }
       BInfo binfo = new BInfo(b, true);