Skip to content
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

Issue 1540: Bookie/BookieServer components shutdown will fail to end exit the BookieProcess #1543

Closed
wants to merge 4 commits into from

Conversation

sijie
Copy link
Member

@sijie sijie commented Jul 14, 2018

Descriptions of the changes in this PR:

Motivation

Fixes the issue at #1540.

If Bookie/BookieServer components are shutdown internally because of any fatal errors
(ExitCode - INVALID_CONF, SERVER_EXCEPTION, ZK_EXPIRED, ZK_REG_FAIL, BOOKIE_EXCEPTION) then
it will go through shutdown method logic and shutdowns components internal to Bookie/BookieServer
but it will not succeed in bringing down the bookie process.

This is because in BookieServer.main / server.Main.doMain it would wait for the startComponent
future to complete
http://github.com/apache/bookkeeper/blob/master/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/Main.java#L227 .
The startComponent future will be market complete only in runtime shutdownhook -
https://github.com/apache/bookkeeper/blob/master/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/component/ComponentStarter.java#L66.

But the problem is nowhere in Bookie/BookieProcess shutdown we are calling System.exit() and hence
the runtime shutdownhook is not executed to mark the startComponent future to complete. Hence
Main.doMain will wait forever on this future though Bookie/BookieServer components are shutdown
because of known fatal errors.

Regression

Issue #508 introduced this regression. Before this change, the main thread is blocking using BookieServer#join().
When bookie is dead for any reason, the DeathWatchThread will kill the bookie and bookie server. so the main thread will quite.
However after #508 is introduced, the lifecycle management is disconnected from the bookie and bookie server. so when they are dead,
lifecycle management is unaware of the situation and the main thread doesn't quite.

Changes

  • Add UncaughtExceptionHandler to lifecycle components
  • When a lifecycle component hits an error, it can use UncaughtExceptionHandler to notify lifecycle component stack to shutdown the whole stack

Master Issue: #1540


In order to uphold a high standard for quality for code contributions, Apache BookKeeper runs various precommit
checks for pull requests. A pull request can only be merged when it passes precommit checks. However running all
the precommit checks can take a long time, some trivial changes don't need to run all the precommit checks. You
can check following list to skip the tests that don't need to run for your pull request. Leave them unchecked if
you are not sure, committers will help you:

  • [skip bookkeeper-server bookie tests]: skip testing org.apache.bookkeeper.bookie in bookkeeper-server module.
  • [skip bookkeeper-server client tests]: skip testing org.apache.bookkeeper.client in bookkeeper-server module.
  • [skip bookkeeper-server replication tests]: skip testing org.apache.bookkeeper.replication in bookkeeper-server module.
  • [skip bookkeeper-server tls tests]: skip testing org.apache.bookkeeper.tls in bookkeeper-server module.
  • [skip bookkeeper-server remaining tests]: skip testing all other tests in bookkeeper-server module.
  • [skip integration tests]: skip docker based integration tests. if you make java code changes, you shouldn't skip integration tests.
  • [skip build java8]: skip build on java8. ONLY skip this when ONLY changing files under documentation under site.
  • [skip build java9]: skip build on java9. ONLY skip this when ONLY changing files under documentation under site.


Be sure to do all of the following to help us incorporate your contribution
quickly and easily:

If this PR is a BookKeeper Proposal (BP):

  • Make sure the PR title is formatted like:
    <BP-#>: Description of bookkeeper proposal
    e.g. BP-1: 64 bits ledger is support
  • Attach the master issue link in the description of this PR.
  • Attach the google doc link if the BP is written in Google Doc.

Otherwise:

  • Make sure the PR title is formatted like:
    <Issue #>: Description of pull request
    e.g. Issue 123: Description ...
  • Make sure tests pass via mvn clean apache-rat:check install spotbugs:check.
  • Replace <Issue #> in the title with the actual Issue number.

…exit the BookieProcess

 ### Motivation

Fixes the issue at apache#1540.

If Bookie/BookieServer components are shutdown internally because of any fatal errors
(ExitCode - INVALID_CONF, SERVER_EXCEPTION, ZK_EXPIRED, ZK_REG_FAIL, BOOKIE_EXCEPTION) then
it will go through shutdown method logic and shutdowns components internal to Bookie/BookieServer
but it will not succeed in bringing down the bookie process.

