Skip to content

Zookeeper loss#6740

Merged
clintropolis merged 13 commits intoapache:masterfrom
michael-trelinski:zookeeper_loss
Mar 29, 2019
Merged

Zookeeper loss#6740
clintropolis merged 13 commits intoapache:masterfrom
michael-trelinski:zookeeper_loss

Conversation

@michael-trelinski
Copy link
Copy Markdown
Contributor

This pull request addresses Proposal #6518. The main idea is that upon zookeeper being unreachable after retries, it should forcefully exit so that a daemon monitor can restart the process. This new behavior is disabled by default and enabled by a config file setting.

michael-trelinski and others added 3 commits October 24, 2018 16:44
Fix bin/init to source from proper directory.
…to zookeeper_loss

# Conflicts:
#	extensions-core/parquet-extensions/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule
@sascha-coenen
Copy link
Copy Markdown

Hi. I'm running Druid on Kubernetes and the fact that the Curator library is not dealing with connection disruptions properly is the one single remaining issue that prevents my setup from being resilient. So I love your motion as it is much needed. I'm not a committer, so I cannot comment on whether the approach chosen is sound, but in one way or another, something has to be done for sure.

As as side note, out of shear curiosity I wonder whether there is any good reason to use Curator at all. I know that there are plans to move away from Zookeeper, but it is not Zookeeper that is causing all the issues - it is the Curator client library that is the culprit. I wonder why the Druid Devs don't just throw out Curator and be done with it. Would be much easier than getting rid of Zookeeper, which they can still do, but as a migration path action, I would throw out Curator first.

The most natural thing to do would have been to fix the following Curator issue:
https://issues.apache.org/jira/browse/CURATOR-229
But this issue has been reported in 2015 and it still did not get fixed although it is a severe bug that is impeding many people.

Druid is also not using the most recent version of Curator if I'm not mistaken. Version 4.0.1 has been released in Feb 2018 while Druid master uses 4.0.0.

@michael-trelinski
Copy link
Copy Markdown
Contributor Author

michael-trelinski commented Dec 15, 2018 via email

@kaijianding
Copy link
Copy Markdown
Contributor

+1
LGTM
I did the same thing in my production environment to let druid process exit when zookeeper server is down for a very long time. This PR is definitely useful.


private final Function<Void, Void> exitFunction;

public BoundedExponentialBackoffRetryWithQuit(Function<Void, Void> exitFunction, int baseSleepTimeMs, int maxSleepTimeMs, int maxRetries)
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

Line longer than 120 cols.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

Fixed

@JsonProperty("authScheme")
private String authScheme = "digest";

@JsonProperty("quitOnConnectFail")
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

IMO "quit" is a little vague. Maybe "terminateDruidProcess".

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

Please add documentation for this property.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

Fixed.

Added documentation.


private final Function<Void, Void> exitFunction;

public BoundedExponentialBackoffRetryWithQuit(Function<Void, Void> exitFunction, int baseSleepTimeMs, int maxSleepTimeMs, int maxRetries)
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

Line longer than 120 cols.

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

Why using Function<Void, Void> rather than Runnable?

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

Fixed line length.

Why using Function<Void, Void> rather than Runnable?

Initially, I used a Callable, but discovered that the method signature included a "throws" which played badly with the allowRetry method signature. I switched to function because it doesn't suffer from that. I don't want to use Runnable because it would lead people to believe that it needs to be executed from a Thread, which it does not.

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

To me, Function<Void, Void> is more confusing. Could you please define an interface like "ExitAction"?

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

I switched it to your original choice of Runnable rather than to write an interface that looked identical to Runnable/Callable.

return shouldRetry;
}

public Function<Void, Void> getExitFunction()
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

Why this getter is needed?

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

I removed the getter. I didn't know that's how we did things. My apologies.


import java.util.function.Function;

public class BoundedExponentialBackoffRetryWithQuit extends BoundedExponentialBackoffRetry
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

Maybe use composition instead of inheritance?

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

Point taken, but I think this is simple, and if the base class changes its constructor, we'll have to refactor either way. I wanted this based off of the BoundedExponentialBackoffRetry logic, so I subclassed it.

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

