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

feat: Provide upper limit on number of push queries #4581

Merged
merged 5 commits into from
Feb 19, 2020

Conversation

purplefox
Copy link
Contributor

Description

Provides an upper limit on the number of push queries that can be running in the server at any one time.

Push queries are expensive as they own their own Kafka streams topology and associated threads. We can't have too many in the server at once.

Testing done

Added new test

Reviewer checklist

  • Ensure docs are updated if necessary. (eg. if a user visible feature is being added or changed).
  • Ensure relevant issues are linked (description should include text like "Fixes #")

@purplefox purplefox marked this pull request as ready for review February 18, 2020 11:28
@purplefox purplefox requested a review from a team as a code owner February 18, 2020 11:28
Copy link
Contributor

@big-andy-coates big-andy-coates left a comment

Choose a reason for hiding this comment

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

Thanks @purplefox

There seems to be changes in the PR unrelated to the description. Should these be in this PR? e.g. the TLS stuff is surely unrelated and could be in a different PR.

Few suggestions / requests below on code quality side too.


public static final String WORKER_POOL_SIZE = propertyName("worker.pool.size");
public static final String WORKER_POOL_DOC =
"Max number of worker threads for executing blocking code";
public static final int DEFAULT_WORKER_POOL_SIZE = 100;

public static final String MAX_PUSH_QUERIES = propertyName("max.push.queries");
public static final int DEFAULT_MAX_PUSH_QUERIES = 100;
Copy link
Contributor

Choose a reason for hiding this comment

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

I'd say 100 is way too high! Though, of course, it depends on what queries they're running.

@rodesai did some work investigating this, so he may have some more thoughts on what's a good limit.

Personally, I'd be starting with more like 10.

Copy link
Contributor

Choose a reason for hiding this comment

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

+1 - when we benchmarked, we saw performance start to drop off around 30 queries (presumably due to overhead from all the threads added by each new streams app - streams threads, kafka client threads))

Copy link
Contributor Author

Choose a reason for hiding this comment

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

I haven't tuned anything yet, so this is just a placeholder.

Copy link
Contributor

Choose a reason for hiding this comment

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

FWIW - we were talking to someone who ran 415 (albeit super simple) queries on one node (we should verify), so might be better to just let the users tune this and start with something somewhat aggressive

Copy link
Contributor

Choose a reason for hiding this comment

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

Erm.... I think it being a config allows the user to tune it @agavra :p lol.

Copy link
Contributor

Choose a reason for hiding this comment

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

-___-


