diff --git a/.github/workflows/bot.yml b/.github/workflows/bot.yml index 432d359817282..a26d3c40fb081 100644 --- a/.github/workflows/bot.yml +++ b/.github/workflows/bot.yml @@ -241,7 +241,7 @@ jobs: mvn verify -Pintegration-tests -D"$SCALA_PROFILE" -D"$FLINK_PROFILE" -pl hudi-flink-datasource/hudi-flink $MVN_ARGS docker-java17-test: - runs-on: ubuntu-latest + runs-on: ubuntu-22.04 strategy: matrix: include: @@ -269,7 +269,7 @@ jobs: ./packaging/bundle-validation/run_docker_java17.sh validate-bundles: - runs-on: ubuntu-latest + runs-on: ubuntu-22.04 strategy: matrix: include: diff --git a/azure-pipelines-20230430.yml b/azure-pipelines-20230430.yml index 85d185fbc2c5c..3fb901d27b8ce 100644 --- a/azure-pipelines-20230430.yml +++ b/azure-pipelines-20230430.yml @@ -160,7 +160,31 @@ stages: inputs: mavenPomFile: 'pom.xml' goals: 'test' - options: $(MVN_OPTS_TEST) -Pfunctional-tests -pl $(JOB2_MODULES) + options: $(MVN_OPTS_TEST) -pl $(JOB2_MODULES) -Pfunctional-tests -debug + publishJUnitResults: false + jdkVersionOption: '1.8' + mavenOptions: '-Xmx4g' + - script: | + grep "testcase" */target/surefire-reports/*.xml */*/target/surefire-reports/*.xml | awk -F'"' ' { print $6,$4,$2 } ' | sort -nr | head -n 100 + displayName: Top 100 long-running testcases + - job: UT_FT_5 + displayName: FT client/spark-client Long running + timeoutInMinutes: '150' + steps: + - task: Maven@4 + displayName: maven install + inputs: + mavenPomFile: 'pom.xml' + goals: 'clean install' + options: $(MVN_OPTS_INSTALL) + publishJUnitResults: false + jdkVersionOption: '1.8' + - task: Maven@4 + displayName: FT client/spark-client Long running + inputs: + mavenPomFile: 'pom.xml' + goals: 'test' + options: $(MVN_OPTS_TEST) -pl $(JOB2_MODULES) -Pisolated-tests publishJUnitResults: false jdkVersionOption: '1.8' mavenOptions: '-Xmx4g' diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieWriteClient.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieWriteClient.java index bfb811a159904..1b3edb9105f12 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieWriteClient.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieWriteClient.java @@ -40,9 +40,11 @@ import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecordLocation; +import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.model.HoodieWriteStat; import org.apache.hudi.common.model.TableServiceType; import org.apache.hudi.common.model.WriteOperationType; +import org.apache.hudi.common.table.HoodieTableConfig; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.HoodieTableVersion; import org.apache.hudi.common.table.TableSchemaResolver; @@ -101,6 +103,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Properties; import java.util.Set; import java.util.function.BiConsumer; import java.util.stream.Collectors; @@ -108,6 +111,8 @@ import static org.apache.hudi.avro.AvroSchemaUtils.getAvroRecordQualifiedName; import static org.apache.hudi.common.model.HoodieCommitMetadata.SCHEMA_KEY; import static org.apache.hudi.common.util.StringUtils.getUTF8Bytes; +import static org.apache.hudi.keygen.KeyGenUtils.getComplexKeygenErrorMessage; +import static org.apache.hudi.keygen.KeyGenUtils.isComplexKeyGeneratorWithSingleRecordKeyField; import static org.apache.hudi.metadata.HoodieTableMetadata.getMetadataTableBasePath; /** @@ -1323,7 +1328,7 @@ public final HoodieTable initTable(WriteOperationType operationType, Option COMPLEX_KEYGEN_NEW_ENCODING = ConfigProperty + .key("hoodie.write.complex.keygen.new.encoding") + .defaultValue(false) + .markAdvanced() + .sinceVersion("0.14.2") + .supportedVersions("0.14.2") + .withDocumentation("If set to false, the record key field name is encoded and prepended " + + "in the case where a single record key field is used in the complex key generator, " + + "i.e., record keys stored in _hoodie_record_key meta field is in the format of " + + "`:`, which conforms to the behavior " + + "in 0.14.0 release and older. If set to true, the record key field name is not " + + "encoded under the same case in the complex key generator, i.e., record keys stored " + + "in _hoodie_record_key meta field is in the format of ``, " + + "which conforms to the behavior in 0.14.1 release."); + + public static final ConfigProperty ENABLE_COMPLEX_KEYGEN_VALIDATION = ConfigProperty + .key("hoodie.write.complex.keygen.validation.enable") + .defaultValue(true) + .markAdvanced() + .sinceVersion("0.14.2") + .supportedVersions("0.14.2", "0.15.1", "1.0.3", "1.1.0") + .withDocumentation("This config only takes effect for writing table version 8 and below, " + + "upgrade or downgrade. If set to true, the writer enables the validation on whether the " + + "table uses the complex key generator with a single record key field, which can be affected " + + "by a breaking change in 0.14.1, 0.15.0, 1.0.0, 1.0.1, 1.0.2 releases, causing key " + + "encoding change and potential duplicates in the table. The validation fails the " + + "pipeline if the table meets the condition for the user to take proper action. " + + "The user can turn this validation off by setting the config to false, after " + + "evaluating the table and situation and doing table repair if needed."); + public static final ConfigProperty ROLLBACK_USING_MARKERS_ENABLE = ConfigProperty .key("hoodie.rollback.using.markers") .defaultValue("true") @@ -1317,6 +1347,10 @@ public boolean shouldRollbackUsingMarkers() { return getBoolean(ROLLBACK_USING_MARKERS_ENABLE); } + public boolean enableComplexKeygenValidation() { + return getBoolean(ENABLE_COMPLEX_KEYGEN_VALIDATION); + } + public int getWriteBufferLimitBytes() { return Integer.parseInt(getStringOrDefault(WRITE_BUFFER_LIMIT_BYTES_VALUE)); } @@ -2745,6 +2779,11 @@ public Builder withRollbackUsingMarkers(boolean rollbackUsingMarkers) { return this; } + public Builder withComplexKeygenValidation(boolean enableComplexKeygenValidation) { + writeConfig.setValue(ENABLE_COMPLEX_KEYGEN_VALIDATION, String.valueOf(enableComplexKeygenValidation)); + return this; + } + public Builder withWriteBufferLimitBytes(int writeBufferLimit) { writeConfig.setValue(WRITE_BUFFER_LIMIT_BYTES_VALUE, String.valueOf(writeBufferLimit)); return this; diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/ComplexAvroKeyGenerator.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/ComplexAvroKeyGenerator.java index 743aef1174a73..12cfac1b83210 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/ComplexAvroKeyGenerator.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/ComplexAvroKeyGenerator.java @@ -19,6 +19,7 @@ import org.apache.avro.generic.GenericRecord; import org.apache.hudi.common.config.TypedProperties; +import org.apache.hudi.common.function.SerializableFunctionUnchecked; import org.apache.hudi.keygen.constant.KeyGeneratorOptions; import java.util.Arrays; @@ -29,6 +30,7 @@ */ public class ComplexAvroKeyGenerator extends BaseKeyGenerator { public static final String DEFAULT_RECORD_KEY_SEPARATOR = ":"; + private final SerializableFunctionUnchecked recordKeyFunction; public ComplexAvroKeyGenerator(TypedProperties props) { super(props); @@ -37,18 +39,26 @@ public ComplexAvroKeyGenerator(TypedProperties props) { .map(String::trim) .filter(s -> !s.isEmpty()) .collect(Collectors.toList()); + this.recordKeyFunction = getRecordKeyFunc(KeyGenUtils.encodeSingleKeyFieldNameForComplexKeyGen(props)); } @Override public String getRecordKey(GenericRecord record) { - if (getRecordKeyFieldNames().size() == 1) { - return KeyGenUtils.getRecordKey(record, getRecordKeyFieldNames().get(0), isConsistentLogicalTimestampEnabled()); - } - return KeyGenUtils.getRecordKey(record, getRecordKeyFieldNames(), isConsistentLogicalTimestampEnabled()); + return recordKeyFunction.apply(record); } @Override public String getPartitionPath(GenericRecord record) { return KeyGenUtils.getRecordPartitionPath(record, getPartitionPathFields(), hiveStylePartitioning, encodePartitionPath, isConsistentLogicalTimestampEnabled()); } + + private SerializableFunctionUnchecked getRecordKeyFunc(boolean encodeSingleKeyFieldName) { + if (getRecordKeyFieldNames().size() == 1) { + if (encodeSingleKeyFieldName) { + return record -> KeyGenUtils.getRecordKey(record, getRecordKeyFieldNames(), isConsistentLogicalTimestampEnabled()); + } + return record -> KeyGenUtils.getRecordKey(record, getRecordKeyFieldNames().get(0), isConsistentLogicalTimestampEnabled()); + } + return record -> KeyGenUtils.getRecordKey(record, getRecordKeyFieldNames(), isConsistentLogicalTimestampEnabled()); + } } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/KeyGenUtils.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/KeyGenUtils.java index 7b88a0ab979b4..9dfb44c0c3af7 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/KeyGenUtils.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/KeyGenUtils.java @@ -21,6 +21,8 @@ import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.table.HoodieTableConfig; +import org.apache.hudi.common.util.ConfigUtils; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.PartitionPathEncodeUtils; import org.apache.hudi.common.util.ReflectionUtils; @@ -39,6 +41,8 @@ import java.util.List; import java.util.stream.Collectors; +import static org.apache.hudi.config.HoodieWriteConfig.COMPLEX_KEYGEN_NEW_ENCODING; + public class KeyGenUtils { protected static final String NULL_RECORDKEY_PLACEHOLDER = "__null__"; @@ -47,7 +51,7 @@ public class KeyGenUtils { protected static final String HUDI_DEFAULT_PARTITION_PATH = PartitionPathEncodeUtils.DEFAULT_PARTITION_PATH; public static final String DEFAULT_PARTITION_PATH_SEPARATOR = "/"; public static final String DEFAULT_RECORD_KEY_PARTS_SEPARATOR = ","; - public static final String DEFAULT_COMPOSITE_KEY_FILED_VALUE = ":"; + public static final String DEFAULT_COLUMN_VALUE_SEPARATOR = ":"; public static final String RECORD_KEY_GEN_PARTITION_ID_CONFIG = "_hoodie.record.key.gen.partition.id"; public static final String RECORD_KEY_GEN_INSTANT_TIME_CONFIG = "_hoodie.record.key.gen.instant.time"; @@ -128,7 +132,7 @@ public static String[] extractRecordKeys(String recordKey) { public static String[] extractRecordKeysByFields(String recordKey, List fields) { String[] fieldKV = recordKey.split(DEFAULT_RECORD_KEY_PARTS_SEPARATOR); - return Arrays.stream(fieldKV).map(kv -> kv.split(DEFAULT_COMPOSITE_KEY_FILED_VALUE, 2)) + return Arrays.stream(fieldKV).map(kv -> kv.split(DEFAULT_COLUMN_VALUE_SEPARATOR, 2)) .filter(kvArray -> kvArray.length == 1 || fields.isEmpty() || (fields.contains(kvArray[0]))) .map(kvArray -> { if (kvArray.length == 1) { @@ -149,11 +153,11 @@ public static String getRecordKey(GenericRecord record, List recordKeyFi for (String recordKeyField : recordKeyFields) { String recordKeyValue = HoodieAvroUtils.getNestedFieldValAsString(record, recordKeyField, true, consistentLogicalTimestampEnabled); if (recordKeyValue == null) { - recordKey.append(recordKeyField + DEFAULT_COMPOSITE_KEY_FILED_VALUE + NULL_RECORDKEY_PLACEHOLDER + DEFAULT_RECORD_KEY_PARTS_SEPARATOR); + recordKey.append(recordKeyField).append(DEFAULT_COLUMN_VALUE_SEPARATOR).append(NULL_RECORDKEY_PLACEHOLDER).append(DEFAULT_RECORD_KEY_PARTS_SEPARATOR); } else if (recordKeyValue.isEmpty()) { - recordKey.append(recordKeyField + DEFAULT_COMPOSITE_KEY_FILED_VALUE + EMPTY_RECORDKEY_PLACEHOLDER + DEFAULT_RECORD_KEY_PARTS_SEPARATOR); + recordKey.append(recordKeyField).append(DEFAULT_COLUMN_VALUE_SEPARATOR).append(EMPTY_RECORDKEY_PLACEHOLDER).append(DEFAULT_RECORD_KEY_PARTS_SEPARATOR); } else { - recordKey.append(recordKeyField + DEFAULT_COMPOSITE_KEY_FILED_VALUE + recordKeyValue + DEFAULT_RECORD_KEY_PARTS_SEPARATOR); + recordKey.append(recordKeyField).append(DEFAULT_COLUMN_VALUE_SEPARATOR).append(recordKeyValue).append(DEFAULT_RECORD_KEY_PARTS_SEPARATOR); keyIsNullEmpty = false; } } @@ -260,4 +264,30 @@ public static List getRecordKeyFields(TypedProperties props) { public static boolean enableAutoGenerateRecordKeys(TypedProperties props) { return !props.containsKey(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key()); } + + public static boolean isComplexKeyGeneratorWithSingleRecordKeyField(HoodieTableConfig tableConfig) { + Option recordKeyFields = tableConfig.getRecordKeyFields(); + return KeyGeneratorType.isComplexKeyGenerator(tableConfig) + && recordKeyFields.isPresent() && recordKeyFields.get().length == 1; + } + + public static String getComplexKeygenErrorMessage(String operation) { + return "This table uses the complex key generator with a single record " + + "key field. If the table is written with Hudi 0.14.1, 0.15.0, 1.0.0, 1.0.1, or 1.0.2 " + + "release before, the table may potentially contain duplicates due to a breaking " + + "change in the key encoding in the _hoodie_record_key meta field (HUDI-7001) which " + + "is crucial for upserts. Please take action based on the details on the deployment " + + "guide (https://hudi.apache.org/docs/deployment#complex-key-generator) " + + "before resuming the " + operation + " to the table. If you're certain " + + "that the table is not affected by the key encoding change, set " + + "`hoodie.write.complex.keygen.validation.enable=false` to skip this validation."; + } + + public static boolean encodeSingleKeyFieldNameForComplexKeyGen(TypedProperties props) { + return !ConfigUtils.getBooleanWithAltKeys(props, COMPLEX_KEYGEN_NEW_ENCODING); + } + + public static boolean mayUseNewEncodingForComplexKeyGen(HoodieTableConfig tableConfig) { + return isComplexKeyGeneratorWithSingleRecordKeyField(tableConfig); + } } diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/TestBaseHoodieWriteClient.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/TestBaseHoodieWriteClient.java new file mode 100644 index 0000000000000..afa7df3f89fb3 --- /dev/null +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/TestBaseHoodieWriteClient.java @@ -0,0 +1,288 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.client; + +import org.apache.hudi.client.embedded.EmbeddedTimelineService; +import org.apache.hudi.common.engine.HoodieLocalEngineContext; +import org.apache.hudi.common.model.HoodieCommitMetadata; +import org.apache.hudi.common.table.HoodieTableConfig; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.table.view.FileSystemViewStorageType; +import org.apache.hudi.common.testutils.HoodieCommonTestHarness; +import org.apache.hudi.common.testutils.HoodieTestUtils; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.model.WriteOperationType; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.index.HoodieIndex; +import org.apache.hudi.index.simple.HoodieSimpleIndex; +import org.apache.hudi.keygen.ComplexAvroKeyGenerator; +import org.apache.hudi.keygen.constant.KeyGeneratorOptions; +import org.apache.hudi.table.BulkInsertPartitioner; +import org.apache.hudi.table.HoodieTable; + +import org.apache.hadoop.conf.Configuration; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.MethodSource; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.function.BiConsumer; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static org.apache.hudi.common.testutils.HoodieTestUtils.getDefaultHadoopConf; +import static org.apache.hudi.testutils.Assertions.assertComplexKeyGeneratorValidationThrows; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +class TestBaseHoodieWriteClient extends HoodieCommonTestHarness { + + private static Stream testWithComplexKeyGeneratorValidation() { + List arguments = new ArrayList<>(); + + List keyAndPartitionFieldOptions = Arrays.asList( + Arguments.of("r1", "p1"), + Arguments.of("r1", "p1,p2"), + Arguments.of("r1", ""), + Arguments.of("r1,r2", "p1") + ); + + List booleanOptions = Arrays.asList( + Arguments.of(false, true), + Arguments.of(true, true), + Arguments.of(true, false) + ); + + List tableVersionOptions = Arrays.asList(6); + + arguments.addAll(Stream.of("org.apache.hudi.keygen.ComplexAvroKeyGenerator", + "org.apache.hudi.keygen.ComplexKeyGenerator") + .flatMap(keyGenClass -> keyAndPartitionFieldOptions.stream() + .flatMap(keyAndPartitionField -> booleanOptions.stream() + .flatMap(booleans -> tableVersionOptions.stream() + .map(tableVersion -> Arguments.of( + keyGenClass, + keyAndPartitionField.get()[0], + keyAndPartitionField.get()[1], + booleans.get()[0], + booleans.get()[1], + tableVersion + )) + ) + )) + .collect(Collectors.toList())); + arguments.addAll(Stream.of("org.apache.hudi.keygen.SimpleAvroKeyGenerator", + "org.apache.hudi.keygen.SimpleKeyGenerator", + "org.apache.hudi.keygen.TimestampBasedAvroKeyGenerator", + "org.apache.hudi.keygen.TimestampBasedKeyGenerator") + .flatMap(keyGenClass -> booleanOptions.stream() + .flatMap(booleans -> tableVersionOptions.stream() + .map(tableVersion -> Arguments.of( + keyGenClass, + "r1", + "p1", + booleans.get()[0], + booleans.get()[1], + tableVersion + )) + ) + ) + .collect(Collectors.toList())); + arguments.addAll(Stream.of("org.apache.hudi.keygen.NonpartitionedAvroKeyGenerator", + "org.apache.hudi.keygen.NonpartitionedKeyGenerator") + .flatMap(keyGenClass -> booleanOptions.stream() + .flatMap(booleans -> tableVersionOptions.stream() + .map(tableVersion -> Arguments.of( + keyGenClass, + "r1", + "", + booleans.get()[0], + booleans.get()[1], + tableVersion + )) + ) + ) + .collect(Collectors.toList())); + arguments.addAll(Stream.of("org.apache.hudi.keygen.CustomAvroKeyGenerator", + "org.apache.hudi.keygen.CustomKeyGenerator") + .flatMap(keyGenClass -> booleanOptions.stream() + .flatMap(booleans -> tableVersionOptions.stream() + .map(tableVersion -> Arguments.of( + keyGenClass, + "r1", + "p1:SIMPLE", + booleans.get()[0], + booleans.get()[1], + tableVersion + )) + ) + ) + .collect(Collectors.toList())); + + return arguments.stream(); + } + + @ParameterizedTest + @MethodSource + void testWithComplexKeyGeneratorValidation(String keyGeneratorClass, + String recordKeyFields, + String partitionPathFields, + boolean setComplexKeyGeneratorValidationConfig, + boolean enableComplexKeyGeneratorValidation, + int tableVersion) throws IOException { + if (basePath == null) { + initPath(); + } + Properties tableProperties = new Properties(); + tableProperties.put(HoodieTableConfig.KEY_GENERATOR_CLASS_NAME.key(), keyGeneratorClass); + tableProperties.put(HoodieTableConfig.RECORDKEY_FIELDS.key(), recordKeyFields); + tableProperties.put(HoodieTableConfig.PARTITION_FIELDS.key(), partitionPathFields); + tableProperties.put(HoodieTableConfig.VERSION.key(), String.valueOf(tableVersion)); + Properties writeProperties = new Properties(); + writeProperties.put(HoodieWriteConfig.KEYGENERATOR_CLASS_NAME.key(), keyGeneratorClass); + writeProperties.put(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key(), recordKeyFields); + writeProperties.put(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key(), partitionPathFields); + if (setComplexKeyGeneratorValidationConfig) { + writeProperties.put( + HoodieWriteConfig.ENABLE_COMPLEX_KEYGEN_VALIDATION.key(), enableComplexKeyGeneratorValidation); + } + metaClient = HoodieTestUtils.init( + HoodieTestUtils.getDefaultHadoopConf(), basePath, getTableType(), tableProperties); + HoodieWriteConfig.Builder writeConfigBuilder = HoodieWriteConfig.newBuilder() + .withPath(basePath) + .withProperties(writeProperties); + HoodieTable table = mock(HoodieTable.class); + BaseHoodieTableServiceClient tableServiceClient = mock(BaseHoodieTableServiceClient.class); + TestWriteClient writeClient = new TestWriteClient(writeConfigBuilder.build(), table, Option.empty(), tableServiceClient); + + if (tableVersion <= 8 && enableComplexKeyGeneratorValidation + && (ComplexAvroKeyGenerator.class.getCanonicalName().equals(keyGeneratorClass) + || "org.apache.hudi.keygen.ComplexKeyGenerator".equals(keyGeneratorClass)) + && recordKeyFields.split(",").length == 1) { + assertComplexKeyGeneratorValidationThrows(() -> writeClient.initTable(WriteOperationType.INSERT, Option.empty()), "ingestion"); + } else { + writeClient.initTable(WriteOperationType.INSERT, Option.empty()); + String requestedTime = writeClient.startCommit(); + + HoodieTimeline writeTimeline = metaClient.getActiveTimeline().getWriteTimeline(); + assertTrue(writeTimeline.lastInstant().isPresent()); + assertEquals("commit", writeTimeline.lastInstant().get().getAction()); + assertEquals(requestedTime, writeTimeline.lastInstant().get().getTimestamp()); + } + } + + private static class TestWriteClient extends BaseHoodieWriteClient { + private final HoodieTable table; + + public TestWriteClient(HoodieWriteConfig writeConfig, HoodieTable table, Option timelineService, + BaseHoodieTableServiceClient tableServiceClient) { + super(new HoodieLocalEngineContext(getDefaultHadoopConf()), writeConfig, timelineService, null); + this.table = table; + this.tableServiceClient = tableServiceClient; + } + + @Override + protected HoodieIndex createIndex(HoodieWriteConfig writeConfig) { + return new HoodieSimpleIndex(config, Option.empty()); + } + + @Override + public boolean commit(String instantTime, String writeStatuses, Option> extraMetadata, String commitActionType, Map> partitionToReplacedFileIds, + Option> extraPreCommitFunc) { + return false; + } + + @Override + protected HoodieTable createTable(HoodieWriteConfig config, Configuration hadoopConf) { + // table should only be made with remote view config for these tests + FileSystemViewStorageType storageType = config.getViewStorageConfig().getStorageType(); + Assertions.assertTrue(storageType == FileSystemViewStorageType.REMOTE_FIRST || storageType == FileSystemViewStorageType.REMOTE_ONLY); + return table; + } + + @Override + protected HoodieTable createTable(HoodieWriteConfig config, Configuration hadoopConf, HoodieTableMetaClient metaClient) { + // table should only be made with remote view config for these tests + FileSystemViewStorageType storageType = config.getViewStorageConfig().getStorageType(); + Assertions.assertTrue(storageType == FileSystemViewStorageType.REMOTE_FIRST || storageType == FileSystemViewStorageType.REMOTE_ONLY); + // Ensure the returned table has the correct metaClient + when(table.getMetaClient()).thenReturn(metaClient); + return table; + } + + @Override + public String filterExists(String hoodieRecords) { + return ""; + } + + @Override + public String upsert(String records, String instantTime) { + return ""; + } + + @Override + public String upsertPreppedRecords(String preppedRecords, String instantTime) { + return ""; + } + + @Override + public String insert(String records, String instantTime) { + return ""; + } + + @Override + public String insertPreppedRecords(String preppedRecords, String instantTime) { + return ""; + } + + @Override + public String bulkInsert(String records, String instantTime) { + return ""; + } + + @Override + public String bulkInsert(String records, String instantTime, Option userDefinedBulkInsertPartitioner) { + return ""; + } + + @Override + public String bulkInsertPreppedRecords(String preppedRecords, String instantTime, Option bulkInsertPartitioner) { + return ""; + } + + @Override + public String delete(String keys, String instantTime) { + return ""; + } + + @Override + public String deletePrepped(String preppedRecords, String instantTime) { + return ""; + } + } +} \ No newline at end of file diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/keygen/TestComplexAvroKeyGenerator.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/keygen/TestComplexAvroKeyGenerator.java index 0f6afd2ade6b2..dce53907aef7b 100644 --- a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/keygen/TestComplexAvroKeyGenerator.java +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/keygen/TestComplexAvroKeyGenerator.java @@ -21,37 +21,57 @@ import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.testutils.HoodieTestDataGenerator; +import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.keygen.constant.KeyGeneratorOptions; import org.apache.avro.generic.GenericRecord; -import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.CsvSource; -import static junit.framework.TestCase.assertEquals; +import static org.junit.jupiter.api.Assertions.assertEquals; public class TestComplexAvroKeyGenerator { - @Test - public void testSingleValueKeyGenerator() { + @ParameterizedTest + @CsvSource(value = {"false,true", "true,false", "true,true"}) + void testSingleValueKeyGenerator(boolean setNewEncodingConfig, + boolean encodeSingleKeyFieldValueOnly) { + String recordKeyFieldName = "_row_key"; TypedProperties properties = new TypedProperties(); - properties.setProperty(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key(), "_row_key"); + properties.setProperty(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key(), recordKeyFieldName); properties.setProperty(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key(), "timestamp"); + if (setNewEncodingConfig) { + properties.setProperty( + HoodieWriteConfig.COMPLEX_KEYGEN_NEW_ENCODING.key(), + String.valueOf(encodeSingleKeyFieldValueOnly)); + } ComplexAvroKeyGenerator compositeKeyGenerator = new ComplexAvroKeyGenerator(properties); assertEquals(compositeKeyGenerator.getRecordKeyFieldNames().size(), 1); assertEquals(compositeKeyGenerator.getPartitionPathFields().size(), 1); HoodieTestDataGenerator dataGenerator = new HoodieTestDataGenerator(); GenericRecord record = dataGenerator.generateGenericRecords(1).get(0); - String rowKey = record.get("_row_key").toString(); + String rowKey = record.get(recordKeyFieldName).toString(); String partitionPath = record.get("timestamp").toString(); HoodieKey hoodieKey = compositeKeyGenerator.getKey(record); - assertEquals(rowKey, hoodieKey.getRecordKey()); + // Table version 8 may use new encoding config if set + String expectedRecordKey = setNewEncodingConfig && encodeSingleKeyFieldValueOnly + ? rowKey : recordKeyFieldName + ":" + rowKey; + assertEquals(expectedRecordKey, hoodieKey.getRecordKey()); assertEquals(partitionPath, hoodieKey.getPartitionPath()); } - @Test - public void testMultipleValueKeyGenerator() { + @ParameterizedTest + @CsvSource(value = {"false,true", "true,false", "true,true"}) + void testMultipleValueKeyGenerator(boolean setNewEncodingConfig, + boolean encodeSingleKeyFieldValueOnly) { TypedProperties properties = new TypedProperties(); properties.setProperty(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key(), "_row_key,timestamp"); properties.setProperty(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key(), "rider,driver"); + if (setNewEncodingConfig) { + properties.setProperty( + HoodieWriteConfig.COMPLEX_KEYGEN_NEW_ENCODING.key(), + String.valueOf(encodeSingleKeyFieldValueOnly)); + } ComplexAvroKeyGenerator compositeKeyGenerator = new ComplexAvroKeyGenerator(properties); assertEquals(compositeKeyGenerator.getRecordKeyFieldNames().size(), 2); assertEquals(compositeKeyGenerator.getPartitionPathFields().size(), 2); @@ -66,11 +86,18 @@ public void testMultipleValueKeyGenerator() { assertEquals(partitionPath, hoodieKey.getPartitionPath()); } - @Test - public void testMultipleValueKeyGeneratorNonPartitioned() { + @ParameterizedTest + @CsvSource(value = {"false,true", "true,false", "true,true"}) + void testMultipleValueKeyGeneratorNonPartitioned(boolean setNewEncodingConfig, + boolean encodeSingleKeyFieldValueOnly) { TypedProperties properties = new TypedProperties(); properties.setProperty(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key(), "_row_key,timestamp"); properties.setProperty(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key(), ""); + if (setNewEncodingConfig) { + properties.setProperty( + HoodieWriteConfig.COMPLEX_KEYGEN_NEW_ENCODING.key(), + String.valueOf(encodeSingleKeyFieldValueOnly)); + } ComplexAvroKeyGenerator compositeKeyGenerator = new ComplexAvroKeyGenerator(properties); assertEquals(compositeKeyGenerator.getRecordKeyFieldNames().size(), 2); assertEquals(compositeKeyGenerator.getPartitionPathFields().size(), 0); diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/keygen/TestKeyGenUtils.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/keygen/TestKeyGenUtils.java index ae0b259dd73cb..830c3b27b0a48 100644 --- a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/keygen/TestKeyGenUtils.java +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/keygen/TestKeyGenUtils.java @@ -18,16 +18,27 @@ package org.apache.hudi.keygen; +import org.apache.hudi.common.table.HoodieTableConfig; import org.apache.hudi.common.util.Option; +import org.apache.hudi.exception.HoodieKeyException; import org.apache.hudi.keygen.constant.KeyGeneratorType; +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericData; +import org.apache.avro.generic.GenericRecord; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; import java.util.ArrayList; +import java.util.Arrays; import java.util.List; +import static org.apache.hudi.common.table.HoodieTableConfig.KEY_GENERATOR_CLASS_NAME; +import static org.apache.hudi.common.table.HoodieTableConfig.RECORDKEY_FIELDS; import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; public class TestKeyGenUtils { @@ -87,4 +98,109 @@ public void testExtractRecordKeysWithFields() { String[] s2 = KeyGenUtils.extractRecordKeysByFields("id1:1,id2:2,2,id3:3", fields); Assertions.assertArrayEquals(new String[] {"2", "2"}, s2); } + + @Test + void testGetRecordKey() { + Schema nullableStringSchema = Schema.createUnion(Schema.create(Schema.Type.NULL), Schema.create(Schema.Type.STRING)); + Schema schema = Schema.createRecord("TestRecord", "doc", "test", false, + Arrays.asList( + new Schema.Field("key1", nullableStringSchema, "", null), + new Schema.Field("key2", nullableStringSchema, "", null), + new Schema.Field("key3", nullableStringSchema, "", null), + new Schema.Field("key4", nullableStringSchema, "", null) + )); + GenericRecord avroRecord = new GenericData.Record(schema); + avroRecord.put("key1", "value1"); + avroRecord.put("key2", "value2"); + avroRecord.put("key3", null); + avroRecord.put("key4", ""); + + assertEquals("key1:value1", + KeyGenUtils.getRecordKey(avroRecord, Arrays.asList("key1"), true)); + assertThrows(HoodieKeyException.class, + () -> KeyGenUtils.getRecordKey(avroRecord, Arrays.asList("key3"), true), + "recordKey values: \"key3:__null__\" for fields: [key3] cannot be entirely null or empty."); + assertThrows(HoodieKeyException.class, + () -> KeyGenUtils.getRecordKey(avroRecord, Arrays.asList("key4"), true), + "recordKey values: \"key4:__empty__\" for fields: [key4] cannot be entirely null or empty."); + assertEquals("key1:value1,key2:value2", + KeyGenUtils.getRecordKey(avroRecord, Arrays.asList("key1", "key2"), true)); + assertEquals("key1:value1,key3:__null__", + KeyGenUtils.getRecordKey(avroRecord, Arrays.asList("key1", "key3"), true)); + assertEquals("key1:value1,key4:__empty__", + KeyGenUtils.getRecordKey(avroRecord, Arrays.asList("key1", "key4"), true)); + + assertEquals("value1", + KeyGenUtils.getRecordKey(avroRecord, "key1", true)); + assertThrows(HoodieKeyException.class, + () -> KeyGenUtils.getRecordKey(avroRecord, "key3", true), + "recordKey value: \"null\" for field: \"key3\" cannot be null or empty."); + assertThrows(HoodieKeyException.class, + () -> KeyGenUtils.getRecordKey(avroRecord, "key4", true), + "recordKey value: \"\" for field: \"key4\" cannot be null or empty."); + } + + @Test + void testIsComplexKeyGeneratorWithSingleRecordKeyField() { + HoodieTableConfig tableConfig = new HoodieTableConfig(); + tableConfig.setValue(KEY_GENERATOR_CLASS_NAME, "org.apache.hudi.keygen.ComplexKeyGenerator"); + tableConfig.setValue(RECORDKEY_FIELDS, "id"); + assertTrue(KeyGenUtils.isComplexKeyGeneratorWithSingleRecordKeyField(tableConfig)); + + tableConfig = new HoodieTableConfig(); + tableConfig.setValue(RECORDKEY_FIELDS, "userId"); + tableConfig.setValue(KEY_GENERATOR_CLASS_NAME, "org.apache.hudi.keygen.ComplexAvroKeyGenerator"); + assertTrue(KeyGenUtils.isComplexKeyGeneratorWithSingleRecordKeyField(tableConfig)); + } + + @Test + void testIsComplexKeyGeneratorWithSingleRecordKeyFieldOnMultipleFields() { + HoodieTableConfig tableConfig = new HoodieTableConfig(); + tableConfig.setValue(KEY_GENERATOR_CLASS_NAME, "org.apache.hudi.keygen.ComplexKeyGenerator"); + tableConfig.setValue(RECORDKEY_FIELDS, "id,userId"); + assertFalse(KeyGenUtils.isComplexKeyGeneratorWithSingleRecordKeyField(tableConfig)); + + tableConfig = new HoodieTableConfig(); + tableConfig.setValue(KEY_GENERATOR_CLASS_NAME, "org.apache.hudi.keygen.ComplexAvroKeyGenerator"); + tableConfig.setValue(RECORDKEY_FIELDS, "id,userId,name"); + assertFalse(KeyGenUtils.isComplexKeyGeneratorWithSingleRecordKeyField(tableConfig)); + } + + @Test + void testIsComplexKeyGeneratorWithSingleRecordKeyFieldOnNonComplexGenerator() { + HoodieTableConfig tableConfig = new HoodieTableConfig(); + tableConfig.setValue(KEY_GENERATOR_CLASS_NAME, "org.apache.hudi.keygen.SimpleKeyGenerator"); + tableConfig.setValue(RECORDKEY_FIELDS, "id"); + assertFalse(KeyGenUtils.isComplexKeyGeneratorWithSingleRecordKeyField(tableConfig)); + + tableConfig = new HoodieTableConfig(); + tableConfig.setValue(KEY_GENERATOR_CLASS_NAME, "org.apache.hudi.keygen.SimpleAvroKeyGenerator"); + tableConfig.setValue(RECORDKEY_FIELDS, "userId"); + assertFalse(KeyGenUtils.isComplexKeyGeneratorWithSingleRecordKeyField(tableConfig)); + + tableConfig = new HoodieTableConfig(); + tableConfig.setValue(KEY_GENERATOR_CLASS_NAME, "org.apache.hudi.keygen.TimestampBasedKeyGenerator"); + tableConfig.setValue(RECORDKEY_FIELDS, "id"); + assertFalse(KeyGenUtils.isComplexKeyGeneratorWithSingleRecordKeyField(tableConfig)); + + tableConfig = new HoodieTableConfig(); + tableConfig.setValue(KEY_GENERATOR_CLASS_NAME, "org.apache.hudi.keygen.CustomKeyGenerator"); + tableConfig.setValue(RECORDKEY_FIELDS, "id"); + assertFalse(KeyGenUtils.isComplexKeyGeneratorWithSingleRecordKeyField(tableConfig)); + } + + @Test + void testIsComplexKeyGeneratorWithSingleRecordKeyFieldOnNoRecordKeyFields() { + HoodieTableConfig tableConfig = new HoodieTableConfig(); + tableConfig.setValue(KEY_GENERATOR_CLASS_NAME, "org.apache.hudi.keygen.ComplexKeyGenerator"); + assertFalse(KeyGenUtils.isComplexKeyGeneratorWithSingleRecordKeyField(tableConfig)); + } + + @Test + void testIsComplexKeyGeneratorWithSingleRecordKeyFieldEmptyRecordKeyFields() { + HoodieTableConfig tableConfig = new HoodieTableConfig(); + tableConfig.setValue(KEY_GENERATOR_CLASS_NAME, "org.apache.hudi.keygen.ComplexKeyGenerator"); + tableConfig.setValue(RECORDKEY_FIELDS, ""); + assertFalse(KeyGenUtils.isComplexKeyGeneratorWithSingleRecordKeyField(tableConfig)); + } } diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/testutils/Assertions.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/testutils/Assertions.java index bb2ba84f8c9bd..cb4ef64289deb 100644 --- a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/testutils/Assertions.java +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/testutils/Assertions.java @@ -21,12 +21,17 @@ import org.apache.hudi.client.WriteStatus; import org.apache.hudi.common.testutils.CheckedFunction; +import org.apache.hudi.exception.HoodieException; + +import org.junit.jupiter.api.function.Executable; import java.util.List; +import static org.apache.hudi.keygen.KeyGenUtils.getComplexKeygenErrorMessage; import static org.junit.jupiter.api.Assertions.assertAll; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertThrows; /** * Commonly used assertion functions. @@ -51,4 +56,8 @@ public static void assertFileSizesEqual(List statuses, CheckedFunct assertEquals(fileSizeGetter.apply(status), status.getStat().getFileSizeInBytes()))); } + public static void assertComplexKeyGeneratorValidationThrows(Executable writeOperation, String operation) { + HoodieException exception = assertThrows(HoodieException.class, writeOperation); + assertEquals(getComplexKeygenErrorMessage(operation), exception.getMessage()); + } } diff --git a/hudi-client/hudi-spark-client/pom.xml b/hudi-client/hudi-spark-client/pom.xml index fa437494fd9f5..e2e7191038cb5 100644 --- a/hudi-client/hudi-spark-client/pom.xml +++ b/hudi-client/hudi-spark-client/pom.xml @@ -253,6 +253,23 @@ org.apache.rat apache-rat-plugin + + org.apache.maven.plugins + maven-surefire-plugin + + + + **/TestSparkHoodieHBaseIndex.java + **/TestHoodieSparkMergeOnReadTableRollback.java + **/TestHoodieBackedMetadata.java + **/TestHoodieSparkCopyOnWriteTableArchiveWithReplace.java + **/TestHoodieClientOnCopyOnWriteStorage.java + **/TestSavepointRestoreMergeOnRead.java + **/TestHoodieIndex.java + **/TestConsistentBucketIndex.java + + + @@ -264,4 +281,31 @@ + + + + + isolated-tests + + + + org.apache.maven.plugins + maven-surefire-plugin + + + **/TestSparkHoodieHBaseIndex.java + **/TestHoodieSparkMergeOnReadTableRollback.java + **/TestHoodieBackedMetadata.java + **/TestHoodieSparkCopyOnWriteTableArchiveWithReplace.java + **/TestHoodieClientOnCopyOnWriteStorage.java + **/TestSavepointRestoreMergeOnRead.java + **/TestHoodieIndex.java + **/TestConsistentBucketIndex.java + + + + + + + diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/BuiltinKeyGenerator.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/BuiltinKeyGenerator.java index 58350b0d49460..3fdb8ef065e8d 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/BuiltinKeyGenerator.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/BuiltinKeyGenerator.java @@ -51,7 +51,7 @@ import scala.Function1; import static org.apache.hudi.common.util.CollectionUtils.tail; -import static org.apache.hudi.keygen.KeyGenUtils.DEFAULT_COMPOSITE_KEY_FILED_VALUE; +import static org.apache.hudi.keygen.KeyGenUtils.DEFAULT_COLUMN_VALUE_SEPARATOR; import static org.apache.hudi.keygen.KeyGenUtils.DEFAULT_RECORD_KEY_PARTS_SEPARATOR; import static org.apache.hudi.keygen.KeyGenUtils.EMPTY_RECORDKEY_PLACEHOLDER; import static org.apache.hudi.keygen.KeyGenUtils.NULL_RECORDKEY_PLACEHOLDER; @@ -71,8 +71,6 @@ public abstract class BuiltinKeyGenerator extends BaseKeyGenerator implements Sp private static final Logger LOG = LoggerFactory.getLogger(BuiltinKeyGenerator.class); - private static final String COMPOSITE_KEY_FIELD_VALUE_INFIX = ":"; - protected static final String FIELDS_SEP = ","; protected static final UTF8String NULL_RECORD_KEY_PLACEHOLDER_UTF8 = UTF8String.fromString(NULL_RECORDKEY_PLACEHOLDER); @@ -183,8 +181,10 @@ protected final UTF8String combineRecordKeyUnsafe(List fieldNames, List< * NOTE: This method has to stay final (so that it's easier for JIT compiler to apply certain * optimizations, like inlining) */ - protected final String combineCompositeRecordKey(Object... recordKeyParts) { + protected final String combineCompositeRecordKey(boolean encodeSingleKeyFieldName, + Object... recordKeyParts) { return combineCompositeRecordKeyInternal( + encodeSingleKeyFieldName, StringPartitionPathFormatter.JavaStringBuilder::new, BuiltinKeyGenerator::toString, BuiltinKeyGenerator::handleNullOrEmptyCompositeKeyPart, @@ -197,8 +197,10 @@ protected final String combineCompositeRecordKey(Object... recordKeyParts) { * NOTE: This method has to stay final (so that it's easier for JIT compiler to apply certain * optimizations, like inlining) */ - protected final UTF8String combineCompositeRecordKeyUnsafe(Object... recordKeyParts) { + protected final UTF8String combineCompositeRecordKeyUnsafe(boolean encodeSingleKeyFieldName, + Object... recordKeyParts) { return combineCompositeRecordKeyInternal( + encodeSingleKeyFieldName, UTF8StringPartitionPathFormatter.UTF8StringBuilder::new, BuiltinKeyGenerator::toUTF8String, BuiltinKeyGenerator::handleNullOrEmptyCompositeKeyPartUTF8, @@ -220,7 +222,7 @@ private S combineRecordKeyInternal( PartitionPathFormatterBase.StringBuilder 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_COLUMN_VALUE_SEPARATOR); // NOTE: If record-key part has already been a string [[toString]] will be a no-op sb.append(emptyKeyPartHandler.apply(converter.apply(recordKeyParts.get(i)))); @@ -233,23 +235,43 @@ private S combineRecordKeyInternal( } private S combineCompositeRecordKeyInternal( + boolean encodeSingleKeyFieldName, Supplier> builderFactory, Function converter, Function emptyKeyPartHandler, Predicate isNullOrEmptyKeyPartPredicate, Object... recordKeyParts ) { - boolean hasNonNullNonEmptyPart = false; + if (recordKeyParts.length == 0) { + throw new HoodieKeyException(String.format("All of the values for (%s) were either null or empty", recordKeyFields)); + } PartitionPathFormatterBase.StringBuilder sb = builderFactory.get(); + + if (recordKeyParts.length == 1) { + // NOTE: If record-key part has already been a string [[toString]] will be a no-op + S convertedKeyPart = emptyKeyPartHandler.apply(converter.apply(recordKeyParts[0])); + + if (encodeSingleKeyFieldName) { + sb.appendJava(recordKeyFields.get(0)); + sb.appendJava(DEFAULT_COLUMN_VALUE_SEPARATOR); + } + sb.append(convertedKeyPart); + // This check is to validate that overall composite-key has at least one non-null, non-empty + // segment + if (isNullOrEmptyKeyPartPredicate.test(convertedKeyPart)) { + throw new HoodieKeyException(String.format("All of the values for (%s) were either null or empty", recordKeyFields)); + } + return sb.build(); + } + + boolean hasNonNullNonEmptyPart = false; for (int i = 0; i < recordKeyParts.length; ++i) { // NOTE: If record-key part has already been a string [[toString]] will be a no-op S convertedKeyPart = emptyKeyPartHandler.apply(converter.apply(recordKeyParts[i])); - if (recordKeyParts.length > 1) { - sb.appendJava(recordKeyFields.get(i)); - sb.appendJava(COMPOSITE_KEY_FIELD_VALUE_INFIX); - } + sb.appendJava(recordKeyFields.get(i)); + sb.appendJava(DEFAULT_COLUMN_VALUE_SEPARATOR); sb.append(convertedKeyPart); // This check is to validate that overall composite-key has at least one non-null, non-empty // segment diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/ComplexKeyGenerator.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/ComplexKeyGenerator.java index d00ca066cedde..67be836f79fb9 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/ComplexKeyGenerator.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/ComplexKeyGenerator.java @@ -38,6 +38,7 @@ public class ComplexKeyGenerator extends BuiltinKeyGenerator { private final ComplexAvroKeyGenerator complexAvroKeyGenerator; + private final boolean encodeSingleKeyFieldName; public ComplexKeyGenerator(TypedProperties props) { super(props); @@ -47,6 +48,7 @@ public ComplexKeyGenerator(TypedProperties props) { .filter(s -> !s.isEmpty()) .collect(Collectors.toList()); this.complexAvroKeyGenerator = new ComplexAvroKeyGenerator(props); + this.encodeSingleKeyFieldName = KeyGenUtils.encodeSingleKeyFieldNameForComplexKeyGen(props); } @Override @@ -62,13 +64,15 @@ public String getPartitionPath(GenericRecord record) { @Override public String getRecordKey(Row row) { tryInitRowAccessor(row.schema()); - return combineCompositeRecordKey(rowAccessor.getRecordKeyParts(row)); + return combineCompositeRecordKey( + encodeSingleKeyFieldName, rowAccessor.getRecordKeyParts(row)); } @Override public UTF8String getRecordKey(InternalRow internalRow, StructType schema) { tryInitRowAccessor(schema); - return combineCompositeRecordKeyUnsafe(rowAccessor.getRecordKeyParts(internalRow)); + return combineCompositeRecordKeyUnsafe( + encodeSingleKeyFieldName, rowAccessor.getRecordKeyParts(internalRow)); } @Override diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/GlobalDeleteKeyGenerator.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/GlobalDeleteKeyGenerator.java index 8a22938430751..d4e037a07d96a 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/GlobalDeleteKeyGenerator.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/GlobalDeleteKeyGenerator.java @@ -60,13 +60,13 @@ public String getPartitionPath(GenericRecord record) { @Override public String getRecordKey(Row row) { tryInitRowAccessor(row.schema()); - return combineCompositeRecordKey(rowAccessor.getRecordKeyParts(row)); + return combineCompositeRecordKey(true, rowAccessor.getRecordKeyParts(row)); } @Override public UTF8String getRecordKey(InternalRow internalRow, StructType schema) { tryInitRowAccessor(schema); - return combineCompositeRecordKeyUnsafe(rowAccessor.getRecordKeyParts(internalRow)); + return combineCompositeRecordKeyUnsafe(true, rowAccessor.getRecordKeyParts(internalRow)); } @Override diff --git a/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/util/JavaScalaConverters.scala b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/util/JavaScalaConverters.scala new file mode 100644 index 0000000000000..4a55bc7f2f076 --- /dev/null +++ b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/util/JavaScalaConverters.scala @@ -0,0 +1,72 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.hudi.util + +import scala.collection.JavaConverters._ + +/** + * Utils that do conversion between Java and Scala collections, used by classes in Java code only. + * For classes in Scala code, import `scala.collection.JavaConverters._` directly. + */ +object JavaScalaConverters { + /** + * @param scalaList list in Scala [[Seq]]. + * @tparam A type of item. + * @return list in [[java.util.List]]. + */ + def convertScalaListToJavaList[A](scalaList: Seq[A]): java.util.List[A] = { + scalaList.asJava + } + + /** + * @param javaList list in [[java.util.List]]. + * @tparam A type of item. + * @return list in Scala immutable [[List]]. + */ + def convertJavaListToScalaList[A](javaList: java.util.List[A]): List[A] = { + javaList.asScala.toList + } + + /** + * @param javaList list in [[java.util.List]]. + * @tparam A type of item. + * @return list in Scala [[Seq]]. + */ + def convertJavaListToScalaSeq[A](javaList: java.util.List[A]): Seq[A] = { + javaList.asScala.toSeq + } + + /** + * @param javaIterator iterator in [[java.util.Iterator]] + * @tparam A type of item. + * @return iterator in Scala [[Iterator]]. + */ + def convertJavaIteratorToScalaIterator[A](javaIterator: java.util.Iterator[A]): Iterator[A] = { + javaIterator.asScala + } + + /** + * @param javaProperties properties in [[java.util.Properties]] + * @return map in Scala [[Map]]. + */ + def convertJavaPropertiesToScalaMap(javaProperties: java.util.Properties): Map[String, String] = { + javaProperties.asScala.toMap + } +} diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/keygen/KeyGeneratorTestUtilities.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/keygen/KeyGeneratorTestUtilities.java new file mode 100644 index 0000000000000..29ee6af5a7a94 --- /dev/null +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/keygen/KeyGeneratorTestUtilities.java @@ -0,0 +1,200 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.hudi.keygen; + +import org.apache.hudi.util.JavaScalaConverters; + +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericData; +import org.apache.avro.generic.GenericRecord; +import org.apache.avro.util.Utf8; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.catalyst.expressions.GenericRowWithSchema; +import org.apache.spark.sql.types.ArrayType; +import org.apache.spark.sql.types.DataType; +import org.apache.spark.sql.types.DataTypes; +import org.apache.spark.sql.types.DecimalType; +import org.apache.spark.sql.types.MapType; +import org.apache.spark.sql.types.Metadata; +import org.apache.spark.sql.types.StringType; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; + +import java.math.BigDecimal; +import java.math.BigInteger; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; +import java.util.stream.IntStream; + +class KeyGeneratorTestUtilities { + + public static final String NESTED_COL_SCHEMA = "{\"type\":\"record\", \"name\":\"nested_col\",\"fields\": [" + + "{\"name\": \"prop1\",\"type\": [\"null\", \"string\"]},{\"name\": \"prop2\", \"type\": \"long\"}]}"; + public static final String EXAMPLE_SCHEMA = "{\"type\": \"record\",\"name\": \"testrec\",\"fields\": [ " + + "{\"name\": \"timestamp\",\"type\": \"long\"},{\"name\": \"_row_key\", \"type\": \"string\"}," + + "{\"name\": \"ts_ms\", \"type\": \"string\"}," + + "{\"name\": \"pii_col\", \"type\": \"string\"}," + + "{\"name\": \"nested_col\",\"type\": [\"null\", " + NESTED_COL_SCHEMA + "]}" + + "]}"; + + private static final StructType FARE_STRUCT_TYPE = new StructType(new StructField[] { + new StructField("amount", DataTypes.DoubleType, false, Metadata.empty()), + new StructField("currency", DataTypes.StringType, false, Metadata.empty()) + }); + protected static final StructType TRIP_STRUCT_TYPE = new StructType(new StructField[] { + new StructField("timestamp", DataTypes.LongType, false, Metadata.empty()), + new StructField("_row_key", DataTypes.StringType, false, Metadata.empty()), + new StructField("partition_path", DataTypes.StringType, true, Metadata.empty()), + new StructField("trip_type", DataTypes.StringType, false, Metadata.empty()), + new StructField("rider", DataTypes.StringType, false, Metadata.empty()), + new StructField("driver", DataTypes.StringType, false, Metadata.empty()), + new StructField("begin_lat", DataTypes.DoubleType, false, Metadata.empty()), + new StructField("begin_lon", DataTypes.DoubleType, false, Metadata.empty()), + new StructField("end_lat", DataTypes.DoubleType, false, Metadata.empty()), + new StructField("end_lon", DataTypes.DoubleType, false, Metadata.empty()), + new StructField("distance_in_meters", DataTypes.IntegerType, false, Metadata.empty()), + new StructField("seconds_since_epoch", DataTypes.LongType, false, Metadata.empty()), + new StructField("weight", DataTypes.FloatType, false, Metadata.empty()), + new StructField("nation", DataTypes.BinaryType, false, Metadata.empty()), + new StructField("current_date", DataTypes.DateType, false, Metadata.empty()), + new StructField("current_ts", DataTypes.LongType, false, Metadata.empty()), + new StructField("height", new DecimalType(10, 6), false, Metadata.empty()), + new StructField("city_to_state", DataTypes.createMapType( + DataTypes.StringType, DataTypes.StringType, false), false, Metadata.empty()), + new StructField("fare", FARE_STRUCT_TYPE, false, Metadata.empty()), + new StructField("tip_history", DataTypes.createArrayType(FARE_STRUCT_TYPE, false), false, Metadata.empty()), + new StructField("_hoodie_is_deleted", DataTypes.BooleanType, false, Metadata.empty()) + }); + private static final StructType NESTED_TYPE = new StructType(new StructField[] { + new StructField("prop1", DataTypes.StringType, true, Metadata.empty()), + new StructField("prop2", DataTypes.LongType, false, Metadata.empty()) + }); + private static final StructType STRUCT_TYPE = new StructType(new StructField[] { + new StructField("timestamp", DataTypes.LongType, false, Metadata.empty()), + new StructField("_row_key", DataTypes.StringType, false, Metadata.empty()), + new StructField("ts_ms", DataTypes.StringType, false, Metadata.empty()), + new StructField("pii_col", DataTypes.StringType, false, Metadata.empty()), + new StructField("nested_col", NESTED_TYPE, true, Metadata.empty()) + }); + + public static GenericRecord getRecord() { + return getRecord(getNestedColRecord("val1", 10L)); + } + + public static GenericRecord getNestedColRecord(String prop1Value, Long prop2Value) { + GenericRecord nestedColRecord = new GenericData.Record(new Schema.Parser().parse(NESTED_COL_SCHEMA)); + nestedColRecord.put("prop1", prop1Value); + nestedColRecord.put("prop2", prop2Value); + return nestedColRecord; + } + + public static GenericRecord getRecord(GenericRecord nestedColRecord) { + GenericRecord avroRecord = new GenericData.Record(new Schema.Parser().parse(EXAMPLE_SCHEMA)); + avroRecord.put("timestamp", 4357686L); + avroRecord.put("_row_key", "key1"); + avroRecord.put("ts_ms", "2020-03-21"); + avroRecord.put("pii_col", "pi"); + avroRecord.put("nested_col", nestedColRecord); + return avroRecord; + } + + public static Row getRow(GenericRecord genericRecord) { + return getRow(genericRecord, STRUCT_TYPE); + } + + public static Row getRow(GenericRecord genericRecord, StructType structType) { + Row row = genericRecordToRow(genericRecord, structType); + int fieldCount = structType.fieldNames().length; + Object[] values = new Object[fieldCount]; + for (int i = 0; i < fieldCount; i++) { + values[i] = row.get(i); + } + return new GenericRowWithSchema(values, structType); + } + + public static InternalRow getInternalRow(Row row) { + List values = IntStream.range(0, row.schema().fieldNames().length) + .mapToObj(row::get).collect(Collectors.toList()); + return InternalRow.apply(JavaScalaConverters.convertJavaListToScalaList(values)); + } + + /** + * Converts an Avro GenericRecord to a Spark SQL Row based on a given schema. + * Only used by key generator tests. + * + * @param genericRecord the input Avro GenericRecord + * @param schema the target Spark SQL StructType that defines the structure of the output Row + * @return a Spark SQL Row containing the data from the GenericRecord + */ + public static Row genericRecordToRow(GenericRecord genericRecord, StructType schema) { + StructField[] fields = schema.fields(); + Object[] values = new Object[fields.length]; + + for (int i = 0; i < fields.length; i++) { + StructField field = fields[i]; + Object avroValue = genericRecord.get(field.name()); + values[i] = convertAvroValue(avroValue, field.dataType()); + } + + return new GenericRowWithSchema(values, schema); + } + + /** + * Recursively converts an Avro value to its corresponding Spark SQL data type. + * Only used by key generator tests. + * + * @param value the value to convert + * @param dataType the target Spark SQL DataType + * @return the converted value compatible with Spark SQL + */ + private static Object convertAvroValue(Object value, DataType dataType) { + if (value == null) { + return null; + } + + if (dataType instanceof StringType + && (value instanceof Utf8 || value instanceof GenericData.EnumSymbol || value instanceof String)) { + return value.toString(); + } else if (dataType instanceof DecimalType && value instanceof GenericData.Fixed) { + DecimalType decimalType = (DecimalType) dataType; + byte[] bytes = ((GenericData.Fixed) value).bytes(); + return new BigDecimal(new BigInteger(bytes), decimalType.scale()); + } else if (dataType instanceof StructType && value instanceof GenericRecord) { + return genericRecordToRow((GenericRecord) value, (StructType) dataType); + } else if (dataType instanceof ArrayType && value instanceof List) { + ArrayType arrayType = (ArrayType) dataType; + List avroList = (List) value; + return avroList.stream() + .map(item -> convertAvroValue(item, arrayType.elementType())) + .collect(Collectors.toList()); + } else if (dataType instanceof MapType && value instanceof Map) { + MapType mapType = (MapType) dataType; + Map avroMap = (Map) value; + return avroMap.entrySet().stream() + .collect(Collectors.toMap( + e -> e.getKey().toString(), // Convert key to String + e -> convertAvroValue(e.getValue(), mapType.valueType()) + )); + } + return value; + } +} \ No newline at end of file diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/keygen/TestComplexKeyGenerator.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/keygen/TestComplexKeyGenerator.java new file mode 100644 index 0000000000000..5ea3c28def6f9 --- /dev/null +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/keygen/TestComplexKeyGenerator.java @@ -0,0 +1,214 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.hudi.keygen; + +import org.apache.hudi.common.config.TypedProperties; +import org.apache.hudi.common.model.HoodieKey; +import org.apache.hudi.common.testutils.HoodieTestDataGenerator; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.exception.HoodieKeyException; +import org.apache.hudi.keygen.constant.KeyGeneratorOptions; + +import org.apache.avro.generic.GenericRecord; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.unsafe.types.UTF8String; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.CsvSource; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertThrows; + +public class TestComplexKeyGenerator extends KeyGeneratorTestUtilities { + + private TypedProperties getCommonProps(boolean getComplexRecordKey) { + TypedProperties properties = new TypedProperties(); + if (getComplexRecordKey) { + properties.put(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key(), "_row_key, pii_col"); + } else { + properties.put(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key(), "_row_key"); + } + properties.put(KeyGeneratorOptions.HIVE_STYLE_PARTITIONING_ENABLE.key(), "true"); + return properties; + } + + private TypedProperties getPropertiesWithoutPartitionPathProp() { + return getCommonProps(false); + } + + private TypedProperties getPropertiesWithoutRecordKeyProp() { + TypedProperties properties = new TypedProperties(); + properties.put(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key(), "timestamp"); + return properties; + } + + private TypedProperties getWrongRecordKeyFieldProps() { + TypedProperties properties = new TypedProperties(); + properties.put(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key(), "timestamp"); + properties.put(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key(), "_wrong_key"); + return properties; + } + + private TypedProperties getProps() { + TypedProperties properties = getCommonProps(true); + properties.put(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key(), "timestamp,ts_ms"); + return properties; + } + + @Test + void testNullPartitionPathFields() { + assertThrows(IllegalArgumentException.class, () -> new ComplexKeyGenerator(getPropertiesWithoutPartitionPathProp())); + } + + @Test + void testNullRecordKeyFields() { + GenericRecord avroRecord = getRecord(); + assertThrows(StringIndexOutOfBoundsException.class, () -> { + ComplexKeyGenerator keyGenerator = new ComplexKeyGenerator(getPropertiesWithoutRecordKeyProp()); + keyGenerator.getRecordKey(avroRecord); + }); + } + + @Test + void testWrongRecordKeyField() { + ComplexKeyGenerator keyGenerator = new ComplexKeyGenerator(getWrongRecordKeyFieldProps()); + assertThrows(HoodieKeyException.class, () -> keyGenerator.getRecordKey(getRecord())); + } + + @Test + void testHappyFlow() { + ComplexKeyGenerator keyGenerator = new ComplexKeyGenerator(getProps()); + GenericRecord avroRecord = getRecord(); + HoodieKey key = keyGenerator.getKey(avroRecord); + assertEquals("_row_key:key1,pii_col:pi", key.getRecordKey()); + assertEquals("timestamp=4357686/ts_ms=2020-03-21", key.getPartitionPath()); + Row row = KeyGeneratorTestUtilities.getRow(avroRecord); + assertEquals("_row_key:key1,pii_col:pi", keyGenerator.getRecordKey(row)); + assertEquals("timestamp=4357686/ts_ms=2020-03-21", keyGenerator.getPartitionPath(row)); + + InternalRow internalRow = KeyGeneratorTestUtilities.getInternalRow(row); + assertEquals(UTF8String.fromString("timestamp=4357686/ts_ms=2020-03-21"), keyGenerator.getPartitionPath(internalRow, row.schema())); + } + + @ParameterizedTest + @CsvSource(value = {"false,true", "true,false", "true,true"}) + void testSingleValueKeyGenerator(boolean setNewEncodingConfig, + boolean encodeSingleKeyFieldValueOnly) { + String recordKeyFieldName = "_row_key"; + TypedProperties properties = new TypedProperties(); + properties.setProperty(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key(), recordKeyFieldName); + properties.setProperty(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key(), "timestamp"); + if (setNewEncodingConfig) { + properties.setProperty( + HoodieWriteConfig.COMPLEX_KEYGEN_NEW_ENCODING.key(), + String.valueOf(encodeSingleKeyFieldValueOnly)); + } + ComplexKeyGenerator compositeKeyGenerator = new ComplexKeyGenerator(properties); + assertEquals(1, compositeKeyGenerator.getRecordKeyFieldNames().size()); + assertEquals(1, compositeKeyGenerator.getPartitionPathFields().size()); + try (HoodieTestDataGenerator dataGenerator = new HoodieTestDataGenerator(System.currentTimeMillis())) { + GenericRecord avroRecord = dataGenerator.generateGenericRecords(1).get(0); + String rowKey = avroRecord.get(recordKeyFieldName).toString(); + String partitionPath = avroRecord.get("timestamp").toString(); + HoodieKey hoodieKey = compositeKeyGenerator.getKey(avroRecord); + // For table version 9, new encoding config should have no effect + // Table version 8 may use new encoding config if set + String expectedRecordKey = setNewEncodingConfig && encodeSingleKeyFieldValueOnly + ? rowKey : recordKeyFieldName + ":" + rowKey; + assertEquals(expectedRecordKey, hoodieKey.getRecordKey()); + assertEquals(partitionPath, hoodieKey.getPartitionPath()); + + Row row = KeyGeneratorTestUtilities.getRow(avroRecord, TRIP_STRUCT_TYPE); + assertEquals(partitionPath, compositeKeyGenerator.getPartitionPath(row)); + InternalRow internalRow = KeyGeneratorTestUtilities.getInternalRow(row); + assertEquals(UTF8String.fromString(expectedRecordKey), compositeKeyGenerator.getRecordKey(internalRow, TRIP_STRUCT_TYPE)); + assertEquals(UTF8String.fromString(partitionPath), compositeKeyGenerator.getPartitionPath(internalRow, row.schema())); + } + } + + @ParameterizedTest + @CsvSource(value = {"false,true", "true,false", "true,true"}) + void testMultipleValueKeyGenerator(boolean setNewEncodingConfig, + boolean encodeSingleKeyFieldValueOnly) { + TypedProperties properties = new TypedProperties(); + properties.setProperty(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key(), "_row_key,timestamp"); + properties.setProperty(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key(), "rider,driver"); + if (setNewEncodingConfig) { + properties.setProperty( + HoodieWriteConfig.COMPLEX_KEYGEN_NEW_ENCODING.key(), + String.valueOf(encodeSingleKeyFieldValueOnly)); + } + ComplexKeyGenerator compositeKeyGenerator = new ComplexKeyGenerator(properties); + assertEquals(2, compositeKeyGenerator.getRecordKeyFieldNames().size()); + assertEquals(2, compositeKeyGenerator.getPartitionPathFields().size()); + try (HoodieTestDataGenerator dataGenerator = new HoodieTestDataGenerator(System.currentTimeMillis())) { + GenericRecord avroRecord = dataGenerator.generateGenericRecords(1).get(0); + String rowKey = + "_row_key" + ComplexAvroKeyGenerator.DEFAULT_RECORD_KEY_SEPARATOR + avroRecord.get("_row_key").toString() + "," + + "timestamp" + ComplexAvroKeyGenerator.DEFAULT_RECORD_KEY_SEPARATOR + avroRecord.get("timestamp").toString(); + String partitionPath = avroRecord.get("rider").toString() + "/" + avroRecord.get("driver").toString(); + HoodieKey hoodieKey = compositeKeyGenerator.getKey(avroRecord); + assertEquals(rowKey, hoodieKey.getRecordKey()); + assertEquals(partitionPath, hoodieKey.getPartitionPath()); + + Row row = KeyGeneratorTestUtilities.getRow(avroRecord, TRIP_STRUCT_TYPE); + assertEquals(partitionPath, compositeKeyGenerator.getPartitionPath(row)); + + InternalRow internalRow = KeyGeneratorTestUtilities.getInternalRow(row); + assertEquals(UTF8String.fromString(rowKey), compositeKeyGenerator.getRecordKey(internalRow, TRIP_STRUCT_TYPE)); + assertEquals(UTF8String.fromString(partitionPath), compositeKeyGenerator.getPartitionPath(internalRow, row.schema())); + } + } + + @ParameterizedTest + @CsvSource(value = {"false,true", "true,false", "true,true"}) + void testMultipleValueKeyGeneratorNonPartitioned(boolean setNewEncodingConfig, + boolean encodeSingleKeyFieldValueOnly) { + TypedProperties properties = new TypedProperties(); + properties.setProperty(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key(), "_row_key,timestamp"); + properties.setProperty(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key(), ""); + if (setNewEncodingConfig) { + properties.setProperty( + HoodieWriteConfig.COMPLEX_KEYGEN_NEW_ENCODING.key(), + String.valueOf(encodeSingleKeyFieldValueOnly)); + } + ComplexKeyGenerator compositeKeyGenerator = new ComplexKeyGenerator(properties); + assertEquals(2, compositeKeyGenerator.getRecordKeyFieldNames().size()); + assertEquals(0, compositeKeyGenerator.getPartitionPathFields().size()); + try (HoodieTestDataGenerator dataGenerator = new HoodieTestDataGenerator(System.currentTimeMillis())) { + GenericRecord avroRecord = dataGenerator.generateGenericRecords(1).get(0); + String rowKey = + "_row_key" + ComplexAvroKeyGenerator.DEFAULT_RECORD_KEY_SEPARATOR + avroRecord.get("_row_key").toString() + "," + + "timestamp" + ComplexAvroKeyGenerator.DEFAULT_RECORD_KEY_SEPARATOR + avroRecord.get("timestamp").toString(); + String partitionPath = ""; + HoodieKey hoodieKey = compositeKeyGenerator.getKey(avroRecord); + assertEquals(rowKey, hoodieKey.getRecordKey()); + assertEquals(partitionPath, hoodieKey.getPartitionPath()); + + Row row = KeyGeneratorTestUtilities.getRow(avroRecord, TRIP_STRUCT_TYPE); + assertEquals(partitionPath, compositeKeyGenerator.getPartitionPath(row)); + + InternalRow internalRow = KeyGeneratorTestUtilities.getInternalRow(row); + assertEquals(UTF8String.fromString(rowKey), compositeKeyGenerator.getRecordKey(internalRow, TRIP_STRUCT_TYPE)); + assertEquals(UTF8String.fromString(partitionPath), compositeKeyGenerator.getPartitionPath(internalRow, row.schema())); + } + } +} diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/keygen/TestCustomKeyGenerator.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/keygen/TestCustomKeyGenerator.java similarity index 65% rename from hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/keygen/TestCustomKeyGenerator.java rename to hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/keygen/TestCustomKeyGenerator.java index 0ba8d1425e725..171f516e52614 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/keygen/TestCustomKeyGenerator.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/keygen/TestCustomKeyGenerator.java @@ -24,20 +24,22 @@ import org.apache.hudi.keygen.constant.KeyGeneratorOptions; import org.apache.hudi.keygen.constant.KeyGeneratorType; import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory; -import org.apache.hudi.testutils.KeyGeneratorTestUtilities; import org.apache.avro.generic.GenericRecord; import org.apache.spark.sql.Row; import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.unsafe.types.UTF8String; -import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; import java.io.IOException; import java.io.PrintWriter; import java.io.StringWriter; -public class TestCustomKeyGenerator extends KeyGeneratorTestUtilities { +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assertions.fail; + +class TestCustomKeyGenerator extends KeyGeneratorTestUtilities { /** * Method to create props used for common cases. @@ -133,36 +135,36 @@ private String stackTraceToString(Throwable e) { } @Test - public void testSimpleKeyGeneratorWithKeyGeneratorClass() throws IOException { + void testSimpleKeyGeneratorWithKeyGeneratorClass() throws IOException { testSimpleKeyGenerator(getPropertiesForSimpleKeyGen(true)); } @Test - public void testSimpleKeyGeneratorWithKeyGeneratorType() throws IOException { + void testSimpleKeyGeneratorWithKeyGeneratorType() throws IOException { testSimpleKeyGenerator(getPropertiesForSimpleKeyGen(false)); } public void testSimpleKeyGenerator(TypedProperties props) throws IOException { BuiltinKeyGenerator keyGenerator = (BuiltinKeyGenerator) HoodieSparkKeyGeneratorFactory.createKeyGenerator(props); - GenericRecord record = getRecord(); - HoodieKey key = keyGenerator.getKey(record); - Assertions.assertEquals("key1", key.getRecordKey()); - Assertions.assertEquals("timestamp=4357686", key.getPartitionPath()); - Row row = KeyGeneratorTestUtilities.getRow(record); - Assertions.assertEquals("key1", keyGenerator.getRecordKey(row)); - Assertions.assertEquals("timestamp=4357686", keyGenerator.getPartitionPath(row)); + GenericRecord avroRecord = getRecord(); + HoodieKey key = keyGenerator.getKey(avroRecord); + assertEquals("key1", key.getRecordKey()); + assertEquals("timestamp=4357686", key.getPartitionPath()); + Row row = KeyGeneratorTestUtilities.getRow(avroRecord); + assertEquals("key1", keyGenerator.getRecordKey(row)); + assertEquals("timestamp=4357686", keyGenerator.getPartitionPath(row)); InternalRow internalRow = KeyGeneratorTestUtilities.getInternalRow(row); - Assertions.assertEquals(UTF8String.fromString("timestamp=4357686"), keyGenerator.getPartitionPath(internalRow, row.schema())); + assertEquals(UTF8String.fromString("timestamp=4357686"), keyGenerator.getPartitionPath(internalRow, row.schema())); } @Test - public void testTimestampBasedKeyGeneratorWithKeyGeneratorClass() throws IOException { + void testTimestampBasedKeyGeneratorWithKeyGeneratorClass() throws IOException { testTimestampBasedKeyGenerator(getPropertiesForTimestampBasedKeyGen(true)); } @Test - public void testTimestampBasedKeyGeneratorWithKeyGeneratorType() throws IOException { + void testTimestampBasedKeyGeneratorWithKeyGeneratorType() throws IOException { testTimestampBasedKeyGenerator(getPropertiesForTimestampBasedKeyGen(false)); } @@ -170,24 +172,24 @@ public void testTimestampBasedKeyGenerator(TypedProperties props) throws IOExcep BuiltinKeyGenerator keyGenerator = (BuiltinKeyGenerator) HoodieSparkKeyGeneratorFactory.createKeyGenerator(props); - GenericRecord record = getRecord(); - HoodieKey key = keyGenerator.getKey(record); - Assertions.assertEquals("key1", key.getRecordKey()); - Assertions.assertEquals("ts_ms=20200321", key.getPartitionPath()); - Row row = KeyGeneratorTestUtilities.getRow(record); - Assertions.assertEquals("key1", keyGenerator.getRecordKey(row)); - Assertions.assertEquals("ts_ms=20200321", keyGenerator.getPartitionPath(row)); + GenericRecord avroRecord = getRecord(); + HoodieKey key = keyGenerator.getKey(avroRecord); + assertEquals("key1", key.getRecordKey()); + assertEquals("ts_ms=20200321", key.getPartitionPath()); + Row row = KeyGeneratorTestUtilities.getRow(avroRecord); + assertEquals("key1", keyGenerator.getRecordKey(row)); + assertEquals("ts_ms=20200321", keyGenerator.getPartitionPath(row)); InternalRow internalRow = KeyGeneratorTestUtilities.getInternalRow(row); - Assertions.assertEquals(UTF8String.fromString("ts_ms=20200321"), keyGenerator.getPartitionPath(internalRow, row.schema())); + assertEquals(UTF8String.fromString("ts_ms=20200321"), keyGenerator.getPartitionPath(internalRow, row.schema())); } @Test - public void testNonPartitionedKeyGeneratorWithKeyGeneratorClass() throws IOException { + void testNonPartitionedKeyGeneratorWithKeyGeneratorClass() throws IOException { testNonPartitionedKeyGenerator(getPropertiesForNonPartitionedKeyGen(true)); } @Test - public void testNonPartitionedKeyGeneratorWithKeyGeneratorType() throws IOException { + void testNonPartitionedKeyGeneratorWithKeyGeneratorType() throws IOException { testNonPartitionedKeyGenerator(getPropertiesForNonPartitionedKeyGen(false)); } @@ -195,25 +197,25 @@ public void testNonPartitionedKeyGenerator(TypedProperties props) throws IOExcep BuiltinKeyGenerator keyGenerator = (BuiltinKeyGenerator) HoodieSparkKeyGeneratorFactory.createKeyGenerator(props); - GenericRecord record = getRecord(); - HoodieKey key = keyGenerator.getKey(record); - Assertions.assertEquals(key.getRecordKey(), "key1"); - Assertions.assertTrue(key.getPartitionPath().isEmpty()); - Row row = KeyGeneratorTestUtilities.getRow(record); - Assertions.assertEquals(keyGenerator.getRecordKey(row), "key1"); - Assertions.assertTrue(keyGenerator.getPartitionPath(row).isEmpty()); + GenericRecord avroRecord = getRecord(); + HoodieKey key = keyGenerator.getKey(avroRecord); + assertEquals("key1", key.getRecordKey()); + assertTrue(key.getPartitionPath().isEmpty()); + Row row = KeyGeneratorTestUtilities.getRow(avroRecord); + assertEquals("key1", keyGenerator.getRecordKey(row)); + assertTrue(keyGenerator.getPartitionPath(row).isEmpty()); InternalRow internalRow = KeyGeneratorTestUtilities.getInternalRow(row); - Assertions.assertEquals(0, keyGenerator.getPartitionPath(internalRow, row.schema()).numBytes()); + assertEquals(0, keyGenerator.getPartitionPath(internalRow, row.schema()).numBytes()); } @Test - public void testInvalidPartitionKeyTypeWithKeyGeneratorClass() { + void testInvalidPartitionKeyTypeWithKeyGeneratorClass() { testInvalidPartitionKeyType(getInvalidPartitionKeyTypeProps(true)); } @Test - public void testInvalidPartitionKeyTypeWithKeyGeneratorType() { + void testInvalidPartitionKeyTypeWithKeyGeneratorType() { testInvalidPartitionKeyType(getInvalidPartitionKeyTypeProps(false)); } @@ -223,31 +225,31 @@ public void testInvalidPartitionKeyType(TypedProperties props) { (BuiltinKeyGenerator) HoodieSparkKeyGeneratorFactory.createKeyGenerator(props); keyGenerator.getKey(getRecord()); - Assertions.fail("should fail when invalid PartitionKeyType is provided!"); + fail("should fail when invalid PartitionKeyType is provided!"); } catch (Exception e) { - Assertions.assertTrue(getNestedConstructorErrorCause(e).getMessage().contains("No enum constant org.apache.hudi.keygen.CustomAvroKeyGenerator.PartitionKeyType.DUMMY")); + assertTrue(getNestedConstructorErrorCause(e).getMessage().contains("No enum constant org.apache.hudi.keygen.CustomAvroKeyGenerator.PartitionKeyType.DUMMY")); } try { BuiltinKeyGenerator keyGenerator = (BuiltinKeyGenerator) HoodieSparkKeyGeneratorFactory.createKeyGenerator(props); - GenericRecord record = getRecord(); - Row row = KeyGeneratorTestUtilities.getRow(record); + GenericRecord avroRecord = getRecord(); + Row row = KeyGeneratorTestUtilities.getRow(avroRecord); keyGenerator.getPartitionPath(row); - Assertions.fail("should fail when invalid PartitionKeyType is provided!"); + fail("should fail when invalid PartitionKeyType is provided!"); } catch (Exception e) { - Assertions.assertTrue(getNestedConstructorErrorCause(e).getMessage().contains("No enum constant org.apache.hudi.keygen.CustomAvroKeyGenerator.PartitionKeyType.DUMMY")); + assertTrue(getNestedConstructorErrorCause(e).getMessage().contains("No enum constant org.apache.hudi.keygen.CustomAvroKeyGenerator.PartitionKeyType.DUMMY")); } } @Test - public void testNoRecordKeyFieldPropWithKeyGeneratorClass() { + void testNoRecordKeyFieldPropWithKeyGeneratorClass() { testNoRecordKeyFieldProp(true); } @Test - public void testNoRecordKeyFieldPropWithKeyGeneratorType() { + void testNoRecordKeyFieldPropWithKeyGeneratorType() { testNoRecordKeyFieldProp(false); } @@ -257,14 +259,14 @@ public void testNoRecordKeyFieldProp(boolean useKeyGeneratorClassName) { BuiltinKeyGenerator keyGenerator = new CustomKeyGenerator(propsWithoutRecordKeyFieldProps); keyGenerator.getKey(getRecord()); - Assertions.fail("should fail when record key field is not provided!"); + fail("should fail when record key field is not provided!"); } catch (Exception e) { if (useKeyGeneratorClassName) { // "Property hoodie.datasource.write.recordkey.field not found" exception cause CustomKeyGenerator init fail - Assertions.assertTrue(e.getMessage() + assertTrue(e.getMessage() .contains("Unable to find field names for record key in cfg")); } else { - Assertions.assertTrue(stackTraceToString(e).contains("Unable to find field names for record key in cfg")); + assertTrue(stackTraceToString(e).contains("Unable to find field names for record key in cfg")); } } @@ -272,28 +274,28 @@ public void testNoRecordKeyFieldProp(boolean useKeyGeneratorClassName) { try { BuiltinKeyGenerator keyGenerator = new CustomKeyGenerator(propsWithoutRecordKeyFieldProps); - GenericRecord record = getRecord(); - Row row = KeyGeneratorTestUtilities.getRow(record); + GenericRecord avroRecord = getRecord(); + Row row = KeyGeneratorTestUtilities.getRow(avroRecord); keyGenerator.getRecordKey(row); - Assertions.fail("should fail when record key field is not provided!"); + fail("should fail when record key field is not provided!"); } catch (Exception e) { if (useKeyGeneratorClassName) { // "Property hoodie.datasource.write.recordkey.field not found" exception cause CustomKeyGenerator init fail - Assertions.assertTrue(e.getMessage() + assertTrue(e.getMessage() .contains("All of the values for ([]) were either null or empty")); } else { - Assertions.assertTrue(stackTraceToString(e).contains("All of the values for ([]) were either null or empty")); + assertTrue(stackTraceToString(e).contains("All of the values for ([]) were either null or empty")); } } } @Test - public void testPartitionFieldsInImproperFormatWithKeyGeneratorClass() { + void testPartitionFieldsInImproperFormatWithKeyGeneratorClass() { testPartitionFieldsInImproperFormat(getImproperPartitionFieldFormatProp(true)); } @Test - public void testPartitionFieldsInImproperFormatWithKeyGeneratorType() { + void testPartitionFieldsInImproperFormatWithKeyGeneratorType() { testPartitionFieldsInImproperFormat(getImproperPartitionFieldFormatProp(false)); } @@ -303,31 +305,31 @@ public void testPartitionFieldsInImproperFormat(TypedProperties props) { (BuiltinKeyGenerator) HoodieSparkKeyGeneratorFactory.createKeyGenerator(props); keyGenerator.getKey(getRecord()); - Assertions.fail("should fail when partition key field is provided in improper format!"); + fail("should fail when partition key field is provided in improper format!"); } catch (Exception e) { - Assertions.assertTrue(getNestedConstructorErrorCause(e).getMessage().contains("Unable to find field names for partition path in proper format")); + assertTrue(getNestedConstructorErrorCause(e).getMessage().contains("Unable to find field names for partition path in proper format")); } try { BuiltinKeyGenerator keyGenerator = (BuiltinKeyGenerator) HoodieSparkKeyGeneratorFactory.createKeyGenerator(props); - GenericRecord record = getRecord(); - Row row = KeyGeneratorTestUtilities.getRow(record); + GenericRecord avroRecord = getRecord(); + Row row = KeyGeneratorTestUtilities.getRow(avroRecord); keyGenerator.getPartitionPath(row); - Assertions.fail("should fail when partition key field is provided in improper format!"); + fail("should fail when partition key field is provided in improper format!"); } catch (Exception e) { - Assertions.assertTrue(getNestedConstructorErrorCause(e).getMessage().contains("Unable to find field names for partition path in proper format")); + assertTrue(getNestedConstructorErrorCause(e).getMessage().contains("Unable to find field names for partition path in proper format")); } } @Test - public void testComplexRecordKeyWithSimplePartitionPathWithKeyGeneratorClass() throws IOException { + void testComplexRecordKeyWithSimplePartitionPathWithKeyGeneratorClass() throws IOException { testComplexRecordKeyWithSimplePartitionPath(getComplexRecordKeyWithSimplePartitionProps(true)); } @Test - public void testComplexRecordKeyWithSimplePartitionPathWithKeyGeneratorType() throws IOException { + void testComplexRecordKeyWithSimplePartitionPathWithKeyGeneratorType() throws IOException { testComplexRecordKeyWithSimplePartitionPath(getComplexRecordKeyWithSimplePartitionProps(false)); } @@ -335,26 +337,26 @@ public void testComplexRecordKeyWithSimplePartitionPath(TypedProperties props) t BuiltinKeyGenerator keyGenerator = (BuiltinKeyGenerator) HoodieSparkKeyGeneratorFactory.createKeyGenerator(props); - GenericRecord record = getRecord(); - HoodieKey key = keyGenerator.getKey(record); - Assertions.assertEquals("_row_key:key1,pii_col:pi", key.getRecordKey()); - Assertions.assertEquals("timestamp=4357686", key.getPartitionPath()); + GenericRecord avroRecord = getRecord(); + HoodieKey key = keyGenerator.getKey(avroRecord); + assertEquals("_row_key:key1,pii_col:pi", key.getRecordKey()); + assertEquals("timestamp=4357686", key.getPartitionPath()); - Row row = KeyGeneratorTestUtilities.getRow(record); - Assertions.assertEquals("_row_key:key1,pii_col:pi", keyGenerator.getRecordKey(row)); - Assertions.assertEquals("timestamp=4357686", keyGenerator.getPartitionPath(row)); + Row row = KeyGeneratorTestUtilities.getRow(avroRecord); + assertEquals("_row_key:key1,pii_col:pi", keyGenerator.getRecordKey(row)); + assertEquals("timestamp=4357686", keyGenerator.getPartitionPath(row)); InternalRow internalRow = KeyGeneratorTestUtilities.getInternalRow(row); - Assertions.assertEquals(UTF8String.fromString("timestamp=4357686"), keyGenerator.getPartitionPath(internalRow, row.schema())); + assertEquals(UTF8String.fromString("timestamp=4357686"), keyGenerator.getPartitionPath(internalRow, row.schema())); } @Test - public void testComplexRecordKeysWithComplexPartitionPathWithKeyGeneratorClass() throws IOException { + void testComplexRecordKeysWithComplexPartitionPathWithKeyGeneratorClass() throws IOException { testComplexRecordKeysWithComplexPartitionPath(getComplexRecordKeyAndPartitionPathProps(true)); } @Test - public void testComplexRecordKeysWithComplexPartitionPathWithKeyGeneratorType() throws IOException { + void testComplexRecordKeysWithComplexPartitionPathWithKeyGeneratorType() throws IOException { testComplexRecordKeysWithComplexPartitionPath(getComplexRecordKeyAndPartitionPathProps(false)); } @@ -362,17 +364,17 @@ public void testComplexRecordKeysWithComplexPartitionPath(TypedProperties props) BuiltinKeyGenerator keyGenerator = (BuiltinKeyGenerator) HoodieSparkKeyGeneratorFactory.createKeyGenerator(props); - GenericRecord record = getRecord(); - HoodieKey key = keyGenerator.getKey(record); - Assertions.assertEquals("_row_key:key1,pii_col:pi", key.getRecordKey()); - Assertions.assertEquals("timestamp=4357686/ts_ms=20200321", key.getPartitionPath()); + GenericRecord avroRecord = getRecord(); + HoodieKey key = keyGenerator.getKey(avroRecord); + assertEquals("_row_key:key1,pii_col:pi", key.getRecordKey()); + assertEquals("timestamp=4357686/ts_ms=20200321", key.getPartitionPath()); - Row row = KeyGeneratorTestUtilities.getRow(record); - Assertions.assertEquals("_row_key:key1,pii_col:pi", keyGenerator.getRecordKey(row)); - Assertions.assertEquals("timestamp=4357686/ts_ms=20200321", keyGenerator.getPartitionPath(row)); + Row row = KeyGeneratorTestUtilities.getRow(avroRecord); + assertEquals("_row_key:key1,pii_col:pi", keyGenerator.getRecordKey(row)); + assertEquals("timestamp=4357686/ts_ms=20200321", keyGenerator.getPartitionPath(row)); InternalRow internalRow = KeyGeneratorTestUtilities.getInternalRow(row); - Assertions.assertEquals(UTF8String.fromString("timestamp=4357686/ts_ms=20200321"), keyGenerator.getPartitionPath(internalRow, row.schema())); + assertEquals(UTF8String.fromString("timestamp=4357686/ts_ms=20200321"), keyGenerator.getPartitionPath(internalRow, row.schema())); } private static Throwable getNestedConstructorErrorCause(Exception e) { diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/keygen/TestGlobalDeleteRecordGenerator.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/keygen/TestGlobalDeleteRecordGenerator.java similarity index 60% rename from hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/keygen/TestGlobalDeleteRecordGenerator.java rename to hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/keygen/TestGlobalDeleteRecordGenerator.java index df69279cc89f0..14b29760f764c 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/keygen/TestGlobalDeleteRecordGenerator.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/keygen/TestGlobalDeleteRecordGenerator.java @@ -18,28 +18,19 @@ package org.apache.hudi.keygen; -import org.apache.avro.generic.GenericRecord; import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.exception.HoodieKeyException; import org.apache.hudi.keygen.constant.KeyGeneratorOptions; -import org.apache.hudi.testutils.KeyGeneratorTestUtilities; + +import org.apache.avro.generic.GenericRecord; import org.apache.spark.sql.Row; -import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; -public class TestGlobalDeleteRecordGenerator extends KeyGeneratorTestUtilities { +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertThrows; - private TypedProperties getCommonProps(boolean getComplexRecordKey) { - TypedProperties properties = new TypedProperties(); - if (getComplexRecordKey) { - properties.put(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key(), "_row_key,pii_col"); - } else { - properties.put(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key(), "_row_key"); - } - properties.put(KeyGeneratorOptions.HIVE_STYLE_PARTITIONING_ENABLE.key(), "true"); - return properties; - } +class TestGlobalDeleteRecordGenerator extends KeyGeneratorTestUtilities { private TypedProperties getPropertiesWithoutRecordKeyProp() { TypedProperties properties = new TypedProperties(); @@ -53,36 +44,54 @@ private TypedProperties getWrongRecordKeyFieldProps() { return properties; } - private TypedProperties getProps() { - TypedProperties properties = getCommonProps(true); + private TypedProperties getProps(boolean multipleRecordKeyFields) { + TypedProperties properties = new TypedProperties(); + if (multipleRecordKeyFields) { + properties.put(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key(), "_row_key,pii_col"); + } else { + properties.put(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key(), "_row_key"); + } + properties.put(KeyGeneratorOptions.HIVE_STYLE_PARTITIONING_ENABLE.key(), "true"); properties.put(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key(), "timestamp,ts_ms"); return properties; } @Test public void testNullRecordKeyFields() { - GenericRecord record = getRecord(); - Assertions.assertThrows(StringIndexOutOfBoundsException.class, () -> { + GenericRecord avroRecord = getRecord(); + assertThrows(StringIndexOutOfBoundsException.class, () -> { BaseKeyGenerator keyGenerator = new GlobalDeleteKeyGenerator(getPropertiesWithoutRecordKeyProp()); - keyGenerator.getRecordKey(record); + keyGenerator.getRecordKey(avroRecord); }); } @Test - public void testWrongRecordKeyField() { + void testWrongRecordKeyField() { GlobalDeleteKeyGenerator keyGenerator = new GlobalDeleteKeyGenerator(getWrongRecordKeyFieldProps()); - Assertions.assertThrows(HoodieKeyException.class, () -> keyGenerator.getRecordKey(getRecord())); + assertThrows(HoodieKeyException.class, () -> keyGenerator.getRecordKey(getRecord())); + } + + @Test + void testSingleRecordKeyField() { + GlobalDeleteKeyGenerator keyGenerator = new GlobalDeleteKeyGenerator(getProps(false)); + GenericRecord avroRecord = getRecord(); + HoodieKey key = keyGenerator.getKey(avroRecord); + assertEquals("_row_key:key1", key.getRecordKey()); + assertEquals("", key.getPartitionPath()); + Row row = KeyGeneratorTestUtilities.getRow(avroRecord); + assertEquals("_row_key:key1", keyGenerator.getRecordKey(row)); + assertEquals("", keyGenerator.getPartitionPath(row)); } @Test - public void testHappyFlow() { - GlobalDeleteKeyGenerator keyGenerator = new GlobalDeleteKeyGenerator(getProps()); - GenericRecord record = getRecord(); - HoodieKey key = keyGenerator.getKey(record); - Assertions.assertEquals(key.getRecordKey(), "_row_key:key1,pii_col:pi"); - Assertions.assertEquals(key.getPartitionPath(), ""); - Row row = KeyGeneratorTestUtilities.getRow(record); - Assertions.assertEquals(keyGenerator.getRecordKey(row), "_row_key:key1,pii_col:pi"); - Assertions.assertEquals(keyGenerator.getPartitionPath(row), ""); + void testMultipleRecordKeyFields() { + GlobalDeleteKeyGenerator keyGenerator = new GlobalDeleteKeyGenerator(getProps(true)); + GenericRecord avroRecord = getRecord(); + HoodieKey key = keyGenerator.getKey(avroRecord); + assertEquals("_row_key:key1,pii_col:pi", key.getRecordKey()); + assertEquals("", key.getPartitionPath()); + Row row = KeyGeneratorTestUtilities.getRow(avroRecord); + assertEquals("_row_key:key1,pii_col:pi", keyGenerator.getRecordKey(row)); + assertEquals("", keyGenerator.getPartitionPath(row)); } } diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/keygen/TestNonpartitionedKeyGenerator.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/keygen/TestNonpartitionedKeyGenerator.java similarity index 60% rename from hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/keygen/TestNonpartitionedKeyGenerator.java rename to hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/keygen/TestNonpartitionedKeyGenerator.java index fb740d00e2a5e..b5545f501a1d2 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/keygen/TestNonpartitionedKeyGenerator.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/keygen/TestNonpartitionedKeyGenerator.java @@ -18,20 +18,20 @@ package org.apache.hudi.keygen; -import org.apache.avro.generic.GenericRecord; import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.testutils.HoodieTestDataGenerator; import org.apache.hudi.exception.HoodieKeyException; import org.apache.hudi.keygen.constant.KeyGeneratorOptions; -import org.apache.hudi.testutils.KeyGeneratorTestUtilities; + +import org.apache.avro.generic.GenericRecord; import org.apache.spark.sql.Row; -import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; -import static junit.framework.TestCase.assertEquals; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertThrows; -public class TestNonpartitionedKeyGenerator extends KeyGeneratorTestUtilities { +class TestNonpartitionedKeyGenerator extends KeyGeneratorTestUtilities { private TypedProperties getCommonProps(boolean getComplexRecordKey) { TypedProperties properties = new TypedProperties(); @@ -67,72 +67,74 @@ private TypedProperties getWrongRecordKeyFieldProps() { } @Test - public void testNullRecordKeyFields() { - GenericRecord record = getRecord(); - Assertions.assertThrows(StringIndexOutOfBoundsException.class, () -> { + void testNullRecordKeyFields() { + GenericRecord avroRecord = getRecord(); + assertThrows(StringIndexOutOfBoundsException.class, () -> { BaseKeyGenerator keyGenerator = new NonpartitionedKeyGenerator(getPropertiesWithoutRecordKeyProp()); - keyGenerator.getRecordKey(record); + keyGenerator.getRecordKey(avroRecord); }); } @Test - public void testNonNullPartitionPathFields() { + void testNonNullPartitionPathFields() { TypedProperties properties = getPropertiesWithPartitionPathProp(); NonpartitionedKeyGenerator keyGenerator = new NonpartitionedKeyGenerator(properties); - GenericRecord record = getRecord(); - Row row = KeyGeneratorTestUtilities.getRow(record); - Assertions.assertEquals(properties.getString(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key()), "timestamp,ts_ms"); - Assertions.assertEquals(keyGenerator.getPartitionPath(row), ""); + GenericRecord avroRecord = getRecord(); + Row row = KeyGeneratorTestUtilities.getRow(avroRecord); + assertEquals("timestamp,ts_ms", properties.getString(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key())); + assertEquals("", keyGenerator.getPartitionPath(row)); } @Test - public void testNullPartitionPathFields() { + void testNullPartitionPathFields() { TypedProperties properties = getPropertiesWithoutPartitionPathProp(); NonpartitionedKeyGenerator keyGenerator = new NonpartitionedKeyGenerator(properties); - GenericRecord record = getRecord(); - Row row = KeyGeneratorTestUtilities.getRow(record); - Assertions.assertEquals(keyGenerator.getPartitionPath(row), ""); + GenericRecord avroRecord = getRecord(); + Row row = KeyGeneratorTestUtilities.getRow(avroRecord); + assertEquals("", keyGenerator.getPartitionPath(row)); } @Test - public void testWrongRecordKeyField() { + void testWrongRecordKeyField() { NonpartitionedKeyGenerator keyGenerator = new NonpartitionedKeyGenerator(getWrongRecordKeyFieldProps()); - Assertions.assertThrows(HoodieKeyException.class, () -> keyGenerator.getRecordKey(getRecord())); + assertThrows(HoodieKeyException.class, () -> keyGenerator.getRecordKey(getRecord())); } @Test - public void testSingleValueKeyGeneratorNonPartitioned() { + void testSingleValueKeyGeneratorNonPartitioned() { TypedProperties properties = new TypedProperties(); properties.setProperty(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key(), "timestamp"); properties.setProperty(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key(), ""); NonpartitionedKeyGenerator keyGenerator = new NonpartitionedKeyGenerator(properties); - assertEquals(keyGenerator.getRecordKeyFieldNames().size(), 1); - assertEquals(keyGenerator.getPartitionPathFields().size(), 0); - - HoodieTestDataGenerator dataGenerator = new HoodieTestDataGenerator(); - GenericRecord record = dataGenerator.generateGenericRecords(1).get(0); - String rowKey = record.get("timestamp").toString(); - HoodieKey hoodieKey = keyGenerator.getKey(record); - assertEquals(rowKey, hoodieKey.getRecordKey()); - assertEquals("", hoodieKey.getPartitionPath()); + assertEquals(1, keyGenerator.getRecordKeyFieldNames().size()); + assertEquals(0, keyGenerator.getPartitionPathFields().size()); + + try (HoodieTestDataGenerator dataGenerator = new HoodieTestDataGenerator(System.currentTimeMillis())) { + GenericRecord avroRecord = dataGenerator.generateGenericRecords(1).get(0); + String rowKey = avroRecord.get("timestamp").toString(); + HoodieKey hoodieKey = keyGenerator.getKey(avroRecord); + assertEquals(rowKey, hoodieKey.getRecordKey()); + assertEquals("", hoodieKey.getPartitionPath()); + } } @Test - public void testMultipleValueKeyGeneratorNonPartitioned1() { + void testMultipleValueKeyGeneratorNonPartitioned1() { TypedProperties properties = new TypedProperties(); properties.setProperty(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key(), "timestamp,driver"); properties.setProperty(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key(), ""); NonpartitionedKeyGenerator keyGenerator = new NonpartitionedKeyGenerator(properties); - assertEquals(keyGenerator.getRecordKeyFieldNames().size(), 2); - assertEquals(keyGenerator.getPartitionPathFields().size(), 0); - HoodieTestDataGenerator dataGenerator = new HoodieTestDataGenerator(); - GenericRecord record = dataGenerator.generateGenericRecords(1).get(0); - String rowKey = - "timestamp" + ComplexAvroKeyGenerator.DEFAULT_RECORD_KEY_SEPARATOR + record.get("timestamp").toString() + "," - + "driver" + ComplexAvroKeyGenerator.DEFAULT_RECORD_KEY_SEPARATOR + record.get("driver").toString(); - String partitionPath = ""; - HoodieKey hoodieKey = keyGenerator.getKey(record); - assertEquals(rowKey, hoodieKey.getRecordKey()); - assertEquals(partitionPath, hoodieKey.getPartitionPath()); + assertEquals(2, keyGenerator.getRecordKeyFieldNames().size()); + assertEquals(0, keyGenerator.getPartitionPathFields().size()); + try (HoodieTestDataGenerator dataGenerator = new HoodieTestDataGenerator(System.currentTimeMillis())) { + GenericRecord avroRecord = dataGenerator.generateGenericRecords(1).get(0); + String rowKey = + "timestamp" + ComplexAvroKeyGenerator.DEFAULT_RECORD_KEY_SEPARATOR + avroRecord.get("timestamp").toString() + "," + + "driver" + ComplexAvroKeyGenerator.DEFAULT_RECORD_KEY_SEPARATOR + avroRecord.get("driver").toString(); + String partitionPath = ""; + HoodieKey hoodieKey = keyGenerator.getKey(avroRecord); + assertEquals(rowKey, hoodieKey.getRecordKey()); + assertEquals(partitionPath, hoodieKey.getPartitionPath()); + } } } diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/keygen/TestSimpleKeyGenerator.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/keygen/TestSimpleKeyGenerator.java similarity index 79% rename from hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/keygen/TestSimpleKeyGenerator.java rename to hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/keygen/TestSimpleKeyGenerator.java index adf522f8354b2..5c277d2e7b5da 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/keygen/TestSimpleKeyGenerator.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/keygen/TestSimpleKeyGenerator.java @@ -7,24 +7,25 @@ * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. */ package org.apache.hudi.keygen; -import org.apache.avro.generic.GenericRecord; import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.exception.HoodieKeyException; import org.apache.hudi.keygen.constant.KeyGeneratorOptions; -import org.apache.hudi.testutils.KeyGeneratorTestUtilities; + +import org.apache.avro.generic.GenericRecord; import org.apache.spark.sql.Row; import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.unsafe.types.UTF8String; @@ -38,7 +39,7 @@ import static org.apache.hudi.keygen.KeyGenUtils.HUDI_DEFAULT_PARTITION_PATH; import static org.junit.jupiter.api.Assertions.assertThrows; -public class TestSimpleKeyGenerator extends KeyGeneratorTestUtilities { +class TestSimpleKeyGenerator extends KeyGeneratorTestUtilities { private TypedProperties getCommonProps() { TypedProperties properties = new TypedProperties(); properties.put(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key(), "_row_key"); @@ -90,54 +91,52 @@ private TypedProperties getPropsWithNestedPartitionPathField() { } @Test - public void testNullPartitionPathFields() { + void testNullPartitionPathFields() { assertThrows(IllegalArgumentException.class, () -> new SimpleKeyGenerator(getPropertiesWithoutPartitionPathProp())); } @Test - public void testNullRecordKeyFields() { - GenericRecord record = getRecord(); + void testNullRecordKeyFields() { + GenericRecord avroRecord = getRecord(); Assertions.assertThrows(IndexOutOfBoundsException.class, () -> { BaseKeyGenerator keyGenerator = new SimpleKeyGenerator(getPropertiesWithoutRecordKeyProp()); - keyGenerator.getRecordKey(record); + keyGenerator.getRecordKey(avroRecord); }); } @Test - public void testWrongRecordKeyField() { + void testWrongRecordKeyField() { SimpleKeyGenerator keyGenerator = new SimpleKeyGenerator(getWrongRecordKeyFieldProps()); assertThrows(HoodieKeyException.class, () -> keyGenerator.getRecordKey(getRecord())); } @Test - public void testWrongPartitionPathField() { + void testWrongPartitionPathField() { SimpleKeyGenerator keyGenerator = new SimpleKeyGenerator(getWrongPartitionPathFieldProps()); - GenericRecord record = getRecord(); + GenericRecord avroRecord = getRecord(); // TODO this should throw as well //assertThrows(HoodieException.class, () -> { // keyGenerator.getPartitionPath(record); //}); - assertThrows(HoodieException.class, () -> { - keyGenerator.getPartitionPath(KeyGeneratorTestUtilities.getRow(record)); - }); + assertThrows(HoodieException.class, + () -> keyGenerator.getPartitionPath(KeyGeneratorTestUtilities.getRow(avroRecord))); } @Test - public void testComplexRecordKeyField() { - assertThrows(IllegalArgumentException.class, () -> { - new SimpleKeyGenerator(getComplexRecordKeyProp()); - }); + void testComplexRecordKeyField() { + assertThrows(IllegalArgumentException.class, + () -> new SimpleKeyGenerator(getComplexRecordKeyProp())); } @Test - public void testHappyFlow() { + void testHappyFlow() { SimpleKeyGenerator keyGenerator = new SimpleKeyGenerator(getProps()); - GenericRecord record = getRecord(); + GenericRecord avroRecord = getRecord(); HoodieKey key = keyGenerator.getKey(getRecord()); Assertions.assertEquals("key1", key.getRecordKey()); Assertions.assertEquals("timestamp=4357686", key.getPartitionPath()); - Row row = KeyGeneratorTestUtilities.getRow(record); + Row row = KeyGeneratorTestUtilities.getRow(avroRecord); Assertions.assertEquals("key1", keyGenerator.getRecordKey(row)); Assertions.assertEquals("timestamp=4357686", keyGenerator.getPartitionPath(row)); @@ -152,20 +151,20 @@ private static Stream nestedColTestRecords() { @ParameterizedTest @MethodSource("nestedColTestRecords") - public void testNestedPartitionPathField(GenericRecord nestedColRecord) { + void testNestedPartitionPathField(GenericRecord nestedColRecord) { SimpleKeyGenerator keyGenerator = new SimpleKeyGenerator(getPropsWithNestedPartitionPathField()); - GenericRecord record = getRecord(nestedColRecord); + GenericRecord avroRecord = getRecord(nestedColRecord); String partitionPathFieldValue = null; if (nestedColRecord != null) { partitionPathFieldValue = (String) nestedColRecord.get("prop1"); } String expectedPartitionPath = "nested_col.prop1=" + (partitionPathFieldValue != null && !partitionPathFieldValue.isEmpty() ? partitionPathFieldValue : HUDI_DEFAULT_PARTITION_PATH); - HoodieKey key = keyGenerator.getKey(record); + HoodieKey key = keyGenerator.getKey(avroRecord); Assertions.assertEquals("key1", key.getRecordKey()); Assertions.assertEquals(expectedPartitionPath, key.getPartitionPath()); - Row row = KeyGeneratorTestUtilities.getRow(record); + Row row = KeyGeneratorTestUtilities.getRow(avroRecord); Assertions.assertEquals("key1", keyGenerator.getRecordKey(row)); Assertions.assertEquals(expectedPartitionPath, keyGenerator.getPartitionPath(row)); } diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/keygen/TestTimestampBasedKeyGenerator.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/keygen/TestTimestampBasedKeyGenerator.java similarity index 79% rename from hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/keygen/TestTimestampBasedKeyGenerator.java rename to hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/keygen/TestTimestampBasedKeyGenerator.java index ec93ea229accb..136ed7c4b5107 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/keygen/TestTimestampBasedKeyGenerator.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/keygen/TestTimestampBasedKeyGenerator.java @@ -18,14 +18,12 @@ package org.apache.hudi.keygen; -import org.apache.hudi.AvroConversionUtils; import org.apache.hudi.avro.AvroSchemaUtils; import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.testutils.SchemaTestUtil; import org.apache.hudi.exception.HoodieKeyGeneratorException; import org.apache.hudi.keygen.constant.KeyGeneratorOptions; -import org.apache.hudi.testutils.KeyGeneratorTestUtilities; import org.apache.avro.Conversions; import org.apache.avro.LogicalTypes; @@ -34,18 +32,17 @@ import org.apache.avro.generic.GenericRecord; import org.apache.spark.sql.Row; import org.apache.spark.sql.catalyst.InternalRow; -import org.apache.spark.sql.catalyst.expressions.GenericRowWithSchema; +import org.apache.spark.sql.types.DecimalType; +import org.apache.spark.sql.types.Metadata; +import org.apache.spark.sql.types.StructField; import org.apache.spark.sql.types.StructType; import org.apache.spark.unsafe.types.UTF8String; -import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import java.io.IOException; import java.math.BigDecimal; -import scala.Function1; - import static org.apache.hudi.common.config.TimestampKeyGeneratorConfig.TIMESTAMP_INPUT_DATE_FORMAT; import static org.apache.hudi.common.config.TimestampKeyGeneratorConfig.TIMESTAMP_INPUT_DATE_FORMAT_LIST_DELIMITER_REGEX; import static org.apache.hudi.common.config.TimestampKeyGeneratorConfig.TIMESTAMP_INPUT_TIMEZONE_FORMAT; @@ -53,9 +50,13 @@ import static org.apache.hudi.common.config.TimestampKeyGeneratorConfig.TIMESTAMP_OUTPUT_TIMEZONE_FORMAT; import static org.apache.hudi.common.config.TimestampKeyGeneratorConfig.TIMESTAMP_TIMEZONE_FORMAT; import static org.apache.hudi.common.config.TimestampKeyGeneratorConfig.TIMESTAMP_TYPE_FIELD; +import static org.apache.spark.sql.types.DataTypes.LongType; +import static org.apache.spark.sql.types.DataTypes.StringType; +import static org.apache.spark.sql.types.DataTypes.TimestampType; import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertThrows; -public class TestTimestampBasedKeyGenerator { +class TestTimestampBasedKeyGenerator { private GenericRecord baseRecord; private TypedProperties properties = new TypedProperties(); @@ -66,9 +67,14 @@ public class TestTimestampBasedKeyGenerator { private InternalRow internalRow; @BeforeEach - public void initialize() throws IOException { + void initialize() throws IOException { schema = SchemaTestUtil.getTimestampEvolvedSchema(); - structType = AvroConversionUtils.convertAvroSchemaToStructType(schema); + structType = new StructType(new StructField[] { + new StructField("field1", StringType, true, Metadata.empty()), + new StructField("createTime", LongType, true, Metadata.empty()), + new StructField("createTimeString", StringType, true, Metadata.empty()), + new StructField("createTimeDecimal", new DecimalType(20, 4), true, Metadata.empty()) + }); baseRecord = SchemaTestUtil .generateAvroRecordFromJson(schema, 1, "001", "f1"); baseRow = genericRecordToRow(baseRecord); @@ -80,29 +86,22 @@ public void initialize() throws IOException { } private Row genericRecordToRow(GenericRecord baseRecord) { - Function1 convertor = AvroConversionUtils.createConverterToRow(baseRecord.getSchema(), structType); - Row row = convertor.apply(baseRecord); - int fieldCount = structType.fieldNames().length; - Object[] values = new Object[fieldCount]; - for (int i = 0; i < fieldCount; i++) { - values[i] = row.get(i); - } - return new GenericRowWithSchema(values, structType); + return KeyGeneratorTestUtilities.genericRecordToRow(baseRecord, structType); } private TypedProperties getBaseKeyConfig(String partitionPathField, String timestampType, String dateFormat, String timezone, String scalarType) { - TypedProperties properties = new TypedProperties(this.properties); + TypedProperties newProperties = new TypedProperties(this.properties); - properties.setProperty(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key(), partitionPathField); - properties.setProperty(TIMESTAMP_TYPE_FIELD.key(), timestampType); - properties.setProperty(TIMESTAMP_OUTPUT_DATE_FORMAT.key(), dateFormat); - properties.setProperty(TIMESTAMP_TIMEZONE_FORMAT.key(), timezone); + newProperties.setProperty(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key(), partitionPathField); + newProperties.setProperty(TIMESTAMP_TYPE_FIELD.key(), timestampType); + newProperties.setProperty(TIMESTAMP_OUTPUT_DATE_FORMAT.key(), dateFormat); + newProperties.setProperty(TIMESTAMP_TIMEZONE_FORMAT.key(), timezone); if (scalarType != null) { - properties.setProperty("hoodie.deltastreamer.keygen.timebased.timestamp.scalar.time.unit", scalarType); + newProperties.setProperty("hoodie.deltastreamer.keygen.timebased.timestamp.scalar.time.unit", scalarType); } - return properties; + return newProperties; } private TypedProperties getBaseKeyConfig(String partitionPathField, @@ -112,33 +111,33 @@ private TypedProperties getBaseKeyConfig(String partitionPathField, String inputTimezone, String outputFormat, String outputTimezone) { - TypedProperties properties = new TypedProperties(this.properties); + TypedProperties newProperties = new TypedProperties(this.properties); - properties.setProperty(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key(), partitionPathField); + newProperties.setProperty(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key(), partitionPathField); if (timestampType != null) { - properties.setProperty(TIMESTAMP_TYPE_FIELD.key(), timestampType); + newProperties.setProperty(TIMESTAMP_TYPE_FIELD.key(), timestampType); } if (inputFormatList != null) { - properties.setProperty(TIMESTAMP_INPUT_DATE_FORMAT.key(), inputFormatList); + newProperties.setProperty(TIMESTAMP_INPUT_DATE_FORMAT.key(), inputFormatList); } if (inputFormatDelimiterRegex != null) { - properties.setProperty(TIMESTAMP_INPUT_DATE_FORMAT_LIST_DELIMITER_REGEX.key(), inputFormatDelimiterRegex); + newProperties.setProperty(TIMESTAMP_INPUT_DATE_FORMAT_LIST_DELIMITER_REGEX.key(), inputFormatDelimiterRegex); } if (inputTimezone != null) { - properties.setProperty(TIMESTAMP_INPUT_TIMEZONE_FORMAT.key(), inputTimezone); + newProperties.setProperty(TIMESTAMP_INPUT_TIMEZONE_FORMAT.key(), inputTimezone); } if (outputFormat != null) { - properties.setProperty(TIMESTAMP_OUTPUT_DATE_FORMAT.key(), outputFormat); + newProperties.setProperty(TIMESTAMP_OUTPUT_DATE_FORMAT.key(), outputFormat); } if (outputTimezone != null) { - properties.setProperty(TIMESTAMP_OUTPUT_TIMEZONE_FORMAT.key(), outputTimezone); + newProperties.setProperty(TIMESTAMP_OUTPUT_TIMEZONE_FORMAT.key(), outputTimezone); } - return properties; + return newProperties; } @Test - public void testTimestampBasedKeyGenerator() throws IOException { + void testTimestampBasedKeyGenerator() throws IOException { // timezone is GMT+8:00 baseRecord.put("createTime", 1578283932000L); properties = getBaseKeyConfig("createTime", "EPOCHMILLISECONDS", "yyyy-MM-dd hh", "GMT+8:00", null); @@ -212,7 +211,7 @@ public void testTimestampBasedKeyGenerator() throws IOException { } @Test - public void testScalar() throws IOException { + void testScalar() throws IOException { // timezone is GMT+8:00 baseRecord.put("createTime", 20000L); @@ -220,7 +219,7 @@ public void testScalar() throws IOException { properties = getBaseKeyConfig("createTime", "SCALAR", "yyyy-MM-dd hh", "GMT", "days"); TimestampBasedKeyGenerator keyGen = new TimestampBasedKeyGenerator(properties); HoodieKey hk1 = keyGen.getKey(baseRecord); - assertEquals(hk1.getPartitionPath(), "2024-10-04 12"); + assertEquals("2024-10-04 12", hk1.getPartitionPath()); // test w/ Row baseRow = genericRecordToRow(baseRecord); @@ -254,9 +253,12 @@ public void testScalar() throws IOException { } @Test - public void testScalarWithLogicalType() throws IOException { + void testScalarWithLogicalType() throws IOException { schema = SchemaTestUtil.getTimestampWithLogicalTypeSchema(); - structType = AvroConversionUtils.convertAvroSchemaToStructType(schema); + structType = new StructType(new StructField[] { + new StructField("field1", StringType, true, Metadata.empty()), + new StructField("createTime", TimestampType, true, Metadata.empty()) + }); baseRecord = SchemaTestUtil.generateAvroRecordFromJson(schema, 1, "001", "f1"); baseRecord.put("createTime", 1638513806000000L); @@ -288,7 +290,7 @@ public void testScalarWithLogicalType() throws IOException { } @Test - public void test_ExpectsMatch_SingleInputFormat_ISO8601WithMsZ_OutputTimezoneAsUTC() throws IOException { + void test_ExpectsMatch_SingleInputFormat_ISO8601WithMsZ_OutputTimezoneAsUTC() throws IOException { baseRecord.put("createTimeString", "2020-04-01T13:01:33.428Z"); properties = this.getBaseKeyConfig( "createTimeString", @@ -300,14 +302,14 @@ public void test_ExpectsMatch_SingleInputFormat_ISO8601WithMsZ_OutputTimezoneAsU "GMT"); BuiltinKeyGenerator keyGen = new TimestampBasedKeyGenerator(properties); HoodieKey hk1 = keyGen.getKey(baseRecord); - Assertions.assertEquals("2020040113", hk1.getPartitionPath()); + assertEquals("2020040113", hk1.getPartitionPath()); baseRow = genericRecordToRow(baseRecord); assertEquals("2020040113", keyGen.getPartitionPath(baseRow)); } @Test - public void test_ExpectsMatch_SingleInputFormats_ISO8601WithMsZ_OutputTimezoneAsInputDateTimeZone() throws IOException { + void test_ExpectsMatch_SingleInputFormats_ISO8601WithMsZ_OutputTimezoneAsInputDateTimeZone() throws IOException { baseRecord.put("createTimeString", "2020-04-01T13:01:33.428Z"); properties = this.getBaseKeyConfig( "createTimeString", @@ -319,14 +321,14 @@ public void test_ExpectsMatch_SingleInputFormats_ISO8601WithMsZ_OutputTimezoneAs ""); BuiltinKeyGenerator keyGen = new TimestampBasedKeyGenerator(properties); HoodieKey hk1 = keyGen.getKey(baseRecord); - Assertions.assertEquals("2020040113", hk1.getPartitionPath()); + assertEquals("2020040113", hk1.getPartitionPath()); baseRow = genericRecordToRow(baseRecord); assertEquals("2020040113", keyGen.getPartitionPath(baseRow)); } @Test - public void test_ExpectsMatch_MultipleInputFormats_ISO8601WithMsZ_OutputTimezoneAsUTC() throws IOException { + void test_ExpectsMatch_MultipleInputFormats_ISO8601WithMsZ_OutputTimezoneAsUTC() throws IOException { baseRecord.put("createTimeString", "2020-04-01T13:01:33.428Z"); properties = this.getBaseKeyConfig( "createTimeString", @@ -338,14 +340,14 @@ public void test_ExpectsMatch_MultipleInputFormats_ISO8601WithMsZ_OutputTimezone "UTC"); BuiltinKeyGenerator keyGen = new TimestampBasedKeyGenerator(properties); HoodieKey hk1 = keyGen.getKey(baseRecord); - Assertions.assertEquals("2020040113", hk1.getPartitionPath()); + assertEquals("2020040113", hk1.getPartitionPath()); baseRow = genericRecordToRow(baseRecord); assertEquals("2020040113", keyGen.getPartitionPath(baseRow)); } @Test - public void test_ExpectsMatch_MultipleInputFormats_ISO8601NoMsZ_OutputTimezoneAsUTC() throws IOException { + void test_ExpectsMatch_MultipleInputFormats_ISO8601NoMsZ_OutputTimezoneAsUTC() throws IOException { baseRecord.put("createTimeString", "2020-04-01T13:01:33Z"); properties = this.getBaseKeyConfig( "createTimeString", @@ -357,14 +359,14 @@ public void test_ExpectsMatch_MultipleInputFormats_ISO8601NoMsZ_OutputTimezoneAs "UTC"); BuiltinKeyGenerator keyGen = new TimestampBasedKeyGenerator(properties); HoodieKey hk1 = keyGen.getKey(baseRecord); - Assertions.assertEquals("2020040113", hk1.getPartitionPath()); + assertEquals("2020040113", hk1.getPartitionPath()); baseRow = genericRecordToRow(baseRecord); assertEquals("2020040113", keyGen.getPartitionPath(baseRow)); } @Test - public void test_ExpectsMatch_MultipleInputFormats_ISO8601NoMsWithOffset_OutputTimezoneAsUTC() throws IOException { + void test_ExpectsMatch_MultipleInputFormats_ISO8601NoMsWithOffset_OutputTimezoneAsUTC() throws IOException { baseRecord.put("createTimeString", "2020-04-01T13:01:33-05:00"); properties = this.getBaseKeyConfig( "createTimeString", @@ -376,14 +378,14 @@ public void test_ExpectsMatch_MultipleInputFormats_ISO8601NoMsWithOffset_OutputT "UTC"); BuiltinKeyGenerator keyGen = new TimestampBasedKeyGenerator(properties); HoodieKey hk1 = keyGen.getKey(baseRecord); - Assertions.assertEquals("2020040118", hk1.getPartitionPath()); + assertEquals("2020040118", hk1.getPartitionPath()); baseRow = genericRecordToRow(baseRecord); assertEquals("2020040118", keyGen.getPartitionPath(baseRow)); } @Test - public void test_ExpectsMatch_MultipleInputFormats_ISO8601WithMsWithOffset_OutputTimezoneAsUTC() throws IOException { + void test_ExpectsMatch_MultipleInputFormats_ISO8601WithMsWithOffset_OutputTimezoneAsUTC() throws IOException { baseRecord.put("createTimeString", "2020-04-01T13:01:33.123-05:00"); properties = this.getBaseKeyConfig( "createTimeString", @@ -395,14 +397,14 @@ public void test_ExpectsMatch_MultipleInputFormats_ISO8601WithMsWithOffset_Outpu "UTC"); BuiltinKeyGenerator keyGen = new TimestampBasedKeyGenerator(properties); HoodieKey hk1 = keyGen.getKey(baseRecord); - Assertions.assertEquals("2020040118", hk1.getPartitionPath()); + assertEquals("2020040118", hk1.getPartitionPath()); baseRow = genericRecordToRow(baseRecord); assertEquals("2020040118", keyGen.getPartitionPath(baseRow)); } @Test - public void test_ExpectsMatch_MultipleInputFormats_ISO8601WithMsZ_OutputTimezoneAsEST() throws IOException { + void test_ExpectsMatch_MultipleInputFormats_ISO8601WithMsZ_OutputTimezoneAsEST() throws IOException { baseRecord.put("createTimeString", "2020-04-01T13:01:33.123Z"); properties = this.getBaseKeyConfig( "createTimeString", @@ -414,14 +416,14 @@ public void test_ExpectsMatch_MultipleInputFormats_ISO8601WithMsZ_OutputTimezone "EST"); BuiltinKeyGenerator keyGen = new TimestampBasedKeyGenerator(properties); HoodieKey hk1 = keyGen.getKey(baseRecord); - Assertions.assertEquals("2020040109", hk1.getPartitionPath()); + assertEquals("2020040109", hk1.getPartitionPath()); baseRow = genericRecordToRow(baseRecord); assertEquals("2020040109", keyGen.getPartitionPath(baseRow)); } @Test - public void test_Throws_MultipleInputFormats_InputDateNotMatchingFormats() throws IOException { + void test_Throws_MultipleInputFormats_InputDateNotMatchingFormats() throws IOException { baseRecord.put("createTimeString", "2020-04-01 13:01:33.123-05:00"); properties = this.getBaseKeyConfig( "createTimeString", @@ -432,14 +434,14 @@ public void test_Throws_MultipleInputFormats_InputDateNotMatchingFormats() throw "yyyyMMddHH", "UTC"); BuiltinKeyGenerator keyGen = new TimestampBasedKeyGenerator(properties); - Assertions.assertThrows(HoodieKeyGeneratorException.class, () -> keyGen.getKey(baseRecord)); + assertThrows(HoodieKeyGeneratorException.class, () -> keyGen.getKey(baseRecord)); baseRow = genericRecordToRow(baseRecord); - Assertions.assertThrows(HoodieKeyGeneratorException.class, () -> keyGen.getPartitionPath(baseRow)); + assertThrows(HoodieKeyGeneratorException.class, () -> keyGen.getPartitionPath(baseRow)); } @Test - public void test_ExpectsMatch_MultipleInputFormats_ShortDate_OutputCustomDate() throws IOException { + void test_ExpectsMatch_MultipleInputFormats_ShortDate_OutputCustomDate() throws IOException { baseRecord.put("createTimeString", "20200401"); properties = this.getBaseKeyConfig( "createTimeString", @@ -451,7 +453,7 @@ public void test_ExpectsMatch_MultipleInputFormats_ShortDate_OutputCustomDate() "UTC"); BuiltinKeyGenerator keyGen = new TimestampBasedKeyGenerator(properties); HoodieKey hk1 = keyGen.getKey(baseRecord); - Assertions.assertEquals("04/01/2020", hk1.getPartitionPath()); + assertEquals("04/01/2020", hk1.getPartitionPath()); baseRow = genericRecordToRow(baseRecord); assertEquals("04/01/2020", keyGen.getPartitionPath(baseRow)); diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/keygen/factory/TestCreateKeyGeneratorByTypeWithFactory.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/keygen/factory/TestCreateKeyGeneratorByTypeWithFactory.java similarity index 88% rename from hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/keygen/factory/TestCreateKeyGeneratorByTypeWithFactory.java rename to hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/keygen/factory/TestCreateKeyGeneratorByTypeWithFactory.java index dc597df2cf5c2..238c45c81c1ac 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/keygen/factory/TestCreateKeyGeneratorByTypeWithFactory.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/keygen/factory/TestCreateKeyGeneratorByTypeWithFactory.java @@ -7,13 +7,14 @@ * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. */ package org.apache.hudi.keygen.factory; @@ -43,7 +44,7 @@ import java.io.IOException; import java.util.stream.Stream; -public class TestCreateKeyGeneratorByTypeWithFactory { +class TestCreateKeyGeneratorByTypeWithFactory { private TypedProperties props; @@ -54,7 +55,7 @@ private static Stream configParams() { } @BeforeEach - public void init() { + void init() { props = new TypedProperties(); props.put(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key(), "_row_key"); props.put(KeyGeneratorOptions.HIVE_STYLE_PARTITIONING_ENABLE.key(), "true"); @@ -67,13 +68,13 @@ public void init() { } @AfterEach - public void teardown() { + void teardown() { props = null; } @ParameterizedTest @MethodSource("configParams") - public void testKeyGeneratorTypes(String keyGenType) throws IOException { + void testKeyGeneratorTypes(String keyGenType) throws IOException { props.put(HoodieWriteConfig.KEYGENERATOR_TYPE.key(), keyGenType); KeyGeneratorType keyType = KeyGeneratorType.valueOf(keyGenType); @@ -107,7 +108,7 @@ public void testKeyGeneratorTypes(String keyGenType) throws IOException { } @Test - public void testAutoRecordKeyGenerator() throws IOException { + void testAutoRecordKeyGenerator() throws IOException { props = new TypedProperties(); props.put(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key(), "partition"); props.put(KeyGenUtils.RECORD_KEY_GEN_INSTANT_TIME_CONFIG, "100"); diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/keygen/factory/TestHoodieSparkKeyGeneratorFactory.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/keygen/factory/TestHoodieSparkKeyGeneratorFactory.java similarity index 91% rename from hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/keygen/factory/TestHoodieSparkKeyGeneratorFactory.java rename to hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/keygen/factory/TestHoodieSparkKeyGeneratorFactory.java index 3cc30e86399f0..3c2bd1c4b68f6 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/keygen/factory/TestHoodieSparkKeyGeneratorFactory.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/keygen/factory/TestHoodieSparkKeyGeneratorFactory.java @@ -7,13 +7,14 @@ * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. */ package org.apache.hudi.keygen.factory; @@ -45,7 +46,7 @@ */ public class TestHoodieSparkKeyGeneratorFactory { @Test - public void testInferKeyGeneratorTypeFromWriteConfig() { + void testInferKeyGeneratorTypeFromWriteConfig() { assertEquals( KeyGeneratorType.NON_PARTITION, HoodieSparkKeyGeneratorFactory.inferKeyGeneratorTypeFromWriteConfig(new TypedProperties())); @@ -67,7 +68,7 @@ public void testInferKeyGeneratorTypeFromWriteConfig() { } @Test - public void testKeyGeneratorFactory() throws IOException { + void testKeyGeneratorFactory() throws IOException { TypedProperties props = getCommonProps(); // set KeyGenerator type only diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestUtils.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestUtils.java index a7808ea938248..f5f17e1d9bc55 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestUtils.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestUtils.java @@ -36,6 +36,7 @@ import org.apache.hudi.common.table.view.FileSystemViewStorageConfig; import org.apache.hudi.common.table.view.HoodieTableFileSystemView; import org.apache.hudi.common.table.view.TableFileSystemView.BaseFileOnlyView; +import org.apache.hudi.common.testutils.HoodieTestUtils; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.ReflectionUtils; import org.apache.hudi.config.HoodieWriteConfig; @@ -57,6 +58,7 @@ import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -344,4 +346,8 @@ private static boolean canLoadClass(String className) { return false; } } + + public static HoodieTableMetaClient createMetaClient(SparkSession spark, String basePath) { + return HoodieTestUtils.createMetaClient(spark.sessionState().newHadoopConf(), basePath); + } } diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/SparkDatasetTestUtils.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/SparkDatasetTestUtils.java index 09e6bd699bce1..59b4417376f84 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/SparkDatasetTestUtils.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/SparkDatasetTestUtils.java @@ -94,7 +94,7 @@ public class SparkDatasetTestUtils { * @param schema instance of {@link StructType} for which encoder is requested. * @return the encoder thus generated. */ - private static ExpressionEncoder getEncoder(StructType schema) { + public static ExpressionEncoder getEncoder(StructType schema) { List attributes = JavaConversions.asJavaCollection(schema.toAttributes()).stream() .map(Attribute::toAttribute).collect(Collectors.toList()); return RowEncoder.apply(schema) @@ -191,7 +191,7 @@ public static HoodieWriteConfig.Builder getConfigBuilder(String basePath, int ti .withBulkInsertParallelism(2); } - private static InternalRow serializeRow(ExpressionEncoder encoder, Row row) + public static InternalRow serializeRow(ExpressionEncoder encoder, Row row) throws InvocationTargetException, IllegalAccessException, NoSuchMethodException, ClassNotFoundException { // TODO remove reflection if Spark 2.x support is dropped if (package$.MODULE$.SPARK_VERSION().startsWith("2.")) { diff --git a/hudi-common/src/main/java/org/apache/hudi/common/config/ConfigProperty.java b/hudi-common/src/main/java/org/apache/hudi/common/config/ConfigProperty.java index d4ed193a0416a..f7e28655031ac 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/config/ConfigProperty.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/config/ConfigProperty.java @@ -31,6 +31,7 @@ import java.util.Objects; import java.util.Set; import java.util.function.Function; +import java.util.stream.Collectors; /** * ConfigProperty describes a configuration property. It contains the configuration @@ -54,6 +55,8 @@ public class ConfigProperty implements Serializable { private final Option deprecatedVersion; + private final List supportedVersions; + private final Set validValues; private final boolean advanced; @@ -65,6 +68,7 @@ public class ConfigProperty implements Serializable { ConfigProperty(String key, T defaultValue, String docOnDefaultValue, String doc, Option sinceVersion, Option deprecatedVersion, + List supportedVersions, Option>> inferFunc, Set validValues, boolean advanced, String... alternatives) { this.key = Objects.requireNonNull(key); @@ -73,6 +77,7 @@ public class ConfigProperty implements Serializable { this.doc = doc; this.sinceVersion = sinceVersion; this.deprecatedVersion = deprecatedVersion; + this.supportedVersions = supportedVersions; this.inferFunction = inferFunc; this.validValues = validValues; this.advanced = advanced; @@ -111,6 +116,10 @@ public Option getDeprecatedVersion() { return deprecatedVersion; } + public List getSupportedVersions() { + return supportedVersions; + } + public boolean hasInferFunction() { return getInferFunction().isPresent(); } @@ -141,7 +150,7 @@ public boolean isAdvanced() { public ConfigProperty withDocumentation(String doc) { Objects.requireNonNull(doc); - return new ConfigProperty<>(key, defaultValue, docOnDefaultValue, doc, sinceVersion, deprecatedVersion, inferFunction, validValues, advanced, alternatives); + return new ConfigProperty<>(key, defaultValue, docOnDefaultValue, doc, sinceVersion, deprecatedVersion, supportedVersions, inferFunction, validValues, advanced, alternatives); } public > ConfigProperty withDocumentation(Class e) { @@ -184,39 +193,45 @@ public > ConfigProperty withDocumentation(Class e, Strin } } - return new ConfigProperty<>(key, defaultValue, docOnDefaultValue, sb.toString(), sinceVersion, deprecatedVersion, inferFunction, validValues, advanced, alternatives); + return new ConfigProperty<>(key, defaultValue, docOnDefaultValue, sb.toString(), sinceVersion, deprecatedVersion, supportedVersions, inferFunction, validValues, advanced, alternatives); } public ConfigProperty withValidValues(String... validValues) { Objects.requireNonNull(validValues); - return new ConfigProperty<>(key, defaultValue, docOnDefaultValue, doc, sinceVersion, deprecatedVersion, inferFunction, new HashSet<>(Arrays.asList(validValues)), advanced, alternatives); + return new ConfigProperty<>(key, defaultValue, docOnDefaultValue, doc, sinceVersion, deprecatedVersion, supportedVersions, inferFunction, new HashSet<>(Arrays.asList(validValues)), advanced, + alternatives); } public ConfigProperty withAlternatives(String... alternatives) { Objects.requireNonNull(alternatives); - return new ConfigProperty<>(key, defaultValue, docOnDefaultValue, doc, sinceVersion, deprecatedVersion, inferFunction, validValues, advanced, alternatives); + return new ConfigProperty<>(key, defaultValue, docOnDefaultValue, doc, sinceVersion, deprecatedVersion, supportedVersions, inferFunction, validValues, advanced, alternatives); } public ConfigProperty sinceVersion(String sinceVersion) { Objects.requireNonNull(sinceVersion); - return new ConfigProperty<>(key, defaultValue, docOnDefaultValue, doc, Option.of(sinceVersion), deprecatedVersion, inferFunction, validValues, advanced, alternatives); + return new ConfigProperty<>(key, defaultValue, docOnDefaultValue, doc, Option.of(sinceVersion), deprecatedVersion, supportedVersions, inferFunction, validValues, advanced, alternatives); } public ConfigProperty deprecatedAfter(String deprecatedVersion) { Objects.requireNonNull(deprecatedVersion); - return new ConfigProperty<>(key, defaultValue, docOnDefaultValue, doc, sinceVersion, Option.of(deprecatedVersion), inferFunction, validValues, advanced, alternatives); + return new ConfigProperty<>(key, defaultValue, docOnDefaultValue, doc, sinceVersion, Option.of(deprecatedVersion), supportedVersions, inferFunction, validValues, advanced, alternatives); + } + + public ConfigProperty supportedVersions(String... supportedVersions) { + return new ConfigProperty<>(key, defaultValue, docOnDefaultValue, doc, sinceVersion, deprecatedVersion, + Arrays.stream(supportedVersions).collect(Collectors.toList()), inferFunction, validValues, advanced, alternatives); } public ConfigProperty withInferFunction(Function> inferFunction) { Objects.requireNonNull(inferFunction); - return new ConfigProperty<>(key, defaultValue, docOnDefaultValue, doc, sinceVersion, deprecatedVersion, Option.of(inferFunction), validValues, advanced, alternatives); + return new ConfigProperty<>(key, defaultValue, docOnDefaultValue, doc, sinceVersion, deprecatedVersion, supportedVersions, Option.of(inferFunction), validValues, advanced, alternatives); } /** * Marks the config as an advanced config. */ public ConfigProperty markAdvanced() { - return new ConfigProperty<>(key, defaultValue, docOnDefaultValue, doc, sinceVersion, deprecatedVersion, inferFunction, validValues, true, alternatives); + return new ConfigProperty<>(key, defaultValue, docOnDefaultValue, doc, sinceVersion, deprecatedVersion, supportedVersions, inferFunction, validValues, true, alternatives); } /** @@ -256,8 +271,8 @@ public ConfigProperty defaultValue(T value) { public ConfigProperty defaultValue(T value, String docOnDefaultValue) { Objects.requireNonNull(value); Objects.requireNonNull(docOnDefaultValue); - ConfigProperty configProperty = new ConfigProperty<>(key, value, docOnDefaultValue, "", Option.empty(), Option.empty(), Option.empty(), Collections.emptySet(), false); - return configProperty; + return new ConfigProperty<>(key, value, docOnDefaultValue, "", Option.empty(), + Option.empty(), Collections.emptyList(), Option.empty(), Collections.emptySet(), false); } public ConfigProperty noDefaultValue() { @@ -265,9 +280,8 @@ public ConfigProperty noDefaultValue() { } public ConfigProperty noDefaultValue(String docOnDefaultValue) { - ConfigProperty configProperty = new ConfigProperty<>(key, null, docOnDefaultValue, "", Option.empty(), - Option.empty(), Option.empty(), Collections.emptySet(), false); - return configProperty; + return new ConfigProperty<>(key, null, docOnDefaultValue, "", Option.empty(), + Option.empty(), Collections.emptyList(), Option.empty(), Collections.emptySet(), false); } } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java b/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java index 589f1e6cfbf77..c84ce4a6320dd 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java @@ -433,40 +433,6 @@ private HoodieArchivedTimeline instantiateArchivedTimeline(String startTs) { : new HoodieArchivedTimeline(this, startTs); } - /** - * Validate table properties. - * - * @param properties Properties from writeConfig. - */ - public void validateTableProperties(Properties properties) { - // Once meta fields are disabled, it cant be re-enabled for a given table. - if (!getTableConfig().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 (!getTableConfig().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); - } - } - - //Check to make sure it's not a COW table with consistent hashing bucket index - if (tableType == HoodieTableType.COPY_ON_WRITE) { - String indexType = properties.getProperty("hoodie.index.type"); - if (indexType != null && indexType.equals("BUCKET")) { - String bucketEngine = properties.getProperty("hoodie.index.bucket.engine"); - if (bucketEngine != null && bucketEngine.equals("CONSISTENT_HASHING")) { - throw new HoodieException("Consistent hashing bucket index does not work with COW table. Use simple bucket index or an MOR table."); - } - } - } - } - /** * Helper method to initialize a given path as a hoodie table with configs passed in as Properties. * diff --git a/hudi-common/src/main/java/org/apache/hudi/keygen/constant/KeyGeneratorType.java b/hudi-common/src/main/java/org/apache/hudi/keygen/constant/KeyGeneratorType.java index 5434b4901c0ce..296a97139d7bb 100644 --- a/hudi-common/src/main/java/org/apache/hudi/keygen/constant/KeyGeneratorType.java +++ b/hudi-common/src/main/java/org/apache/hudi/keygen/constant/KeyGeneratorType.java @@ -20,11 +20,14 @@ import org.apache.hudi.common.config.EnumDescription; import org.apache.hudi.common.config.EnumFieldDescription; +import org.apache.hudi.common.config.HoodieConfig; import java.util.ArrayList; import java.util.Arrays; import java.util.List; +import static org.apache.hudi.common.table.HoodieTableConfig.KEY_GENERATOR_CLASS_NAME; + /** * Types of {@link org.apache.hudi.keygen.KeyGenerator}. */ @@ -60,4 +63,18 @@ public static List getNames() { .forEach(x -> names.add(x.name())); return names; } + + public static boolean isComplexKeyGenerator(HoodieConfig config) { + if (config.contains(KEY_GENERATOR_CLASS_NAME)) { + try { + String keyGeneratorClass = config.getString(KEY_GENERATOR_CLASS_NAME).trim(); + return keyGeneratorClass.equals("org.apache.hudi.keygen.ComplexAvroKeyGenerator") + || keyGeneratorClass.equals("org.apache.hudi.keygen.ComplexKeyGenerator"); + } catch (IllegalArgumentException e) { + return false; + } + } else { + return false; + } + } } diff --git a/hudi-common/src/test/java/org/apache/hudi/common/config/TestConfigProperty.java b/hudi-common/src/test/java/org/apache/hudi/common/config/TestConfigProperty.java index dbbd45a4e4a71..910bd2d2e6b5f 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/config/TestConfigProperty.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/config/TestConfigProperty.java @@ -22,7 +22,10 @@ import org.junit.jupiter.api.Test; +import java.util.Arrays; +import java.util.Collections; import java.util.Properties; +import java.util.stream.Collectors; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; @@ -37,6 +40,7 @@ public class TestConfigProperty extends HoodieConfig { public static ConfigProperty FAKE_STRING_CONFIG = ConfigProperty .key("test.fake.string.config") .defaultValue("1") + .supportedVersions("a.b.c", "d.e.f") .withAlternatives("test.fake.string.alternative.config") .withDocumentation("Fake config only for testing"); @@ -195,4 +199,12 @@ void testEnumConfigs() { assertEquals(" TEST_VAL_B(default): Test val b", lines[2]); assertEquals(" OTHER_VAL: Other val", lines[3]); } + + @Test + void testGetSupportedVersions() { + assertEquals( + Arrays.stream(new String[] {"a.b.c", "d.e.f"}).collect(Collectors.toList()), + FAKE_STRING_CONFIG.getSupportedVersions()); + assertEquals(Collections.EMPTY_LIST, FAKE_INTEGER_CONFIG.getSupportedVersions()); + } } diff --git a/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestUtils.java b/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestUtils.java index 7100ab9af3419..a11e65dc78045 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestUtils.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestUtils.java @@ -255,4 +255,8 @@ public static DistributedFileSystem useExternalHdfs() throws IOException { conf.set("dfs.replication", "3"); return (DistributedFileSystem) DistributedFileSystem.get(conf); } + + public static HoodieTableMetaClient createMetaClient(Configuration conf, String basePath) { + return HoodieTableMetaClient.builder().setConf(conf).setBasePath(basePath).build(); + } } diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/configuration/OptionsResolver.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/configuration/OptionsResolver.java index 934e22f11397f..4554d7b65c471 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/configuration/OptionsResolver.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/configuration/OptionsResolver.java @@ -365,6 +365,14 @@ public static boolean isLazyFailedWritesCleanPolicy(Configuration conf) { .equalsIgnoreCase(HoodieFailedWritesCleaningPolicy.LAZY.name()); } + /** + * Returns whether complex keygen encodes single record key with field name. + */ + public static boolean useComplexKeygenNewEncoding(Configuration conf) { + return Boolean.parseBoolean(conf.getString(HoodieWriteConfig.COMPLEX_KEYGEN_NEW_ENCODING.key(), + HoodieWriteConfig.COMPLEX_KEYGEN_NEW_ENCODING.defaultValue().toString())); + } + // ------------------------------------------------------------------------- // Utilities // ------------------------------------------------------------------------- diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/bulk/AutoRowDataKeyGen.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/bulk/AutoRowDataKeyGen.java index bcae6fcd7b6cb..0b169425c466a 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/bulk/AutoRowDataKeyGen.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/bulk/AutoRowDataKeyGen.java @@ -21,6 +21,7 @@ import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.util.Option; import org.apache.hudi.configuration.FlinkOptions; +import org.apache.hudi.configuration.OptionsResolver; import org.apache.flink.configuration.Configuration; import org.apache.flink.table.data.RowData; @@ -40,15 +41,18 @@ public AutoRowDataKeyGen( String partitionFields, RowType rowType, boolean hiveStylePartitioning, - boolean encodePartitionPath) { - super(Option.empty(), partitionFields, rowType, hiveStylePartitioning, encodePartitionPath, false, Option.empty()); + boolean encodePartitionPath, + boolean useCompkexKeygenNewEncoding) { + super(Option.empty(), partitionFields, rowType, hiveStylePartitioning, encodePartitionPath, false, Option.empty(), + useCompkexKeygenNewEncoding); this.taskId = taskId; this.instantTime = instantTime; } public static RowDataKeyGen instance(Configuration conf, RowType rowType, int taskId, String instantTime) { return new AutoRowDataKeyGen(taskId, instantTime, conf.getString(FlinkOptions.PARTITION_PATH_FIELD), - rowType, conf.getBoolean(FlinkOptions.HIVE_STYLE_PARTITIONING), conf.getBoolean(FlinkOptions.URL_ENCODE_PARTITIONING)); + rowType, conf.getBoolean(FlinkOptions.HIVE_STYLE_PARTITIONING), conf.getBoolean(FlinkOptions.URL_ENCODE_PARTITIONING), + OptionsResolver.useComplexKeygenNewEncoding(conf)); } @Override diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/bulk/RowDataKeyGen.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/bulk/RowDataKeyGen.java index a9f34b36d2772..ab1a0d4c1b588 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/bulk/RowDataKeyGen.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/bulk/RowDataKeyGen.java @@ -18,6 +18,7 @@ package org.apache.hudi.sink.bulk; +import org.apache.hudi.common.function.SerializableFunctionUnchecked; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.StringUtils; @@ -41,6 +42,7 @@ import static org.apache.hudi.common.util.PartitionPathEncodeUtils.DEFAULT_PARTITION_PATH; import static org.apache.hudi.common.util.PartitionPathEncodeUtils.escapePathName; +import static org.apache.hudi.keygen.KeyGenUtils.DEFAULT_COLUMN_VALUE_SEPARATOR; /** * Key generator for {@link RowData}. @@ -70,7 +72,7 @@ public class RowDataKeyGen implements Serializable { private final Option keyGenOpt; // efficient code path - private boolean simpleRecordKey = false; + private SerializableFunctionUnchecked simpleRecordKeyFunc; private RowData.FieldGetter recordKeyFieldGetter; private boolean simplePartitionPath = false; @@ -85,7 +87,8 @@ protected RowDataKeyGen( boolean hiveStylePartitioning, boolean encodePartitionPath, boolean consistentLogicalTimestampEnabled, - Option keyGenOpt) { + Option keyGenOpt, + boolean useComplexKeygenNewEncoding) { this.partitionPathFields = partitionFields.split(","); this.hiveStylePartitioning = hiveStylePartitioning; this.encodePartitionPath = encodePartitionPath; @@ -94,6 +97,8 @@ protected RowDataKeyGen( List fieldNames = rowType.getFieldNames(); List fieldTypes = rowType.getChildren(); + boolean simpleRecordKey = false; + boolean multiplePartitions = false; if (!recordKeys.isPresent()) { this.recordKeyFields = null; this.recordKeyProjection = null; @@ -101,7 +106,7 @@ protected RowDataKeyGen( this.recordKeyFields = recordKeys.get().split(","); if (this.recordKeyFields.length == 1) { // efficient code path - this.simpleRecordKey = true; + simpleRecordKey = true; int recordKeyIdx = fieldNames.indexOf(this.recordKeyFields[0]); this.recordKeyFieldGetter = RowData.createFieldGetter(fieldTypes.get(recordKeyIdx), recordKeyIdx); this.recordKeyProjection = null; @@ -122,6 +127,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_COLUMN_VALUE_SEPARATOR).append(oriKey).toString(); + }; + } else { + this.simpleRecordKeyFunc = rowData -> getRecordKey(recordKeyFieldGetter.getFieldOrNull(rowData), this.recordKeyFields[0], consistentLogicalTimestampEnabled); + } } this.keyGenOpt = keyGenOpt; } @@ -138,7 +155,7 @@ public static RowDataKeyGen instance(Configuration conf, RowType rowType) { boolean consistentLogicalTimestampEnabled = OptionsResolver.isConsistentLogicalTimestampEnabled(conf); return new RowDataKeyGen(Option.of(conf.getString(FlinkOptions.RECORD_KEY_FIELD)), conf.getString(FlinkOptions.PARTITION_PATH_FIELD), rowType, conf.getBoolean(FlinkOptions.HIVE_STYLE_PARTITIONING), conf.getBoolean(FlinkOptions.URL_ENCODE_PARTITIONING), - consistentLogicalTimestampEnabled, keyGeneratorOpt); + consistentLogicalTimestampEnabled, keyGeneratorOpt, OptionsResolver.useComplexKeygenNewEncoding(conf)); } public HoodieKey getHoodieKey(RowData rowData) { @@ -146,8 +163,8 @@ public HoodieKey getHoodieKey(RowData rowData) { } public String getRecordKey(RowData rowData) { - if (this.simpleRecordKey) { - return getRecordKey(recordKeyFieldGetter.getFieldOrNull(rowData), this.recordKeyFields[0], consistentLogicalTimestampEnabled); + if (this.simpleRecordKeyFunc != null) { + return this.simpleRecordKeyFunc.apply(rowData); } else { Object[] keyValues = this.recordKeyProjection.projectAsValues(rowData); return getRecordKey(keyValues, this.recordKeyFields, consistentLogicalTimestampEnabled); diff --git a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/bulk/TestRowDataKeyGen.java b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/bulk/TestRowDataKeyGen.java index d222489bd80d3..4364f61cc6c34 100644 --- a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/bulk/TestRowDataKeyGen.java +++ b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/bulk/TestRowDataKeyGen.java @@ -18,6 +18,7 @@ package org.apache.hudi.sink.bulk; +import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.configuration.FlinkOptions; import org.apache.hudi.exception.HoodieKeyException; import org.apache.hudi.keygen.constant.KeyGeneratorOptions; @@ -73,6 +74,38 @@ void testSimpleKeyAndPartition() { assertThat(keyGen2.getPartitionPath(rowData3), is(String.format("partition=%s", DEFAULT_PARTITION_PATH))); } + @ParameterizedTest + @ValueSource(booleans = {true, false}) + void testSingleKeyMultiplePartitionFields(boolean encodeSingleKeyFieldValueOnly) { + Configuration conf = TestConfigurations.getDefaultConf("path1"); + conf.set(FlinkOptions.RECORD_KEY_FIELD, "uuid"); + conf.set(FlinkOptions.PARTITION_PATH_FIELD, "partition,ts"); + conf.setString(HoodieWriteConfig.COMPLEX_KEYGEN_NEW_ENCODING.key(), String.valueOf(encodeSingleKeyFieldValueOnly)); + RowData rowData1 = insertRow(StringData.fromString("id1"), StringData.fromString("Danny"), 23, + TimestampData.fromEpochMillis(1), StringData.fromString("par1")); + RowDataKeyGen keyGen1 = RowDataKeyGen.instance(conf, TestConfigurations.ROW_TYPE); + String expectedKey = encodeSingleKeyFieldValueOnly ? "id1" : "uuid:id1"; + assertThat(keyGen1.getRecordKey(rowData1), is(expectedKey)); + assertThat(keyGen1.getPartitionPath(rowData1), is("par1/1970-01-01T00:00:00.001")); + + // null record key and partition path + final RowData rowData2 = insertRow(TestConfigurations.ROW_TYPE, null, null, 23, null, null); + assertThrows(HoodieKeyException.class, () -> keyGen1.getRecordKey(rowData2)); + assertThat(keyGen1.getPartitionPath(rowData2), is(String.format("%s/%s", DEFAULT_PARTITION_PATH, DEFAULT_PARTITION_PATH))); + // empty record key and partition path + final RowData rowData3 = insertRow(StringData.fromString(""), StringData.fromString(""), 23, + TimestampData.fromEpochMillis(1), StringData.fromString("")); + assertThrows(HoodieKeyException.class, () -> keyGen1.getRecordKey(rowData3)); + assertThat(keyGen1.getPartitionPath(rowData3), is(String.format("%s/1970-01-01T00:00:00.001", DEFAULT_PARTITION_PATH))); + + // hive style partitioning + conf.set(FlinkOptions.HIVE_STYLE_PARTITIONING, true); + final RowDataKeyGen keyGen2 = RowDataKeyGen.instance(conf, TestConfigurations.ROW_TYPE); + assertThat(keyGen2.getPartitionPath(rowData1), is(String.format("partition=%s/ts=%s", "par1", "1970-01-01T00:00:00.001"))); + assertThat(keyGen2.getPartitionPath(rowData2), is(String.format("partition=%s/ts=%s", DEFAULT_PARTITION_PATH, DEFAULT_PARTITION_PATH))); + assertThat(keyGen2.getPartitionPath(rowData3), is(String.format("partition=%s/ts=%s", DEFAULT_PARTITION_PATH, "1970-01-01T00:00:00.001"))); + } + @Test void testComplexKeyAndPartition() { Configuration conf = TestConfigurations.getDefaultConf("path1"); @@ -209,5 +242,4 @@ void testRecordKeyContainsTimestamp() { assertThat(keyGen2.getRecordKey(rowData1), is("uuid:id1,ts:1675841687000")); } - } diff --git a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/table/ITTestSchemaEvolution.java b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/table/ITTestSchemaEvolution.java index 0417285815a97..c73f1c83a7007 100644 --- a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/table/ITTestSchemaEvolution.java +++ b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/table/ITTestSchemaEvolution.java @@ -26,7 +26,6 @@ import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.configuration.FlinkOptions; import org.apache.hudi.internal.schema.Types; -import org.apache.hudi.keygen.ComplexAvroKeyGenerator; import org.apache.hudi.keygen.constant.KeyGeneratorOptions; import org.apache.hudi.util.AvroSchemaConverter; import org.apache.hudi.util.FlinkWriteClients; @@ -320,7 +319,6 @@ private TableOptions defaultTableOptions(String tablePath) { KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key(), "uuid", KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key(), "partition", KeyGeneratorOptions.HIVE_STYLE_PARTITIONING_ENABLE.key(), true, - HoodieWriteConfig.KEYGENERATOR_CLASS_NAME.key(), ComplexAvroKeyGenerator.class.getName(), FlinkOptions.WRITE_BATCH_SIZE.key(), 0.000001, // each record triggers flush FlinkOptions.SOURCE_AVRO_SCHEMA.key(), AvroSchemaConverter.convertToSchema(ROW_TYPE_EVOLUTION_BEFORE), FlinkOptions.READ_TASKS.key(), 1, @@ -493,7 +491,7 @@ private ExpectedResult(String[] evolvedRows, String[] rowsWithMeta, String[] row "+I[id6, Emma, null, 20, +I[null, s6, 6, null, null, 6], {Emma=2020.0}, [20.0], null, null, null]", "+I[id7, Bob, null, 44, +I[null, s7, 7, null, null, 7], {Bob=4444.0}, [44.0, 44.0], null, null, null]", "+I[id8, Han, null, 56, +I[null, s8, 8, null, null, 8], {Han=5656.0}, [56.0, 56.0, 56.0], null, null, null]", - "+I[id9, Alice, 90000.9, unknown, +I[9, s9, 99, t9, drop_add9, 9], {Alice=9999.99}, [9999.0, 9999.0], +I[9, 9], [9], {k9=v9}]", + "+I[id9, Alice, 90000.9, unknown, +I[9, s9, 99, t9, drop_add9, 9], {Alice=9999.99}, [9999.0, 9999.0], +I[9, 9], [9], {k9=v9}]" }, new String[] { "+I[1]", @@ -531,7 +529,7 @@ private ExpectedResult(String[] evolvedRows, String[] rowsWithMeta, String[] row "+I[Han, null, 56, +I[null, s8, 8, null, null, 8], {Han=5656.0}, [56.0, 56.0, 56.0], null, null, null]", "+I[Alice, 90000.9, unknown, +I[9, s9, 99, t9, drop_add9, 9], {Alice=9999.99}, [9999.0, 9999.0], +I[9, 9], [9], {k9=v9}]", "+I[Danny, 10000.1, 23, +I[1, s1, 11, t1, drop_add1, 1], {Danny=2323.23}, [23.0, 23.0, 23.0], +I[1, 1], [1], {k1=v1}]", - "+I[Julian, 30000.3, 53, +I[3, s3, 33, t3, drop_add3, 3], {Julian=5353.53}, [53.0], +I[3, 3], [3], {k3=v3}]", + "+I[Julian, 30000.3, 53, +I[3, s3, 33, t3, drop_add3, 3], {Julian=5353.53}, [53.0], +I[3, 3], [3], {k3=v3}]" }, new String[] { "+I[id0, Indica, null, 12, null, {Indica=1212.0}, [12.0], null, null, null]", @@ -545,7 +543,7 @@ private ExpectedResult(String[] evolvedRows, String[] rowsWithMeta, String[] row "+I[id8, Han, null, 56, +I[null, s8, 8, null, null, 8], {Han=5656.0}, [56.0, 56.0, 56.0], null, null, null]", "+I[id9, Alice, 90000.9, unknown, +I[9, s9, 99, t9, drop_add9, 9], {Alice=9999.99}, [9999.0, 9999.0], +I[9, 9], [9], {k9=v9}]", "+I[id1, Danny, 10000.1, 23, +I[1, s1, 11, t1, drop_add1, 1], {Danny=2323.23}, [23.0, 23.0, 23.0], +I[1, 1], [1], {k1=v1}]", - "+I[id3, Julian, 30000.3, 53, +I[3, s3, 33, t3, drop_add3, 3], {Julian=5353.53}, [53.0], +I[3, 3], [3], {k3=v3}]", + "+I[id3, Julian, 30000.3, 53, +I[3, s3, 33, t3, drop_add3, 3], {Julian=5353.53}, [53.0], +I[3, 3], [3], {k3=v3}]" }, new String[] { "+I[1]", diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/internal/DataSourceInternalWriterHelper.java b/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/internal/DataSourceInternalWriterHelper.java index 4ad6c2066a3c5..00421b1b34595 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/internal/DataSourceInternalWriterHelper.java +++ b/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/internal/DataSourceInternalWriterHelper.java @@ -70,7 +70,7 @@ public DataSourceInternalWriterHelper(String instantTime, HoodieWriteConfig writ this.writeClient.initTable(operationType, Option.of(instantTime)); this.metaClient = HoodieTableMetaClient.builder().setConf(configuration).setBasePath(writeConfig.getBasePath()).build(); - this.metaClient.validateTableProperties(writeConfig.getProps()); + this.writeClient.validateTableProperties(this.metaClient.getTableConfig(), writeConfig.getProps()); this.hoodieTable = HoodieSparkTable.create(writeConfig, new HoodieSparkEngineContext(new JavaSparkContext(sparkSession.sparkContext())), metaClient); this.writeClient.preWrite(instantTime, WriteOperationType.BULK_INSERT, metaClient); } diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/RecordLevelIndexSupport.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/RecordLevelIndexSupport.scala index 743ce0cc6c1df..6ee3bd3c92fde 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/RecordLevelIndexSupport.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/RecordLevelIndexSupport.scala @@ -23,6 +23,8 @@ import org.apache.hudi.common.config.HoodieMetadataConfig import org.apache.hudi.common.fs.FSUtils import org.apache.hudi.common.model.HoodieRecord.HoodieMetadataField import org.apache.hudi.common.table.HoodieTableMetaClient +import org.apache.hudi.keygen.KeyGenUtils +import org.apache.hudi.keygen.constant.KeyGeneratorType import org.apache.hudi.metadata.{HoodieTableMetadata, HoodieTableMetadataUtil} import org.apache.hudi.util.JFunction import org.apache.spark.api.java.JavaSparkContext @@ -124,7 +126,10 @@ class RecordLevelIndexSupport(spark: SparkSession, * @return Tuple of List of filtered queries and list of record key literals that need to be matched */ def filterQueriesWithRecordKey(queryFilters: Seq[Expression]): (List[Expression], List[String]) = { - if (!isIndexAvailable) { + if (!isIndexAvailable || KeyGenUtils.mayUseNewEncodingForComplexKeyGen(metaClient.getTableConfig)) { + (List.empty, List.empty) + } else if (KeyGeneratorType.isComplexKeyGenerator(metaClient.getTableConfig)) { + // Complex record keys filtering is not yet supported. Support was added in HUDI-8432. (List.empty, List.empty) } else { var recordKeyQueries: List[Expression] = List.empty diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestBootstrapRead.java b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestBootstrapRead.java index d926a3be5a4e2..c2a4626dfc1b8 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestBootstrapRead.java +++ b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestBootstrapRead.java @@ -19,6 +19,7 @@ package org.apache.hudi.functional; import org.apache.hudi.common.model.HoodieTableType; +import org.apache.hudi.config.HoodieWriteConfig; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; @@ -28,6 +29,7 @@ import org.junit.jupiter.params.provider.Arguments; import org.junit.jupiter.params.provider.MethodSource; +import java.util.HashMap; import java.util.Map; import java.util.stream.Stream; @@ -68,13 +70,18 @@ public void testBootstrapFunctional(String bootstrapType, Boolean dashPartitions this.dashPartitions = dashPartitions; this.tableType = tableType; this.nPartitions = nPartitions; - setupDirs(); + Map overrideOpts = new HashMap<>(); + if (nPartitions > 1) { + overrideOpts.put(HoodieWriteConfig.ENABLE_COMPLEX_KEYGEN_VALIDATION.key(), "false"); + } + setupDirs(overrideOpts); // do bootstrap Map options = setBootstrapOptions(); Dataset bootstrapDf = sparkSession.emptyDataFrame(); bootstrapDf.write().format("hudi") .options(options) + .options(overrideOpts) .mode(SaveMode.Overwrite) .save(bootstrapTargetPath); compareTables(); @@ -82,6 +89,7 @@ public void testBootstrapFunctional(String bootstrapType, Boolean dashPartitions // do upserts options = basicOptions(); + options.putAll(overrideOpts); doUpdate(options, "001"); compareTables(); verifyMetaColOnlyRead(1); diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestBootstrapReadBase.java b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestBootstrapReadBase.java index d3246f7c4da83..d80c2a0e836ad 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestBootstrapReadBase.java +++ b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestBootstrapReadBase.java @@ -220,7 +220,7 @@ protected void compareDf(Dataset df1, Dataset df2) { assertEquals(0, df2.except(df1).count()); } - protected void setupDirs() { + protected void setupDirs(Map overrideOpts) { dataGen = new HoodieTestDataGenerator(dashPartitions ? dashPartitionPaths : slashPartitionPaths); Dataset inserts = generateTestInserts("000", nInserts); if (dashPartitions) { @@ -240,6 +240,7 @@ protected void setupDirs() { inserts.write().format("hudi") .options(basicOptions()) + .options(overrideOpts) .mode(SaveMode.Overwrite) .save(hudiBasePath); } diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestNewHoodieParquetFileFormat.java b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestNewHoodieParquetFileFormat.java index ec719414dc8b9..c546e8c980e6f 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestNewHoodieParquetFileFormat.java +++ b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestNewHoodieParquetFileFormat.java @@ -31,6 +31,7 @@ import org.junit.jupiter.params.provider.MethodSource; import java.util.Arrays; +import java.util.Collections; import java.util.Map; import java.util.stream.Stream; @@ -61,7 +62,7 @@ public void testNewParquetFileFormat(HoodieTableType tableType, Integer nPartiti this.dashPartitions = true; this.tableType = tableType; this.nPartitions = nPartitions; - setupDirs(); + setupDirs(Collections.emptyMap()); //do bootstrap Map options = setBootstrapOptions(); diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/keygen/TestComplexKeyGenerator.java b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/keygen/TestComplexKeyGenerator.java index 296cf3d6e0db1..d9d1e51059b7c 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/keygen/TestComplexKeyGenerator.java +++ b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/keygen/TestComplexKeyGenerator.java @@ -118,7 +118,7 @@ public void testSingleValueKeyGenerator() { String rowKey = record.get("_row_key").toString(); String partitionPath = record.get("timestamp").toString(); HoodieKey hoodieKey = compositeKeyGenerator.getKey(record); - assertEquals(rowKey, hoodieKey.getRecordKey()); + assertEquals("_row_key:" + rowKey, hoodieKey.getRecordKey()); assertEquals(partitionPath, hoodieKey.getPartitionPath()); Row row = KeyGeneratorTestUtilities.getRow(record, HoodieTestDataGenerator.AVRO_SCHEMA, diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestDataSourceDefaults.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestDataSourceDefaults.scala index a2598c766b193..09364d8835249 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestDataSourceDefaults.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestDataSourceDefaults.scala @@ -17,20 +17,23 @@ package org.apache.hudi -import org.apache.avro.Schema -import org.apache.avro.generic.GenericRecord +import org.apache.hudi.HoodieSparkUtils.sparkAdapter import org.apache.hudi.avro.HoodieAvroUtils import org.apache.hudi.common.config.TypedProperties import org.apache.hudi.common.model._ -import org.apache.hudi.common.testutils.{SchemaTestUtil, PreCombineTestUtils} +import org.apache.hudi.common.testutils.{PreCombineTestUtils, SchemaTestUtil} import org.apache.hudi.common.util.Option import org.apache.hudi.common.util.PartitionPathEncodeUtils.DEFAULT_PARTITION_PATH -import org.apache.hudi.config.HoodiePayloadConfig +import org.apache.hudi.config.{HoodiePayloadConfig, HoodieWriteConfig} import org.apache.hudi.exception.{HoodieException, HoodieKeyException} import org.apache.hudi.keygen._ -import org.apache.hudi.testutils.KeyGeneratorTestUtilities +import org.apache.hudi.testutils.SparkDatasetTestUtils +import org.apache.avro.Schema +import org.apache.avro.generic.GenericRecord import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder +import org.apache.spark.sql.catalyst.expressions.GenericRowWithSchema import org.apache.spark.sql.types.StructType import org.apache.spark.unsafe.types.UTF8String import org.junit.jupiter.api.Assertions.assertEquals @@ -50,12 +53,13 @@ class TestDataSourceDefaults extends ScalaAssertionSupport { var internalRow: InternalRow = _ val testStructName = "testStructName" val testNamespace = "testNamespace" + val encoder: ExpressionEncoder[_] = SparkDatasetTestUtils.getEncoder(structType) @BeforeEach def initialize(): Unit = { baseRecord = SchemaTestUtil .generateAvroRecordFromJson(schema, 1, "001", "f1") - baseRow = KeyGeneratorTestUtilities.getRow(baseRecord, schema, structType) - internalRow = KeyGeneratorTestUtilities.getInternalRow(baseRow) + baseRow = getRow(baseRecord, schema, structType) + internalRow = getInternalRow(baseRow) } private def getKeyConfig(recordKeyFieldName: String, partitionPathField: String, hiveStylePartitioning: String): TypedProperties = { @@ -146,16 +150,16 @@ class TestDataSourceDefaults extends ScalaAssertionSupport { val keyGen = new SimpleKeyGenerator(getKeyConfig("field1", "name", "false")) baseRecord.put("name", "") - baseRow = KeyGeneratorTestUtilities.getRow(baseRecord, schema, structType) - internalRow = KeyGeneratorTestUtilities.getInternalRow(baseRow) + baseRow = getRow(baseRecord, schema, structType) + internalRow = getInternalRow(baseRow) assertEquals(DEFAULT_PARTITION_PATH, keyGen.getKey(baseRecord).getPartitionPath) assertEquals(DEFAULT_PARTITION_PATH, keyGen.getPartitionPath(baseRow)) assertEquals(UTF8String.fromString(DEFAULT_PARTITION_PATH), keyGen.getPartitionPath(internalRow, structType)) baseRecord.put("name", null) - baseRow = KeyGeneratorTestUtilities.getRow(baseRecord, schema, structType) - internalRow = KeyGeneratorTestUtilities.getInternalRow(baseRow) + baseRow = getRow(baseRecord, schema, structType) + internalRow = getInternalRow(baseRow) assertEquals(DEFAULT_PARTITION_PATH, keyGen.getKey(baseRecord).getPartitionPath) assertEquals(DEFAULT_PARTITION_PATH, keyGen.getPartitionPath(baseRow)) @@ -170,8 +174,8 @@ class TestDataSourceDefaults extends ScalaAssertionSupport { val keyGen = new SimpleKeyGenerator(props) baseRecord.put("field1", "") - baseRow = KeyGeneratorTestUtilities.getRow(baseRecord, schema, structType) - internalRow = KeyGeneratorTestUtilities.getInternalRow(baseRow) + baseRow = getRow(baseRecord, schema, structType) + internalRow = getInternalRow(baseRow) assertThrows(classOf[HoodieKeyException]) { keyGen.getKey(baseRecord) @@ -184,8 +188,8 @@ class TestDataSourceDefaults extends ScalaAssertionSupport { } baseRecord.put("field1", null) - baseRow = KeyGeneratorTestUtilities.getRow(baseRecord, schema, structType) - internalRow = KeyGeneratorTestUtilities.getInternalRow(baseRow) + baseRow = getRow(baseRecord, schema, structType) + internalRow = getInternalRow(baseRow) assertThrows(classOf[HoodieKeyException]) { keyGen.getKey(baseRecord) @@ -336,8 +340,8 @@ class TestDataSourceDefaults extends ScalaAssertionSupport { val keyGen = new ComplexKeyGenerator(getKeyConfig("field1,name", "field1,name", "false")) baseRecord.put("name", "") - baseRow = KeyGeneratorTestUtilities.getRow(baseRecord, schema, structType) - internalRow = KeyGeneratorTestUtilities.getInternalRow(baseRow) + baseRow = getRow(baseRecord, schema, structType) + internalRow = getInternalRow(baseRow) val expectedKey = new HoodieKey("field1:field1,name:__empty__", "field1/" + DEFAULT_PARTITION_PATH) @@ -354,8 +358,8 @@ class TestDataSourceDefaults extends ScalaAssertionSupport { val keyGen = new ComplexKeyGenerator(getKeyConfig("field1,name", "field1,name", "false")) baseRecord.put("name", null) - baseRow = KeyGeneratorTestUtilities.getRow(baseRecord, schema, structType) - internalRow = KeyGeneratorTestUtilities.getInternalRow(baseRow) + baseRow = getRow(baseRecord, schema, structType) + internalRow = getInternalRow(baseRow) val expectedKey = new HoodieKey("field1:field1,name:__null__", "field1/" + DEFAULT_PARTITION_PATH) @@ -377,8 +381,8 @@ class TestDataSourceDefaults extends ScalaAssertionSupport { baseRecord.put("name", "") baseRecord.put("field1", null) - baseRow = KeyGeneratorTestUtilities.getRow(baseRecord, schema, structType) - internalRow = KeyGeneratorTestUtilities.getInternalRow(baseRow) + baseRow = getRow(baseRecord, schema, structType) + internalRow = getInternalRow(baseRow) assertThrows(classOf[HoodieKeyException]) { keyGen.getKey(baseRecord) @@ -398,8 +402,27 @@ class TestDataSourceDefaults extends ScalaAssertionSupport { baseRecord.put("name", "name1") baseRecord.put("field1", "field1") - baseRow = KeyGeneratorTestUtilities.getRow(baseRecord, schema, structType) - internalRow = KeyGeneratorTestUtilities.getInternalRow(baseRow) + baseRow = getRow(baseRecord, schema, structType) + internalRow = getInternalRow(baseRow) + + val expectedKey = new HoodieKey("field1:field1,name:name1", "field1/name1") + + assertEquals(expectedKey, keyGen.getKey(baseRecord)) + + assertEquals(expectedKey.getRecordKey, keyGen.getRecordKey(baseRow)) + assertEquals(expectedKey.getPartitionPath, keyGen.getPartitionPath(baseRow)) + assertEquals(UTF8String.fromString(expectedKey.getRecordKey), keyGen.getRecordKey(internalRow, structType)) + assertEquals(UTF8String.fromString(expectedKey.getPartitionPath), keyGen.getPartitionPath(internalRow, structType)) + } + + { + val config = getKeyConfig("field1, name", "field1, name", "false") + // This config should not affect record key encoding for multiple record key fields + config.put(HoodieWriteConfig.COMPLEX_KEYGEN_NEW_ENCODING.key, "true") + val keyGen = new ComplexKeyGenerator(config) + + baseRow = getRow(baseRecord, schema, structType) + internalRow = getInternalRow(baseRow) val expectedKey = new HoodieKey("field1:field1,name:name1", "field1/name1") @@ -412,9 +435,48 @@ class TestDataSourceDefaults extends ScalaAssertionSupport { } { - val keyGen = new ComplexKeyGenerator(getKeyConfig("field1,", "field1,", "false")) + baseRecord.put("name", "value1") + baseRecord.put("field1", "value2") + // Default: encode record key field name if there is a single record key field + val keyGen = new ComplexKeyGenerator(getKeyConfig("name,", "field1,", "false")) - val expectedKey = new HoodieKey("field1", "field1") + val expectedKey = new HoodieKey("name:value1", "value2") + + assertEquals(expectedKey, keyGen.getKey(baseRecord)) + + baseRow = getRow(baseRecord, schema, structType) + internalRow = getInternalRow(baseRow) + + assertEquals(expectedKey.getRecordKey, keyGen.getRecordKey(baseRow)) + assertEquals(expectedKey.getPartitionPath, keyGen.getPartitionPath(baseRow)) + assertEquals(UTF8String.fromString(expectedKey.getRecordKey), keyGen.getRecordKey(internalRow, structType)) + assertEquals(UTF8String.fromString(expectedKey.getPartitionPath), keyGen.getPartitionPath(internalRow, structType)) + } + + { + // Testing that config set to true but expecting old format (field name included) + val config = getKeyConfig("name,", "field1,", "false") + config.put(HoodieWriteConfig.COMPLEX_KEYGEN_NEW_ENCODING.key, "false") + val keyGen = new ComplexKeyGenerator(config) + + val expectedKey = new HoodieKey("name:value1", "value2") + + assertEquals(expectedKey, keyGen.getKey(baseRecord)) + + assertEquals(expectedKey.getRecordKey, keyGen.getRecordKey(baseRow)) + assertEquals(expectedKey.getPartitionPath, keyGen.getPartitionPath(baseRow)) + assertEquals(UTF8String.fromString(expectedKey.getRecordKey), keyGen.getRecordKey(internalRow, structType)) + assertEquals(UTF8String.fromString(expectedKey.getPartitionPath), keyGen.getPartitionPath(internalRow, structType)) + } + + { + // Turning off the encoding of record key field name if there is a single record key field + val config = getKeyConfig("name,", "field1,", "false") + // This config should affect record key encoding for single record key fields + config.put(HoodieWriteConfig.COMPLEX_KEYGEN_NEW_ENCODING.key, "true") + val keyGen = new ComplexKeyGenerator(config) + + val expectedKey = new HoodieKey("value1", "value2") assertEquals(expectedKey, keyGen.getKey(baseRecord)) @@ -460,8 +522,8 @@ class TestDataSourceDefaults extends ScalaAssertionSupport { { // If one part of the record key is empty, replace with "__empty__" baseRecord.put("name", "") - baseRow = KeyGeneratorTestUtilities.getRow(baseRecord, schema, structType) - internalRow = KeyGeneratorTestUtilities.getInternalRow(baseRow) + baseRow = getRow(baseRecord, schema, structType) + internalRow = getInternalRow(baseRow) val keyGen = new GlobalDeleteKeyGenerator(getKeyConfig("field1,name", "field1,name", "false")) @@ -476,8 +538,8 @@ class TestDataSourceDefaults extends ScalaAssertionSupport { { // If one part of the record key is null, replace with "__null__" baseRecord.put("name", null) - baseRow = KeyGeneratorTestUtilities.getRow(baseRecord, schema, structType) - internalRow = KeyGeneratorTestUtilities.getInternalRow(baseRow) + baseRow = getRow(baseRecord, schema, structType) + internalRow = getInternalRow(baseRow) val keyGen = new GlobalDeleteKeyGenerator(getKeyConfig("field1,name", "field1,name", "false")) @@ -522,8 +584,8 @@ class TestDataSourceDefaults extends ScalaAssertionSupport { baseRecord.put("name", "") baseRecord.put("field1", null) - baseRow = KeyGeneratorTestUtilities.getRow(baseRecord, schema, structType) - internalRow = KeyGeneratorTestUtilities.getInternalRow(baseRow) + baseRow = getRow(baseRecord, schema, structType) + internalRow = getInternalRow(baseRow) assertThrows(classOf[HoodieKeyException]) { keyGen.getKey(baseRecord) @@ -622,4 +684,12 @@ class TestDataSourceDefaults extends ScalaAssertionSupport { val combined12 = combinedPayload12.getInsertValue(schema) assertEquals(Option.empty(), combined12) } + + def getRow(record: GenericRecord, schema: Schema, structType: StructType): Row = { + val converterFn = AvroConversionUtils.createConverterToRow(schema, structType) + val row = converterFn.apply(record) + new GenericRowWithSchema(structType.fieldNames.indices.map(i => row.get(i)).toArray, structType) + } + + def getInternalRow(row: Row): InternalRow = SparkDatasetTestUtils.serializeRow(encoder, row) } diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieFileIndex.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieFileIndex.scala index 803702addb489..1ba2b0e34e15f 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieFileIndex.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieFileIndex.scala @@ -122,6 +122,7 @@ class TestHoodieFileIndex extends HoodieSparkClientTestBase with ScalaAssertionS .option(TIMESTAMP_TYPE_FIELD.key, TimestampType.DATE_STRING.name()) .option(TIMESTAMP_INPUT_DATE_FORMAT.key, "yyyy/MM/dd") .option(TIMESTAMP_OUTPUT_DATE_FORMAT.key, "yyyy-MM-dd") + .option(HoodieWriteConfig.ENABLE_COMPLEX_KEYGEN_VALIDATION.key(), "false") .mode(SaveMode.Overwrite) if (isNullOrEmpty(keyGenerator)) { @@ -315,7 +316,8 @@ class TestHoodieFileIndex extends HoodieSparkClientTestBase with ScalaAssertionS RECORDKEY_FIELD.key -> "id", PRECOMBINE_FIELD.key -> "version", PARTITIONPATH_FIELD.key -> "dt,hh", - HoodieMetadataConfig.ENABLE.key -> useMetadataTable.toString + HoodieMetadataConfig.ENABLE.key -> useMetadataTable.toString, + HoodieWriteConfig.ENABLE_COMPLEX_KEYGEN_VALIDATION.key -> "false" ) val readerOpts: Map[String, String] = queryOpts ++ Map( @@ -458,7 +460,8 @@ class TestHoodieFileIndex extends HoodieSparkClientTestBase with ScalaAssertionS RECORDKEY_FIELD.key -> "id", PRECOMBINE_FIELD.key -> "version", PARTITIONPATH_FIELD.key -> partitionNames.mkString(","), - HoodieMetadataConfig.ENABLE.key -> useMetadataTable.toString + HoodieMetadataConfig.ENABLE.key -> useMetadataTable.toString, + HoodieWriteConfig.ENABLE_COMPLEX_KEYGEN_VALIDATION.key() -> "false" ) val readerOpts: Map[String, String] = queryOpts ++ Map( @@ -512,7 +515,8 @@ class TestHoodieFileIndex extends HoodieSparkClientTestBase with ScalaAssertionS DataSourceWriteOptions.OPERATION.key -> DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL, RECORDKEY_FIELD.key -> "id", PRECOMBINE_FIELD.key -> "version", - PARTITIONPATH_FIELD.key -> "dt,hh" + PARTITIONPATH_FIELD.key -> "dt,hh", + HoodieWriteConfig.ENABLE_COMPLEX_KEYGEN_VALIDATION.key() -> "false" ) val readerOpts: Map[String, String] = queryOpts ++ Map( @@ -572,7 +576,8 @@ class TestHoodieFileIndex extends HoodieSparkClientTestBase with ScalaAssertionS HoodieMetadataConfig.ENABLE.key -> enableMetadataTable.toString, RECORDKEY_FIELD.key -> "id", PARTITIONPATH_FIELD.key -> "region_code,dt", - DataSourceWriteOptions.PRECOMBINE_FIELD.key -> "price" + DataSourceWriteOptions.PRECOMBINE_FIELD.key -> "price", + HoodieWriteConfig.ENABLE_COMPLEX_KEYGEN_VALIDATION.key() -> "false" ) val readerOpts: Map[String, String] = queryOpts ++ Map( diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestCOWDataSource.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestCOWDataSource.scala index f500ea83120dc..6aca6d0fb027c 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestCOWDataSource.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestCOWDataSource.scala @@ -185,13 +185,14 @@ class TestCOWDataSource extends HoodieSparkClientTestBase with ScalaAssertionSup val inputDF1 = spark.read.json(spark.sparkContext.parallelize(records1, 2)) val records2 = recordsToStrings(dataGen.generateInserts("000", 200)).toList val inputDF2 = spark.read.json(spark.sparkContext.parallelize(records2, 2)) - // hard code the value for rider and fare so that we can verify the partitions paths with hudi + // hard code the value for rider and fare so that we can verify the partition paths with hudi val toInsertDf = inputDF1.withColumn("fare", lit(100)).withColumn("rider", lit("rider-123")) .union(inputDF2.withColumn("fare", lit(200)).withColumn("rider", lit("rider-456"))) toInsertDf.write.partitionBy("fare", "rider").format("hudi") .options(commonOptsNoPreCombine) .option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL) + .option(HoodieWriteConfig.ENABLE_COMPLEX_KEYGEN_VALIDATION.key(), "false") .mode(SaveMode.Overwrite) .save(basePath) @@ -211,6 +212,7 @@ class TestCOWDataSource extends HoodieSparkClientTestBase with ScalaAssertionSup toInsertDf.write.partitionBy("fare", "rider").format("hudi") .options(commonOptsNoPreCombine) .option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL) + .option(HoodieWriteConfig.ENABLE_COMPLEX_KEYGEN_VALIDATION.key(), "false") .mode(SaveMode.Append) .save(basePath) @@ -218,6 +220,7 @@ class TestCOWDataSource extends HoodieSparkClientTestBase with ScalaAssertionSup toInsertDf.write.partitionBy("fare", "rider").format("hudi") .options(commonOptsNoPreCombine) .option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL) + .option(HoodieWriteConfig.ENABLE_COMPLEX_KEYGEN_VALIDATION.key(), "false") .option(KEYGENERATOR_CLASS_NAME.key(), classOf[NonpartitionedKeyGenerator].getName) .mode(SaveMode.Overwrite) .save(basePath) @@ -1021,10 +1024,16 @@ class TestCOWDataSource extends HoodieSparkClientTestBase with ScalaAssertionSup private def getDataFrameWriter(keyGenerator: String, opts: Map[String, String]): DataFrameWriter[Row] = { val records = recordsToStrings(dataGen.generateInserts("000", 100)).toList val inputDF = spark.read.json(spark.sparkContext.parallelize(records, 2)) - inputDF.write.format("hudi") + val writer = inputDF.write.format("hudi") .options(opts) .option(DataSourceWriteOptions.KEYGENERATOR_CLASS_NAME.key, keyGenerator) .mode(SaveMode.Overwrite) + if (classOf[ComplexKeyGenerator].getCanonicalName.equals(keyGenerator)) { + // Disable complex key generator validation so that the writer can succeed + writer.option(HoodieWriteConfig.ENABLE_COMPLEX_KEYGEN_VALIDATION.key, "false") + } else { + writer + } } @ParameterizedTest @@ -1759,7 +1768,8 @@ class TestCOWDataSource extends HoodieSparkClientTestBase with ScalaAssertionSup HoodieWriteConfig.KEYGENERATOR_CLASS_NAME.key() -> "org.apache.hudi.keygen.ComplexKeyGenerator", KeyGeneratorOptions.HIVE_STYLE_PARTITIONING_ENABLE.key() -> "true", HiveSyncConfigHolder.HIVE_SYNC_ENABLED.key() -> "false", - HoodieWriteConfig.RECORD_MERGER_IMPLS.key() -> "org.apache.hudi.HoodieSparkRecordMerger" + HoodieWriteConfig.RECORD_MERGER_IMPLS.key() -> "org.apache.hudi.HoodieSparkRecordMerger", + HoodieWriteConfig.ENABLE_COMPLEX_KEYGEN_VALIDATION.key() -> "false" ) df1.write.format("hudi").options(hudiOptions).mode(SaveMode.Append).save(basePath) diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestGetPartitionValuesFromPath.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestGetPartitionValuesFromPath.scala index aadd9397f47d4..50d3db3a316f9 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestGetPartitionValuesFromPath.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestGetPartitionValuesFromPath.scala @@ -18,6 +18,7 @@ package org.apache.hudi.functional +import org.apache.hudi.config.HoodieWriteConfig import org.apache.spark.sql.hudi.HoodieSparkSqlTestBase class TestGetPartitionValuesFromPath extends HoodieSparkSqlTestBase { @@ -38,7 +39,8 @@ class TestGetPartitionValuesFromPath extends HoodieSparkSqlTestBase { |tblproperties ( | primaryKey = 'id', | type='mor', - | hoodie.datasource.write.hive_style_partitioning='$hiveStylePartitioning') + | hoodie.datasource.write.hive_style_partitioning='$hiveStylePartitioning', + | ${HoodieWriteConfig.ENABLE_COMPLEX_KEYGEN_VALIDATION.key()} = 'false') |partitioned by (region, dt)""".stripMargin) spark.sql(s"insert into $tableName partition (region='reg1', dt='2023-08-01') select 1, 'name1'") @@ -66,7 +68,8 @@ class TestGetPartitionValuesFromPath extends HoodieSparkSqlTestBase { | primaryKey = 'id', | type = 'mor', | preCombineField = 'ts', - | hoodie.datasource.write.drop.partition.columns = 'true' + | hoodie.datasource.write.drop.partition.columns = 'true', + | ${HoodieWriteConfig.ENABLE_COMPLEX_KEYGEN_VALIDATION.key()} = 'false' |) |partitioned by (region, dt)""".stripMargin) diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestRecordLevelIndexWithSQL.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestRecordLevelIndexWithSQL.scala index 8e235960fba33..41ebb01d44e1b 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestRecordLevelIndexWithSQL.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestRecordLevelIndexWithSQL.scala @@ -17,6 +17,7 @@ package org.apache.hudi.functional +import org.apache.hudi.DataSourceWriteOptions.RECORDKEY_FIELD import org.apache.hudi.common.model.{FileSlice, HoodieTableType} import org.apache.hudi.common.table.HoodieTableMetaClient import org.apache.hudi.metadata.HoodieMetadataFileSystemView @@ -26,7 +27,7 @@ import org.apache.spark.sql.SaveMode import org.apache.spark.sql.catalyst.expressions.{AttributeReference, EqualTo, Expression, GreaterThan, GreaterThanOrEqual, In, Literal, Or} import org.apache.spark.sql.types.StringType import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue} -import org.junit.jupiter.api.Tag +import org.junit.jupiter.api.{Tag, Test} import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.params.provider.ValueSource @@ -155,4 +156,32 @@ class TestRecordLevelIndexWithSQL extends RecordLevelIndexTestBase { val readDf = spark.read.format("hudi").options(hudiOpts).load(basePath) readDf.registerTempTable(sqlTempTable) } + + @Test + def testRLINoPruningWithComplexRecordKeys(): Unit = { + var hudiOpts = commonOpts + { + RECORDKEY_FIELD.key -> "_row_key,rider" + } + hudiOpts = hudiOpts + ( + DataSourceWriteOptions.TABLE_TYPE.key -> "COPY_ON_WRITE", + DataSourceReadOptions.ENABLE_DATA_SKIPPING.key -> "true") + + doWriteAndValidateDataAndRecordIndex(hudiOpts, + operation = DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL, + saveMode = SaveMode.Overwrite, + validate = false) + doWriteAndValidateDataAndRecordIndex(hudiOpts, + operation = DataSourceWriteOptions.UPSERT_OPERATION_OPT_VAL, + saveMode = SaveMode.Append, + validate = false) + + val indexOpts = hudiOpts + {"path" -> basePath} + metaClient = HoodieTableMetaClient.reload(metaClient) + val fileIndex = HoodieFileIndex(spark, metaClient, None, indexOpts, includeLogFiles = true) + // random data filter + val filteredPartitionDirectories = fileIndex.listFiles(Seq(), Seq(EqualTo(attribute("_row_key"), Literal("abc")))) + val filteredFilesCount = filteredPartitionDirectories.flatMap(s => s.files).size + // Assert no pruning with complex record keys + assertEquals(getLatestDataFilesCount(indexOpts, includeLogFiles = false), filteredFilesCount) + } } diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/cdc/TestCDCDataFrameSuite.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/cdc/TestCDCDataFrameSuite.scala index 210ea00048ef4..1865985811949 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/cdc/TestCDCDataFrameSuite.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/cdc/TestCDCDataFrameSuite.scala @@ -784,6 +784,7 @@ class TestCDCDataFrameSuite extends HoodieCDCTestBase { .option(HoodieWriteConfig.TBL_NAME.key(), tableName + loggingMode.name()) .option("hoodie.datasource.write.operation", "upsert") .option("hoodie.datasource.write.keygenerator.class", "org.apache.hudi.keygen.ComplexKeyGenerator") + .option(HoodieWriteConfig.ENABLE_COMPLEX_KEYGEN_VALIDATION.key(), "false") .option("hoodie.datasource.write.payload.class", "org.apache.hudi.common.model.AWSDmsAvroPayload") .option("hoodie.table.cdc.enabled", "true") .option("hoodie.table.cdc.supplemental.logging.mode", loggingMode.name()) @@ -804,6 +805,7 @@ class TestCDCDataFrameSuite extends HoodieCDCTestBase { .option(HoodieWriteConfig.TBL_NAME.key(), tableName + loggingMode.name()) .option("hoodie.datasource.write.operation", "upsert") .option("hoodie.datasource.write.keygenerator.class", "org.apache.hudi.keygen.ComplexKeyGenerator") + .option(HoodieWriteConfig.ENABLE_COMPLEX_KEYGEN_VALIDATION.key(), "false") .option("hoodie.datasource.write.payload.class", "org.apache.hudi.common.model.AWSDmsAvroPayload") .option("hoodie.table.cdc.enabled", "true") .option("hoodie.table.cdc.supplemental.logging.mode", loggingMode.name()) diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/HoodieSparkSqlTestBase.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/HoodieSparkSqlTestBase.scala index 2e06da2aee512..83f4c07f35a72 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/HoodieSparkSqlTestBase.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/HoodieSparkSqlTestBase.scala @@ -267,6 +267,26 @@ object HoodieSparkSqlTestBase { .getActiveTimeline.getInstantDetails(cleanInstant).get) } + def enableComplexKeygenValidation(spark: SparkSession, + tableName: String): Unit = { + setComplexKeygenValidation(spark, tableName, value = true) + } + + def disableComplexKeygenValidation(spark: SparkSession, + tableName: String): Unit = { + setComplexKeygenValidation(spark, tableName, value = false) + } + + private def setComplexKeygenValidation(spark: SparkSession, + tableName: String, + value: Boolean): Unit = { + spark.sql( + s""" + |ALTER TABLE $tableName + |SET TBLPROPERTIES (hoodie.write.complex.keygen.validation.enable = '$value') + |""".stripMargin) + } + private def checkMessageContains(e: Throwable, text: String): Boolean = e.getMessage.trim.contains(text.trim) diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestAlterTableDropPartition.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestAlterTableDropPartition.scala index 2c592f5a8159a..84cc67858fbab 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestAlterTableDropPartition.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestAlterTableDropPartition.scala @@ -281,6 +281,9 @@ class TestAlterTableDropPartition extends HoodieSparkSqlTestBase { val df = Seq((1, "z3", "v1", "2021", "10", "01"), (2, "l4", "v1", "2021", "10", "02")) .toDF("id", "name", "ts", "year", "month", "day") + // Need to set hoodie.write.complex.keygen.validation.enable to false since alter table command also writes to the table + spark.sql(s"set ${HoodieWriteConfig.ENABLE_COMPLEX_KEYGEN_VALIDATION.key()}=false") + df.write.format("hudi") .option(HoodieWriteConfig.TBL_NAME.key, tableName) .option(TABLE_TYPE.key, COW_TABLE_TYPE_OPT_VAL) @@ -288,6 +291,7 @@ class TestAlterTableDropPartition extends HoodieSparkSqlTestBase { .option(PRECOMBINE_FIELD.key, "ts") .option(PARTITIONPATH_FIELD.key, "year,month,day") .option(HIVE_STYLE_PARTITIONING.key, hiveStyle) + .option(HoodieWriteConfig.ENABLE_COMPLEX_KEYGEN_VALIDATION.key(), "false") .option(HoodieWriteConfig.INSERT_PARALLELISM_VALUE.key, "1") .option(HoodieWriteConfig.UPSERT_PARALLELISM_VALUE.key, "1") .mode(SaveMode.Overwrite) @@ -312,6 +316,7 @@ class TestAlterTableDropPartition extends HoodieSparkSqlTestBase { .option(PRECOMBINE_FIELD.key, "ts") .option(PARTITIONPATH_FIELD.key, "year,month,day") .option(HIVE_STYLE_PARTITIONING.key, hiveStyle) + .option(HoodieWriteConfig.ENABLE_COMPLEX_KEYGEN_VALIDATION.key(), "false") .option(KEYGENERATOR_CLASS_NAME.key, classOf[ComplexKeyGenerator].getName) .option(HoodieWriteConfig.INSERT_PARALLELISM_VALUE.key, "1") .option(HoodieWriteConfig.UPSERT_PARALLELISM_VALUE.key, "1") @@ -345,6 +350,8 @@ class TestAlterTableDropPartition extends HoodieSparkSqlTestBase { } else { checkAnswer(s"show partitions $tableName")(Seq("2021/10/02")) } + + spark.sql(s"set ${HoodieWriteConfig.ENABLE_COMPLEX_KEYGEN_VALIDATION.key()}=") } } } @@ -358,6 +365,9 @@ class TestAlterTableDropPartition extends HoodieSparkSqlTestBase { import spark.implicits._ val df = Seq((1, "z3", "v1", "2021", "10", "01")).toDF("id", "name", "ts", "year", "month", "day") + // Need to set hoodie.write.complex.keygen.validation.enable to false since alter table command also writes to the table + spark.sql(s"set ${HoodieWriteConfig.ENABLE_COMPLEX_KEYGEN_VALIDATION.key()}=false") + df.write.format("hudi") .option(HoodieWriteConfig.TBL_NAME.key, tableName) .option(TABLE_TYPE.key, COW_TABLE_TYPE_OPT_VAL) @@ -365,6 +375,7 @@ class TestAlterTableDropPartition extends HoodieSparkSqlTestBase { .option(PRECOMBINE_FIELD.key, "ts") .option(PARTITIONPATH_FIELD.key, "year,month,day") .option(HIVE_STYLE_PARTITIONING.key, hiveStyle) + .option(HoodieWriteConfig.ENABLE_COMPLEX_KEYGEN_VALIDATION.key(), "false") .option(HoodieWriteConfig.INSERT_PARALLELISM_VALUE.key, "1") .option(HoodieWriteConfig.UPSERT_PARALLELISM_VALUE.key, "1") .mode(SaveMode.Overwrite) @@ -389,6 +400,7 @@ class TestAlterTableDropPartition extends HoodieSparkSqlTestBase { .option(PRECOMBINE_FIELD.key, "ts") .option(PARTITIONPATH_FIELD.key, "year,month,day") .option(HIVE_STYLE_PARTITIONING.key, hiveStyle) + .option(HoodieWriteConfig.ENABLE_COMPLEX_KEYGEN_VALIDATION.key(), "false") .option(KEYGENERATOR_CLASS_NAME.key, classOf[ComplexKeyGenerator].getName) .option(HoodieWriteConfig.INSERT_PARALLELISM_VALUE.key, "1") .option(HoodieWriteConfig.UPSERT_PARALLELISM_VALUE.key, "1") @@ -429,6 +441,8 @@ class TestAlterTableDropPartition extends HoodieSparkSqlTestBase { } else { checkAnswer(s"show partitions $tableName")(Seq("2021/10/02")) } + + spark.sql(s"set ${HoodieWriteConfig.ENABLE_COMPLEX_KEYGEN_VALIDATION.key()}=") } } } diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestCreateTable.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestCreateTable.scala index 937d11af6be65..00920c81beebd 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestCreateTable.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestCreateTable.scala @@ -21,16 +21,18 @@ import org.apache.hudi.DataSourceWriteOptions._ import org.apache.hudi.HoodieSparkUtils import org.apache.hudi.common.model.{HoodieRecord, WriteOperationType} import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient} -import org.apache.hudi.common.util.PartitionPathEncodeUtils.escapePathName import org.apache.hudi.config.HoodieWriteConfig import org.apache.hudi.hadoop.realtime.HoodieParquetRealtimeInputFormat import org.apache.hudi.keygen.SimpleKeyGenerator -import org.apache.spark.sql.SaveMode +import org.apache.hudi.testutils.Assertions +import org.apache.hudi.testutils.HoodieClientTestUtils.createMetaClient +import org.apache.spark.sql.{SaveMode, SparkSession} import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.catalog.{CatalogTableType, HoodieCatalogTable} import org.apache.spark.sql.hudi.HoodieSparkSqlTestBase.getLastCommitMetadata import org.apache.spark.sql.types._ import org.junit.jupiter.api.Assertions.{assertFalse, assertTrue} +import org.junit.jupiter.api.function.Executable import scala.collection.JavaConverters._ @@ -957,13 +959,15 @@ class TestCreateTable extends HoodieSparkSqlTestBase { } } - test("Test Create Table From Existing Hoodie Table For Multi-Level Partitioned Table") { + test("Test Create Table with Complex Key Generator and Key Encoding") { withTempDir { tmp => - Seq("2021-08-02", "2021/08/02").foreach { day => + Seq(false, true).foreach { encodeSingleKeyFieldValue => val tableName = generateTableName val tablePath = s"${tmp.getCanonicalPath}/$tableName" import spark.implicits._ - val df = Seq((1, "a1", 10, 1000, day, 12)).toDF("id", "name", "value", "ts", "day", "hh") + // The COMPLEX_KEYGEN_NEW_ENCODING config only works for table version 8 and below + val keyPrefix = if (encodeSingleKeyFieldValue) "" else "id:" + val df = Seq((1, "a1", 10, 1000, "2025-07-29", 12)).toDF("id", "name", "value", "ts", "day", "hh") // Write a table by spark dataframe. df.write.format("hudi") .option(HoodieWriteConfig.TBL_NAME.key, tableName) @@ -974,6 +978,10 @@ class TestCreateTable extends HoodieSparkSqlTestBase { .option(URL_ENCODE_PARTITIONING.key, "true") .option(HoodieWriteConfig.INSERT_PARALLELISM_VALUE.key, "1") .option(HoodieWriteConfig.UPSERT_PARALLELISM_VALUE.key, "1") + .option( + HoodieWriteConfig.COMPLEX_KEYGEN_NEW_ENCODING.key, + encodeSingleKeyFieldValue.toString) + .option(HoodieWriteConfig.ENABLE_COMPLEX_KEYGEN_VALIDATION.key, "false") .mode(SaveMode.Overwrite) .save(tablePath) @@ -983,54 +991,150 @@ class TestCreateTable extends HoodieSparkSqlTestBase { |create table $tableName using hudi |location '$tablePath' |""".stripMargin) - checkAnswer(s"select id, name, value, ts, day, hh from $tableName")( - Seq(1, "a1", 10, 1000, day, 12) + checkAnswer(s"select _hoodie_record_key, id, name, value, ts, day, hh from $tableName")( + Seq(keyPrefix + "1", 1, "a1", 10, 1000, "2025-07-29", 12) ) + + spark.sql(s"set hoodie.write.complex.keygen.new.encoding = $encodeSingleKeyFieldValue") + // Check the missing properties for spark sql - val metaClient = HoodieTableMetaClient.builder() - .setBasePath(tablePath) - .setConf(spark.sessionState.newHadoopConf()) - .build() + val metaClient = createMetaClient(spark, tablePath) val properties = metaClient.getTableConfig.getProps.asScala.toMap assertResult(true)(properties.contains(HoodieTableConfig.CREATE_SCHEMA.key)) assertResult("day,hh")(properties(HoodieTableConfig.PARTITION_FIELDS.key)) assertResult("ts")(properties(HoodieTableConfig.PRECOMBINE_FIELD.key)) - val escapedPathPart = escapePathName(day) + val query = s"select _hoodie_record_key, _hoodie_partition_path, id, name, value, ts, day, hh from $tableName order by id" // Test insert into - spark.sql(s"insert into $tableName values(2, 'a2', 10, 1000, '$day', 12)") - checkAnswer(s"select _hoodie_record_key, _hoodie_partition_path, id, name, value, ts, day, hh from $tableName order by id")( - Seq("1", s"$escapedPathPart/12", 1, "a1", 10, 1000, day, 12), - Seq("2", s"$escapedPathPart/12", 2, "a2", 10, 1000, day, 12) + writeAndValidateWithComplexKeyGenerator( + spark, tableName, + s"insert into $tableName values(2, 'a2', 10, 1000, '2025-07-29', 12)", query + )( + Seq(keyPrefix + "1", "2025-07-29/12", 1, "a1", 10, 1000, "2025-07-29", 12) + )( + Seq(keyPrefix + "1", "2025-07-29/12", 1, "a1", 10, 1000, "2025-07-29", 12), + Seq(keyPrefix + "2", "2025-07-29/12", 2, "a2", 10, 1000, "2025-07-29", 12) ) + // Test merge into - spark.sql( + writeAndValidateWithComplexKeyGenerator(spark, tableName, s""" |merge into $tableName h0 - |using (select 1 as id, 'a1' as name, 11 as value, 1001 as ts, '$day' as day, 12 as hh) s0 + |using (select 1 as id, 'a1' as name, 11 as value, 1001 as ts, '2025-07-29' as day, 12 as hh) s0 |on h0.id = s0.id |when matched then update set * - |""".stripMargin) - checkAnswer(s"select _hoodie_record_key, _hoodie_partition_path, id, name, value, ts, day, hh from $tableName order by id")( - Seq("1", s"$escapedPathPart/12", 1, "a1", 11, 1001, day, 12), - Seq("2", s"$escapedPathPart/12", 2, "a2", 10, 1000, day, 12) + |""".stripMargin, + query + )( + Seq(keyPrefix + "1", "2025-07-29/12", 1, "a1", 10, 1000, "2025-07-29", 12), + Seq(keyPrefix + "2", "2025-07-29/12", 2, "a2", 10, 1000, "2025-07-29", 12) + )( + Seq(keyPrefix + "1", "2025-07-29/12", 1, "a1", 11, 1001, "2025-07-29", 12), + Seq(keyPrefix + "2", "2025-07-29/12", 2, "a2", 10, 1000, "2025-07-29", 12) ) + // Test update - spark.sql(s"update $tableName set value = value + 1 where id = 2") - checkAnswer(s"select _hoodie_record_key, _hoodie_partition_path, id, name, value, ts, day, hh from $tableName order by id")( - Seq("1", s"$escapedPathPart/12", 1, "a1", 11, 1001, day, 12), - Seq("2", s"$escapedPathPart/12", 2, "a2", 11, 1000, day, 12) + writeAndValidateWithComplexKeyGenerator( + spark, tableName, + s"update $tableName set value = value + 1 where id = 2", query + )( + Seq(keyPrefix + "1", "2025-07-29/12", 1, "a1", 11, 1001, "2025-07-29", 12), + Seq(keyPrefix + "2", "2025-07-29/12", 2, "a2", 10, 1000, "2025-07-29", 12) + )( + Seq(keyPrefix + "1", "2025-07-29/12", 1, "a1", 11, 1001, "2025-07-29", 12), + Seq(keyPrefix + "2", "2025-07-29/12", 2, "a2", 11, 1000, "2025-07-29", 12) ) + // Test delete - spark.sql(s"delete from $tableName where id = 1") - checkAnswer(s"select _hoodie_record_key, _hoodie_partition_path, id, name, value, ts, day, hh from $tableName order by id")( - Seq("2", s"$escapedPathPart/12", 2, "a2", 11, 1000, day, 12) + writeAndValidateWithComplexKeyGenerator( + spark, tableName, + s"delete from $tableName where id = 1", query + )( + Seq(keyPrefix + "1", "2025-07-29/12", 1, "a1", 11, 1001, "2025-07-29", 12), + Seq(keyPrefix + "2", "2025-07-29/12", 2, "a2", 11, 1000, "2025-07-29", 12) + )( + Seq(keyPrefix + "2", "2025-07-29/12", 2, "a2", 11, 1000, "2025-07-29", 12) ) + + spark.sql(s"set hoodie.write.complex.keygen.new.encoding=") } } } + test("Test Create Table with Complex Key Generator with multiple partition fields and record key fields") { + withTempDir { tmp => + val tableName = generateTableName + val tablePath = s"${tmp.getCanonicalPath}/$tableName" + import spark.implicits._ + val df = Seq((1, "a1", 10, 1000, "2025-07-29", 12)).toDF("id", "name", "value", "ts", "day", "hh") + // Write a table by spark dataframe. + df.write.format("hudi") + .option(HoodieWriteConfig.TBL_NAME.key, tableName) + .option(TABLE_TYPE.key, MOR_TABLE_TYPE_OPT_VAL) + .option(RECORDKEY_FIELD.key, "id,name") + .option(PRECOMBINE_FIELD.key, "ts") + .option(PARTITIONPATH_FIELD.key, "day,hh") + .option(URL_ENCODE_PARTITIONING.key, "true") + .option(HoodieWriteConfig.INSERT_PARALLELISM_VALUE.key, "1") + .option(HoodieWriteConfig.UPSERT_PARALLELISM_VALUE.key, "1") + .mode(SaveMode.Overwrite) + .save(tablePath) + + // Create a table over the existing table. + spark.sql( + s""" + |create table $tableName using hudi + |location '$tablePath' + |""".stripMargin) + checkAnswer(s"select _hoodie_record_key, id, name, value, ts, day, hh from $tableName")( + Seq("id:1,name:a1", 1, "a1", 10, 1000, "2025-07-29", 12) + ) + // Check the missing properties for spark sql + val metaClient = createMetaClient(spark, tablePath) + val properties = metaClient.getTableConfig.getProps.asScala.toMap + assertResult(true)(properties.contains(HoodieTableConfig.CREATE_SCHEMA.key)) + assertResult("id,name")(properties(HoodieTableConfig.RECORDKEY_FIELDS.key)) + assertResult("day,hh")(properties(HoodieTableConfig.PARTITION_FIELDS.key)) + assertResult("ts")(properties(HoodieTableConfig.PRECOMBINE_FIELD.key)) + + val query = s"select _hoodie_record_key, _hoodie_partition_path, id, name, value, ts, day, hh from $tableName order by id" + + // Test insert into + spark.sql(s"insert into $tableName values(2, 'a2', 10, 1000, '2025-07-29', 12)") + checkAnswer(query)( + Seq("id:1,name:a1", "2025-07-29/12", 1, "a1", 10, 1000, "2025-07-29", 12), + Seq("id:2,name:a2", "2025-07-29/12", 2, "a2", 10, 1000, "2025-07-29", 12) + ) + + // Test merge into + spark.sql( + s""" + |merge into $tableName h0 + |using (select 1 as id, 'a1' as name, 11 as value, 1001 as ts, '2025-07-29' as day, 12 as hh) s0 + |on h0.id = s0.id + |when matched then update set * + |""".stripMargin) + checkAnswer(query)( + Seq("id:1,name:a1", "2025-07-29/12", 1, "a1", 11, 1001, "2025-07-29", 12), + Seq("id:2,name:a2", "2025-07-29/12", 2, "a2", 10, 1000, "2025-07-29", 12) + ) + + // Test update + spark.sql(s"update $tableName set value = value + 1 where id = 2") + checkAnswer(query)( + Seq("id:1,name:a1", "2025-07-29/12", 1, "a1", 11, 1001, "2025-07-29", 12), + Seq("id:2,name:a2", "2025-07-29/12", 2, "a2", 11, 1000, "2025-07-29", 12) + ) + + // Test delete + spark.sql(s"delete from $tableName where id = 1") + checkAnswer(query)( + Seq("id:2,name:a2", "2025-07-29/12", 2, "a2", 11, 1000, "2025-07-29", 12) + ) + } + } + test("Test Create Table From Existing Hoodie Table For None Partitioned Table") { withTempDir { tmp => // Write a table by spark dataframe. @@ -1487,4 +1591,25 @@ class TestCreateTable extends HoodieSparkSqlTestBase { } } } + + def writeAndValidateWithComplexKeyGenerator(spark: SparkSession, + tableName: String, + dmlToWrite: String, + query: String) + (expectedRowsBefore: Seq[Any]*) + (expectedRowsAfter: Seq[Any]*): Unit = { + // By default, the complex key generator validation is enabled and should throw exception on DML + Assertions.assertComplexKeyGeneratorValidationThrows(new Executable() { + override def execute(): Unit = { + spark.sql(dmlToWrite) + } + }, "ingestion") + // Query should still succeed + checkAnswer(query)(expectedRowsBefore: _*) + // Disabling the complex key generator validation should let write succeed + spark.sql(s"set ${HoodieWriteConfig.ENABLE_COMPLEX_KEYGEN_VALIDATION.key()}=false") + spark.sql(dmlToWrite) + spark.sql(s"set ${HoodieWriteConfig.ENABLE_COMPLEX_KEYGEN_VALIDATION.key()}=true") + checkAnswer(query)(expectedRowsAfter: _*) + } } diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestInsertTable.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestInsertTable.scala index 9d14064f3987f..410390df4811b 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestInsertTable.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestInsertTable.scala @@ -193,7 +193,10 @@ class TestInsertTable extends HoodieSparkSqlTestBase { | ht string, | ts long |) using hudi - | tblproperties (primaryKey = 'id') + | tblproperties ( + | primaryKey = 'id', + | ${HoodieWriteConfig.ENABLE_COMPLEX_KEYGEN_VALIDATION.key()} = 'false' + |) | partitioned by (dt, ht) | location '${tmp.getCanonicalPath}' """.stripMargin) @@ -520,7 +523,8 @@ class TestInsertTable extends HoodieSparkSqlTestBase { |) using hudi | tblproperties ( | type = '$tableType', - | primaryKey = 'id' + | primaryKey = 'id', + | ${HoodieWriteConfig.ENABLE_COMPLEX_KEYGEN_VALIDATION.key()} = 'false' | ) | partitioned by (dt, hh) """.stripMargin @@ -818,7 +822,8 @@ class TestInsertTable extends HoodieSparkSqlTestBase { |) using hudi | tblproperties ( | type = '$tableType', - | primaryKey = 'id' + | primaryKey = 'id', + | ${HoodieWriteConfig.ENABLE_COMPLEX_KEYGEN_VALIDATION.key()} = 'false' | ) | partitioned by (dt, hh) | location '${tmp.getCanonicalPath}/$tableMultiPartition' @@ -1193,6 +1198,7 @@ class TestInsertTable extends HoodieSparkSqlTestBase { .option(HoodieWriteConfig.INSERT_PARALLELISM_VALUE.key, "1") .option(HoodieWriteConfig.UPSERT_PARALLELISM_VALUE.key, "1") .option(HoodieWriteConfig.ALLOW_OPERATION_METADATA_FIELD.key, "true") + .option(HoodieWriteConfig.ENABLE_COMPLEX_KEYGEN_VALIDATION.key(), "false") .mode(SaveMode.Overwrite) .save(tablePath) diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestLazyPartitionPathFetching.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestLazyPartitionPathFetching.scala index e2635c0cba879..236110932d777 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestLazyPartitionPathFetching.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestLazyPartitionPathFetching.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.hudi +import org.apache.hudi.config.HoodieWriteConfig + class TestLazyPartitionPathFetching extends HoodieSparkSqlTestBase { test("Test querying with string column + partition pruning") { @@ -102,7 +104,8 @@ class TestLazyPartitionPathFetching extends HoodieSparkSqlTestBase { | tblproperties ( | primaryKey ='id', | type = 'cow', - | preCombineField = 'ts' + | preCombineField = 'ts', + | ${HoodieWriteConfig.ENABLE_COMPLEX_KEYGEN_VALIDATION.key()} = 'false' | ) | PARTITIONED BY (country, date_par) """.stripMargin) diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestPartitionPushDownWhenListingPaths.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestPartitionPushDownWhenListingPaths.scala index 1b5e590913f3b..9b469bd487801 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestPartitionPushDownWhenListingPaths.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestPartitionPushDownWhenListingPaths.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.hudi import org.apache.hudi.common.config.HoodieMetadataConfig +import org.apache.hudi.config.HoodieWriteConfig class TestPartitionPushDownWhenListingPaths extends HoodieSparkSqlTestBase { @@ -45,7 +46,8 @@ class TestPartitionPushDownWhenListingPaths extends HoodieSparkSqlTestBase { | type = '$tableType', | preCombineField = 'ts', | hoodie.datasource.write.hive_style_partitioning = 'true', - | hoodie.datasource.write.partitionpath.urlencode = 'true' + | hoodie.datasource.write.partitionpath.urlencode = 'true', + | ${HoodieWriteConfig.ENABLE_COMPLEX_KEYGEN_VALIDATION.key()} = 'false' | ) | PARTITIONED BY (date_par, country, hour, longValue)""".stripMargin) spark.sql(s"insert into $tableName values(1, 'a1', 10, 1000, date '2023-02-27', 'ID', 1, 102345L)") diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestRepairTable.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestRepairTable.scala index 8078ed29bd7e4..89449959b5f3f 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestRepairTable.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestRepairTable.scala @@ -21,6 +21,7 @@ package org.apache.spark.sql.hudi import org.apache.hudi.DataSourceWriteOptions.{PARTITIONPATH_FIELD, PRECOMBINE_FIELD, RECORDKEY_FIELD} import org.apache.hudi.HoodieSparkUtils import org.apache.hudi.common.table.HoodieTableConfig.HIVE_STYLE_PARTITIONING_ENABLE +import org.apache.hudi.config.HoodieWriteConfig import org.apache.hudi.config.HoodieWriteConfig.TBL_NAME import org.apache.spark.sql.SaveMode @@ -74,7 +75,8 @@ class TestRepairTable extends HoodieSparkSqlTestBase { | tblproperties ( | primaryKey = 'id', | preCombineField = 'ts', - | hoodie.datasource.write.hive_style_partitioning = '$hiveStylePartitionEnable' + | hoodie.datasource.write.hive_style_partitioning = '$hiveStylePartitionEnable', + | ${HoodieWriteConfig.ENABLE_COMPLEX_KEYGEN_VALIDATION.key()} = 'false' | ) """.stripMargin) val table = spark.sessionState.sqlParser.parseTableIdentifier(tableName) @@ -87,6 +89,7 @@ class TestRepairTable extends HoodieSparkSqlTestBase { .option(PRECOMBINE_FIELD.key, "ts") .option(PARTITIONPATH_FIELD.key, "dt,hh") .option(HIVE_STYLE_PARTITIONING_ENABLE.key, hiveStylePartitionEnable) + .option(HoodieWriteConfig.ENABLE_COMPLEX_KEYGEN_VALIDATION.key(), "false") .mode(SaveMode.Append) .save(basePath) @@ -112,6 +115,7 @@ class TestRepairTable extends HoodieSparkSqlTestBase { .option(RECORDKEY_FIELD.key, "id") .option(PRECOMBINE_FIELD.key, "ts") .option(PARTITIONPATH_FIELD.key, "dt,hh") + .option(HoodieWriteConfig.ENABLE_COMPLEX_KEYGEN_VALIDATION.key(), "false") .option(HIVE_STYLE_PARTITIONING_ENABLE.key, hiveStylePartitionEnable) .mode(SaveMode.Append) .save(basePath) diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestShowPartitions.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestShowPartitions.scala index 968d7a168aa38..0d7f7a0918bd0 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestShowPartitions.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestShowPartitions.scala @@ -19,6 +19,8 @@ package org.apache.spark.sql.hudi import org.apache.hudi.HoodieSparkUtils.isSpark2 import org.apache.hudi.common.util.PartitionPathEncodeUtils.DEFAULT_PARTITION_PATH +import org.apache.hudi.config.HoodieWriteConfig +import org.apache.spark.sql.hudi.HoodieSparkSqlTestBase class TestShowPartitions extends HoodieSparkSqlTestBase { @@ -123,7 +125,8 @@ class TestShowPartitions extends HoodieSparkSqlTestBase { | partitioned by (year, month, day) | tblproperties ( | primaryKey = 'id', - | preCombineField = 'ts' + | preCombineField = 'ts', + | ${HoodieWriteConfig.ENABLE_COMPLEX_KEYGEN_VALIDATION.key()} = 'false' | ) """.stripMargin) // Empty partitions @@ -228,7 +231,8 @@ class TestShowPartitions extends HoodieSparkSqlTestBase { | partitioned by (year, month, day) | tblproperties ( | primaryKey = 'id', - | preCombineField = 'ts' + | preCombineField = 'ts', + | ${HoodieWriteConfig.ENABLE_COMPLEX_KEYGEN_VALIDATION.key()} = 'false' | ) """.stripMargin) diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestSpark3DDL.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestSpark3DDL.scala index 6ca1a72edcdb2..6afc46a064266 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestSpark3DDL.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestSpark3DDL.scala @@ -1030,7 +1030,8 @@ class TestSpark3DDL extends HoodieSparkSqlTestBase { |tblproperties ( | primaryKey = 'id', | type = 'cow', - | preCombineField = 'ts' + | preCombineField = 'ts', + | ${HoodieWriteConfig.ENABLE_COMPLEX_KEYGEN_VALIDATION.key()} = 'false' |) |partitioned by (region, dt)""".stripMargin) diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestTruncateTable.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestTruncateTable.scala index 808bfebb802c0..7f980197bc61b 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestTruncateTable.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestTruncateTable.scala @@ -107,6 +107,8 @@ class TestTruncateTable extends HoodieSparkSqlTestBase { val df = Seq((1, "z3", "v1", "2021", "10", "01"), (2, "l4", "v1", "2021", "10","02")) .toDF("id", "name", "ts", "year", "month", "day") + // Need to set hoodie.write.complex.keygen.validation.enable to false since truncate table command also writes to the table + spark.sql(s"set ${HoodieWriteConfig.ENABLE_COMPLEX_KEYGEN_VALIDATION.key()}=false") df.write.format("hudi") .option(HoodieWriteConfig.TBL_NAME.key, tableName) .option(TABLE_TYPE.key, COW_TABLE_TYPE_OPT_VAL) @@ -114,6 +116,7 @@ class TestTruncateTable extends HoodieSparkSqlTestBase { .option(PRECOMBINE_FIELD.key, "ts") .option(PARTITIONPATH_FIELD.key, "year,month,day") .option(HIVE_STYLE_PARTITIONING.key, hiveStyle) + .option(HoodieWriteConfig.ENABLE_COMPLEX_KEYGEN_VALIDATION.key(), "false") .option(HoodieWriteConfig.INSERT_PARALLELISM_VALUE.key, "1") .option(HoodieWriteConfig.UPSERT_PARALLELISM_VALUE.key, "1") .mode(SaveMode.Overwrite) @@ -141,6 +144,8 @@ class TestTruncateTable extends HoodieSparkSqlTestBase { // Truncate table spark.sql(s"truncate table $tableName") checkAnswer(s"select count(1) from $tableName")(Seq(0)) + + spark.sql(s"set ${HoodieWriteConfig.ENABLE_COMPLEX_KEYGEN_VALIDATION.key()}=") } } } diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestFsViewProcedure.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestFsViewProcedure.scala index 9de1f1b0ee855..94d94fefb47f7 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestFsViewProcedure.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestFsViewProcedure.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.hudi.procedure +import org.apache.hudi.config.HoodieWriteConfig + class TestFsViewProcedure extends HoodieSparkProcedureTestBase { test("Test Call show_fsview_all Procedure") { withTempDir { tmp => @@ -114,7 +116,8 @@ class TestFsViewProcedure extends HoodieSparkProcedureTestBase { | location '${tmp.getCanonicalPath}/$tableName' | tblproperties ( | primaryKey = 'id', - | preCombineField = 'ts' + | preCombineField = 'ts', + | ${HoodieWriteConfig.ENABLE_COMPLEX_KEYGEN_VALIDATION.key()} = 'false' | ) """.stripMargin) // insert data to table diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestStatsProcedure.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestStatsProcedure.scala index c600d45e56828..8c57839e754ab 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestStatsProcedure.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestStatsProcedure.scala @@ -19,6 +19,8 @@ package org.apache.spark.sql.hudi.procedure +import org.apache.hudi.config.HoodieWriteConfig + class TestStatsProcedure extends HoodieSparkProcedureTestBase { test("Test Call stats_wa Procedure") { withTempDir { tmp => @@ -75,7 +77,8 @@ class TestStatsProcedure extends HoodieSparkProcedureTestBase { | location '$tablePath' | tblproperties ( | primaryKey = 'id', - | preCombineField = 'ts' + | preCombineField = 'ts', + | ${HoodieWriteConfig.ENABLE_COMPLEX_KEYGEN_VALIDATION.key()} = 'false' | ) """.stripMargin) // insert data to table