瀏覽代碼

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

git-svn-id: https://svn.apache.org/repos/asf/hadoop/core/branches/branch-0.19@743896 13f79535-47bb-0310-9956-ffa450edef68
Hairong Kuang 16 年之前
父節點
當前提交
991cc1138a

+ 3 - 0
CHANGES.txt

@@ -1082,6 +1082,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/hdfs/server/datanode/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.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);
+  }
+}

+ 6 - 4
src/hdfs/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java

@@ -107,9 +107,11 @@ class BlockReceiver implements java.io.Closeable, FSConstants {
           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 @@ class BlockReceiver implements java.io.Closeable, FSConstants {
       if (responder != null) {
         responder.interrupt();
       }
-      removeBlock();
+      cleanupBlock();
       throw ioe;
     } finally {
       if (responder != null) {
@@ -571,10 +573,10 @@ class BlockReceiver implements java.io.Closeable, FSConstants {
     }
   }
 
-  /** 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);
     }

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

@@ -975,7 +975,7 @@ public 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 successfully finalized because all packets
       // were successfully processed at the Datanode but the ack for
@@ -1001,7 +1001,7 @@ public 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/hdfs/server/datanode/SimulatedFSDataset.java

@@ -373,7 +373,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);