Early stop on failed start (Alternative to #13087) (#13258)

* Make halt configurable. Don't halt in tests
This commit is contained in:
AmatyaAvadhanula 2022-12-05 21:05:07 +05:30 committed by GitHub
parent 65945a686f
commit 658a9c2d35
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
3 changed files with 45 additions and 2 deletions

View File

@ -454,8 +454,9 @@
<excludes>
<!-- There are UTs for this class but it is very difficult to get required branch coverage -->
<exclude>org/apache/druid/metadata/BasicDataSourceExt.class</exclude>
<!-- Tested in the SQL layer, but, oddly, not in this module. -->
<!-- Tested in the SQL layer, but, oddly, not in this module. -->
<exclude>org/apache/druid/server/QueryResponse.class</exclude>
<exclude>org/apache/druid/curator/CuratorModule.class</exclude>
</excludes>
</configuration>
</plugin>

View File

@ -30,6 +30,7 @@ import org.apache.curator.framework.api.ACLProvider;
import org.apache.curator.framework.imps.DefaultACLProvider;
import org.apache.curator.retry.BoundedExponentialBackoffRetry;
import org.apache.curator.shaded.com.google.common.base.Strings;
import org.apache.druid.concurrent.Threads;
import org.apache.druid.guice.JsonConfigProvider;
import org.apache.druid.guice.LazySingleton;
import org.apache.druid.java.util.common.StringUtils;
@ -40,6 +41,7 @@ import org.apache.zookeeper.data.ACL;
import java.nio.charset.StandardCharsets;
import java.util.List;
import java.util.concurrent.TimeUnit;
public class CuratorModule implements Module
{
@ -49,6 +51,23 @@ public class CuratorModule implements Module
static final int MAX_SLEEP_TIME_MS = 45000;
private static final int MAX_RETRIES = 29;
private final boolean haltOnFailedStart;
public CuratorModule()
{
this(true);
}
/**
*
* @param haltOnFailedStart set to true if the JVM needs to be halted within 30 seconds of failed initialization
* due to unhandled curator exceptions.
*/
public CuratorModule(boolean haltOnFailedStart)
{
this.haltOnFailedStart = haltOnFailedStart;
}
@Override
public void configure(Binder binder)
{
@ -97,6 +116,29 @@ public class CuratorModule implements Module
framework.getUnhandledErrorListenable().addListener((message, e) -> {
log.error(e, "Unhandled error in Curator, stopping server.");
if (haltOnFailedStart) {
final long startTime = System.currentTimeMillis();
final Thread halter = new Thread(
() -> {
try {
Threads.sleepFor(30, TimeUnit.SECONDS);
}
catch (InterruptedException ignored) {
}
log.warn(
"Could not stop server within %,d millis after unhandled Curator error. Halting immediately.",
System.currentTimeMillis() - startTime
);
Runtime.getRuntime().halt(1);
},
"exiter-thread"
);
halter.setDaemon(true);
halter.start();
}
shutdown(lifecycle);
});

View File

@ -120,7 +120,7 @@ public final class CuratorModuleTest
return new StartupInjectorBuilder()
.add(
new LifecycleModule(),
new CuratorModule(),
new CuratorModule(false),
binder -> binder.bind(Properties.class).toInstance(props)
)
.build();