1
0

[HUDI-2255] Refactor Datasource options (#3373)

Co-authored-by: Wenning Ding <wenningd@amazon.com>
This commit is contained in:
wenningd
2021-08-03 17:50:30 -07:00
committed by GitHub
parent 1ff2d3459a
commit 91bb0d1318
83 changed files with 887 additions and 887 deletions

View File

@@ -247,7 +247,7 @@ public class FlinkOptions extends HoodieConfig {
+ "By default true (in favor of streaming progressing over data integrity)");
public static final ConfigOption<String> RECORD_KEY_FIELD = ConfigOptions
.key(KeyGeneratorOptions.RECORDKEY_FIELD_OPT_KEY.key())
.key(KeyGeneratorOptions.RECORDKEY_FIELD.key())
.stringType()
.defaultValue("uuid")
.withDescription("Record key field. Value to be used as the `recordKey` component of `HoodieKey`.\n"
@@ -255,20 +255,20 @@ public class FlinkOptions extends HoodieConfig {
+ "the dot notation eg: `a.b.c`");
public static final ConfigOption<String> PARTITION_PATH_FIELD = ConfigOptions
.key(KeyGeneratorOptions.PARTITIONPATH_FIELD_OPT_KEY.key())
.key(KeyGeneratorOptions.PARTITIONPATH_FIELD.key())
.stringType()
.defaultValue("")
.withDescription("Partition path field. Value to be used at the `partitionPath` component of `HoodieKey`.\n"
+ "Actual value obtained by invoking .toString(), default ''");
public static final ConfigOption<Boolean> URL_ENCODE_PARTITIONING = ConfigOptions
.key(KeyGeneratorOptions.URL_ENCODE_PARTITIONING_OPT_KEY.key())
.key(KeyGeneratorOptions.URL_ENCODE_PARTITIONING.key())
.booleanType()
.defaultValue(false)
.withDescription("Whether to encode the partition path url, default false");
public static final ConfigOption<Boolean> HIVE_STYLE_PARTITIONING = ConfigOptions
.key(KeyGeneratorOptions.HIVE_STYLE_PARTITIONING_OPT_KEY.key())
.key(KeyGeneratorOptions.HIVE_STYLE_PARTITIONING.key())
.booleanType()
.defaultValue(false)
.withDescription("Whether to use Hive style partitioning.\n"