This is because in BookieServer.main / server.Main.doMain it would wait for the startComponent
future to complete
http://github.com/apache/bookkeeper/blob/master/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/Main.java#L227 .
The startComponent future will be market complete only in runtime shutdownhook -
https://github.com/apache/bookkeeper/blob/master/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/component/ComponentStarter.java#L66.

But the problem is nowhere in Bookie/BookieProcess shutdown we are calling System.exit() and hence
the runtime shutdownhook is not executed to mark the startComponent future to complete. Hence
Main.doMain will wait forever on this future though Bookie/BookieServer components are shutdown
because of known fatal errors.

 ### Regression

Issue apache#508 introduced this regression. Before this change, the main thread is blocking using `BookieServer#join()`.
When bookie is dead for any reason, the DeathWatchThread will kill the bookie and bookie server. so the main thread will quite.
However after apache#508 is introduced, the lifecycle management is disconnected from the bookie and bookie server. so when they are dead,
lifecycle management is unaware of the situation and the main thread doesn't quite.

 ### Changes

- Add `UncaughtExceptionHandler` to lifecycle components
- When a lifecycle component hits an error, it can use `UncaughtExceptionHandler` to notify lifecycle component stack to shutdown the whole stack
@sijie sijie added this to the 4.8.0 milestone Jul 14, 2018
@sijie sijie self-assigned this Jul 14, 2018
@sijie
Copy link
Member Author

sijie commented Jul 14, 2018

@reddycharan @jvrao this is the fix.

Copy link
Contributor

@eolivelli eolivelli left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Awesome work

+1

