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

fix: allow streams topic prefixed configs #3691

Merged
merged 1 commit into from
Oct 29, 2019

Conversation

vinothchandar
Copy link
Contributor

Fixes #817

Description

  • Adds new property file for production settings
  • Changes to allow topic prefixed streams configs such as min.insync.replicas

Testing done

  • Unit tests added
  • Verfied locally that the property file, creates topics with correct configuration
[kafka]$ bin/kafka-topics.sh --bootstrap-server localhost:9092 --topic _confluent-ksql-default_query_CTAS_ORDER_QUANTITIES_12-Aggregate-aggregate-changelog,_confluent-ksql-default_query_CTAS_ORDER_QUANTITIES_12-Aggregate-groupby-repartition --describe
...
Topic: _confluent-ksql-default_query_CTAS_ORDER_QUANTITIES_12-Aggregate-groupby-repartition	PartitionCount: 1	ReplicationFactor: 3	Configs: min.insync.replicas=2,cleanup.policy=delete,segment.bytes=52428800,retention.ms=-1
	Topic: _confluent-ksql-default_query_CTAS_ORDER_QUANTITIES_12-Aggregate-groupby-repartition	Partition: 0	Leader: 2	Replicas: 2,1,0	Isr: 2,1,0
Topic: _confluent-ksql-default_query_CTAS_ORDER_QUANTITIES_12-Aggregate-aggregate-changelog	PartitionCount: 1	ReplicationFactor: 3	Configs: min.insync.replicas=2,cleanup.policy=compact,segment.bytes=1073741824
	Topic: _confluent-ksql-default_query_CTAS_ORDER_QUANTITIES_12-Aggregate-aggregate-changelog	Partition: 0	Leader: 0	Replicas: 0,2,1	Isr: 0,2,1
15:31:05 [kafka]$

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 #")

@vinothchandar vinothchandar requested review from JimGalasyn and a team as code owners October 29, 2019 01:24
# Note: the value 3 requires at least 3 brokers in your Kafka cluster.
# Configure underlying Kafka Streams internal topics in order to achieve better fault tolerance and
# durability, even in the face of Kafka broker failures. Highly recommended for mission critical applications.
# Note that value 3 requires at least 3 brokers in your kafka cluster
Copy link
Member

Choose a reason for hiding this comment

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

Suggested change
# Note that value 3 requires at least 3 brokers in your kafka cluster
# Note that value 3 requires at least 3 brokers in your Kafka cluster.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

done

@@ -566,5 +568,8 @@ When deploying KSQL to production, the following settings are recommended in you
# Bump the number of replicas for state storage for stateful operations
# like aggregations and joins. By having two replicas (one main and one
# standby) recovery from node failures is quicker since the state doesn't
# have to be rebuilt from scratch.
# have to be rebuilt from scratch. This configuration is also essential for
# pull queries to be highly available during node failures
Copy link
Member

Choose a reason for hiding this comment

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

Suggested change
# pull queries to be highly available during node failures
# pull queries to be highly available during node failures.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

done

ksql.streams.num.standby.replicas=1

For your convenience, a sample file is provided at ``<path-to-ksql-repo>/config/ksql-production-server.properties``
Copy link
Member

Choose a reason for hiding this comment

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

People may not have cloned the KSQl repo, should we just put the URL here? https://github.com/confluentinc/ksql/tree/master/config

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 considered that. But what if this config changes based on versions? The installation directory seems to be only binaries?

Copy link
Member

@JimGalasyn JimGalasyn left a comment

Choose a reason for hiding this comment

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

LGTM, with a couple of suggestions.

@vinothchandar vinothchandar changed the title [WIP] fix: allow streams topic prefixed configs fix: allow streams topic prefixed configs Oct 29, 2019
Copy link
Contributor

@agavra agavra left a comment

Choose a reason for hiding this comment

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

some inline comments and suggestions, nothing major