final QueryResponseMetadata metadata = new QueryResponseMetadata(query.getId().toString(),
final QueryResponseMetadata metadata = new QueryResponseMetadata(
query == null ? null : query.getId().toString(),
Copy link
Contributor

Choose a reason for hiding this comment

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

Can we please not have methods taking and returning null. If it's optional, make it Optional! Jackson knows how to deal with Optional.

Or just use an empty string for pull query ids and mark QueryResponseMetadata with @JsonInclude(NON_EMPTY), which will mean it doesn't include a queryId field in the response if the id is empty.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Well, according to Brian Goetz, Optional was not designed to be used in method args (and IntelliJ will, by default, warn you if you do this) ;)

So this is very much a matter of personal style, there is no objective "right" way to do this.

We could argue about this until the cows come home, but imho it's of little importance and we have more important things to spend our time on :)

Copy link
Contributor

@big-andy-coates big-andy-coates Feb 19, 2020

Choose a reason for hiding this comment

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

What is your objection to using an empty string?

By using null anyone using this type must null check the Id everytime be for accessing it and do something if its null. This has a tendency to produce NPEs and hence bugs.

Why not just an empty string?

The issue with nulls is that all callers need to null check and if they forget we end up with an NPE. Using Optional encodes the optionality into the type system so callers know it is optional, without having to check the implementation of the code. Using a default value, e.g. empty string, may mean calls can just use the value without caring. Of course a default value many not be the correct approach if callers need to do different things for the two difference cases.

Copy link
Contributor Author

@purplefox purplefox Feb 19, 2020

Choose a reason for hiding this comment

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

Empty string seems hacky to me. The query has no id, the correct way to represent that inside an object is as a null field (Optionals weren't designed for this case).

Copy link
Contributor Author

Choose a reason for hiding this comment

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

I have fixed this in a simple way which avoids nulls but doesn't require any new types. I hope you agree this is preferable.

Comment on lines 82 to 83
final PushQueryHolder query = queryPublisher.isPullQuery() ? null :
connectionQueryManager.createApiQuery(queryPublisher, routingContext.request());
Copy link
Contributor

Choose a reason for hiding this comment

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

Rather than having null here, why not introduce a QueryHolder interface with the getId and close methods. Then connectionQueryManager.createApiQuery can return a PushQueryHolder for push queries and a PullQueryHolder for pull. The Pull version would have a no-op close method.

This uses the power of polymorphism to avoids all the null checking and ifs.

Copy link
Contributor

Choose a reason for hiding this comment

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

+1 this would avoid (at a minimum) the two if checks below and at best future NPEs from less detailed oriented coders (like me)

Copy link
Contributor Author

@purplefox purplefox Feb 18, 2020

Choose a reason for hiding this comment

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

I think this is a matter of personal style. I think the current code is simple and straight to the point. Adding new types here seems overkill imho.

Copy link
Contributor

Choose a reason for hiding this comment

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

This is not a matter of style. Style is about where you put your brackets or tabs vs spaces. This is about good programming patterns and OO principles. In this case its leveraging polymorphism to avoid conditionals.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

I'd consider trading a single succint line of code for a new interface, probably 20+ lines of new code and changes, as overengineering it.

Comment on lines 70 to 77
public static final String TLS_CLIENT_AUTH_REQUIRED = propertyName("tls.client.auth.required");
public static final boolean DEFAULT_TLS_CLIENT_AUTH_REQUIRED = false;
public static final String TLS_CLIENT_AUTH_REQUIRED_DOC =
"Is client auth required?";
Copy link
Contributor

Choose a reason for hiding this comment

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

Isn't client auth normally off, requested or required rather than simply on-off?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Yes, but I looked at these docs which are linked to from the KSQL docs and it appears to be binary for KSQL currently: https://docs.confluent.io/current/kafka-rest/config.html#kafka-rest-https-config which is why I have also made it binary here

Copy link
Contributor

Choose a reason for hiding this comment

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

Problem is, we can't extend this later as its a boolean. If you want binary, then I'd go with making this a string property that must be either off or required. We can then add requested later if we want without having to add a new config type and deprecating this one.

Alternatively, just add the three now.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

I've been trying to keep the configs as compatible with the current ksql as possible. If that's not. requirement then fine.

@Override
protected ApiServerConfig createServerConfig() {
ApiServerConfig config = super.createServerConfig();
Map<String, Object> origs = config.originalsWithPrefix("");
Copy link
Contributor

Choose a reason for hiding this comment

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

Maybe:

Suggested change
Map<String, Object> origs = config.originalsWithPrefix("");
Map<String, Object> origs = config.originals();

Copy link
Contributor Author

Choose a reason for hiding this comment

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

That's the first thing I looked for, but there doesn't seem to be any such method.

Copy link
Contributor

Choose a reason for hiding this comment

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

There really is!

Copy link
Contributor Author

Choose a reason for hiding this comment

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

It seems there are two AbstractConfig classes - one in common and one in Kafka, don't know why!

import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

public class BaseApiTest {
Copy link
Contributor

Choose a reason for hiding this comment

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

It's preferable to create a test fixture, rather than use base classes in tests. You can have any number of test fixtures in a class, but you can only have one base classs!

Take a look at TestKsqlRestApp or EmbeddedSingleNodeKafkaCluster for examples.

Copy link
Contributor Author

@purplefox purplefox Feb 18, 2020

Choose a reason for hiding this comment

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

I like the pattern of base classes for tests, we want to run the same tests for different config and that seems the simplest way to do it. I don't really see how a test fixture would help or add here.

Copy link
Contributor

Choose a reason for hiding this comment

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

There aren't any tests that I can see in BaseApiTest, so this doesn't seem to be about running a common set of tests with multiple configs.

BaseApiTest looks to be more a setUp method, a overrideaable set of config and some helper methods. The pattern we use for such is a unit TestResource as this allows composition of different test resources, where as a base class only allows one.

Please take a look at TestKsqlRestApp as a good example of how to knock up a simple junit TestResouce. It really is a better pattern than a base class.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

If it's not broke don't fix it. The base class abstraction works fine for now. I've been using this pattern for around 20 years and so far haven't come across a situation where I've had a requirement to inherit from multiple base classes. I suggest the most efficient thing to do would be to wait until that requirement happens and refactor it to.a fixture then, rather than expend unnecessary effort doing it now.

Comment on lines 49 to 75
public static final String TLS_ENABLED = propertyName("tls.enabled");
public static final boolean DEFAULT_TLS_ENABLED = false;
public static final String TLS_ENABLED_DOC =
"Is TLS enabled?";

public static final String CERT_PATH = propertyName("cert.path");
public static final String CERT_PATH_DOC =
"Path to cert file";
public static final String TLS_KEY_STORE_PATH = propertyName("tls.keystore.path");
public static final String TLS_KEY_STORE_PATH_DOC =
"Path to server key store";

public static final String TLS_KEY_STORE_PASSWORD = propertyName("tls.keystore.password");
public static final String TLS_KEY_STORE_PASSWORD_DOC =
"Password for server key store";

public static final String TLS_TRUST_STORE_PATH = propertyName("tls.truststore.path");
public static final String TLS_TRUST_STORE_PATH_DOC =
"Path to client trust store";

public static final String TLS_TRUST_STORE_PASSWORD = propertyName("tls.truststore.password");
public static final String TLS_TRUST_STORE_PASSWORD_DOC =
"Password for client trust store";

public static final String TLS_CLIENT_AUTH_REQUIRED = propertyName("tls.client.auth.required");
public static final boolean DEFAULT_TLS_CLIENT_AUTH_REQUIRED = false;
public static final String TLS_CLIENT_AUTH_REQUIRED_DOC =
Copy link
Contributor

Choose a reason for hiding this comment

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

Not sure how this change is related to the subject of this PR. Should this be in this PR?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

From previously stacked PR

Copy link
Contributor

Choose a reason for hiding this comment

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

Yeah, worked it out eventually - can you call this out in the description next time please?

@big-andy-coates big-andy-coates self-assigned this Feb 18, 2020

public static final String WORKER_POOL_SIZE = propertyName("worker.pool.size");
public static final String WORKER_POOL_DOC =
"Max number of worker threads for executing blocking code";
public static final int DEFAULT_WORKER_POOL_SIZE = 100;

public static final String MAX_PUSH_QUERIES = propertyName("max.push.queries");
Copy link
Contributor

Choose a reason for hiding this comment

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

We have a similar config for limiting the number of persistent push queries:

"ksql.query.persistent.active.limit";

If this limit is for transient push queries, we should look to use consistent naming. If its for all push queries, we should try to consolidate the configs.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

As a general thing, right now, all the new api config are prefixed with "ksql.api" - they need to have some common prefix as that's what the CommonConfig class expects. Once we've migrated all the old API stuff we can consolidate things.

@purplefox
Copy link
Contributor Author

There seems to be changes in the PR unrelated to the description. Should these be in this PR? e.g. the TLS stuff is surely unrelated and could be in a different PR.

It's a stacked commit (stacked on the TLS commit), that's why you see the changes from the other one here. On the files view you can select which commits you want to view changes for.

@purplefox purplefox force-pushed the query_limits branch 2 times, most recently from db521aa to 66f1535 Compare February 18, 2020 22:43
Comment on lines 169 to 183
private static Validator oneOrMore() {
return (name, val) -> {
if (val instanceof Long) {
if (((Long) val) < 1) {
throw new ConfigException(name, val, "Not >= 1");
}
} else if (val instanceof Integer) {
if (((Integer) val) < 1) {
throw new ConfigException(name, val, "Not >= 1");
}
} else {
throw new IllegalArgumentException("validator should only be used with int, long");
}
};
}
Copy link
Contributor

@big-andy-coates big-andy-coates Feb 19, 2020

Choose a reason for hiding this comment

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

Can .you move this to ConfigValidators please?

Even better, change the existing zeroOrPositive to be more flexible, e.g. to become equalToOrGreaterThan(0), equalToOrGreaterThan(1), etc.

import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

public class BaseApiTest {
Copy link
Contributor

Choose a reason for hiding this comment

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

There aren't any tests that I can see in BaseApiTest, so this doesn't seem to be about running a common set of tests with multiple configs.

BaseApiTest looks to be more a setUp method, a overrideaable set of config and some helper methods. The pattern we use for such is a unit TestResource as this allows composition of different test resources, where as a base class only allows one.

Please take a look at TestKsqlRestApp as a good example of how to knock up a simple junit TestResouce. It really is a better pattern than a base class.

Copy link
Contributor

@big-andy-coates big-andy-coates left a comment

Choose a reason for hiding this comment

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

Hey @purplefox,

I've replied to comments above, plus another request below.

Github's being weird and refusing to add my comment - it was just a request to move the oneOrMore validator you added to ConfigValidators please.

Copy link
Contributor

@big-andy-coates big-andy-coates left a comment

Choose a reason for hiding this comment

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

.

Copy link
Contributor

@big-andy-coates big-andy-coates left a comment

Choose a reason for hiding this comment

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

Discussed offline: Tim's going to make the changes outlined above, or something close. Approving on that basis.

@purplefox purplefox merged commit 2cd66c7 into confluentinc:master Feb 19, 2020
@purplefox purplefox deleted the query_limits branch February 19, 2020 21:40
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

Successfully merging this pull request may close these issues.

4 participants