ソースを参照

commit 48476ea59216e92d7b398c6af701f473b94ccb50
Author: Yahoo\! <ltucker@yahoo-inc.com>
Date: Mon Aug 17 14:10:57 2009 -0700

Revert "Applying patch 2950012.mr838.patch"

This reverts commit 1e58f90ccda7bb7c9c0aff9e3fd58497c94600ff.

Conflicts:

YAHOO-CHANGES.txt
build.xml


git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-0.20-security-patches@1076973 13f79535-47bb-0310-9956-ffa450edef68

Owen O'Malley 14 年 前
コミット
c6d6aada28

+ 12 - 15
src/mapred/org/apache/hadoop/mapred/Task.java

@@ -781,30 +781,27 @@ abstract public class Task implements Writable, Configurable {
           }
           reporter.setProgressFlag();
         }
-        break;
+        // task can Commit now  
+        try {
+          LOG.info("Task " + taskId + " is allowed to commit now");
+          committer.commitTask(taskContext);
+          return;
+        } catch (IOException iee) {
+          LOG.warn("Failure committing: " + 
+                    StringUtils.stringifyException(iee));
+          discardOutput(taskContext);
+          throw iee;
+        }
       } catch (IOException ie) {
         LOG.warn("Failure asking whether task can commit: " + 
             StringUtils.stringifyException(ie));
         if (--retries == 0) {
-          //if it couldn't query successfully then delete the output
+          //if it couldn't commit a successfully then delete the output
           discardOutput(taskContext);
           System.exit(68);
         }
       }
     }
-    
-    // task can Commit now  
-    try {
-      LOG.info("Task " + taskId + " is allowed to commit now");
-      committer.commitTask(taskContext);
-      return;
-    } catch (IOException iee) {
-      LOG.warn("Failure committing: " + 
-        StringUtils.stringifyException(iee));
-      //if it couldn't commit a successfully then delete the output
-      discardOutput(taskContext);
-      throw iee;
-    }
   }
 
   private 

+ 0 - 62
src/test/org/apache/hadoop/mapred/TestTaskCommit.java

@@ -1,62 +0,0 @@
-/**
- * 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.mapred;
-
-import java.io.IOException;
-
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-
-public class TestTaskCommit extends HadoopTestCase {
-
-  static class CommitterWithCommitFail extends FileOutputCommitter {
-    public void commitTask(TaskAttemptContext context) throws IOException {
-      Path taskOutputPath = getTempTaskOutputPath(context);
-      TaskAttemptID attemptId = context.getTaskAttemptID();
-      JobConf job = context.getJobConf();
-      if (taskOutputPath != null) {
-        FileSystem fs = taskOutputPath.getFileSystem(job);
-        if (fs.exists(taskOutputPath)) {
-          throw new IOException();
-        }
-      }
-    }
-  }
-
-  public TestTaskCommit() throws IOException {
-    super(LOCAL_MR, LOCAL_FS, 1, 1);
-  }
-  
-  public void testCommitFail() throws IOException {
-    Path rootDir = 
-      new Path(System.getProperty("test.build.data",  "/tmp"), "test");
-    final Path inDir = new Path(rootDir, "input");
-    final Path outDir = new Path(rootDir, "output");
-    JobConf jobConf = createJobConf();
-    jobConf.setMaxMapAttempts(1);
-    jobConf.setOutputCommitter(CommitterWithCommitFail.class);
-    RunningJob rJob = UtilsForTests.runJob(jobConf, inDir, outDir, 1, 0);
-    rJob.waitForCompletion();
-    assertEquals(JobStatus.FAILED, rJob.getJobState());
-  }
-
-  public static void main(String[] argv) throws Exception {
-    TestTaskCommit td = new TestTaskCommit();
-    td.testCommitFail();
-  }
-}