Could you please add a Javadoc comment "BoundedExponentialBackoffRetryWithQuit extends BoundedExponentialBackoffRetry for simplicity. It's not actually a BoundedExponentialBackoffRetry from the Liskov substitution principle point of view, but it doesn't matter in this code."

.ensembleProvider(ensembleProvider)
.sessionTimeoutMs(config.getZkSessionTimeoutMs())
.retryPolicy(new BoundedExponentialBackoffRetry(BASE_SLEEP_TIME_MS, MAX_SLEEP_TIME_MS, MAX_RETRIES))
.retryPolicy(config.getQuitOnConnectFail() ? new BoundedExponentialBackoffRetryWithQuit(exitFunction, BASE_SLEEP_TIME_MS, MAX_SLEEP_TIME_MS, MAX_RETRIES) : new BoundedExponentialBackoffRetry(BASE_SLEEP_TIME_MS, MAX_SLEEP_TIME_MS, MAX_RETRIES))
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

Line longer than 120 cols. Extract a variable.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

This has been extracted to a variable, but I couldn't find a clean way to do the conditional on multiple lines, so I just did an if-else refactor.

exConfig.getRestUriPath(),
exConfig.getPollingMs(),
new BoundedExponentialBackoffRetry(BASE_SLEEP_TIME_MS, MAX_SLEEP_TIME_MS, MAX_RETRIES)
config.getQuitOnConnectFail() ? new BoundedExponentialBackoffRetryWithQuit(exitFunction, BASE_SLEEP_TIME_MS, MAX_SLEEP_TIME_MS, MAX_RETRIES) : new BoundedExponentialBackoffRetry(BASE_SLEEP_TIME_MS, MAX_SLEEP_TIME_MS, MAX_RETRIES)
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

Line longer than 120 cols. Extract a variable.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

Fixed

