Browse Source

Merge -r 743891:743892 from trunk to move the change of HADOOP-5192 to branch 0.18.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/core/branches/branch-0.18@743935 13f79535-47bb-0310-9956-ffa450edef68
Hairong Kuang 16 years ago
parent
commit
601ff57ef8

+ 3 - 0
CHANGES.txt

@@ -7,6 +7,9 @@ Release 0.18.4 - Unreleased
     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

+ 38 - 0
src/hdfs/org/apache/hadoop/dfs/BlockAlreadyExistsException.java

@@ -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.dfs;
+
+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);
+  }
+}

+ 6 - 4
src/hdfs/org/apache/hadoop/dfs/DataNode.java

@@ -2389,9 +2389,11 @@ public class DataNode extends Configured
                                                     streams.checksumOut, 
                                                     SMALL_BUFFER_SIZE));
         }
+      } 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);
@@ -2805,7 +2807,7 @@ public class DataNode extends Configured
         if (responder != null) {
           responder.interrupt();
         }
-        removeBlock();
+        cleanupBlock();
         throw ioe;
       } finally {
         if (responder != null) {
@@ -2819,10 +2821,10 @@ public class DataNode extends Configured
       }
     }
 
-    /** 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
         data.unfinalizeBlock(block);
       }

+ 2 - 2
src/hdfs/org/apache/hadoop/dfs/FSDataset.java

@@ -939,7 +939,7 @@ class FSDataset implements FSConstants, FSDatasetInterface {
     //
     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 succesfully finalized because all packets
       // were successfully processed at the Datanode but the ack for
@@ -966,7 +966,7 @@ class FSDataset implements FSConstants, FSDatasetInterface {
         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) {

+ 1 - 1
src/test/org/apache/hadoop/dfs/SimulatedFSDataset.java

@@ -372,7 +372,7 @@ public class SimulatedFSDataset  implements FSConstants, FSDatasetInterface, Con
                                             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);