-
Notifications
You must be signed in to change notification settings - Fork 1k
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
Added self documentation to the KSQL config public config variables. #422
Changes from 2 commits
db38492
17de98e
34f2231
b155a65
94cc2db
02a2f32
b3438c6
5104828
7026f77
ae5689a
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -36,18 +36,14 @@ public class KsqlConfig extends AbstractConfig implements Cloneable { | |
|
||
public static final String SINK_NUMBER_OF_PARTITIONS = "PARTITIONS"; | ||
public static final String SINK_NUMBER_OF_PARTITIONS_PROPERTY = "ksql.sink.partitions"; | ||
public static final String DEFAULT_SINK_NUMBER_OF_PARTITIONS = "ksql.sink.partitions.default"; | ||
|
||
public static final String SINK_NUMBER_OF_REPLICATIONS = "REPLICATIONS"; | ||
public static final String SINK_NUMBER_OF_REPLICATIONS_PROPERTY = "ksql.sink.replications"; | ||
public static final String DEFAULT_SINK_NUMBER_OF_REPLICATIONS = "ksql.sink.replications.default"; | ||
public static final String SINK_NUMBER_OF_REPLICAS = "REPLICAS"; | ||
public static final String SINK_NUMBER_OF_REPLICAS_PROPERTY = "ksql.sink.replicas"; | ||
|
||
public static final String SINK_WINDOW_CHANGE_LOG_ADDITIONAL_RETENTION = | ||
"WINDOW_CHANGE_LOG_ADDITIONAL_RETENTION"; | ||
public static final String SINK_WINDOW_CHANGE_LOG_ADDITIONAL_RETENTION_PROPERTY = | ||
"ksql.sink.window.change.log.additional.retention"; | ||
public static final String DEFAULT_SINK_WINDOW_CHANGE_LOG_ADDITIONAL_RETENTION = | ||
"ksql.sink.window.change.log.additional.retention.default"; | ||
|
||
public static final String STREAM_INTERNAL_CHANGELOG_TOPIC_SUFFIX = "-changelog"; | ||
|
||
|
@@ -118,18 +114,18 @@ public class KsqlConfig extends AbstractConfig implements Cloneable { | |
"Suffix for state store names in Tables. For instance if the suffix is _ksql_statestore the state " | ||
+ "store name would be ksql_query_1_ksql_statestore" | ||
+ "_ksql_statestore ") | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Did you add this extra line by mistake? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Good catch for |
||
.define(DEFAULT_SINK_NUMBER_OF_PARTITIONS, | ||
.define(SINK_NUMBER_OF_PARTITIONS_PROPERTY, | ||
ConfigDef.Type.INT, | ||
defaultSinkNumberOfPartitions, | ||
ConfigDef.Importance.MEDIUM, | ||
"The default number of partitions for the topics created by KSQL.") | ||
.define(DEFAULT_SINK_NUMBER_OF_REPLICATIONS, | ||
.define(SINK_NUMBER_OF_REPLICAS_PROPERTY, | ||
ConfigDef.Type.SHORT, | ||
defaultSinkNumberOfReplications, | ||
ConfigDef.Importance.MEDIUM, | ||
"The default number of replicas for the topics created by KSQL." | ||
) | ||
.define(DEFAULT_SINK_WINDOW_CHANGE_LOG_ADDITIONAL_RETENTION, | ||
.define(SINK_WINDOW_CHANGE_LOG_ADDITIONAL_RETENTION_PROPERTY, | ||
ConfigDef.Type.LONG, | ||
defaultSinkWindowChangeLogAdditionalRetention, | ||
ConfigDef.Importance.MEDIUM, | ||
|
@@ -144,40 +140,15 @@ public KsqlConfig(Map<?, ?> props) { | |
|
||
ksqlConfigProps = new HashMap<>(); | ||
ksqlStreamConfigProps = new HashMap<>(); | ||
ksqlConfigProps.put(KSQL_SERVICE_ID_CONFIG, KSQL_SERVICE_ID_DEFAULT); | ||
ksqlConfigProps.put(KSQL_PERSISTENT_QUERY_NAME_PREFIX_CONFIG, KSQL_PERSISTENT_QUERY_NAME_PREFIX_DEFAULT); | ||
ksqlConfigProps.put(KSQL_TRANSIENT_QUERY_NAME_PREFIX_CONFIG, KSQL_TRANSIENT_QUERY_NAME_PREFIX_DEFAULT); | ||
ksqlConfigProps.put(KSQL_TABLE_STATESTORE_NAME_SUFFIX_CONFIG, KSQL_TABLE_STATESTORE_NAME_SUFFIX_DEFAULT); | ||
|
||
if (props.containsKey(DEFAULT_SINK_NUMBER_OF_PARTITIONS)) { | ||
ksqlConfigProps.put(SINK_NUMBER_OF_PARTITIONS_PROPERTY, | ||
Integer.parseInt(props.get(DEFAULT_SINK_NUMBER_OF_PARTITIONS).toString())); | ||
} else { | ||
ksqlConfigProps.put(SINK_NUMBER_OF_PARTITIONS_PROPERTY, defaultSinkNumberOfPartitions); | ||
} | ||
|
||
if (props.containsKey(DEFAULT_SINK_NUMBER_OF_REPLICATIONS)) { | ||
ksqlConfigProps.put(SINK_NUMBER_OF_REPLICATIONS_PROPERTY, | ||
Short.parseShort(props.get(DEFAULT_SINK_NUMBER_OF_REPLICATIONS).toString())); | ||
} else { | ||
ksqlConfigProps.put(SINK_NUMBER_OF_REPLICATIONS_PROPERTY, defaultSinkNumberOfReplications); | ||
} | ||
|
||
if (props.containsKey(DEFAULT_SINK_WINDOW_CHANGE_LOG_ADDITIONAL_RETENTION)) { | ||
ksqlConfigProps.put(SINK_WINDOW_CHANGE_LOG_ADDITIONAL_RETENTION_PROPERTY, | ||
Long.parseLong(props.get(DEFAULT_SINK_WINDOW_CHANGE_LOG_ADDITIONAL_RETENTION).toString())); | ||
} else { | ||
ksqlConfigProps.put(SINK_WINDOW_CHANGE_LOG_ADDITIONAL_RETENTION_PROPERTY, | ||
defaultSinkWindowChangeLogAdditionalRetention); | ||
} | ||
ksqlConfigProps.putAll(super.values()); | ||
|
||
ksqlStreamConfigProps.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, defaultAutoOffsetRestConfig); | ||
ksqlStreamConfigProps.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, defaultCommitIntervalMsConfig); | ||
ksqlStreamConfigProps.put( | ||
StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, defaultCacheMaxBytesBufferingConfig); | ||
ksqlStreamConfigProps.put(StreamsConfig.NUM_STREAM_THREADS_CONFIG, defaultNumberOfStreamsThreads); | ||
|
||
for (Map.Entry<?, ?> entry : props.entrySet()) { | ||
for (Map.Entry<?, ?> entry : originals().entrySet()) { | ||
final String key = entry.getKey().toString(); | ||
if (key.toLowerCase().startsWith(KSQL_CONFIG_PREPERTY_PREFIX)) { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. i think we'd already have all the properties in the |
||
ksqlConfigProps.put(key, entry.getValue()); | ||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
The name and the value of this property don't seem to line up. What does the
REPLICAS
value mean exactly? Would it be more appropriate to have a numeric value? This applies to theSINK_NUMBER_OF_PARTITIONS
property above as well. Also where is thisSINK_NUMBER_OF_REPLICAS
string used?There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
These are not part of the config variables that can be set by the user via the KsqlConfig but they will be set in the query to specify the number of partitions, replications and the timestamp column for the result stream/table. This is just a constant to represent the name of the attributes.
Here is an example:
CREATE STREAM enrichedpv1 with (timestamp='pageid', partitions = 3) AS SELECT users.userid AS userid, pageid as pageid, region, gender FROM pageview LEFT JOIN users ON pageview.userid = users.userid;
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Hmm. Then why is it in
KsqlConfig
? There is also aSINK_NUMBER_OF_REPLICAS_PROPERTY
which I assume does go in theConfigDef
? It would be better to have the literals that are in the query moved to a separate file.There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Good idea, I'll move them to a new file, KsqlConstants.