public Void apply(Void someVoid)
{
log.error("Zookeeper can't be reached, forcefully stopping lifecycle...");
lifecycle.stop();
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

Are you sure that this function will actually run before the function configured in makeEnsembleProvider() will terminate the process?

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

I'm not sure I understand the question. Can you please rephrase?

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

EnsembleProvider created in makeEnsembleProvider() method, where you also added exit (but without stopping a lifecycle), is injected only in this method. So they are somehow related. I don't know what EnsembleProvider provider is and which termination action will kick in first.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

When I was testing this, I could never make the makeEnsembleProvider(...)-created RetryPolicy actually fire. It was always the makeCurator(...)-created RetryPolicy that fired (and exited). I'm just covering all the bases.

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

I think the reason of this should be understood. What if under some conditions the other exit function configured in makeEnsembleProvider() will suddenly kick in first and lifecycle will be left unfinished?

At very least, the exit function configured in makeEnsembleProvider() should log that as such, to allow easier analysis later. And it should be expressed in comments in that code that there are two functions configured because nobody knows if the one configured in makeCurator() is sufficient.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

It is clearly logged:

makeCurator()'s exit will look like this:

Zookeeper can't be reached, forcefully stopping lifecycle...
Zookeeper can't be reached, forcefully stopping virtual machine...

makeEnsembleProvider()'s exit will look like this:

Zookeeper can't be reached, forcefully stopping virtual machine...

@leventov
Copy link
Copy Markdown
Member

Added Design Review tag because this PR adds a config parameter.

@kaijianding please add such "important" tags (along with Incompatible) when you are the first committer who visits a PR.

@gianm
Copy link
Copy Markdown
Contributor

gianm commented Dec 20, 2018

I haven't read the patch, but just wanted to say that I am generally supportive of the idea of servers nuking themselves if really bad, and possibly unrecoverable, things happen. Like OOMEs and ZK connection loss.

- Add feature documentation
- Cosmetic refactors
- Variable extractions
- Remove getter
@JsonProperty("authScheme")
private String authScheme = "digest";

@JsonProperty("terminateDruidProcess")
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

I meant "terminateDruidProcessOnConnectFail", just replace "quit" part with "terminateDruidProcess"

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

Got it. Fixed.

public Void apply(Void someVoid)
{
log.error("Zookeeper can't be reached, forcefully stopping lifecycle...");
lifecycle.stop();
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

EnsembleProvider created in makeEnsembleProvider() method, where you also added exit (but without stopping a lifecycle), is injected only in this method. So they are somehow related. I don't know what EnsembleProvider provider is and which termination action will kick in first.

Comment thread docs/content/configuration/index.md Outdated
|`druid.zk.service.user`|The username to authenticate with ZooKeeper. This is an optional property.|none|
|`druid.zk.service.pwd`|The [Password Provider](../operations/password-provider.html) or the string password to authenticate with ZooKeeper. This is an optional property.|none|
|`druid.zk.service.authScheme`|digest is the only authentication scheme supported. |digest|
|`druid.zk.service.terminateDruidProcess`|When set to 'true', changes behavior of Druid processes that rely on Zookeeper to forcefully exit. If the connection to Zookeeper fails (after exhausting all exponential backoff retries), the process will die with an errorlevel of 1.|false|
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

I think something like

If set to 'true' and the connection to ZooKeeper fails (after exhausting all potential backoff retires), Druid process terminates itself with exit code 1.

would be clearer

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

Fixed.

log.error("Zookeeper can't be reached, forcefully stopping lifecycle...");
lifecycle.stop();
log.error("Zookeeper can't be reached, forcefully stopping virtual machine...");
System.exit(1);
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

If you really want to exit,

try {
  log.error("Zookeeper can't be reached, forcefully stopping lifecycle...");
  // Maybe wrap with call and log exception too
  lifecycle.stop();
  log.error("Zookeeper can't be reached, forcefully stopping virtual machine...");
} finally {
  System.exit(1);
}

would be more robust

public Void apply(Void aVoid)
{
log.error("Zookeeper can't be reached, forcefully stopping virtual machine...");
System.exit(1);
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

try {
  log.error("Zookeeper can't be reached, forcefully stopping virtual machine...");
} finally {
  System.exit(1);
}

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

Switched to this pattern. I didn't think it was necessary since I've never seen a logger kill a JVM.

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

Should this stop the lifecycle too? If this is getting hit it means that the curator killer is not getting hit and so I think this exit would be unclean.

Also, did you figure out why this needs to kill the process too? I see the discussion here, #6740 (comment) but it doesn't really come to a conclusion about whether it's needed.

I'm not sure it is needed. Presumably if exhibitor is down the zk processes managed by it are dead too, and I would expect the other killer will get hit? Or is this for a startup case where the other killer somehow doesn't get triggered? Regardless, could you also add a comment with the explanation?

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

@clintropolis I think the problem is that nobody knows answers to your questions and nobody has the determination to invest the effort to research this question (yet). I don't think we should demand this from @michael-trelinski. I think the PR in its current form is a move in the right direction anyway.

However, @michael-trelinski could you please add a comment in makeEnsembleProvider() along the lines with // It's unknown whether this precaution is needed, see https://github.com/apache/incubator-druid/pull/6740#discussion_r243368737 for details?

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

Sure, I wasn't expecting a deep investigation, I was suggesting either adding a lifecycle.stop() to make sure it stops cleanly and adding a comment to explain the we aren't sure if it is needed, OR just removing it until a scenario where an Exhibitor run Zookeeper is down but Druid doesn't kill itself and adding this block later when we find out it is needed.

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

Arguments to those provider methods are injected, so it could be available if you added it as an argument, e.g.

makeEnsembleProvider(CuratorConfig config, ExhibitorConfig exConfig, Lifecycle lifecycle)
{
...

Copy link
Copy Markdown
Contributor Author

@michael-trelinski michael-trelinski Feb 5, 2019

Choose a reason for hiding this comment

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

In master and the branch/commit I'm using, makeEnsembleProvider is declared thusly:

@Provides
@LazySingleton
public EnsembleProvider makeEnsembleProvider(CuratorConfig config, ExhibitorConfig exConfig)

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

I see what you're getting at. Apologies.

However, if it wasn't there to begin with and we're all not sure why makeEnsembleProvider is even there, should I be mucking around with it by adding a Lifecycle?

Copy link
Copy Markdown
Member

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 question of why makeEnsembleProvider exists, it provides the EnsembleProvider argument to makeCurator.

The only question I had is whether the quit on retry functionality needs to be wired into it's failures as well as curator's failures. I believe there should be no harm in using the injected lifecycle here as far as I can tell.

Copy link
Copy Markdown
Member

@leventov leventov Mar 5, 2019

Choose a reason for hiding this comment

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

@michael-trelinski could you please link this discussion from the comment? There is no superstition that code should be "self-contained" in Druid, there are many links from the code to Github. It would be easier for readers to navigate. Currently, readers would be forced to dig into Git history to follow this path.

- Switch from Function<Void,Void> to Runnable/Lambda
- try { … } finally { … }

public void setTerminateDruidProcessOnConnectFail(Boolean terminateDruidProcessOnConnectFail)
{
this.terminateDruidProcessOnConnectFail = terminateDruidProcessOnConnectFail == null ? false : terminateDruidProcessOnConnectFail;
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

Line longer than 120 cols. Please use if-else.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

Fixed

@leventov
Copy link
Copy Markdown
Member

@michael-trelinski as a side note, please don't "mark converations as resolved". I wish this Github feature could be turned off. As a reviewer I anyway have to revisit each conversation and verify myself that it's resolved, marking it as resolved just adds clicking work for me.

Copy link
Copy Markdown
Member

@clintropolis clintropolis left a comment

Choose a reason for hiding this comment

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

Overall LGTM 👍

@Override
public boolean allowRetry(int retryCount, long elapsedTimeMs, RetrySleeper sleeper)
{
log.warn("Zookeeper can't be reached, retrying (retryCount = " + retryCount + " out of " + this.getN() + ")...");
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

Could you use a format string here:

log.warn("Zookeeper can't be reached, retrying (retryCount = %s out of %s)...", retryCount, this.getN());

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

Done

try {
log.error("Zookeeper can't be reached, forcefully stopping lifecycle...");
lifecycle.stop();
log.error("Zookeeper can't be reached, forcefully stopping virtual machine...");
Copy link
Copy Markdown
Member

@clintropolis clintropolis Feb 3, 2019

Choose a reason for hiding this comment

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

This should probably be System.err.println since log is probably dead after lifecycle stop. It won't explode, but it probably won't go anywhere either. See #6975, it will require adding @SuppressForbidden(reason = "System#err")

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

Also done.

public Void apply(Void aVoid)
{
log.error("Zookeeper can't be reached, forcefully stopping virtual machine...");
System.exit(1);
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

Should this stop the lifecycle too? If this is getting hit it means that the curator killer is not getting hit and so I think this exit would be unclean.

Also, did you figure out why this needs to kill the process too? I see the discussion here, #6740 (comment) but it doesn't really come to a conclusion about whether it's needed.

I'm not sure it is needed. Presumably if exhibitor is down the zk processes managed by it are dead too, and I would expect the other killer will get hit? Or is this for a startup case where the other killer somehow doesn't get triggered? Regardless, could you also add a comment with the explanation?

@jon-wei jon-wei removed this from the 0.14.0 milestone Feb 12, 2019
System.err.println("Zookeeper can't be reached, forcefully stopping virtual machine...");
}
finally {
System.exit(1);
Copy link
Copy Markdown
Contributor

@egor-ryashin egor-ryashin Feb 15, 2019

Choose a reason for hiding this comment

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

For me it looks like a hack, shouldn't we try to recreate a CuratorFramework?

Copy link
Copy Markdown
Contributor

@egor-ryashin egor-ryashin Feb 15, 2019

Choose a reason for hiding this comment

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

Correct me if I'm wrong, please, we could have the whole Druid clusters restarted due to a ZK cluster shortage?

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

Correct me if I'm wrong, please, we could have the whole Druid clusters restarted due to a ZK cluster shortage?

@egor-ryashin That's the point. It is disabled by default. A lot of us want to have the Java process die (so that an outside process can restart it, like other popular Linux daemons). Some scenarios cause it to not completely die, because a Zookeeper thread hangs on forever.

Our use case was this: DNS/IT services changed the IPs of the zookeeper cluster after the JVM had already cached them. Had they been able to die gracefully, a daemon would have caused the cluster to "self heal." This use case is not an exhaustive list.

Also, we noticed that queries gave bad data during these times. Sometimes, for some use cases (as is ours), no data is preferable to bad data.

For me it looks like a hack, shouldn't we try to recreate a CuratorFramework?

This would not have fixed the issue in our use case.

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

Our use case was this: DNS/IT services changed the IPs of the zookeeper cluster after the JVM had already cached them.

I wonder whether using the option networkaddress.cache.ttl can help in that case?

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

It probably can.

However, all foreseeable use cases cannot be determined at this time.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

Another one may have presented itself today. I contend that having this in there would be a net positive.

https://groups.google.com/forum/#!topic/druid-user/ZK0VMaEIM1w

System.err.println("Zookeeper can't be reached, forcefully stopping virtual machine...");
}
finally {
System.exit(1);
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

I believe you cannot write a unit-test for that.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

You're basically right.

Without adding a bunch of PowerMockito dependencies for catching a static System.exit(..) call (which I don't want to do), I simulated it in a unit test by throwing an unchecked RuntimeException (instead of System.exit(...)) from an identical block of code and catching that.

Copy link
Copy Markdown
Member

@clintropolis clintropolis left a comment

Choose a reason for hiding this comment

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

sorry for delay, LGTM overall 👍. I don't feel strongly enough about makeEnsembleProvider also stopping the lifecycle to hold this PR up, since I think overall it's a useful feature.

@clintropolis
Copy link
Copy Markdown
Member

@leventov or @egor-ryashin any further comments on this PR?

public Void apply(Void aVoid)
{
log.error("Zookeeper can't be reached, forcefully stopping virtual machine...");
System.exit(1);
Copy link
Copy Markdown
Member

@leventov leventov Mar 5, 2019

Choose a reason for hiding this comment

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

@michael-trelinski could you please link this discussion from the comment? There is no superstition that code should be "self-contained" in Druid, there are many links from the code to Github. It would be easier for readers to navigate. Currently, readers would be forced to dig into Git history to follow this path.


private static final Logger log = new Logger(BoundedExponentialBackoffRetryWithQuitTest.class);

/*
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

(Optional) no reason why it shouldn't be a Javadoc comment.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

could you please link this discussion from the comment? There is no superstition that code should be "self-contained" in Druid, there are many links from the code to Github. It would be easier for readers to navigate. Currently, readers would be forced to dig into Git history to follow this path.

@leventov I don't understand what you mean.

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

I mean that comments in code should say see see https://github.com/apache/incubator-druid/pull/6740#discussion_r243368737 or, more generally, see discussions in https://github.com/apache/incubator-druid/pull/6740 somewhere. In other words, the code should back reference to this PR.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

@leventov Ok, done.

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

@michael-trelinski this is a wrong place. This link should be a part of the comment in CuratorModule. Specifically, it should extend your own comment

// It's unknown whether or not this precaution is needed.  Tests revealed that this path was never taken.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

Apologies. It's part of the thread-view for BoundedExponentialBackoffRetryWithQuitTest.java.


import java.util.function.Function;

public class BoundedExponentialBackoffRetryWithQuit extends BoundedExponentialBackoffRetry
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

Could you please add a Javadoc comment "BoundedExponentialBackoffRetryWithQuit extends BoundedExponentialBackoffRetry for simplicity. It's not actually a BoundedExponentialBackoffRetry from the Liskov substitution principle point of view, but it doesn't matter in this code."

RetryPolicy retryPolicy;
if (config.getTerminateDruidProcessOnConnectFail()) {
// It's unknown whether or not this precaution is needed. Tests revealed that this path was never taken.
/** It's unknown whether or not this precaution is needed. Tests revealed that this path was never taken.
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

You cannot Javadoc inner variables in a method. Please make it an ordinary comment.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

Done

@egor-ryashin egor-ryashin dismissed their stale review March 21, 2019 22:21

no time to review currently

@michael-trelinski
Copy link
Copy Markdown
Contributor Author

@clintropolis can you please restart the build (or advise)? I don't believe these errors are my fault, they seem IntelliJ-related on the TeamCity service.

@leventov
Copy link
Copy Markdown
Member

@michael-trelinski you can try to merge master into your branch, it may help.

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.

10 participants