-
Notifications
You must be signed in to change notification settings - Fork 3.8k
Fix some problems with restoring #2141
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
Merged
Merged
Changes from all commits
Commits
Show all changes
2 commits
Select commit
Hold shift + click to select a range
File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -115,7 +115,13 @@ private static String makeDatasource(FireDepartment fireDepartment) | |
| private volatile Firehose firehose = null; | ||
|
|
||
| @JsonIgnore | ||
| private volatile boolean stopped = false; | ||
| private volatile boolean gracefullyStopped = false; | ||
|
|
||
| @JsonIgnore | ||
| private volatile boolean finishingJob = false; | ||
|
|
||
| @JsonIgnore | ||
| private volatile Thread runThread = null; | ||
|
|
||
| @JsonIgnore | ||
| private volatile QueryRunnerFactoryConglomerate queryRunnerFactoryConglomerate = null; | ||
|
|
@@ -172,16 +178,12 @@ public boolean isReady(TaskActionClient taskActionClient) throws Exception | |
| @Override | ||
| public TaskStatus run(final TaskToolbox toolbox) throws Exception | ||
| { | ||
| runThread = Thread.currentThread(); | ||
|
|
||
| if (this.plumber != null) { | ||
| throw new IllegalStateException("WTF?!? run with non-null plumber??!"); | ||
| } | ||
|
|
||
| // Shed any locks we might have (e.g. if we were uncleanly killed and restarted) since we'll reacquire | ||
| // them if we actually need them | ||
| for (final TaskLock taskLock : getTaskLocks(toolbox)) { | ||
| toolbox.getTaskActionClient().submit(new LockReleaseAction(taskLock.getInterval())); | ||
| } | ||
|
|
||
| boolean normalExit = true; | ||
|
|
||
| // It would be nice to get the PlumberSchool in the constructor. Although that will need jackson injectables for | ||
|
|
@@ -313,7 +315,7 @@ public String getVersion(final Interval interval) | |
| committerSupplier = Committers.supplierFromFirehose(firehose); | ||
|
|
||
| // Time to read data! | ||
| while ((!stopped || firehoseDrainableByClosing) && firehose.hasMore()) { | ||
| while ((!gracefullyStopped || firehoseDrainableByClosing) && firehose.hasMore()) { | ||
| final InputRow inputRow; | ||
|
|
||
| try { | ||
|
|
@@ -350,39 +352,55 @@ public String getVersion(final Interval interval) | |
| finally { | ||
| if (normalExit) { | ||
| try { | ||
| if (!stopped) { | ||
| // Hand off all pending data | ||
| log.info("Persisting and handing off pending data."); | ||
| plumber.persist(committerSupplier.get()); | ||
| plumber.finishJob(); | ||
| } else { | ||
| log.info("Persisting pending data without handoff, in preparation for restart."); | ||
| final Committer committer = committerSupplier.get(); | ||
| final CountDownLatch persistLatch = new CountDownLatch(1); | ||
| plumber.persist( | ||
| new Committer() | ||
| // Always want to persist. | ||
| log.info("Persisting remaining data."); | ||
|
|
||
| final Committer committer = committerSupplier.get(); | ||
| final CountDownLatch persistLatch = new CountDownLatch(1); | ||
| plumber.persist( | ||
| new Committer() | ||
| { | ||
| @Override | ||
| public Object getMetadata() | ||
| { | ||
| @Override | ||
| public Object getMetadata() | ||
| { | ||
| return committer.getMetadata(); | ||
| } | ||
| return committer.getMetadata(); | ||
| } | ||
|
|
||
| @Override | ||
| public void run() | ||
| { | ||
| try { | ||
| committer.run(); | ||
| } | ||
| finally { | ||
| persistLatch.countDown(); | ||
| } | ||
| @Override | ||
| public void run() | ||
| { | ||
| try { | ||
| committer.run(); | ||
| } | ||
| finally { | ||
| persistLatch.countDown(); | ||
| } | ||
| } | ||
| ); | ||
| persistLatch.await(); | ||
| } | ||
| ); | ||
| persistLatch.await(); | ||
|
|
||
| if (gracefullyStopped) { | ||
| log.info("Gracefully stopping."); | ||
| } else { | ||
| log.info("Finishing the job."); | ||
| synchronized (this) { | ||
| if (gracefullyStopped) { | ||
| // Someone called stopGracefully after we checked the flag. That's okay, just stop now. | ||
| log.info("Gracefully stopping."); | ||
| } else { | ||
| finishingJob = true; | ||
| } | ||
| } | ||
|
|
||
| if (finishingJob) { | ||
| plumber.finishJob(); | ||
| } | ||
| } | ||
| } | ||
| catch (InterruptedException e) { | ||
| log.debug(e, "Interrupted while finishing the job"); | ||
| } | ||
| catch (Exception e) { | ||
| log.makeAlert(e, "Failed to finish realtime task").emit(); | ||
| throw e; | ||
|
|
@@ -410,13 +428,17 @@ public void stopGracefully() | |
| { | ||
| try { | ||
| synchronized (this) { | ||
| if (!stopped) { | ||
| stopped = true; | ||
| log.info("Gracefully stopping."); | ||
| if (isFirehoseDrainableByClosing(spec.getIOConfig().getFirehoseFactory())) { | ||
| if (!gracefullyStopped) { | ||
| gracefullyStopped = true; | ||
|
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. what happened to the log line?
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. It split in two and moved into the conditional below. |
||
| if (finishingJob) { | ||
| log.info("stopGracefully: Interrupting finishJob."); | ||
| runThread.interrupt(); | ||
| } else if (isFirehoseDrainableByClosing(spec.getIOConfig().getFirehoseFactory())) { | ||
| log.info("stopGracefully: Draining firehose."); | ||
| firehose.close(); | ||
| } else { | ||
| log.debug("Cannot drain firehose[%s] by closing, so skipping closing.", firehose); | ||
| log.info("stopGracefully: Cannot drain firehose by closing, interrupting run thread."); | ||
| runThread.interrupt(); | ||
| } | ||
| } | ||
| } | ||
|
|
||
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
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.
wondering if its better to not release locks only if task is restorable ?
will it be better to always release old locks if there are any in case the task is not restorable (just to be on safer side)?
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.
I don't think there is a reason to release locks here since,