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/13 00:58:24 UTC

svn commit: r743935 - in /hadoop/core/branches/branch-0.18: ./ src/hdfs/org/apache/hadoop/dfs/ src/test/org/apache/hadoop/dfs/

Author: hairong
Date: Thu Feb 12 23:58:23 2009
New Revision: 743935

URL: http://svn.apache.org/viewvc?rev=743935&view=rev
Log:
Merge -r 743891:743892 from trunk to move the change of HADOOP-5192 to branch 0.18.

Added:
    hadoop/core/branches/branch-0.18/src/hdfs/org/apache/hadoop/dfs/BlockAlreadyExistsException.java
Modified:
    hadoop/core/branches/branch-0.18/   (props changed)
    hadoop/core/branches/branch-0.18/CHANGES.txt   (contents, props changed)
    hadoop/core/branches/branch-0.18/src/hdfs/org/apache/hadoop/dfs/DataNode.java
    hadoop/core/branches/branch-0.18/src/hdfs/org/apache/hadoop/dfs/FSDataset.java
    hadoop/core/branches/branch-0.18/src/test/org/apache/hadoop/dfs/SimulatedFSDataset.java

Propchange: hadoop/core/branches/branch-0.18/
------------------------------------------------------------------------------
--- svn:mergeinfo (original)
+++ svn:mergeinfo Thu Feb 12 23:58:23 2009
@@ -1,2 +1,2 @@
 /hadoop/core/branches/branch-0.19:704733
-/hadoop/core/trunk:699517,700163,704701,704732,705420,705430,707258,709040,720602,723460,723831,723918
+/hadoop/core/trunk:699517,700163,704701,704732,705420,705430,707258,709040,720602,723460,723831,723918,743892

Modified: hadoop/core/branches/branch-0.18/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/core/branches/branch-0.18/CHANGES.txt?rev=743935&r1=743934&r2=743935&view=diff
==============================================================================
--- hadoop/core/branches/branch-0.18/CHANGES.txt (original)
+++ hadoop/core/branches/branch-0.18/CHANGES.txt Thu Feb 12 23:58:23 2009
@@ -7,6 +7,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

Propchange: hadoop/core/branches/branch-0.18/CHANGES.txt
------------------------------------------------------------------------------
--- svn:mergeinfo (original)
+++ svn:mergeinfo Thu Feb 12 23:58:23 2009
@@ -1 +1 @@
-/hadoop/core/trunk/CHANGES.txt:699517,700163,700923,704701,705420,705430,707258,709040,720602,723460,723831,723918,732572,741703
+/hadoop/core/trunk/CHANGES.txt:699517,700163,700923,704701,705420,705430,707258,709040,720602,723460,723831,723918,732572,741703,743892

Added: hadoop/core/branches/branch-0.18/src/hdfs/org/apache/hadoop/dfs/BlockAlreadyExistsException.java
URL: http://svn.apache.org/viewvc/hadoop/core/branches/branch-0.18/src/hdfs/org/apache/hadoop/dfs/BlockAlreadyExistsException.java?rev=743935&view=auto
==============================================================================
--- hadoop/core/branches/branch-0.18/src/hdfs/org/apache/hadoop/dfs/BlockAlreadyExistsException.java (added)
+++ hadoop/core/branches/branch-0.18/src/hdfs/org/apache/hadoop/dfs/BlockAlreadyExistsException.java Thu Feb 12 23:58:23 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.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);
+  }
+}

Modified: hadoop/core/branches/branch-0.18/src/hdfs/org/apache/hadoop/dfs/DataNode.java
URL: http://svn.apache.org/viewvc/hadoop/core/branches/branch-0.18/src/hdfs/org/apache/hadoop/dfs/DataNode.java?rev=743935&r1=743934&r2=743935&view=diff
==============================================================================
--- hadoop/core/branches/branch-0.18/src/hdfs/org/apache/hadoop/dfs/DataNode.java (original)
+++ hadoop/core/branches/branch-0.18/src/hdfs/org/apache/hadoop/dfs/DataNode.java Thu Feb 12 23:58:23 2009
@@ -2389,9 +2389,11 @@
                                                     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 @@
         if (responder != null) {
           responder.interrupt();
         }
-        removeBlock();
+        cleanupBlock();
         throw ioe;
       } finally {
         if (responder != null) {
@@ -2819,10 +2821,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
         data.unfinalizeBlock(block);
       }

Modified: hadoop/core/branches/branch-0.18/src/hdfs/org/apache/hadoop/dfs/FSDataset.java
URL: http://svn.apache.org/viewvc/hadoop/core/branches/branch-0.18/src/hdfs/org/apache/hadoop/dfs/FSDataset.java?rev=743935&r1=743934&r2=743935&view=diff
==============================================================================
--- hadoop/core/branches/branch-0.18/src/hdfs/org/apache/hadoop/dfs/FSDataset.java (original)
+++ hadoop/core/branches/branch-0.18/src/hdfs/org/apache/hadoop/dfs/FSDataset.java Thu Feb 12 23:58:23 2009
@@ -939,7 +939,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 succesfully finalized because all packets
       // were successfully processed at the Datanode but the ack for
@@ -966,7 +966,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/branches/branch-0.18/src/test/org/apache/hadoop/dfs/SimulatedFSDataset.java
URL: http://svn.apache.org/viewvc/hadoop/core/branches/branch-0.18/src/test/org/apache/hadoop/dfs/SimulatedFSDataset.java?rev=743935&r1=743934&r2=743935&view=diff
==============================================================================
--- hadoop/core/branches/branch-0.18/src/test/org/apache/hadoop/dfs/SimulatedFSDataset.java (original)
+++ hadoop/core/branches/branch-0.18/src/test/org/apache/hadoop/dfs/SimulatedFSDataset.java Thu Feb 12 23:58:23 2009
@@ -372,7 +372,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);