-
Notifications
You must be signed in to change notification settings - Fork 9.2k
MAPREDUCE-7474. Improve Manifest committer resilience #6716
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
Changes from 6 commits
ed7a884
ba16989
0fbbabc
952d534
330bd61
16e1be4
9193085
3e5e1e6
abed2fe
cd40e7f
2b38434
68dff78
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 |
|---|---|---|
|
|
@@ -132,7 +132,9 @@ public final class ManifestCommitterConstants { | |
| * Should dir cleanup do parallel deletion of task attempt dirs | ||
| * before trying to delete the toplevel dirs. | ||
| * For GCS this may deliver speedup, while on ABFS it may avoid | ||
| * timeouts in certain deployments. | ||
| * timeouts in certain deployments, something | ||
| * {@link #OPT_CLEANUP_PARALLEL_DELETE_BASE_FIRST} | ||
| * can alleviate. | ||
| * Value: {@value}. | ||
| */ | ||
| public static final String OPT_CLEANUP_PARALLEL_DELETE = | ||
|
|
@@ -143,6 +145,20 @@ public final class ManifestCommitterConstants { | |
| */ | ||
| public static final boolean OPT_CLEANUP_PARALLEL_DELETE_DIRS_DEFAULT = true; | ||
|
|
||
| /** | ||
| * Should parallel cleanup try to delete teh base first? | ||
| * Best for azure as it skips the task attempt deletions unless | ||
| * the toplevel delete fails. | ||
| * Value: {@value}. | ||
| */ | ||
| public static final String OPT_CLEANUP_PARALLEL_DELETE_BASE_FIRST = | ||
| OPT_PREFIX + "cleanup.parallel.delete.base.first"; | ||
|
|
||
| /** | ||
| * Default value of option {@link #OPT_CLEANUP_PARALLEL_DELETE_BASE_FIRST}: {@value}. | ||
| */ | ||
| public static final boolean OPT_CLEANUP_PARALLEL_DELETE_BASE_FIRST_DEFAULT = true; | ||
|
||
|
|
||
| /** | ||
| * Threads to use for IO. | ||
| */ | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -56,11 +56,14 @@ | |
| import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterConstants.MANIFEST_SUFFIX; | ||
| import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.OP_COMMIT_FILE_RENAME; | ||
| import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.OP_COMMIT_FILE_RENAME_RECOVERED; | ||
| import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.OP_DELETE_DIR; | ||
| import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.OP_LOAD_MANIFEST; | ||
| import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.OP_MSYNC; | ||
| import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.OP_RENAME_FILE; | ||
| import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.OP_SAVE_TASK_MANIFEST; | ||
| import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.AuditingIntegration.enterStageWorker; | ||
| import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.InternalConstants.SAVE_RETRY_COUNT; | ||
| import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.InternalConstants.SAVE_SLEEP_INTERVAL; | ||
|
|
||
| /** | ||
| * A Stage in Task/Job Commit. | ||
|
|
@@ -445,9 +448,29 @@ protected Boolean delete( | |
| final boolean recursive, | ||
| final String statistic) | ||
| throws IOException { | ||
| return trackDuration(getIOStatistics(), statistic, () -> { | ||
| return operations.delete(path, recursive); | ||
| }); | ||
| if (recursive) { | ||
|
Contributor
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. unable to understand this. How is a recursive flag determining that it is a dir or a file?
Contributor
Author
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. ok, deleteDir will also delete a file. let me highlight that. I'd done this delete dir/file split to support different capacity requests, without that it is a bit over-complex. it does let us collect different statistics though, which may be useful |
||
| return deleteDir(path, statistic); | ||
| } else { | ||
| return deleteFile(path, statistic); | ||
| } | ||
| } | ||
|
|
||
| /** | ||
| * Delete a file at a path. | ||
| * <p> | ||
| * If it returns without an error: there is nothing at | ||
| * the end of the path. | ||
| * @param path path | ||
| * @param statistic statistic to update | ||
| * @return outcome. | ||
| * @throws IOException IO Failure. | ||
| */ | ||
| protected boolean deleteFile( | ||
| final Path path, | ||
| final String statistic) | ||
| throws IOException { | ||
| return trackDuration(getIOStatistics(), statistic, () -> | ||
| operations.deleteFile(path)); | ||
| } | ||
|
|
||
| /** | ||
|
|
@@ -582,19 +605,46 @@ protected final Path directoryMustExist( | |
| * Save a task manifest or summary. This will be done by | ||
| * writing to a temp path and then renaming. | ||
| * If the destination path exists: Delete it. | ||
| * This will retry so that a rename failure from abfs load or IO errors | ||
| * will not fail the task. | ||
| * @param manifestData the manifest/success file | ||
| * @param tempPath temp path for the initial save | ||
| * @param finalPath final path for rename. | ||
| * @throws IOException failure to load/parse | ||
| * @throws IOException failure to rename after retries. | ||
| */ | ||
| @SuppressWarnings("unchecked") | ||
| protected final <T extends AbstractManifestData> void save(T manifestData, | ||
| final Path tempPath, | ||
| final Path finalPath) throws IOException { | ||
| LOG.trace("{}: save('{}, {}, {}')", getName(), manifestData, tempPath, finalPath); | ||
| trackDurationOfInvocation(getIOStatistics(), OP_SAVE_TASK_MANIFEST, () -> | ||
| operations.save(manifestData, tempPath, true)); | ||
| renameFile(tempPath, finalPath); | ||
| boolean success = false; | ||
| int failures = 0; | ||
| while (!success) { | ||
| try { | ||
| LOG.trace("{}: attempt {} save('{}, {}, {}')", | ||
| getName(), failures, manifestData, tempPath, finalPath); | ||
|
|
||
| trackDurationOfInvocation(getIOStatistics(), OP_SAVE_TASK_MANIFEST, () -> | ||
| operations.save(manifestData, tempPath, true)); | ||
|
||
| renameFile(tempPath, finalPath); | ||
mukund-thakur marked this conversation as resolved.
Outdated
Show resolved
Hide resolved
|
||
| // success flag is only set after the rename. | ||
| success = true; | ||
| } catch (IOException e) { | ||
| LOG.warn("Failed to save and commit file {} renamed to {}", | ||
| tempPath, finalPath, e); | ||
| failures++; | ||
| if (failures >= SAVE_RETRY_COUNT) { | ||
| // too many failures: escalate. | ||
| throw e; | ||
| } | ||
| // else, sleep | ||
| try { | ||
| Thread.sleep(SAVE_SLEEP_INTERVAL); | ||
| } catch (InterruptedException ie) { | ||
| Thread.currentThread().interrupt(); | ||
| } | ||
| } | ||
| } | ||
|
|
||
| } | ||
|
|
||
| /** | ||
|
|
@@ -696,12 +746,15 @@ protected boolean storeSupportsResilientCommit() { | |
| */ | ||
| private void maybeDeleteDest(final boolean deleteDest, final Path dest) throws IOException { | ||
|
|
||
| if (deleteDest && getFileStatusOrNull(dest) != null) { | ||
|
|
||
| boolean deleted = delete(dest, true); | ||
| // log the outcome in case of emergency diagnostics traces | ||
| // being needed. | ||
| LOG.debug("{}: delete('{}') returned {}'", getName(), dest, deleted); | ||
| if (deleteDest) { | ||
| final FileStatus st = getFileStatusOrNull(dest); | ||
| if (st != null) { | ||
| if (st.isDirectory()) { | ||
| deleteDir(dest, OP_DELETE_DIR); | ||
| } else { | ||
| deleteFile(dest, OP_DELETE); | ||
| } | ||
| } | ||
| } | ||
| } | ||
|
|
||
|
|
@@ -915,26 +968,36 @@ protected final TaskPool.Submitter getIOProcessors(int size) { | |
| } | ||
|
|
||
| /** | ||
| * Delete a directory, possibly suppressing exceptions. | ||
| * Delete a directory. | ||
| * @param dir directory. | ||
| * @param suppressExceptions should exceptions be suppressed? | ||
| * @param statistic statistic to use | ||
| * @return true if the path is no longer present. | ||
| * @throws IOException exceptions raised in delete if not suppressed. | ||
| * @return any exception caught and suppressed | ||
| */ | ||
| protected IOException deleteDir( | ||
| protected boolean deleteDir( | ||
| final Path dir, | ||
| final String statistic) | ||
| throws IOException { | ||
| return trackDuration(getIOStatistics(), statistic, () -> | ||
| operations.rmdir(dir)); | ||
| } | ||
|
|
||
| /** | ||
| * Delete a directory, suprressing exceptions. | ||
| * @param dir directory. | ||
| * @param statistic statistic to use | ||
| * @return any exception caught. | ||
| */ | ||
| protected IOException deleteDirSuppressingExceptions( | ||
| final Path dir, | ||
| final Boolean suppressExceptions) | ||
| final String statistic) | ||
| throws IOException { | ||
| try { | ||
| delete(dir, true); | ||
| deleteDir(dir, statistic); | ||
| return null; | ||
| } catch (IOException ex) { | ||
| LOG.info("Error deleting {}: {}", dir, ex.toString()); | ||
| if (!suppressExceptions) { | ||
| throw ex; | ||
| } else { | ||
| return ex; | ||
| } | ||
| return ex; | ||
| } | ||
| } | ||
|
|
||
|
|
||
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.
typo: the