nsivabalan commented on code in PR #17834:
URL: https://github.com/apache/hudi/pull/17834#discussion_r2785698033
##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java:
##########
@@ -188,6 +188,37 @@ public class HoodieWriteConfig extends HoodieConfig {
"**Note** This is being actively worked on. Please use "
+ "`hoodie.datasource.write.keygenerator.class` instead.");
+ public static final ConfigProperty<Boolean> COMPLEX_KEYGEN_NEW_ENCODING =
ConfigProperty
+ .key("hoodie.write.complex.keygen.new.encoding")
+ .defaultValue(false)
+ .markAdvanced()
+ .sinceVersion("1.1.0")
Review Comment:
can we fix the sinceVersion
##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java:
##########
@@ -188,6 +188,37 @@ public class HoodieWriteConfig extends HoodieConfig {
"**Note** This is being actively worked on. Please use "
+ "`hoodie.datasource.write.keygenerator.class` instead.");
+ public static final ConfigProperty<Boolean> COMPLEX_KEYGEN_NEW_ENCODING =
ConfigProperty
+ .key("hoodie.write.complex.keygen.new.encoding")
+ .defaultValue(false)
+ .markAdvanced()
+ .sinceVersion("1.1.0")
+ .supportedVersions("0.14.2", "0.15.1", "1.0.3")
Review Comment:
can we fix the supported versions
##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/bulk/RowDataKeyGen.java:
##########
@@ -124,6 +129,18 @@ protected RowDataKeyGen(
this.partitionPathProjection = null;
} else {
this.partitionPathProjection = getProjection(this.partitionPathFields,
fieldNames, fieldTypes);
+ multiplePartitions = true;
+ }
+ if (simpleRecordKey) {
+ if (multiplePartitions && !useComplexKeygenNewEncoding) {
+ // single record key with multiple partition fields
+ this.simpleRecordKeyFunc = rowData -> {
+ String oriKey =
getRecordKey(recordKeyFieldGetter.getFieldOrNull(rowData),
this.recordKeyFields[0], consistentLogicalTimestampEnabled);
+ return new
StringBuilder(this.recordKeyFields[0]).append(DEFAULT_COMPOSITE_KEY_FIELD_VALUE).append(oriKey).toString();
Review Comment:
is this not DEFAULT_COLUMN_VALUE_SEPARATOR ?
##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/BuiltinKeyGenerator.java:
##########
@@ -220,7 +222,7 @@ private <S> S combineRecordKeyInternal(
PartitionPathFormatterBase.StringBuilder<S> sb = builderFactory.get();
for (int i = 0; i < recordKeyParts.size(); ++i) {
-
sb.appendJava(fieldNames.get(i)).appendJava(DEFAULT_COMPOSITE_KEY_FILED_VALUE);
+
sb.appendJava(fieldNames.get(i)).appendJava(DEFAULT_COMPOSITE_KEY_FIELD_VALUE);
Review Comment:
is this not DEFAULT_COLUMN_VALUE_SEPARATOR ?
##########
hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java:
##########
@@ -418,40 +418,6 @@ private HoodieArchivedTimeline
instantiateArchivedTimeline(String startTs) {
: new HoodieArchivedTimeline(this, startTs);
}
- /**
- * Validate table properties.
- *
- * @param properties Properties from writeConfig.
- */
- public void validateTableProperties(Properties properties) {
Review Comment:
oh, we just moved from there to the other class. we are good then.
##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieWriteClient.java:
##########
@@ -1371,6 +1376,44 @@ protected void releaseResources(String instantTime) {
// do nothing here
}
+ /**
+ * Validate table properties.
+ *
+ * @param properties Properties from writeConfig.
+ */
+ public void validateTableProperties(HoodieTableConfig tableConfig,
Properties properties) {
+ // Once meta fields are disabled, it cant be re-enabled for a given table.
+ if (!tableConfig.populateMetaFields()
+ && Boolean.parseBoolean((String)
properties.getOrDefault(HoodieTableConfig.POPULATE_META_FIELDS.key(),
HoodieTableConfig.POPULATE_META_FIELDS.defaultValue().toString()))) {
+ throw new HoodieException(HoodieTableConfig.POPULATE_META_FIELDS.key() +
" already disabled for the table. Can't be re-enabled back");
+ }
+
+ // Meta fields can be disabled only when either {@code
SimpleKeyGenerator}, {@code ComplexKeyGenerator}, {@code
NonpartitionedKeyGenerator} is used
+ if (!tableConfig.populateMetaFields()) {
+ String keyGenClass =
properties.getProperty(HoodieTableConfig.KEY_GENERATOR_CLASS_NAME.key(),
"org.apache.hudi.keygen.SimpleKeyGenerator");
+ if (!keyGenClass.equals("org.apache.hudi.keygen.SimpleKeyGenerator")
+ &&
!keyGenClass.equals("org.apache.hudi.keygen.NonpartitionedKeyGenerator")
+ &&
!keyGenClass.equals("org.apache.hudi.keygen.ComplexKeyGenerator")) {
+ throw new HoodieException("Only simple, non-partitioned or complex key
generator are supported when meta-fields are disabled. Used: " + keyGenClass);
+ }
+ }
+ if (config.enableComplexKeygenValidation()
+ && isComplexKeyGeneratorWithSingleRecordKeyField(tableConfig)) {
+ throw new HoodieException(getComplexKeygenErrorMessage("ingestion"));
+ }
+
+ //Check to make sure it's not a COW table with consistent hashing bucket
index
+ if (tableConfig.getTableType() == HoodieTableType.COPY_ON_WRITE) {
Review Comment:
why are we pulling in more changes.
https://github.com/apache/hudi/pull/13650/changes only has changes in L1400
to 1403 for this class.
if we are pulling in more changes, did we CP those resp PRs?
if not, we might miss tests for those source code changes right.
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]