new ComponentShutdownHook(component, future), "component-shutdown-thread"));
final Thread shutdownHookThread = new Thread(
new ComponentShutdownHook(component, future),
"component-shutdown-thread"
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I wonder if we can use a more meaning name. Not so important

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

what is your suggestion?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Something with the name/class of the component

Copy link
Contributor

@eolivelli eolivelli left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Awesome work

+1

@sijie
Copy link
Member Author

sijie commented Jul 14, 2018

rerun bookkeeper-server bookie tests
rerun bookkeeper-server remaining tests

@sijie
Copy link
Member Author

sijie commented Jul 16, 2018

rerun bookkeeper-server bookie tests

1 similar comment
@sijie
Copy link
Member Author

sijie commented Jul 16, 2018

rerun bookkeeper-server bookie tests

@reddycharan
Copy link
Contributor

reddycharan commented Jul 16, 2018

@sijie

hmm.. I kind of understood what you are doing, but I'm not sure why it is called "ExceptionHandler" (all across the code) and in the fist place why do we need 'UncaughtExceptionHandler'.

In this change, BookieServer is signaling that it is not running and it is done, so https://github.com/apache/bookkeeper/pull/1543/files#diff-e7149cec76f474d8ae8a515cbe0464e8R74 can start ‘shutdownHookThread’ (ComponentShutdownHook) in ComponentStarter. So all it is doing is signaling ‘ComponentStarter’ to start ‘shutdownHookThread’, which completes future, so that Main.doMain need not wait for future anymore. Also exception passed to handler is dropped/ignored in the uncaught exception handler you created https://github.com/apache/bookkeeper/pull/1543/files#diff-e7149cec76f474d8ae8a515cbe0464e8R74. So it doesn’t makes sense completely to call ExceptionHandler.

And the other (major) concern I have is, in the previous versions we make sure “System.Exit” is called in the shutdown path of Bookie/BookieServer - https://github.com/apache/bookkeeper/blob/branch-4.4/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieServer.java#L401 . This is completely missing in the current LifeCycleComponent infrastructure and also in your current change. This is needed because for whatsoever reason non-daemon thread could be running in this Bookie process, though we have come to this (or completed) shutdown path and it could prevent Bookie process from shutdown, no matter how thorough we are with our lifecyclecomponent and core components handling/management.

@sijie
Copy link
Member Author

sijie commented Jul 16, 2018

I'm not sure why it is called "ExceptionHandler" (all across the code) and in the fist place why do we need 'UncaughtExceptionHandler'.

why I use "UncaughtExceptionHandler" is explained here: https://github.com/apache/bookkeeper/pull/1543/files#diff-87fe9faad6f1b8ff70975c11e676b4fbR122

I reused UncaughtExceptionHandler so that it can be easily used to attach to any critical thread, when those threads dead, it will trigger this shutdown logic.

I treated a bookie dead as an "uncaught" exception from a lifecycle component perspective, so registering an uncaught exception handler so the lifecycle management can react to those "uncaught" exception. once this mechanism is in placed, it can also be reused in other uncaught situation.

Hope this make sense.

in the previous versions we make sure “System.Exit” is called in the shutdown path of Bookie/BookieServer

This is a good point. I will add System.Exit when the lifecycle shuts things down.

@reddycharan
Copy link
Contributor

why I use "UncaughtExceptionHandler" is explained here: https://github.com/apache/bookkeeper/pull/1543/files#diff-87fe9faad6f1b8ff70975c11e676b4fbR122

I reused UncaughtExceptionHandler so that it can be easily used to attach to any critical thread, when those threads dead, it will trigger this shutdown logic.

Yeah, I read your comment. But my concern here is it is not just simple exception handler, it starts ComponentShutdownHook (shutdownHookThread) thread, which completes component future and it has got nothing to do with the exception (because it drops/ignores the exception). So probably in LifeCycleComponent instead of “setExceptionHandler” method name “setComponentShutdownHandler” and in BookieServer instead of “setExceptionHandler” method name “bookieServiceShutdownHandler” would be more appropriate. I feel “ExceptionHandler” is being used more generously when we are not really handling exception.

uncaughtExceptionHandler.uncaughtException(
this,
new BookieDeadException("Bookie is not running any more")
.fillInStackTrace());
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Why do we need fillInStackTrace() here?
Stack trace should be filled in by the Throwable's constructor already, assuming BookieException called super's constructor.

Now this got me looking at BookieException.
"public BookieException(int code)" sets private field code but does not call super().
All other constructors call super(..) variants but don't set code so we cannot rely on getCode(). Luckily, getCode() is not used so why do we even bother passing the code and having the field?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Why do we need fillInStackTrace() here?
Stack trace should be filled in by the Throwable's constructor already, assuming BookieException called super's constructor.

good question. I think some of the constructors are not calling super. I can fix the bookie exception constructors and remove fillInStackTrace

variants but don't set code so we cannot rely on getCode()

we should set the code. will fix it.

Luckily, getCode() is not used so why do we even bother passing the code and having the field?

the existing of the code is following BKException. will fix the constructors.

@sijie
Copy link
Member Author

sijie commented Jul 16, 2018

But my concern here is it is not just simple exception handler, it starts ComponentShutdownHook (shutdownHookThread) thread, which completes component future and it has got nothing to do with the exception (because it drops/ignores the exception). So probably in LifeCycleComponent instead of “setExceptionHandler” method name “setComponentShutdownHandler” and in BookieServer instead of “setExceptionHandler” method name “bookieServiceShutdownHandler” would be more appropriate. I feel “ExceptionHandler” is being used more generously when we are not really handling exception.

sure I can rename it to ShutdownHandler. However that would limit the scope to be just shutdown specifically. ExceptionHandler is more a well known interface for handling uncaught exception. How the caller would handle exceptions is up the caller. The caller here is the lifecycle management, when it receives an exception from any lifecycle component, it will trigger the exception handling and the exception handling logic is shutdown the whole lifecycle stack.

If you take a look at the BookieCriticalThread, when it got an uncaught exception, it will calls Runtime.exit to shutdown the whole JVM. the usage in lifecycle component is same here.

@@ -249,6 +266,12 @@ public void run() {
}
if (!isBookieRunning()) {
shutdown();
if (null != uncaughtExceptionHandler) {
uncaughtExceptionHandler.uncaughtException(
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

overall this looks weird. There is no uncaught exception but we are triggering uncaughtExceptionHandler with new exception, but only if it exists.
I'd either:

  • just throw new exception and let the handler really deal with it as with unhandled exception unless caught by higher level component. let app crash if there is no unhandled exception handler.
  • make BookieServer implement lifecyclecomponent, rename shutdown into doStop and call stop() instead of shutdown to ensure proper notification of all subscribers.

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

make BookieServer implement lifecyclecomponent, rename shutdown into doStop and call stop() instead of shutdown to ensure proper notification of all subscribers.

this won't work as shutting down one component will not notify the lifecycle stack to shutdown the whole. so that's why I am using a component being dead as an "uncaught" situation to notify the lifecycle stack to handle this "uncaught" situation and shutdown the whole stack.

just throw new exception and let the handler really deal with it as with unhandled exception unless caught by higher level component. let app crash if there is no unhandled exception handler.

what might be better here is following you suggested:

  • pass the uncaught exception handler to DeathWatchThread. in DeathWatchThread, when it detects a bookie is not running, thrown a BookieBeadException. then the uncaught exception handler will kick in to handle this uncaught exception.

@sijie
Copy link
Member Author

sijie commented Jul 16, 2018

in the previous versions we make sure “System.Exit” is called in the shutdown path of Bookie/BookieServer - https://github.com/apache/bookkeeper/blob/branch-4.4/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieServer.java#L401 . This is completely missing in the current LifeCycleComponent infrastructure and also in your current change.

@reddycharan sorry I took my comment back. System.Exit is called as before. It is called here: https://github.com/apache/bookkeeper/blob/master/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/Main.java#L202

- fix the code in `BookieException`
- `DeathWatcherThread` registers the uncaught exception handler and throw runtime exception when it notices the bookie is not running anymore
@sijie
Copy link
Member Author

sijie commented Jul 16, 2018

@reddycharan @dlg99 I addressed your comments.

  • fix the constructor issue for BookieException
  • change the way how DeathWatcherThread propagates exceptions. so DeatchWatcher will register the uncaught exception handler, when it notices that the bookie exception, it will throw a RuntimeException and it will trigger the Uncaught exception handler to handle this bookie dying situation.

let me know if this approach looks better to you guys.

@@ -248,11 +267,13 @@ public void run() {
Thread.currentThread().interrupt();
}
if (!isBookieRunning()) {
shutdown();
break;
LOG.info("BookieDeathWatcher noticed the bookie is not running any more, exiting the watch loop!");
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

just want to make sure that removal of "shutdown()" call here is intentional.

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

yes it is intentional. because the uncaught exception handle registered in death watcher thread will handle all the shutdown logic.

@reddycharan
Copy link
Contributor

LGTM +1

@sijie
Copy link
Member Author

sijie commented Jul 16, 2018

retest this please

Copy link
Contributor

@dlg99 dlg99 left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

lgtm

@sijie
Copy link
Member Author

sijie commented Jul 17, 2018

@dlg99 @reddycharan can you review my latest commit d7867d9 ? without this change, some of tests will be crashing because it will trigger Runtime.getRuntime().exit(-1). This latest commit will ensure it works for both tests and Main program.

@sijie
Copy link
Member Author

sijie commented Jul 17, 2018

@dlg99 I addressed your comment around the logging part. let me know if it looks okay.

@dlg99
Copy link
Contributor

dlg99 commented Jul 17, 2018

@sijie Looks good!

@reddycharan
Copy link
Contributor

LGTM

@sijie
Copy link
Member Author

sijie commented Jul 18, 2018

rerun bookkeeper-server bookie tests
rerun pr validation
rebuild java8
rebuild java9

@sijie
Copy link
Member Author

sijie commented Jul 23, 2018

retest this please

@sijie sijie closed this in 50f29ed Jul 23, 2018
sijie added a commit that referenced this pull request Jul 23, 2018
… exit the BookieProcess

Descriptions of the changes in this PR:

 ### Motivation

Fixes the issue at #1540.

If Bookie/BookieServer components are shutdown internally because of any fatal errors
(ExitCode - INVALID_CONF, SERVER_EXCEPTION, ZK_EXPIRED, ZK_REG_FAIL, BOOKIE_EXCEPTION) then
it will go through shutdown method logic and shutdowns components internal to Bookie/BookieServer
but it will not succeed in bringing down the bookie process.

This is because in BookieServer.main / server.Main.doMain it would wait for the startComponent
future to complete
http://github.com/apache/bookkeeper/blob/master/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/Main.java#L227 .
The startComponent future will be market complete only in runtime shutdownhook -
https://github.com/apache/bookkeeper/blob/master/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/component/ComponentStarter.java#L66.

But the problem is nowhere in Bookie/BookieProcess shutdown we are calling System.exit() and hence
the runtime shutdownhook is not executed to mark the startComponent future to complete. Hence
Main.doMain will wait forever on this future though Bookie/BookieServer components are shutdown
because of known fatal errors.

 ### Regression

Issue #508 introduced this regression. Before this change, the main thread is blocking using `BookieServer#join()`.
When bookie is dead for any reason, the DeathWatchThread will kill the bookie and bookie server. so the main thread will quite.
However after #508 is introduced, the lifecycle management is disconnected from the bookie and bookie server. so when they are dead,
lifecycle management is unaware of the situation and the main thread doesn't quite.

 ### Changes

- Add `UncaughtExceptionHandler` to lifecycle components
- When a lifecycle component hits an error, it can use `UncaughtExceptionHandler` to notify lifecycle component stack to shutdown the whole stack

Master Issue: #1540

Author: Sijie Guo <sijie@apache.org>

Reviewers: Andrey Yegorov <None>, Charan Reddy Guttapalem <reddycharan18@gmail.com>, Enrico Olivelli <eolivelli@gmail.com>

This closes #1543 from sijie/fix_lifcycle_components, closes #1540

(cherry picked from commit 50f29ed)
Signed-off-by: Sijie Guo <sijie@apache.org>
reddycharan pushed a commit to reddycharan/bookkeeper that referenced this pull request Jul 24, 2018
…hutdown will fail to end exit the BookieProcess

Descriptions of the changes in this PR:

 ### Motivation

Fixes the issue at apache#1540.

If Bookie/BookieServer components are shutdown internally because of any fatal errors
(ExitCode - INVALID_CONF, SERVER_EXCEPTION, ZK_EXPIRED, ZK_REG_FAIL, BOOKIE_EXCEPTION) then
it will go through shutdown method logic and shutdowns components internal to Bookie/BookieServer
but it will not succeed in bringing down the bookie process.

This is because in BookieServer.main / server.Main.doMain it would wait for the startComponent
future to complete
http://github.com/apache/bookkeeper/blob/master/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/Main.java#L227 .
The startComponent future will be market complete only in runtime shutdownhook -
https://github.com/apache/bookkeeper/blob/master/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/component/ComponentStarter.java#L66.

But the problem is nowhere in Bookie/BookieProcess shutdown we are calling System.exit() and hence
the runtime shutdownhook is not executed to mark the startComponent future to complete. Hence
Main.doMain will wait forever on this future though Bookie/BookieServer components are shutdown
because of known fatal errors.

 ### Regression

Issue apache#508 introduced this regression. Before this change, the main thread is blocking using `BookieServer#join()`.
When bookie is dead for any reason, the DeathWatchThread will kill the bookie and bookie server. so the main thread will quite.
However after apache#508 is introduced, the lifecycle management is disconnected from the bookie and bookie server. so when they are dead,
lifecycle management is unaware of the situation and the main thread doesn't quite.

 ### Changes

- Add `UncaughtExceptionHandler` to lifecycle components
- When a lifecycle component hits an error, it can use `UncaughtExceptionHandler` to notify lifecycle component stack to shutdown the whole stack

Master Issue: apache#1540

Author: Sijie Guo <sijie@apache.org>

Reviewers: Andrey Yegorov <None>, Charan Reddy Guttapalem <reddycharan18@gmail.com>, Enrico Olivelli <eolivelli@gmail.com>

This closes apache#1543 from sijie/fix_lifcycle_components, closes apache#1540
reddycharan pushed a commit to reddycharan/bookkeeper that referenced this pull request Jul 24, 2018
…hutdown will fail to end exit the BookieProcess

Descriptions of the changes in this PR:

 ### Motivation

Fixes the issue at apache#1540.

If Bookie/BookieServer components are shutdown internally because of any fatal errors
(ExitCode - INVALID_CONF, SERVER_EXCEPTION, ZK_EXPIRED, ZK_REG_FAIL, BOOKIE_EXCEPTION) then
it will go through shutdown method logic and shutdowns components internal to Bookie/BookieServer
but it will not succeed in bringing down the bookie process.

This is because in BookieServer.main / server.Main.doMain it would wait for the startComponent
future to complete
http://github.com/apache/bookkeeper/blob/master/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/Main.java#L227 .
The startComponent future will be market complete only in runtime shutdownhook -
https://github.com/apache/bookkeeper/blob/master/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/component/ComponentStarter.java#L66.

But the problem is nowhere in Bookie/BookieProcess shutdown we are calling System.exit() and hence
the runtime shutdownhook is not executed to mark the startComponent future to complete. Hence
Main.doMain will wait forever on this future though Bookie/BookieServer components are shutdown
because of known fatal errors.

 ### Regression

Issue apache#508 introduced this regression. Before this change, the main thread is blocking using `BookieServer#join()`.
When bookie is dead for any reason, the DeathWatchThread will kill the bookie and bookie server. so the main thread will quite.
However after apache#508 is introduced, the lifecycle management is disconnected from the bookie and bookie server. so when they are dead,
lifecycle management is unaware of the situation and the main thread doesn't quite.

 ### Changes

- Add `UncaughtExceptionHandler` to lifecycle components
- When a lifecycle component hits an error, it can use `UncaughtExceptionHandler` to notify lifecycle component stack to shutdown the whole stack

Master Issue: apache#1540

Author: Sijie Guo <sijie@apache.org>

Reviewers: Andrey Yegorov <None>, Charan Reddy Guttapalem <reddycharan18@gmail.com>, Enrico Olivelli <eolivelli@gmail.com>

This closes apache#1543 from sijie/fix_lifcycle_components, closes apache#1540
reddycharan pushed a commit to reddycharan/bookkeeper that referenced this pull request Aug 2, 2018
…hutdown will fail to end exit the BookieProcess

Descriptions of the changes in this PR:

 ### Motivation

Fixes the issue at apache#1540.

If Bookie/BookieServer components are shutdown internally because of any fatal errors
(ExitCode - INVALID_CONF, SERVER_EXCEPTION, ZK_EXPIRED, ZK_REG_FAIL, BOOKIE_EXCEPTION) then
it will go through shutdown method logic and shutdowns components internal to Bookie/BookieServer
but it will not succeed in bringing down the bookie process.

This is because in BookieServer.main / server.Main.doMain it would wait for the startComponent
future to complete
http://github.com/apache/bookkeeper/blob/master/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/Main.java#L227 .
The startComponent future will be market complete only in runtime shutdownhook -
https://github.com/apache/bookkeeper/blob/master/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/component/ComponentStarter.java#L66.

But the problem is nowhere in Bookie/BookieProcess shutdown we are calling System.exit() and hence
the runtime shutdownhook is not executed to mark the startComponent future to complete. Hence
Main.doMain will wait forever on this future though Bookie/BookieServer components are shutdown
because of known fatal errors.

 ### Regression

Issue apache#508 introduced this regression. Before this change, the main thread is blocking using `BookieServer#join()`.
When bookie is dead for any reason, the DeathWatchThread will kill the bookie and bookie server. so the main thread will quite.
However after apache#508 is introduced, the lifecycle management is disconnected from the bookie and bookie server. so when they are dead,
lifecycle management is unaware of the situation and the main thread doesn't quite.

 ### Changes

- Add `UncaughtExceptionHandler` to lifecycle components
- When a lifecycle component hits an error, it can use `UncaughtExceptionHandler` to notify lifecycle component stack to shutdown the whole stack

Master Issue: apache#1540

Author: Sijie Guo <sijie@apache.org>

Reviewers: Andrey Yegorov <None>, Charan Reddy Guttapalem <reddycharan18@gmail.com>, Enrico Olivelli <eolivelli@gmail.com>

This closes apache#1543 from sijie/fix_lifcycle_components, closes apache#1540
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Projects
None yet
Development

Successfully merging this pull request may close these issues.

None yet

4 participants