-
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 1 commit
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 |
---|---|---|
|
@@ -57,67 +57,83 @@ public class KsqlConfig extends AbstractConfig implements Cloneable { | |
|
||
public static final String | ||
KSQL_SERVICE_ID_CONFIG = "ksql.service.id"; | ||
public static final ConfigDef.Type | ||
KSQL_SERVICE_ID_TYPE = ConfigDef.Type.STRING; | ||
public static final String | ||
KSQL_SERVICE_ID_DEFAULT = "ksql_"; | ||
public static final ConfigDef.Importance | ||
KSQL_SERVICE_ID_IMPORTANCE = ConfigDef.Importance.MEDIUM; | ||
public static final String | ||
KSQL_SERVICE_ID_DOC = | ||
"Indicates the ID of the ksql service. It will be used as prefix for all KSQL queries in " | ||
+ "this service."; | ||
|
||
public static final String | ||
KSQL_PERSISTENT_QUERY_NAME_PREFIX_CONFIG = "ksql.persistent.prefix"; | ||
public static final ConfigDef.Type | ||
KSQL_PERSISTENT_QUERY_NAME_PREFIX_TYPE = ConfigDef.Type.STRING; | ||
public static final String | ||
KSQL_PERSISTENT_QUERY_NAME_PREFIX_DEFAULT = "query_"; | ||
public static final ConfigDef.Importance | ||
KSQL_PERSISTENT_QUERY_NAME_PREFIX_IMPORTANCE = ConfigDef.Importance.MEDIUM; | ||
public static final String | ||
KSQL_PERSISTENT_QUERY_NAME_PREFIX_DOC = | ||
"Second part of the prefix for persitent queries."; | ||
|
||
public static final String | ||
KSQL_TRANSIENT_QUERY_NAME_PREFIX_CONFIG = "ksql.transient.prefix"; | ||
public static final ConfigDef.Type | ||
KSQL_TRANSIENT_QUERY_NAME_PREFIX_TYPE = ConfigDef.Type.STRING; | ||
public static final String | ||
KSQL_TRANSIENT_QUERY_NAME_PREFIX_DEFAULT = "transient_"; | ||
public static final ConfigDef.Importance | ||
KSQL_TRANSIENT_QUERY_NAME_PREFIX_IMPORTANCE = ConfigDef.Importance.MEDIUM; | ||
public static final String | ||
KSQL_TRANSIENT_QUERY_NAME_PREFIX_DOC = | ||
"Second part of the prefix for transient queries."; | ||
|
||
public static final String | ||
KSQL_TABLE_STATESTORE_NAME_SUFFIX_CONFIG = "ksql.statestore.suffix"; | ||
public static final ConfigDef.Type | ||
KSQL_TABLE_STATESTORE_NAME_SUFFIX_TYPE = ConfigDef.Type.STRING; | ||
public static final String | ||
KSQL_TABLE_STATESTORE_NAME_SUFFIX_DEFAULT = "transient_"; | ||
public static final ConfigDef.Importance | ||
KSQL_TABLE_STATESTORE_NAME_SUFFIX_IMPORTANCE = ConfigDef.Importance.MEDIUM; | ||
public static final String | ||
KSQL_TABLE_STATESTORE_NAME_SUFFIX_DOC = | ||
"Suffix for state store names in Tables."; | ||
KSQL_TABLE_STATESTORE_NAME_SUFFIX_DEFAULT = "_ksql_statestore"; | ||
|
||
public int defaultSinkNumberOfPartitions = 4; | ||
public short defaultSinkNumberOfReplications = 1; | ||
public static int defaultSinkNumberOfPartitions = 4; | ||
public static short defaultSinkNumberOfReplications = 1; | ||
// TODO: Find out the best default value. | ||
public long defaultSinkWindowChangeLogAdditionalRetention = 1000000; | ||
public static long defaultSinkWindowChangeLogAdditionalRetention = 1000000; | ||
|
||
public String defaultAutoOffsetRestConfig = "latest"; | ||
public long defaultCommitIntervalMsConfig = 2000; | ||
public long defaultCacheMaxBytesBufferingConfig = 10000000; | ||
public int defaultNumberOfStreamsThreads = 4; | ||
public static String defaultAutoOffsetRestConfig = "latest"; | ||
public static long defaultCommitIntervalMsConfig = 2000; | ||
public static long defaultCacheMaxBytesBufferingConfig = 10000000; | ||
public static int defaultNumberOfStreamsThreads = 4; | ||
|
||
Map<String, Object> ksqlConfigProps; | ||
Map<String, Object> ksqlStreamConfigProps; | ||
|
||
private static final ConfigDef CONFIG_DEF = new ConfigDef(); | ||
private static final ConfigDef CONFIG_DEF; | ||
|
||
static { | ||
CONFIG_DEF = new ConfigDef() | ||
.define(KSQL_SERVICE_ID_CONFIG, | ||
ConfigDef.Type.STRING, | ||
KSQL_SERVICE_ID_DEFAULT, | ||
ConfigDef.Importance.MEDIUM, | ||
"Indicates the ID of the ksql service. It will be used as prefix for all KSQL queries in " | ||
+ "this service.") | ||
|
||
.define(KSQL_PERSISTENT_QUERY_NAME_PREFIX_CONFIG, | ||
ConfigDef.Type.STRING, | ||
KSQL_PERSISTENT_QUERY_NAME_PREFIX_DEFAULT, | ||
ConfigDef.Importance.MEDIUM, | ||
"Second part of the prefix for persitent queries.") | ||
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. Perhaps a description of what the final query name would look like? 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. Added an example. |
||
.define(KSQL_TRANSIENT_QUERY_NAME_PREFIX_CONFIG, | ||
ConfigDef.Type.STRING, | ||
KSQL_TRANSIENT_QUERY_NAME_PREFIX_DEFAULT, | ||
ConfigDef.Importance.MEDIUM, | ||
"Second part of the prefix for transient queries.") | ||
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. as above 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. Added an example. |
||
.define(KSQL_TABLE_STATESTORE_NAME_SUFFIX_CONFIG, | ||
ConfigDef.Type.STRING, | ||
KSQL_TABLE_STATESTORE_NAME_SUFFIX_DEFAULT, | ||
ConfigDef.Importance.MEDIUM, | ||
"Suffix for state store names in Tables.") | ||
.define(DEFAULT_SINK_NUMBER_OF_PARTITIONS, | ||
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'm not sure why we have 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. The idea is to let user define the default for a set of statements. For instance if user want to have the default value of partitions to be 10 for all of his/her queries they can set the default and just write their queries instead of having to specify it in each of their queries. On the other hand, if user wants to have a different value for a specific query he/she can set the value in the specific query syntax and it will be applied to only that one. User can change the default to another value and that value will be used as default partition number for all the queries after that. |
||
ConfigDef.Type.INT, | ||
defaultSinkNumberOfPartitions, | ||
ConfigDef.Importance.MEDIUM, | ||
"The default number of partitions for the topics created by KSQL.") | ||
.define(DEFAULT_SINK_NUMBER_OF_REPLICATIONS, | ||
ConfigDef.Type.SHORT, | ||
defaultSinkNumberOfReplications, | ||
ConfigDef.Importance.MEDIUM, | ||
"The default number of replications for the topics created by KSQL." | ||
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. replications -> replicas 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. Done! |
||
) | ||
.define(DEFAULT_SINK_WINDOW_CHANGE_LOG_ADDITIONAL_RETENTION, | ||
ConfigDef.Type.LONG, | ||
defaultSinkWindowChangeLogAdditionalRetention, | ||
ConfigDef.Importance.MEDIUM, | ||
"The default window change log additional retention time." | ||
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. You probably want to drop the 'default' here. Also, it may make sense to expand the description. It doesn't make sense as is. Perhaps a sentence like 'The amount of time to retain window change logs'. Actually I am still not sure what that means Also, there should be a time unit in the name of the config and the description. 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 point, added the time unit and more details to the description. |
||
) | ||
; | ||
} | ||
|
||
|
||
public KsqlConfig(Map<?, ?> props) { | ||
super(CONFIG_DEF, props); | ||
|
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.
Do these need to be
public
, i.e., they are set as the defaults anywayThere 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.
Not necessarily, changed them to private.