# Set the batch expiry to Integer.MAX_VALUE to ensure that queries will not
# terminate if the underlying Kafka cluster is unavailable for a period of
# time.
ksql.streams.producer.delivery.timeout.ms=2147483647
Copy link
Contributor

Choose a reason for hiding this comment

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

do we really want the default to be set to something that's so high? will the behavior just hang? sometimes I'd prefer to have a hard error to notify me that something is wrong rather than silently suppress it. Same question goes for the setting below.

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 dont know why this value was chosen myself. I just kept whats already documented here https://docs.confluent.io/current/ksql/docs/installation/server-config/config-reference.html#recommended-ksql-production-settings

I share some views as you, but since this is subjective I will leave it as-is. There are probably other ways of getting alerted or notified (e.g processing lag).

Copy link
Contributor

Choose a reason for hiding this comment

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

I think we should revisit some of these default settings. A lot fo them filtered here from specific applications wherein was better to block forever than time out (and then have the streams app die). But in the general case, these may not be the best values.

Shall we file an issue to track follow up work to update these with a better general set of values?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

sg. created #3706

# Note that value 3 requires at least 3 brokers in your kafka cluster
# See https://docs.confluent.io/current/streams/developer-guide/config-streams.html#recommended-configuration-parameters-for-resiliency
ksql.streams.replication.factor=3
ksql.streams.producer.acks=all
Copy link
Contributor

Choose a reason for hiding this comment

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

do we really need this for all of the producers in ksql streams? I feel like we should have a setting just for our internal topics instead of prescribing this setting to all streams apps (and maybe we shouldn't even make it configurable but rather require that setting)

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Any streams application that prefers not losing data has to have this. This way of configuring kafka for no data loss has been documented even with say transaction. So it's a fairly standard thing to do. Otherwise, what you are computing is just an approximation. The downside of this configuration over not rf=1, acks!=all is slower performance. I still prefer to let users start from a correct state and then tune for performance.

I think there is still merit in making it configurable - for e.g: replication factor > 3


# Set the storage directory for stateful operations like aggregations and
# joins to be at a durable location. By default, they are stored in /tmp.
ksql.streams.state.dir=/some/non-temporary-storage-path/
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 something that the user needs to actively change, is there anyway we can bring attention to it?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

The server will refuse to start and error out opening that path. Hopefully that will call attention. Will add a note similar to what we have for replication factor above.

@@ -77,7 +77,8 @@

if (propertyName.startsWith(KSQL_STREAMS_PREFIX)
&& !propertyName.startsWith(KSQL_STREAMS_PREFIX + StreamsConfig.PRODUCER_PREFIX)
&& !propertyName.startsWith(KSQL_STREAMS_PREFIX + StreamsConfig.CONSUMER_PREFIX)) {
&& !propertyName.startsWith(KSQL_STREAMS_PREFIX + StreamsConfig.CONSUMER_PREFIX)
&& !propertyName.startsWith(KSQL_STREAMS_PREFIX + StreamsConfig.TOPIC_PREFIX)) {
return Optional.empty(); // Unknown streams config
Copy link
Contributor

Choose a reason for hiding this comment

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

out of scope for this PR, but I'm wondering why we don't let people pass in any arbitrary streams config and let streams do the verification here

Copy link
Contributor Author

Choose a reason for hiding this comment

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

we seem to have invested a lot into resolving configs in specific ways. I don't know the full context. But yeah, we could keep it simpler and pass everything with prefix to streams. But as of now, there are only these three configs that Streams documents. So may be ok for now.

@agavra agavra requested a review from a team October 29, 2019 18:26
Fixes confluentinc#817
- Adds new property file for production settings
- Changes to allow topic prefixed streams configs such as min.insync.replicas
- Unit tests added
- Verfied locally that the property file, creates topics with correct configuration
Copy link
Contributor

@agavra agavra left a comment

Choose a reason for hiding this comment

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

LGTM! I'm convinced

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.

Properties file entries for maximum resiliency
4 participants