MAPREDUCE-4898. FileOutputFormat.checkOutputSpecs and FileOutputFormat.setOutputPath incompatible with MR1. (rkanter via tucu)

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1469523 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Alejandro Abdelnur 2013-04-18 18:51:45 +00:00
parent df55edd5de
commit d12c463121
3 changed files with 90 additions and 4 deletions

View File

@ -310,6 +310,8 @@ Release 2.0.5-beta - UNRELEASED
MAPREDUCE-5128. mapred-default.xml is missing a bunch of history server
configs. (sandyr via tucu)
MAPREDUCE-4898. FileOutputFormat.checkOutputSpecs and FileOutputFormat.setOutputPath incompatible with MR1. (rkanter via tucu)
Release 2.0.4-alpha - UNRELEASED
INCOMPATIBLE CHANGES

View File

@ -27,7 +27,7 @@
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.compress.CompressionCodec;
import org.apache.hadoop.fs.FileAlreadyExistsException;
import org.apache.hadoop.mapred.FileAlreadyExistsException;
import org.apache.hadoop.mapred.InvalidJobConfException;
import org.apache.hadoop.mapreduce.Job;
import org.apache.hadoop.mapreduce.JobContext;
@ -150,9 +150,14 @@ public void checkOutputSpecs(JobContext job
* @param outputDir the {@link Path} of the output directory for
* the map-reduce job.
*/
public static void setOutputPath(Job job, Path outputDir) throws IOException {
public static void setOutputPath(Job job, Path outputDir) {
try {
outputDir = outputDir.getFileSystem(job.getConfiguration()).makeQualified(
outputDir);
} catch (IOException e) {
// Throw the IOException as a RuntimeException to be compatible with MR1
throw new RuntimeException(e);
}
job.getConfiguration().set(FileOutputFormat.OUTDIR, outputDir.toString());
}

View File

@ -0,0 +1,79 @@
/**
* 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.mapreduce.lib.output;
import java.io.IOException;
import junit.framework.TestCase;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.mapred.FileAlreadyExistsException;
import org.apache.hadoop.mapreduce.Job;
import org.apache.hadoop.mapreduce.RecordWriter;
import org.apache.hadoop.mapreduce.TaskAttemptContext;
public class TestFileOutputFormat extends TestCase {
public void testSetOutputPathException() throws Exception {
Job job = Job.getInstance();
try {
// Give it an invalid filesystem so it'll throw an exception
FileOutputFormat.setOutputPath(job, new Path("foo:///bar"));
fail("Should have thrown a RuntimeException with an IOException inside");
}
catch (RuntimeException re) {
assertTrue(re.getCause() instanceof IOException);
}
}
public void testCheckOutputSpecsException() throws Exception {
Job job = Job.getInstance();
Path outDir = new Path(System.getProperty("test.build.data", "/tmp"),
"output");
FileSystem fs = outDir.getFileSystem(new Configuration());
// Create the output dir so it already exists and set it for the job
fs.mkdirs(outDir);
FileOutputFormat.setOutputPath(job, outDir);
// We don't need a "full" implementation of FileOutputFormat for this test
FileOutputFormat fof = new FileOutputFormat() {
@Override
public RecordWriter getRecordWriter(TaskAttemptContext job)
throws IOException, InterruptedException {
return null;
}
};
try {
try {
// This should throw a FileAlreadyExistsException because the outputDir
// already exists
fof.checkOutputSpecs(job);
fail("Should have thrown a FileAlreadyExistsException");
}
catch (FileAlreadyExistsException re) {
// correct behavior
}
}
finally {
// Cleanup
if (fs.exists(outDir)) {
fs.delete(outDir, true);
}
}
}
}