-
Notifications
You must be signed in to change notification settings - Fork 746
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Remove empty avro files during compaction #2158
Changes from 1 commit
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -19,6 +19,7 @@ | |
|
||
import java.io.IOException; | ||
import java.net.URI; | ||
import java.util.ArrayList; | ||
import java.util.Collection; | ||
import java.util.List; | ||
import java.util.Map; | ||
|
@@ -28,6 +29,7 @@ | |
import java.util.concurrent.ExecutorService; | ||
import java.util.concurrent.Future; | ||
import java.util.concurrent.TimeUnit; | ||
import java.util.stream.Collectors; | ||
|
||
import org.apache.commons.io.FilenameUtils; | ||
import org.apache.commons.math3.primes.Primes; | ||
|
@@ -40,6 +42,7 @@ | |
import org.apache.hadoop.fs.PathFilter; | ||
import org.apache.hadoop.fs.permission.FsPermission; | ||
import org.apache.hadoop.mapreduce.Job; | ||
import org.apache.hadoop.mapreduce.TaskCompletionEvent; | ||
import org.apache.hadoop.mapreduce.lib.input.FileInputFormat; | ||
import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat; | ||
import org.joda.time.DateTime; | ||
|
@@ -321,9 +324,24 @@ public void run() { | |
this.configureJob(job); | ||
this.submitAndWait(job); | ||
if (shouldPublishData(compactionTimestamp)) { | ||
// remove all invalid empty files due to speculative task execution | ||
List<TaskCompletionEvent> failedEvents = CompactionAvroJobConfigurator.getUnsuccessfulTaskCompletionEvent(job); | ||
List<Path> allFilePaths = DatasetHelper.getApplicableFilePaths(this.tmpFs, this.dataset.outputTmpPath(), Lists.newArrayList("avro")); | ||
List<Path> goodPaths = new ArrayList<>(); | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This code shows up in CompactionCompleteFileOperationAction.java too. Can you make a common method for this? |
||
for (Path filePath: allFilePaths) { | ||
if (isFailedPath(filePath, failedEvents)) { | ||
this.tmpFs.delete(filePath, false); | ||
LOG.error("{} is a bad path so it was deleted", filePath); | ||
} else { | ||
LOG.info("{} is a good path so it was kept", filePath); | ||
goodPaths.add(filePath); | ||
} | ||
} | ||
|
||
|
||
if (!this.recompactAllData && this.recompactFromDestPaths) { | ||
// append new files without deleting output directory | ||
addFilesInTmpPathToOutputPath(); | ||
addGoodFilesToOutputPath(goodPaths); | ||
// clean up late data from outputLateDirectory, which has been set to inputPath | ||
deleteFilesByPaths(this.dataset.inputPaths()); | ||
} else { | ||
|
@@ -352,6 +370,11 @@ public void run() { | |
} | ||
} | ||
|
||
private boolean isFailedPath(Path path, List<TaskCompletionEvent> failedEvents) { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This code is also in CompactionCompleteFileOperationAction.java. |
||
return failedEvents.stream() | ||
.filter(event -> path.toString().contains(event.getTaskAttemptId().toString())) | ||
.collect(Collectors.toList()).size() > 0; | ||
} | ||
|
||
/** | ||
* For regular compactions, compaction timestamp is the time the compaction job starts. | ||
|
@@ -603,9 +626,8 @@ private void moveTmpPathToOutputPath() throws IOException { | |
HadoopUtils.movePath (MRCompactorJobRunner.this.tmpFs, this.dataset.outputTmpPath(), FileSystem.get(this.dataset.outputPath().getParent().toUri(), this.fs.getConf()), this.dataset.outputPath(), false, this.fs.getConf()) ; | ||
} | ||
|
||
private void addFilesInTmpPathToOutputPath () throws IOException { | ||
List<Path> paths = this.getApplicableFilePaths(this.dataset.outputTmpPath(), this.tmpFs); | ||
for (Path path: paths) { | ||
private void addGoodFilesToOutputPath (List<Path> goodPaths) throws IOException { | ||
for (Path path: goodPaths) { | ||
String fileName = path.getName(); | ||
LOG.info(String.format("Adding %s to %s", path.toString(), this.dataset.outputPath())); | ||
Path outPath = MRCompactorJobRunner.this.lateOutputRecordCountProvider.constructLateFilePath(fileName, | ||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Only care about one match, so limit(1) can be added. Also, should put the path separator before and after the attempt id to avoid an incorrect match if the attempt id is a prefix of another attempt id. Like if attempt abc_10 passes, we don't want failed attempt abc_1 to match it.