HADOOP-15469. S3A directory committer commit job fails if _temporary directory created under dest.

Contributed by Steve Loughran.
This commit is contained in:
Steve Loughran 2018-05-17 19:08:05 +01:00
parent f729faaabb
commit 170f1040d4
2 changed files with 5 additions and 10 deletions

View File

@ -93,11 +93,8 @@ public class DirectoryStagingCommitter extends StagingCommitter {
Configuration fsConf = fs.getConf(); Configuration fsConf = fs.getConf();
switch (getConflictResolutionMode(context, fsConf)) { switch (getConflictResolutionMode(context, fsConf)) {
case FAIL: case FAIL:
// this was checked in setupJob, but this avoids some cases where // this was checked in setupJob; temporary files may have been
// output was created while the job was processing // created, so do not check again.
if (fs.exists(outputPath)) {
throw new PathExistsException(outputPath.toString(), E_DEST_EXISTS);
}
break; break;
case APPEND: case APPEND:
// do nothing // do nothing

View File

@ -67,15 +67,14 @@ public class TestStagingDirectoryOutputCommitter
pathExists(mockS3, OUTPUT_PATH); pathExists(mockS3, OUTPUT_PATH);
final DirectoryStagingCommitter committer = newJobCommitter(); final DirectoryStagingCommitter committer = newJobCommitter();
// this should fail
intercept(PathExistsException.class, intercept(PathExistsException.class,
InternalCommitterConstants.E_DEST_EXISTS, InternalCommitterConstants.E_DEST_EXISTS,
"Should throw an exception because the path exists", "Should throw an exception because the path exists",
() -> committer.setupJob(getJob())); () -> committer.setupJob(getJob()));
intercept(PathExistsException.class, // but there are no checks in job commit (HADOOP-15469)
InternalCommitterConstants.E_DEST_EXISTS, committer.commitJob(getJob());
"Should throw an exception because the path exists",
() -> committer.commitJob(getJob()));
reset(mockS3); reset(mockS3);
pathDoesNotExist(mockS3, OUTPUT_PATH); pathDoesNotExist(mockS3, OUTPUT_PATH);
@ -87,7 +86,6 @@ public class TestStagingDirectoryOutputCommitter
reset(mockS3); reset(mockS3);
pathDoesNotExist(mockS3, OUTPUT_PATH); pathDoesNotExist(mockS3, OUTPUT_PATH);
committer.commitJob(getJob()); committer.commitJob(getJob());
verifyExistenceChecked(mockS3, OUTPUT_PATH);
verifyCompletion(mockS3); verifyCompletion(mockS3);
} }