diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java index 1bfc167985c49..1db77f325fc88 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java @@ -169,6 +169,7 @@ public class HoodieWriteConfig extends HoodieConfig { .withDocumentation("Determine what level of persistence is used to cache write RDDs. " + "Refer to org.apache.spark.storage.StorageLevel for different values"); + @Deprecated public static final ConfigProperty PRECOMBINE_FIELD_NAME = ConfigProperty .key("hoodie.datasource.write.precombine.field") .noDefaultValue() @@ -1407,6 +1408,7 @@ public HoodieTableType getTableType() { HoodieTableConfig.TYPE, HoodieTableConfig.TYPE.defaultValue().name()).toUpperCase()); } + @Deprecated public List getPreCombineFields() { return Option.ofNullable(getString(PRECOMBINE_FIELD_NAME)) .map(preCombine -> Arrays.asList(preCombine.split(","))) @@ -3017,6 +3019,7 @@ public Builder forTable(String tableName) { return this; } + @Deprecated public Builder withPreCombineField(String preCombineField) { writeConfig.setValue(PRECOMBINE_FIELD_NAME, preCombineField); return this; diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/EightToNineUpgradeHandler.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/EightToNineUpgradeHandler.java index 472b4157d313b..0deedd7c90022 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/EightToNineUpgradeHandler.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/EightToNineUpgradeHandler.java @@ -104,6 +104,7 @@ public UpgradeDowngrade.TableConfigChangeSet upgrade(HoodieWriteConfig config, SupportsUpgradeDowngrade upgradeDowngradeHelper) { final HoodieTable table = upgradeDowngradeHelper.getTable(config, context); Map tablePropsToAdd = new HashMap<>(); + Set tablePropsToRemove = new HashSet<>(); HoodieTableMetaClient metaClient = table.getMetaClient(); HoodieTableConfig tableConfig = metaClient.getTableConfig(); // Populate missing index versions indexes @@ -117,7 +118,6 @@ public UpgradeDowngrade.TableConfigChangeSet upgrade(HoodieWriteConfig config, indexMetadataOpt.get(), metaClient.getTableConfig().getTableVersion()); } - Set tablePropsToRemove = new HashSet<>(); // Handle merge mode config. reconcileMergeModeConfig(tablePropsToAdd, tableConfig); // Handle partial update mode config. @@ -126,6 +126,8 @@ public UpgradeDowngrade.TableConfigChangeSet upgrade(HoodieWriteConfig config, reconcileMergePropertiesConfig(tablePropsToAdd, tableConfig); // Handle payload class configs. reconcilePayloadClassConfig(tablePropsToAdd, tablePropsToRemove, tableConfig); + // Handle ordering fields config. + reconcileOrderingFieldsConfig(tablePropsToAdd, tablePropsToRemove, tableConfig); return new UpgradeDowngrade.TableConfigChangeSet(tablePropsToAdd, tablePropsToRemove); } @@ -175,8 +177,7 @@ private void reconcilePartialUpdateModeConfig(Map tableP } } - private void reconcileMergePropertiesConfig(Map tablePropsToAdd, - HoodieTableConfig tableConfig) { + private void reconcileMergePropertiesConfig(Map tablePropsToAdd, HoodieTableConfig tableConfig) { String payloadClass = tableConfig.getPayloadClass(); String mergeStrategy = tableConfig.getRecordMergeStrategyId(); if (!BUILTIN_MERGE_STRATEGIES.contains(mergeStrategy) || StringUtils.isNullOrEmpty(payloadClass)) { @@ -193,11 +194,22 @@ private void reconcileMergePropertiesConfig(Map tablePro tablePropsToAdd.put( ConfigProperty.key(RECORD_MERGE_PROPERTY_PREFIX + PARTIAL_UPDATE_UNAVAILABLE_VALUE).noDefaultValue(), // // to be fixed once we land PR #13721. DEBEZIUM_UNAVAILABLE_VALUE); - } else if (payloadClass.equals(MySqlDebeziumAvroPayload.class.getName())) { - tablePropsToAdd.put(HoodieTableConfig.PRECOMBINE_FIELDS, FLATTENED_FILE_COL_NAME + "," + FLATTENED_POS_COL_NAME); } } + private void reconcileOrderingFieldsConfig(Map tablePropsToAdd, + Set tablePropsToRemove, + HoodieTableConfig tableConfig) { + String payloadClass = tableConfig.getPayloadClass(); + Option orderingFieldsOpt = MySqlDebeziumAvroPayload.class.getName().equals(payloadClass) + ? Option.of(FLATTENED_FILE_COL_NAME + "," + FLATTENED_POS_COL_NAME) + : tableConfig.getOrderingFieldsStr(); + orderingFieldsOpt.ifPresent(orderingFields -> { + tablePropsToAdd.put(HoodieTableConfig.ORDERING_FIELDS, orderingFields); + tablePropsToRemove.add(HoodieTableConfig.PRECOMBINE_FIELD); + }); + } + /** * Populates missing version attributes in index definitions based on table version. * diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/NineToEightDowngradeHandler.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/NineToEightDowngradeHandler.java index 2936a158bc507..3f3096cf8754f 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/NineToEightDowngradeHandler.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/NineToEightDowngradeHandler.java @@ -31,6 +31,7 @@ import org.apache.hudi.common.model.debezium.PostgresDebeziumAvroPayload; import org.apache.hudi.common.table.HoodieTableConfig; import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.table.HoodieTable; @@ -81,17 +82,18 @@ public UpgradeDowngrade.TableConfigChangeSet downgrade(HoodieWriteConfig config, // Update table properties. Set propertiesToRemove = new HashSet<>(); Map propertiesToAdd = new HashMap<>(); - reconcileMergeConfigs(propertiesToAdd, propertiesToRemove, metaClient); + HoodieTableConfig tableConfig = metaClient.getTableConfig(); + reconcileMergeConfigs(propertiesToAdd, propertiesToRemove, tableConfig); + reconcileOrderingFieldsConfig(propertiesToAdd, propertiesToRemove, tableConfig); return new UpgradeDowngrade.TableConfigChangeSet(propertiesToAdd, propertiesToRemove); } private void reconcileMergeConfigs(Map propertiesToAdd, Set propertiesToRemove, - HoodieTableMetaClient metaClient) { + HoodieTableConfig tableConfig) { // Update table properties. propertiesToRemove.add(PARTIAL_UPDATE_MODE); // For specified payload classes, add strategy id and custom merge mode. - HoodieTableConfig tableConfig = metaClient.getTableConfig(); String legacyPayloadClass = tableConfig.getLegacyPayloadClass(); if (!StringUtils.isNullOrEmpty(legacyPayloadClass) && (PAYLOAD_CLASSES_TO_HANDLE.contains(legacyPayloadClass))) { propertiesToRemove.add(LEGACY_PAYLOAD_CLASS_NAME); @@ -112,9 +114,18 @@ private void reconcileMergeConfigs(Map propertiesToAdd, propertiesToRemove.add( ConfigProperty.key(RECORD_MERGE_PROPERTY_PREFIX + PARTIAL_UPDATE_UNAVAILABLE_VALUE).noDefaultValue()); } - if (legacyPayloadClass.equals(MySqlDebeziumAvroPayload.class.getName())) { - propertiesToAdd.put(HoodieTableConfig.PRECOMBINE_FIELDS, DebeziumConstants.ADDED_SEQ_COL_NAME); - } } } + + private void reconcileOrderingFieldsConfig(Map propertiesToAdd, + Set propertiesToRemove, + HoodieTableConfig tableConfig) { + Option orderingFieldsOpt = MySqlDebeziumAvroPayload.class.getName().equals(tableConfig.getLegacyPayloadClass()) + ? Option.of(DebeziumConstants.ADDED_SEQ_COL_NAME) + : tableConfig.getOrderingFieldsStr(); + orderingFieldsOpt.ifPresent(orderingFields -> { + propertiesToAdd.put(HoodieTableConfig.PRECOMBINE_FIELD, orderingFields); + propertiesToRemove.add(HoodieTableConfig.ORDERING_FIELDS); + }); + } } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/SevenToEightUpgradeHandler.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/SevenToEightUpgradeHandler.java index a948f14729941..8ee2f68fd728c 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/SevenToEightUpgradeHandler.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/SevenToEightUpgradeHandler.java @@ -148,7 +148,7 @@ static void upgradePartitionFields(HoodieWriteConfig config, HoodieTableConfig t static void upgradeMergeMode(HoodieTableConfig tableConfig, Map tablePropsToAdd) { String payloadClass = tableConfig.getPayloadClass(); - String preCombineFields = tableConfig.getPreCombineFieldsStr().orElse(null); + String orderingFields = tableConfig.getOrderingFieldsStr().orElse(null); if (isCustomPayloadClass(payloadClass)) { // This contains a special case: HoodieMetadataPayload. tablePropsToAdd.put( @@ -163,7 +163,7 @@ static void upgradeMergeMode(HoodieTableConfig tableConfig, Map entry : tablePropsToAdd.entrySet()) { // add alternate keys. metaClient.getTableConfig().setValue(entry.getKey(), entry.getValue()); @@ -241,10 +244,8 @@ public void run(HoodieTableVersion toVersion, String instantTime) { metaClient.getTableConfig().setValue((String) alternateKey, entry.getValue()); }); } - for (ConfigProperty configProperty : tablePropsToRemove) { - metaClient.getTableConfig().clearValue(configProperty); - } + // Write out the current version in hoodie.properties.updated file metaClient.getTableConfig().setTableVersion(toVersion); // Update modified properties. Set propertiesToRemove = diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/table/upgrade/TestEightToNineUpgradeHandler.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/table/upgrade/TestEightToNineUpgradeHandler.java index a61f2b1e4ca72..26eb0ee10612d 100644 --- a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/table/upgrade/TestEightToNineUpgradeHandler.java +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/table/upgrade/TestEightToNineUpgradeHandler.java @@ -122,6 +122,7 @@ public void setUp() throws IOException { when(metaClient.getTableConfig()).thenReturn(tableConfig); when(metaClient.getStorage()).thenReturn(storage); when(tableConfig.getTableVersion()).thenReturn(HoodieTableVersion.EIGHT); + when(tableConfig.getOrderingFieldsStr()).thenReturn(Option.empty()); // Use a temp file for index definition path indexDefPath = new StoragePath(tempDir.resolve("index.json").toString()); @@ -287,15 +288,20 @@ void testUpgradeWithNoIndexMetadata() { private void assertPayloadClassChange(Map propertiesToAdd, Set propertiesToRemove, String payloadClass) { - assertEquals(1, propertiesToRemove.size()); + if (payloadClass.equals(MySqlDebeziumAvroPayload.class.getName())) { + assertEquals(2, propertiesToRemove.size()); + assertTrue(propertiesToRemove.contains(HoodieTableConfig.PRECOMBINE_FIELD)); + } else { + assertEquals(1, propertiesToRemove.size()); + } assertTrue(propertiesToRemove.contains(PAYLOAD_CLASS_NAME)); assertTrue(propertiesToAdd.containsKey(LEGACY_PAYLOAD_CLASS_NAME)); assertEquals( payloadClass, propertiesToAdd.get(LEGACY_PAYLOAD_CLASS_NAME)); if (payloadClass.equals(MySqlDebeziumAvroPayload.class.getName())) { - assertTrue(propertiesToAdd.containsKey(HoodieTableConfig.PRECOMBINE_FIELDS)); - assertEquals(propertiesToAdd.get(HoodieTableConfig.PRECOMBINE_FIELDS).toString(), FLATTENED_FILE_COL_NAME + "," + FLATTENED_POS_COL_NAME); + assertTrue(propertiesToAdd.containsKey(HoodieTableConfig.ORDERING_FIELDS)); + assertEquals(propertiesToAdd.get(HoodieTableConfig.ORDERING_FIELDS), FLATTENED_FILE_COL_NAME + "," + FLATTENED_POS_COL_NAME); } } diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/table/upgrade/TestNineToEightDowngradeHandler.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/table/upgrade/TestNineToEightDowngradeHandler.java index 9971985aa5f5c..8946e3ec68822 100644 --- a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/table/upgrade/TestNineToEightDowngradeHandler.java +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/table/upgrade/TestNineToEightDowngradeHandler.java @@ -64,6 +64,7 @@ import static org.apache.hudi.common.model.DefaultHoodieRecordPayload.DELETE_MARKER; import static org.apache.hudi.common.model.HoodieRecordMerger.PAYLOAD_BASED_MERGE_STRATEGY_UUID; import static org.apache.hudi.common.table.HoodieTableConfig.LEGACY_PAYLOAD_CLASS_NAME; +import static org.apache.hudi.common.table.HoodieTableConfig.ORDERING_FIELDS; import static org.apache.hudi.common.table.HoodieTableConfig.PARTIAL_UPDATE_MODE; import static org.apache.hudi.common.table.HoodieTableConfig.PARTIAL_UPDATE_UNAVAILABLE_VALUE; import static org.apache.hudi.common.table.HoodieTableConfig.PAYLOAD_CLASS_NAME; @@ -98,6 +99,7 @@ public void setUp() { when(upgradeDowngradeHelper.getTable(any(), any())).thenReturn(table); when(table.getMetaClient()).thenReturn(metaClient); when(metaClient.getTableConfig()).thenReturn(tableConfig); + when(tableConfig.getOrderingFieldsStr()).thenReturn(Option.empty()); } static Stream payloadClassTestCases() { @@ -136,8 +138,8 @@ static Stream payloadClassTestCases() { // MySqlDebeziumAvroPayload - requires RECORD_MERGE_MODE and RECORD_MERGE_STRATEGY_ID Arguments.of( MySqlDebeziumAvroPayload.class.getName(), - 2, - LEGACY_PAYLOAD_CLASS_NAME.key() + "," + PARTIAL_UPDATE_MODE.key(), + 3, + LEGACY_PAYLOAD_CLASS_NAME.key() + "," + ORDERING_FIELDS.key() + "," + PARTIAL_UPDATE_MODE.key() + ",", 4, true, true, @@ -225,8 +227,8 @@ void testDowngradeForPayloadClass(String payloadClassName, int expectedPropertie propertiesToChange.propertiesToUpdate().get(RECORD_MERGE_STRATEGY_ID)); } if (payloadClassName.equals(MySqlDebeziumAvroPayload.class.getName())) { - assertTrue(propertiesToChange.propertiesToUpdate().containsKey(HoodieTableConfig.PRECOMBINE_FIELDS)); - assertEquals(propertiesToChange.propertiesToUpdate().get(HoodieTableConfig.PRECOMBINE_FIELDS).toString(), DebeziumConstants.ADDED_SEQ_COL_NAME); + assertTrue(propertiesToChange.propertiesToUpdate().containsKey(HoodieTableConfig.PRECOMBINE_FIELD)); + assertEquals(propertiesToChange.propertiesToUpdate().get(HoodieTableConfig.PRECOMBINE_FIELD), DebeziumConstants.ADDED_SEQ_COL_NAME); } } } diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/table/upgrade/TestSevenToEightUpgradeHandler.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/table/upgrade/TestSevenToEightUpgradeHandler.java index a8c2103e46d31..66d03811b76fc 100644 --- a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/table/upgrade/TestSevenToEightUpgradeHandler.java +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/table/upgrade/TestSevenToEightUpgradeHandler.java @@ -97,7 +97,7 @@ void testPropertyUpgrade() { // Mock record merge mode configuration for merging behavior when(tableConfig.contains(isA(ConfigProperty.class))).thenAnswer(i -> i.getArguments()[0].equals(PAYLOAD_CLASS_NAME)); when(tableConfig.getPayloadClass()).thenReturn(OverwriteWithLatestAvroPayload.class.getName()); - when(tableConfig.getPreCombineFieldsStr()).thenReturn(Option.empty()); + when(tableConfig.getOrderingFieldsStr()).thenReturn(Option.empty()); SevenToEightUpgradeHandler.upgradeMergeMode(tableConfig, tablePropsToAdd); assertTrue(tablePropsToAdd.containsKey(RECORD_MERGE_MODE)); assertNotNull(tablePropsToAdd.get(RECORD_MERGE_MODE)); @@ -140,7 +140,7 @@ void testUpgradeMergeMode(String payloadClass, String preCombineField, String ex Map tablePropsToAdd = new HashMap<>(); when(tableConfig.getPayloadClass()).thenReturn(payloadClass); - when(tableConfig.getPreCombineFieldsStr()).thenReturn(Option.ofNullable(preCombineField)); + when(tableConfig.getOrderingFieldsStr()).thenReturn(Option.ofNullable(preCombineField)); SevenToEightUpgradeHandler.upgradeMergeMode(tableConfig, tablePropsToAdd); diff --git a/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/common/table/read/HoodieFileGroupReaderOnJavaTestBase.java b/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/common/table/read/HoodieFileGroupReaderOnJavaTestBase.java index 4043f9c9153ab..c4e3adeff01f8 100644 --- a/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/common/table/read/HoodieFileGroupReaderOnJavaTestBase.java +++ b/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/common/table/read/HoodieFileGroupReaderOnJavaTestBase.java @@ -28,6 +28,7 @@ import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.table.HoodieTableConfig; import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.util.ConfigUtils; import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.keygen.constant.KeyGeneratorOptions; @@ -89,7 +90,7 @@ public void commitToTable(List recordList, String operation, boole .setKeyGeneratorType(KeyGeneratorType.SIMPLE.name()) .setRecordKeyFields(writeConfigs.get(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key())) .setPartitionFields(writeConfigs.get(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key())) - .setPreCombineFields(writeConfigs.get("hoodie.datasource.write.precombine.field")) + .setOrderingFields(ConfigUtils.getOrderingFieldsStrDuringWrite(writeConfigs)) .setBaseFileFormat(writeConfigs.get(HoodieTableConfig.BASE_FILE_FORMAT.key())) .set(initConfigs); if (writeConfigs.containsKey("hoodie.datasource.write.payload.class")) { diff --git a/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/HoodieDatasetBulkInsertHelper.scala b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/HoodieDatasetBulkInsertHelper.scala index 3f8e832abe00a..4788aae7669ae 100644 --- a/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/HoodieDatasetBulkInsertHelper.scala +++ b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/HoodieDatasetBulkInsertHelper.scala @@ -25,7 +25,8 @@ import org.apache.hudi.common.config.TypedProperties import org.apache.hudi.common.data.HoodieData import org.apache.hudi.common.engine.TaskContextSupplier import org.apache.hudi.common.model.HoodieRecord -import org.apache.hudi.common.util.{ConfigUtils, OrderingValues, ReflectionUtils} +import org.apache.hudi.common.table.HoodieTableConfig +import org.apache.hudi.common.util.{OrderingValues, ReflectionUtils} import org.apache.hudi.config.HoodieWriteConfig import org.apache.hudi.data.HoodieJavaRDD import org.apache.hudi.exception.HoodieException @@ -68,6 +69,7 @@ object HoodieDatasetBulkInsertHelper */ def prepareForBulkInsert(df: DataFrame, config: HoodieWriteConfig, + tableConfig: HoodieTableConfig, partitioner: BulkInsertPartitioner[Dataset[Row]], instantTime: String): Dataset[Row] = { val populateMetaFields = config.populateMetaFields() @@ -121,7 +123,7 @@ object HoodieDatasetBulkInsertHelper } val dedupedRdd = if (config.shouldCombineBeforeInsert) { - dedupeRows(prependedRdd, updatedSchema, ConfigUtils.getOrderingFields(config.getProps).toList, SparkHoodieIndexFactory.isGlobalIndex(config), targetParallelism) + dedupeRows(prependedRdd, updatedSchema, tableConfig.getOrderingFields.asScala.toList, SparkHoodieIndexFactory.isGlobalIndex(config), targetParallelism) } else { prependedRdd } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/engine/HoodieReaderContext.java b/hudi-common/src/main/java/org/apache/hudi/common/engine/HoodieReaderContext.java index 0b065004ca523..fa9c0658e7c02 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/engine/HoodieReaderContext.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/engine/HoodieReaderContext.java @@ -277,13 +277,13 @@ private void initRecordMerger(TypedProperties properties, boolean isIngestion) { // If the provided payload class differs from the table's payload class, we need to infer the correct merging behavior. if (isIngestion && writerPayloadClass.map(className -> !className.equals(tableConfig.getPayloadClass())).orElse(false)) { Triple triple = HoodieTableConfig.inferMergingConfigsForWrites(null, writerPayloadClass.get(), null, - tableConfig.getPreCombineFieldsStr().orElse(null), tableVersion); + tableConfig.getOrderingFieldsStr().orElse(null), tableVersion); recordMergeMode = triple.getLeft(); mergeStrategyId = triple.getRight(); } else if (tableVersion.lesserThan(HoodieTableVersion.EIGHT)) { Triple triple = inferMergingConfigsForPreV9Table( recordMergeMode, tableConfig.getPayloadClass(), - mergeStrategyId, tableConfig.getPreCombineFieldsStr().orElse(null), tableVersion); + mergeStrategyId, tableConfig.getOrderingFieldsStr().orElse(null), tableVersion); recordMergeMode = triple.getLeft(); mergeStrategyId = triple.getRight(); } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieRecordMerger.java b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieRecordMerger.java index d26147f856ae5..90d656fee3604 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieRecordMerger.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieRecordMerger.java @@ -158,7 +158,7 @@ default boolean isProjectionCompatible() { } /** - * Returns a list of fields required for mor merging. The default implementation will return the recordkey field and the precombine + * Returns a list of fields required for mor merging. The default implementation will return the recordKey field and the ordering fields. */ default String[] getMandatoryFieldsForMerging(Schema dataSchema, HoodieTableConfig cfg, TypedProperties properties) { ArrayList requiredFields = new ArrayList<>(); @@ -172,8 +172,8 @@ default String[] getMandatoryFieldsForMerging(Schema dataSchema, HoodieTableConf } } - List preCombineFields = cfg.getPreCombineFields(); - requiredFields.addAll(preCombineFields); + List orderingFields = cfg.getOrderingFields(); + requiredFields.addAll(orderingFields); return requiredFields.toArray(new String[0]); } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableConfig.java b/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableConfig.java index f6463e4280ec5..fcd7998131e9f 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableConfig.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableConfig.java @@ -192,14 +192,25 @@ public class HoodieTableConfig extends HoodieConfig { + " to identify what upgrade/downgrade paths happened on the table. This is only configured " + "when the table is initially setup."); - // TODO: is this this called precombine in 1.0. .. - public static final ConfigProperty PRECOMBINE_FIELDS = ConfigProperty + /** + * @deprecated Use {@link #ORDERING_FIELDS} instead + */ + @Deprecated + public static final ConfigProperty PRECOMBINE_FIELD = ConfigProperty .key("hoodie.table.precombine.field") .noDefaultValue() .withDocumentation("Comma separated fields used in preCombining before actual write. By default, when two records have the same key value, " + "the largest value for the precombine field determined by Object.compareTo(..), is picked. If there are multiple fields configured, " + "comparison is made on the first field. If the first field values are same, comparison is made on the second field and so on."); + public static final ConfigProperty ORDERING_FIELDS = ConfigProperty + .key("hoodie.table.ordering.fields") + .noDefaultValue() + .withAlternatives(PRECOMBINE_FIELD.key()) + .withDocumentation("Comma separated fields used in records merging comparison. By default, when two records have the same key value, " + + "the largest value for the ordering field determined by Object.compareTo(..), is picked. If there are multiple fields configured, " + + "comparison is made on the first field. If the first field values are same, comparison is made on the second field and so on."); + public static final ConfigProperty PARTITION_FIELDS = ConfigProperty .key("hoodie.table.partition.fields") .noDefaultValue() @@ -255,7 +266,7 @@ public class HoodieTableConfig extends HoodieConfig { public static final ConfigProperty RECORD_MERGE_MODE = ConfigProperty .key("hoodie.record.merge.mode") .defaultValue((RecordMergeMode) null, - "COMMIT_TIME_ORDERING if precombine is not set; EVENT_TIME_ORDERING if precombine is set") + "COMMIT_TIME_ORDERING if ordering field is not set; EVENT_TIME_ORDERING if ordering field is set") .sinceVersion("1.0.0") .withDocumentation(RecordMergeMode.class); @@ -951,13 +962,13 @@ public static Triple inferMergingConfigsForPreV // Check ordering field name based on record merge mode if (inferredRecordMergeMode == COMMIT_TIME_ORDERING) { if (nonEmpty(orderingFieldNamesAsString)) { - LOG.warn("The precombine or ordering field ({}) is specified. COMMIT_TIME_ORDERING " - + "merge mode does not use precombine or ordering field anymore.", orderingFieldNamesAsString); + LOG.warn("The ordering field ({}) is specified. COMMIT_TIME_ORDERING " + + "merge mode does not use ordering field anymore.", orderingFieldNamesAsString); } } else if (inferredRecordMergeMode == EVENT_TIME_ORDERING) { if (isNullOrEmpty(orderingFieldNamesAsString)) { - LOG.warn("The precombine or ordering field is not specified. EVENT_TIME_ORDERING " - + "merge mode requires precombine or ordering field to be set for getting the " + LOG.warn("The ordering field is not specified. EVENT_TIME_ORDERING " + + "merge mode requires ordering field to be set for getting the " + "event time. Using commit time-based ordering now."); } } @@ -1018,14 +1029,14 @@ static RecordMergeMode inferRecordMergeModeFromMergeStrategyId(String recordMerg } } - public List getPreCombineFields() { - return getPreCombineFieldsStr() - .map(preCombine -> Arrays.stream(preCombine.split(",")).filter(StringUtils::nonEmpty).collect(Collectors.toList())) + public List getOrderingFields() { + return getOrderingFieldsStr() + .map(orderingFieldsStr -> Arrays.stream(orderingFieldsStr.split(",")).filter(StringUtils::nonEmpty).collect(Collectors.toList())) .orElse(Collections.emptyList()); } - public Option getPreCombineFieldsStr() { - return Option.ofNullable(getString(PRECOMBINE_FIELDS)); + public Option getOrderingFieldsStr() { + return Option.ofNullable(getString(ORDERING_FIELDS)); } public Option getRecordKeyFields() { @@ -1034,7 +1045,7 @@ public Option getRecordKeyFields() { return Option.empty(); } else { return Option.of(Arrays.stream(keyFieldsValue.split(",")) - .filter(p -> p.length() > 0).collect(Collectors.toList()).toArray(new String[] {})); + .filter(p -> !p.isEmpty()).collect(Collectors.toList()).toArray(new String[] {})); } } @@ -1335,11 +1346,13 @@ public Map propsMap() { */ @Deprecated public static final String HOODIE_TABLE_VERSION_PROP_NAME = VERSION.key(); + /** - * @deprecated Use {@link #PRECOMBINE_FIELDS} and its methods. + * @deprecated Use {@link #ORDERING_FIELDS} and its methods. */ @Deprecated - public static final String HOODIE_TABLE_PRECOMBINE_FIELD = PRECOMBINE_FIELDS.key(); + public static final String HOODIE_TABLE_PRECOMBINE_FIELD = PRECOMBINE_FIELD.key(); + /** * @deprecated Use {@link #BASE_FILE_FORMAT} and its methods. */ 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 04bef0657b93f..1a1d072874e2b 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 @@ -1047,7 +1047,7 @@ public static class TableBuilder { private String timelinePath; private String timelineHistoryPath; private String baseFileFormat; - private String preCombineFields; + private String orderingFields; private String partitionFields; private Boolean cdcEnabled; private String cdcSupplementalLoggingMode; @@ -1169,11 +1169,11 @@ public TableBuilder setBaseFileFormat(String baseFileFormat) { } /** - * Sets preCombine fields as a comma separated string in the table - * @param preCombineFieldsAsString - Comma separated preCombine fields which need to be set for the table + * Sets ordering fields as a comma separated string in the table + * @param orderingFieldsAsString - Comma separated ordering fields which need to be set for the table */ - public TableBuilder setPreCombineFields(String preCombineFieldsAsString) { - this.preCombineFields = preCombineFieldsAsString; + public TableBuilder setOrderingFields(String orderingFieldsAsString) { + this.orderingFields = orderingFieldsAsString; return this; } @@ -1376,8 +1376,8 @@ public TableBuilder fromProperties(Properties properties) { setBootstrapIndexEnable(hoodieConfig.getBoolean(HoodieTableConfig.BOOTSTRAP_INDEX_ENABLE)); } - if (hoodieConfig.contains(HoodieTableConfig.PRECOMBINE_FIELDS)) { - setPreCombineFields(hoodieConfig.getString(HoodieTableConfig.PRECOMBINE_FIELDS)); + if (hoodieConfig.contains(HoodieTableConfig.ORDERING_FIELDS)) { + setOrderingFields(hoodieConfig.getString(HoodieTableConfig.ORDERING_FIELDS)); } if (hoodieConfig.contains(HoodieTableConfig.PARTITION_FIELDS)) { setPartitionFields( @@ -1462,14 +1462,14 @@ public Properties build() { if (tableVersion.lesserThan(HoodieTableVersion.NINE)) { Triple mergeConfigs = inferMergingConfigsForPreV9Table( - recordMergeMode, payloadClassName, recordMergerStrategyId, preCombineFields, + recordMergeMode, payloadClassName, recordMergerStrategyId, orderingFields, tableVersion); tableConfig.setValue(RECORD_MERGE_MODE, mergeConfigs.getLeft().name()); tableConfig.setValue(PAYLOAD_CLASS_NAME.key(), mergeConfigs.getMiddle()); tableConfig.setValue(RECORD_MERGE_STRATEGY_ID, mergeConfigs.getRight()); } else { // For table version >= 9 Map mergeConfigs = inferMergingConfigsForV9TableCreation( - recordMergeMode, payloadClassName, recordMergerStrategyId, preCombineFields, tableVersion); + recordMergeMode, payloadClassName, recordMergerStrategyId, orderingFields, tableVersion); for (Map.Entry config : mergeConfigs.entrySet()) { tableConfig.setValue(config.getKey(), config.getValue()); } @@ -1518,8 +1518,8 @@ public Properties build() { tableConfig.setValue(HoodieTableConfig.BOOTSTRAP_BASE_PATH, bootstrapBasePath); } - if (StringUtils.nonEmpty(preCombineFields)) { - tableConfig.setValue(HoodieTableConfig.PRECOMBINE_FIELDS, preCombineFields); + if (StringUtils.nonEmpty(orderingFields)) { + tableConfig.setValue(HoodieTableConfig.ORDERING_FIELDS, orderingFields); } if (null != partitionFields) { diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/log/AbstractHoodieLogRecordScanner.java b/hudi-common/src/main/java/org/apache/hudi/common/table/log/AbstractHoodieLogRecordScanner.java index 4266dd79d6bf8..97d0f41cca60d 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/log/AbstractHoodieLogRecordScanner.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/log/AbstractHoodieLogRecordScanner.java @@ -104,8 +104,6 @@ public abstract class AbstractHoodieLogRecordScanner { private final Option partitionPathFieldOpt; // Partition name override private final Option partitionNameOverrideOpt; - // Pre-combining field - protected final String preCombineFields; // Stateless component for merging records protected final HoodieRecordMerger recordMerger; private final TypedProperties payloadProps; @@ -175,13 +173,12 @@ protected AbstractHoodieLogRecordScanner(HoodieStorage storage, String basePath, // load class from the payload fully qualified class name HoodieTableConfig tableConfig = this.hoodieTableMetaClient.getTableConfig(); this.payloadClassFQN = tableConfig.getPayloadClass(); - this.preCombineFields = tableConfig.getPreCombineFieldsStr().orElse(null); + String orderingFieldsStr = tableConfig.getOrderingFieldsStr().orElse(null); // Log scanner merge log with precombine TypedProperties props = new TypedProperties(); - if (preCombineFields != null) { - props.setProperty(HoodiePayloadProps.PAYLOAD_ORDERING_FIELD_PROP_KEY, preCombineFields); - props.setProperty(HoodieTableConfig.PRECOMBINE_FIELDS.key(), preCombineFields); - props.setProperty("hoodie.datasource.write.precombine.field", preCombineFields); + if (orderingFieldsStr != null) { + props.setProperty(HoodiePayloadProps.PAYLOAD_ORDERING_FIELD_PROP_KEY, orderingFieldsStr); + props.setProperty(HoodieTableConfig.ORDERING_FIELDS.key(), orderingFieldsStr); } this.tableVersion = tableConfig.getTableVersion(); this.payloadProps = props; diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/log/BaseHoodieLogRecordReader.java b/hudi-common/src/main/java/org/apache/hudi/common/table/log/BaseHoodieLogRecordReader.java index 831a86ad3e9aa..4bb3454fbda98 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/log/BaseHoodieLogRecordReader.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/log/BaseHoodieLogRecordReader.java @@ -94,8 +94,8 @@ public abstract class BaseHoodieLogRecordReader { private final String recordKeyField; // Partition name override private final Option partitionNameOverrideOpt; - // Pre-combining field - protected final String preCombineFields; + // Ordering fields + protected final String orderingFields; private final TypedProperties payloadProps; // Log File Paths protected final List logFilePaths; @@ -151,11 +151,11 @@ protected BaseHoodieLogRecordReader(HoodieReaderContext readerContext, Hoodie // load class from the payload fully qualified class name HoodieTableConfig tableConfig = this.hoodieTableMetaClient.getTableConfig(); this.payloadClassFQN = tableConfig.getPayloadClass(); - this.preCombineFields = tableConfig.getPreCombineFieldsStr().orElse(null); + this.orderingFields = tableConfig.getOrderingFieldsStr().orElse(null); // Log scanner merge log with precombine TypedProperties props = new TypedProperties(); - if (this.preCombineFields != null) { - props.setProperty(HoodiePayloadProps.PAYLOAD_ORDERING_FIELD_PROP_KEY, this.preCombineFields); + if (this.orderingFields != null) { + props.setProperty(HoodiePayloadProps.PAYLOAD_ORDERING_FIELD_PROP_KEY, this.orderingFields); } this.payloadProps = props; this.totalLogFiles.addAndGet(logFilePaths.size()); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/read/FileGroupReaderSchemaHandler.java b/hudi-common/src/main/java/org/apache/hudi/common/table/read/FileGroupReaderSchemaHandler.java index 24bbe66b3830c..9ccfcf1218324 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/read/FileGroupReaderSchemaHandler.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/read/FileGroupReaderSchemaHandler.java @@ -214,7 +214,7 @@ private static String[] getMandatoryFieldsForMerging(HoodieTableConfig cfg, cfg.getRecordMergeMode(), cfg.getPayloadClass(), cfg.getRecordMergeStrategyId(), - cfg.getPreCombineFieldsStr().orElse(null), + cfg.getOrderingFieldsStr().orElse(null), cfg.getTableVersion()); mergeMode = mergingConfigs.getLeft(); } @@ -241,7 +241,7 @@ private static String[] getMandatoryFieldsForMerging(HoodieTableConfig cfg, } // Add precombine field for event time ordering merge mode. if (mergeMode == RecordMergeMode.EVENT_TIME_ORDERING) { - List preCombineFields = cfg.getPreCombineFields(); + List preCombineFields = cfg.getOrderingFields(); requiredFields.addAll(preCombineFields); } // Add `HOODIE_IS_DELETED_FIELD` field if exists. diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/ConfigUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/util/ConfigUtils.java index d03182a8dcd1e..bffad1a79ec6a 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/ConfigUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/ConfigUtils.java @@ -82,15 +82,48 @@ public class ConfigUtils { */ @Nullable public static String[] getOrderingFields(Properties properties) { + String orderField = getOrderingFieldsStr(properties); + return StringUtils.isNullOrEmpty(orderField) ? null : orderField.split(","); + } + + /** + * Get ordering fields as comma separated string. + */ + @Nullable + public static String getOrderingFieldsStr(Properties properties) { + String orderField = getOrderingFieldsStrDuringWrite(properties); + if (orderField == null && properties.containsKey(HoodiePayloadProps.PAYLOAD_ORDERING_FIELD_PROP_KEY)) { + orderField = properties.getProperty(HoodiePayloadProps.PAYLOAD_ORDERING_FIELD_PROP_KEY); + } + return orderField; + } + + /** + * Get ordering fields as comma separated string. + */ + @Nullable + public static String getOrderingFieldsStrDuringWrite(Properties properties) { String orderField = null; - if (properties.containsKey("hoodie.datasource.write.precombine.field")) { + if (containsConfigProperty(properties, HoodieTableConfig.ORDERING_FIELDS)) { + orderField = getStringWithAltKeys(properties, HoodieTableConfig.ORDERING_FIELDS); + } else if (properties.containsKey("hoodie.datasource.write.precombine.field")) { orderField = properties.getProperty("hoodie.datasource.write.precombine.field"); - } else if (properties.containsKey(HoodieTableConfig.PRECOMBINE_FIELDS.key())) { - orderField = properties.getProperty(HoodieTableConfig.PRECOMBINE_FIELDS.key()); - } else if (properties.containsKey(HoodiePayloadProps.PAYLOAD_ORDERING_FIELD_PROP_KEY)) { - orderField = properties.getProperty(HoodiePayloadProps.PAYLOAD_ORDERING_FIELD_PROP_KEY); } - return StringUtils.isNullOrEmpty(orderField) ? null : orderField.split(","); + return orderField; + } + + /** + * Get ordering fields as comma separated string. + */ + @Nullable + public static String getOrderingFieldsStrDuringWrite(Map properties) { + String orderField = null; + if (containsConfigProperty(properties, HoodieTableConfig.ORDERING_FIELDS)) { + orderField = getStringWithAltKeys(properties, HoodieTableConfig.ORDERING_FIELDS); + } else if (properties.containsKey("hoodie.datasource.write.precombine.field")) { + orderField = properties.get("hoodie.datasource.write.precombine.field"); + } + return orderField; } /** @@ -101,8 +134,7 @@ public static String[] getOrderingFields(Properties properties) { public static TypedProperties supplementOrderingFields(TypedProperties props, List orderingFields) { String orderingFieldsAsString = String.join(",", orderingFields); props.putIfAbsent(HoodiePayloadProps.PAYLOAD_ORDERING_FIELD_PROP_KEY, orderingFieldsAsString); - props.putIfAbsent(HoodieTableConfig.PRECOMBINE_FIELDS.key(), orderingFieldsAsString); - props.putIfAbsent("hoodie.datasource.write.precombine.field", orderingFieldsAsString); + props.putIfAbsent(HoodieTableConfig.ORDERING_FIELDS.key(), orderingFieldsAsString); return props; } @@ -278,7 +310,7 @@ public static boolean containsConfigProperty(Properties props, * @param configProperty Config to look up. * @return {@code true} if exists; {@code false} otherwise. */ - public static boolean containsConfigProperty(Map props, + public static boolean containsConfigProperty(Map props, ConfigProperty configProperty) { return containsConfigProperty(props::containsKey, configProperty); } @@ -450,8 +482,8 @@ public static String getStringWithAltKeys(Properties props, * and there is default value defined in the {@link ConfigProperty} config and is convertible to * String type; {@code null} otherwise. */ - public static String getStringWithAltKeys(Map props, - ConfigProperty configProperty) { + public static String getStringWithAltKeys(Map props, + ConfigProperty configProperty) { return getStringWithAltKeys(props::get, configProperty); } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/HoodieRecordUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/util/HoodieRecordUtils.java index dd0eccb32e301..4094adbe485b2 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/HoodieRecordUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/HoodieRecordUtils.java @@ -126,6 +126,6 @@ public static List getOrderingFieldNames(RecordMergeMode mergeMode, HoodieTableMetaClient metaClient) { return mergeMode == RecordMergeMode.COMMIT_TIME_ORDERING ? Collections.emptyList() - : Option.ofNullable(ConfigUtils.getOrderingFields(props)).map(Arrays::asList).orElseGet(() -> metaClient.getTableConfig().getPreCombineFields()); + : Option.ofNullable(ConfigUtils.getOrderingFields(props)).map(Arrays::asList).orElseGet(() -> metaClient.getTableConfig().getOrderingFields()); } } \ No newline at end of file diff --git a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java index e99b8b79eb677..52682c7bd74ef 100644 --- a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java @@ -1046,7 +1046,7 @@ private static ClosableIterator> getLogRecords(List(readerContext, writerSchemaOpt.get(), writerSchemaOpt.get(), Option.empty(), properties, datasetMetaClient)); HoodieReadStats readStats = new HoodieReadStats(); KeyBasedFileGroupRecordBuffer recordBuffer = new KeyBasedFileGroupRecordBuffer<>(readerContext, datasetMetaClient, - readerContext.getMergeMode(), Option.empty(), properties, tableConfig.getPreCombineFields(), + readerContext.getMergeMode(), Option.empty(), properties, tableConfig.getOrderingFields(), UpdateProcessor.create(readStats, readerContext, true, Option.empty(), properties)); // CRITICAL: Ensure allowInflightInstants is set to true diff --git a/hudi-common/src/test/java/org/apache/hudi/common/model/TestDefaultHoodieRecordPayload.java b/hudi-common/src/test/java/org/apache/hudi/common/model/TestDefaultHoodieRecordPayload.java index d667176a3c74f..decfa41c57257 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/model/TestDefaultHoodieRecordPayload.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/model/TestDefaultHoodieRecordPayload.java @@ -18,7 +18,7 @@ package org.apache.hudi.common.model; -import org.apache.hudi.common.testutils.PreCombineTestUtils; +import org.apache.hudi.common.testutils.OrderingFieldsTestUtils; import org.apache.hudi.common.util.Option; import org.apache.avro.Schema; @@ -63,9 +63,9 @@ public void setUp() throws Exception { } @ParameterizedTest - @MethodSource("org.apache.hudi.common.testutils.PreCombineTestUtils#configurePreCombine") + @MethodSource("org.apache.hudi.common.testutils.OrderingFieldsTestUtils#configureOrderingFields") public void testActiveRecords(String key) throws IOException { - PreCombineTestUtils.setPreCombineConfig(props, key, "ts"); + OrderingFieldsTestUtils.setOrderingFieldsConfig(props, key, "ts"); GenericRecord record1 = new GenericData.Record(schema); record1.put("id", "1"); record1.put("partition", "partition0"); @@ -91,9 +91,9 @@ public void testActiveRecords(String key) throws IOException { } @ParameterizedTest - @MethodSource("org.apache.hudi.common.testutils.PreCombineTestUtils#configurePreCombine") + @MethodSource("org.apache.hudi.common.testutils.OrderingFieldsTestUtils#configureOrderingFields") public void testDeletedRecord(String key) throws IOException { - PreCombineTestUtils.setPreCombineConfig(props, key, "ts"); + OrderingFieldsTestUtils.setOrderingFieldsConfig(props, key, "ts"); GenericRecord record1 = new GenericData.Record(schema); record1.put("id", "1"); record1.put("partition", "partition0"); diff --git a/hudi-common/src/test/java/org/apache/hudi/common/table/read/SchemaHandlerTestBase.java b/hudi-common/src/test/java/org/apache/hudi/common/table/read/SchemaHandlerTestBase.java index 77d85dc0f5b36..4af0c922c2f2e 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/table/read/SchemaHandlerTestBase.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/table/read/SchemaHandlerTestBase.java @@ -259,11 +259,11 @@ private static void setupMORTable(RecordMergeMode mergeMode, boolean hasPrecombi when(hoodieTableConfig.getRecordMergeMode()).thenReturn(mergeMode); when(hoodieTableConfig.getTableVersion()).thenReturn(HoodieTableVersion.current()); if (hasPrecombine) { - when(hoodieTableConfig.getPreCombineFieldsStr()).thenReturn(Option.of("timestamp")); - when(hoodieTableConfig.getPreCombineFields()).thenReturn(Collections.singletonList("timestamp")); + when(hoodieTableConfig.getOrderingFieldsStr()).thenReturn(Option.of("timestamp")); + when(hoodieTableConfig.getOrderingFields()).thenReturn(Collections.singletonList("timestamp")); } else { - when(hoodieTableConfig.getPreCombineFieldsStr()).thenReturn(Option.empty()); - when(hoodieTableConfig.getPreCombineFields()).thenReturn(Collections.emptyList()); + when(hoodieTableConfig.getOrderingFieldsStr()).thenReturn(Option.empty()); + when(hoodieTableConfig.getOrderingFields()).thenReturn(Collections.emptyList()); } if (mergeMode == CUSTOM) { when(hoodieTableConfig.getRecordMergeStrategyId()).thenReturn("asdf"); diff --git a/hudi-common/src/test/java/org/apache/hudi/common/table/read/TestFileGroupReaderSchemaHandler.java b/hudi-common/src/test/java/org/apache/hudi/common/table/read/TestFileGroupReaderSchemaHandler.java index 2da196905f92d..5d88a929759f5 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/table/read/TestFileGroupReaderSchemaHandler.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/table/read/TestFileGroupReaderSchemaHandler.java @@ -228,8 +228,8 @@ public void testSchemaForMandatoryFields(boolean setPrecombine, when(hoodieTableConfig.getRecordMergeMode()).thenReturn(mergeMode); when(hoodieTableConfig.populateMetaFields()).thenReturn(true); - when(hoodieTableConfig.getPreCombineFieldsStr()).thenReturn(Option.of(setPrecombine ? preCombineField : StringUtils.EMPTY_STRING)); - when(hoodieTableConfig.getPreCombineFields()).thenReturn(setPrecombine ? Collections.singletonList(preCombineField) : Collections.emptyList()); + when(hoodieTableConfig.getOrderingFieldsStr()).thenReturn(Option.of(setPrecombine ? preCombineField : StringUtils.EMPTY_STRING)); + when(hoodieTableConfig.getOrderingFields()).thenReturn(setPrecombine ? Collections.singletonList(preCombineField) : Collections.emptyList()); when(hoodieTableConfig.getTableVersion()).thenReturn(tableVersion); if (hoodieTableConfig.getTableVersion() == HoodieTableVersion.SIX) { if (mergeMode == RecordMergeMode.EVENT_TIME_ORDERING) { diff --git a/hudi-common/src/test/java/org/apache/hudi/common/table/read/TestHoodieFileGroupReaderBase.java b/hudi-common/src/test/java/org/apache/hudi/common/table/read/TestHoodieFileGroupReaderBase.java index a91e78afa584f..bbce066422472 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/table/read/TestHoodieFileGroupReaderBase.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/table/read/TestHoodieFileGroupReaderBase.java @@ -111,7 +111,7 @@ public abstract class TestHoodieFileGroupReaderBase { private static final List DEFAULT_SUPPORTED_FILE_FORMATS = Arrays.asList(HoodieFileFormat.PARQUET, HoodieFileFormat.ORC); protected static List supportedFileFormats; private static final String KEY_FIELD_NAME = "_row_key"; - private static final String PRECOMBINE_FIELD_NAME = "timestamp"; + private static final String ORDERING_FIELD_NAME = "timestamp"; private static final String PARTITION_FIELD_NAME = "partition_path"; private static final String RIDER_FIELD_NAME = "rider"; @TempDir @@ -208,7 +208,7 @@ public void testReadFileGroupWithMultipleOrderingFields() throws Exception { writeConfigs.put("hoodie.datasource.write.table.type", HoodieTableType.MERGE_ON_READ.name()); // Use two precombine values - combination of timestamp and rider String orderingValues = "timestamp,rider"; - writeConfigs.put("hoodie.datasource.write.precombine.field", orderingValues); + writeConfigs.put(HoodieTableConfig.ORDERING_FIELDS.key(), orderingValues); writeConfigs.put("hoodie.payload.ordering.field", orderingValues); try (HoodieTestDataGenerator dataGen = new HoodieTestDataGenerator(0xDEEF)) { @@ -628,8 +628,8 @@ protected Map getCommonConfigs(RecordMergeMode recordMergeMode, Map configMapping = new HashMap<>(); configMapping.put(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key(), KEY_FIELD_NAME); configMapping.put(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key(), PARTITION_FIELD_NAME); - configMapping.put("hoodie.datasource.write.precombine.field", PRECOMBINE_FIELD_NAME); - configMapping.put("hoodie.payload.ordering.field", PRECOMBINE_FIELD_NAME); + configMapping.put(HoodieTableConfig.ORDERING_FIELDS.key(), ORDERING_FIELD_NAME); + configMapping.put("hoodie.payload.ordering.field", ORDERING_FIELD_NAME); configMapping.put(HoodieTableConfig.HOODIE_TABLE_NAME_KEY, "hoodie_test"); configMapping.put("hoodie.insert.shuffle.parallelism", "4"); configMapping.put("hoodie.upsert.shuffle.parallelism", "4"); @@ -713,7 +713,7 @@ private static List getExpectedHoodieRecordsWithOrderingValue(List return expectedHoodieRecords.stream().map(rec -> { RawTripTestPayload oldPayload = (RawTripTestPayload) rec.getData(); try { - List orderingFields = metaClient.getTableConfig().getPreCombineFields(); + List orderingFields = metaClient.getTableConfig().getOrderingFields(); HoodieAvroRecord avroRecord = ((HoodieAvroRecord) rec); Comparable orderingValue = OrderingValues.create(orderingFields, field -> (Comparable) avroRecord.getColumnValueAsJava(avroSchema, field, new TypedProperties())); RawTripTestPayload newPayload = new RawTripTestPayload(Option.ofNullable(oldPayload.getJsonData()), oldPayload.getRowKey(), oldPayload.getPartitionPath(), null, false, orderingValue); @@ -739,20 +739,20 @@ private void validateOutputFromFileGroupReaderWithExistingRecords(StorageConfigu boolean sortOutput = !containsBaseFile; List actualRecordList = convertEngineRecords( readRecordsFromFileGroup(storageConf, tablePath, metaClient, fileSlices, avroSchema, recordMergeMode, false, sortOutput), - avroSchema, readerContext, metaClient.getTableConfig().getPreCombineFields()); + avroSchema, readerContext, metaClient.getTableConfig().getOrderingFields()); // validate size is equivalent to ensure no duplicates are returned assertEquals(expectedRecords.size(), actualRecordList.size()); assertEquals(new HashSet<>(expectedRecords), new HashSet<>(actualRecordList)); // validate records can be read from file group as HoodieRecords actualRecordList = convertHoodieRecords( readHoodieRecordsFromFileGroup(storageConf, tablePath, metaClient, fileSlices, avroSchema, recordMergeMode), - avroSchema, readerContext, metaClient.getTableConfig().getPreCombineFields()); + avroSchema, readerContext, metaClient.getTableConfig().getOrderingFields()); assertEquals(expectedRecords.size(), actualRecordList.size()); assertEquals(new HashSet<>(expectedRecords), new HashSet<>(actualRecordList)); // validate unmerged records actualRecordList = convertEngineRecords( readRecordsFromFileGroup(storageConf, tablePath, metaClient, fileSlices, avroSchema, recordMergeMode, true, false), - avroSchema, readerContext, metaClient.getTableConfig().getPreCombineFields()); + avroSchema, readerContext, metaClient.getTableConfig().getOrderingFields()); assertEquals(expectedUnmergedRecords.size(), actualRecordList.size()); assertEquals(new HashSet<>(expectedUnmergedRecords), new HashSet<>(actualRecordList)); } @@ -888,8 +888,8 @@ private List> readHoodieRecordsFromFileGroup(StorageConfiguratio private TypedProperties buildProperties(HoodieTableMetaClient metaClient, RecordMergeMode recordMergeMode) { TypedProperties props = new TypedProperties(); - props.setProperty("hoodie.datasource.write.precombine.field", metaClient.getTableConfig().getPreCombineFieldsStr().orElse("")); - props.setProperty("hoodie.payload.ordering.field", metaClient.getTableConfig().getPreCombineFieldsStr().orElse("")); + props.setProperty(HoodieTableConfig.ORDERING_FIELDS.key(), metaClient.getTableConfig().getOrderingFieldsStr().orElse("")); + props.setProperty("hoodie.payload.ordering.field", metaClient.getTableConfig().getOrderingFieldsStr().orElse("")); props.setProperty(RECORD_MERGE_MODE.key(), recordMergeMode.name()); if (recordMergeMode.equals(RecordMergeMode.CUSTOM)) { props.setProperty(RECORD_MERGE_STRATEGY_ID.key(), PAYLOAD_BASED_MERGE_STRATEGY_UUID); @@ -949,14 +949,14 @@ private List convertEngineRecords(List } private List convertHoodieRecords(List> records, Schema schema, HoodieReaderContext readerContext, - List preCombineFields) { + List orderingFields) { TypedProperties props = new TypedProperties(); - props.setProperty("hoodie.datasource.write.precombine.field", String.join(",", preCombineFields)); + props.setProperty(HoodieTableConfig.ORDERING_FIELDS.key(), String.join(",", orderingFields)); return records.stream() .map(record -> new HoodieTestDataGenerator.RecordIdentifier( record.getRecordKey(), removeHiveStylePartition(record.getPartitionPath()), - record.getOrderingValue(schema, props, preCombineFields.toArray(new String[0])).toString(), + record.getOrderingValue(schema, props, orderingFields.toArray(new String[0])).toString(), readerContext.getRecordContext().getValue(record.getData(), schema, RIDER_FIELD_NAME).toString())) .collect(Collectors.toList()); } diff --git a/hudi-common/src/test/java/org/apache/hudi/common/table/read/buffer/TestFileGroupRecordBufferLoader.java b/hudi-common/src/test/java/org/apache/hudi/common/table/read/buffer/TestFileGroupRecordBufferLoader.java index 649b6ee1720ca..c13803ede0538 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/table/read/buffer/TestFileGroupRecordBufferLoader.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/table/read/buffer/TestFileGroupRecordBufferLoader.java @@ -64,7 +64,7 @@ public void testDefaultFileGroupBufferRecordLoader(String fileGroupRecordBufferT when(tableConfig.getRecordMergeMode()).thenReturn(RecordMergeMode.COMMIT_TIME_ORDERING); when(tableConfig.getTableVersion()).thenReturn(HoodieTableVersion.NINE); when(tableConfig.getPartialUpdateMode()).thenReturn(Option.empty()); - when(tableConfig.getPreCombineFieldsStr()).thenReturn(Option.empty()); + when(tableConfig.getOrderingFieldsStr()).thenReturn(Option.empty()); when(tableConfig.getRecordKeyFields()).thenReturn(Option.of(new String[] {"record_key"})); StorageConfiguration storageConfiguration = mock(StorageConfiguration.class); HoodieReaderContext readerContext = new HoodieAvroReaderContext(storageConfiguration, tableConfig, Option.empty(), Option.empty()); diff --git a/hudi-common/src/test/java/org/apache/hudi/common/table/read/buffer/TestKeyBasedFileGroupRecordBuffer.java b/hudi-common/src/test/java/org/apache/hudi/common/table/read/buffer/TestKeyBasedFileGroupRecordBuffer.java index 4e85f74bbd374..7dd568bcbac15 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/table/read/buffer/TestKeyBasedFileGroupRecordBuffer.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/table/read/buffer/TestKeyBasedFileGroupRecordBuffer.java @@ -140,7 +140,7 @@ void readWithEventTimeOrderingAndDeleteBlock() throws IOException { void readWithEventTimeOrderingWithRecords() throws IOException { HoodieReadStats readStats = new HoodieReadStats(); TypedProperties properties = new TypedProperties(); - properties.setProperty(HoodieTableConfig.PRECOMBINE_FIELDS.key(), "ts"); + properties.setProperty(HoodieTableConfig.ORDERING_FIELDS.key(), "ts"); properties.setProperty(DELETE_KEY, "counter"); properties.setProperty(DELETE_MARKER, "3"); HoodieTableConfig tableConfig = mock(HoodieTableConfig.class); diff --git a/hudi-common/src/test/java/org/apache/hudi/common/table/read/buffer/TestSortedKeyBasedFileGroupRecordBuffer.java b/hudi-common/src/test/java/org/apache/hudi/common/table/read/buffer/TestSortedKeyBasedFileGroupRecordBuffer.java index 1f2394db848e9..677cde4e9079c 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/table/read/buffer/TestSortedKeyBasedFileGroupRecordBuffer.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/table/read/buffer/TestSortedKeyBasedFileGroupRecordBuffer.java @@ -112,7 +112,7 @@ void readBaseFileAndLogFile() throws IOException { void readWithStreamingRecordBufferLoaderAndEventTimeOrdering() throws IOException { HoodieReadStats readStats = new HoodieReadStats(); TypedProperties properties = new TypedProperties(); - properties.setProperty(HoodieTableConfig.PRECOMBINE_FIELDS.key(), "ts"); + properties.setProperty(HoodieTableConfig.ORDERING_FIELDS.key(), "ts"); properties.setProperty(DELETE_KEY, "counter"); properties.setProperty(DELETE_MARKER, "3"); HoodieTableConfig tableConfig = mock(HoodieTableConfig.class); diff --git a/hudi-common/src/test/java/org/apache/hudi/common/table/read/buffer/TestStreamingKeyBasedFileGroupRecordBuffer.java b/hudi-common/src/test/java/org/apache/hudi/common/table/read/buffer/TestStreamingKeyBasedFileGroupRecordBuffer.java index ee77ee1bf31c5..371cf7245b972 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/table/read/buffer/TestStreamingKeyBasedFileGroupRecordBuffer.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/table/read/buffer/TestStreamingKeyBasedFileGroupRecordBuffer.java @@ -70,7 +70,7 @@ class TestStreamingKeyBasedFileGroupRecordBuffer extends BaseTestFileGroupRecord void readWithEventTimeOrdering() throws IOException { HoodieReadStats readStats = new HoodieReadStats(); TypedProperties properties = new TypedProperties(); - properties.setProperty(HoodieTableConfig.PRECOMBINE_FIELDS.key(), "ts"); + properties.setProperty(HoodieTableConfig.ORDERING_FIELDS.key(), "ts"); properties.setProperty(DELETE_KEY, "counter"); properties.setProperty(DELETE_MARKER, "3"); HoodieTableConfig tableConfig = mock(HoodieTableConfig.class); diff --git a/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieAdaptablePayloadDataGenerator.java b/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieAdaptablePayloadDataGenerator.java index 4d397e8bfac96..5a0106c1b340c 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieAdaptablePayloadDataGenerator.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieAdaptablePayloadDataGenerator.java @@ -65,10 +65,10 @@ public static Properties getKeyGenProps(Class payloadClass) { Properties props = new Properties(); props.put("hoodie.datasource.write.recordkey.field", "id"); props.put("hoodie.datasource.write.partitionpath.field", "pt"); - props.put("hoodie.datasource.write.precombine.field", orderingField); + props.put(HoodieTableConfig.ORDERING_FIELDS.key(), orderingField); props.put(HoodieTableConfig.RECORDKEY_FIELDS.key(), "id"); props.put(HoodieTableConfig.PARTITION_FIELDS.key(), "pt"); - props.put(HoodieTableConfig.PRECOMBINE_FIELDS.key(), orderingField); + props.put(HoodieTableConfig.ORDERING_FIELDS.key(), orderingField); return props; } diff --git a/hudi-common/src/test/java/org/apache/hudi/common/testutils/PreCombineTestUtils.java b/hudi-common/src/test/java/org/apache/hudi/common/testutils/OrderingFieldsTestUtils.java similarity index 78% rename from hudi-common/src/test/java/org/apache/hudi/common/testutils/PreCombineTestUtils.java rename to hudi-common/src/test/java/org/apache/hudi/common/testutils/OrderingFieldsTestUtils.java index 1591433dd1ddd..77c216fe3a52e 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/testutils/PreCombineTestUtils.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/testutils/OrderingFieldsTestUtils.java @@ -28,16 +28,16 @@ import java.util.Properties; import java.util.stream.Stream; -public class PreCombineTestUtils { - private static String[] preCombineConfigs = new String[] { - HoodieTableConfig.PRECOMBINE_FIELDS.key(), +public class OrderingFieldsTestUtils { + private static String[] orderingConfigs = new String[] { + HoodieTableConfig.ORDERING_FIELDS.key(), "hoodie.datasource.write.precombine.field", HoodiePayloadProps.PAYLOAD_ORDERING_FIELD_PROP_KEY }; - public static Stream configurePreCombine() { + public static Stream configureOrderingFields() { return Stream.of( - Arrays.stream(preCombineConfigs).map(Arguments::of).toArray(Arguments[]::new) + Arrays.stream(orderingConfigs).map(Arguments::of).toArray(Arguments[]::new) ); } @@ -45,8 +45,8 @@ public static Stream configurePreCombine() { * Sets specified key to the value provided. The other preCombine related configs are * removed from properties. */ - public static void setPreCombineConfig(Properties props, String key, String value) { - for (String config : preCombineConfigs) { + public static void setOrderingFieldsConfig(Properties props, String key, String value) { + for (String config : orderingConfigs) { if (key.equals(config)) { props.setProperty(key, value); } else { diff --git a/hudi-common/src/test/java/org/apache/hudi/common/util/TestHoodieRecordUtils.java b/hudi-common/src/test/java/org/apache/hudi/common/util/TestHoodieRecordUtils.java index 96a299c9e6ab4..05ed58379d158 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/util/TestHoodieRecordUtils.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/util/TestHoodieRecordUtils.java @@ -71,12 +71,12 @@ void testGetOrderingFields() { // Assert table config precombine fields are returned when props are not set with event time merge mode HoodieTableConfig tableConfig = new HoodieTableConfig(); - tableConfig.setValue(HoodieTableConfig.PRECOMBINE_FIELDS, "tbl"); + tableConfig.setValue(HoodieTableConfig.ORDERING_FIELDS, "tbl"); when(metaClient.getTableConfig()).thenReturn(tableConfig); assertEquals(Collections.singletonList("tbl"), HoodieRecordUtils.getOrderingFieldNames(RecordMergeMode.EVENT_TIME_ORDERING, props, metaClient)); // Assert props value is returned for precombine field configuration when it is set with event time merge mode - props.setProperty("hoodie.datasource.write.precombine.field", "props"); + props.setProperty("hoodie.table.ordering.fields", "props"); assertEquals(Collections.singletonList("props"), HoodieRecordUtils.getOrderingFieldNames(RecordMergeMode.EVENT_TIME_ORDERING, props, metaClient)); } } \ No newline at end of file diff --git a/hudi-examples/hudi-examples-dbt/models/example/hudi_insert_overwrite_table.sql b/hudi-examples/hudi-examples-dbt/models/example/hudi_insert_overwrite_table.sql index e0afa5a456cf6..9334d15b37303 100644 --- a/hudi-examples/hudi-examples-dbt/models/example/hudi_insert_overwrite_table.sql +++ b/hudi-examples/hudi-examples-dbt/models/example/hudi_insert_overwrite_table.sql @@ -27,7 +27,7 @@ incremental_strategy='insert_overwrite', options={ 'type': 'cow', - 'precombineKey': 'ts', + 'orderingFields': 'ts', }, unique_key='id' ) diff --git a/hudi-examples/hudi-examples-dbt/models/example/hudi_upsert_partitioned_cow_table.sql b/hudi-examples/hudi-examples-dbt/models/example/hudi_upsert_partitioned_cow_table.sql index caedcbc5fd055..fbd018a008765 100644 --- a/hudi-examples/hudi-examples-dbt/models/example/hudi_upsert_partitioned_cow_table.sql +++ b/hudi-examples/hudi-examples-dbt/models/example/hudi_upsert_partitioned_cow_table.sql @@ -28,7 +28,7 @@ options={ 'type': 'cow', 'primaryKey': 'id', - 'precombineKey': 'ts', + 'orderingFields': 'ts', }, unique_key='id', partition_by='datestr', diff --git a/hudi-examples/hudi-examples-dbt/models/example/hudi_upsert_partitioned_mor_table.sql b/hudi-examples/hudi-examples-dbt/models/example/hudi_upsert_partitioned_mor_table.sql index 2beab7c4ae466..6035bdccff1fa 100644 --- a/hudi-examples/hudi-examples-dbt/models/example/hudi_upsert_partitioned_mor_table.sql +++ b/hudi-examples/hudi-examples-dbt/models/example/hudi_upsert_partitioned_mor_table.sql @@ -28,7 +28,7 @@ options={ 'type': 'mor', 'primaryKey': 'id', - 'precombineKey': 'ts', + 'orderingFields': 'ts', }, unique_key='id', partition_by='datestr', diff --git a/hudi-examples/hudi-examples-dbt/models/example/hudi_upsert_table.sql b/hudi-examples/hudi-examples-dbt/models/example/hudi_upsert_table.sql index b8ee5b3ed444b..cf47a9c9d6803 100644 --- a/hudi-examples/hudi-examples-dbt/models/example/hudi_upsert_table.sql +++ b/hudi-examples/hudi-examples-dbt/models/example/hudi_upsert_table.sql @@ -28,7 +28,7 @@ options={ 'type': 'cow', 'primaryKey': 'id', - 'precombineKey': 'ts', + 'orderingFields': 'ts', }, unique_key='id' ) diff --git a/hudi-examples/hudi-examples-dbt/models/example_cdc/profiles.sql b/hudi-examples/hudi-examples-dbt/models/example_cdc/profiles.sql index ede8c51de1e74..a53d7dc10c3e3 100644 --- a/hudi-examples/hudi-examples-dbt/models/example_cdc/profiles.sql +++ b/hudi-examples/hudi-examples-dbt/models/example_cdc/profiles.sql @@ -28,7 +28,7 @@ options={ 'type': 'cow', 'primaryKey': 'user_id', - 'preCombineField': 'updated_at', + 'orderingFields': 'updated_at', 'hoodie.table.cdc.enabled': 'true', 'hoodie.table.cdc.supplemental.logging.mode': 'DATA_BEFORE_AFTER' } diff --git a/hudi-examples/hudi-examples-flink/src/main/java/org/apache/hudi/examples/quickstart/utils/QuickstartConfigurations.java b/hudi-examples/hudi-examples-flink/src/main/java/org/apache/hudi/examples/quickstart/utils/QuickstartConfigurations.java index 13c2050b4a48a..dedb65ff78d98 100644 --- a/hudi-examples/hudi-examples-flink/src/main/java/org/apache/hudi/examples/quickstart/utils/QuickstartConfigurations.java +++ b/hudi-examples/hudi-examples-flink/src/main/java/org/apache/hudi/examples/quickstart/utils/QuickstartConfigurations.java @@ -50,7 +50,7 @@ private QuickstartConfigurations() { DataTypes.FIELD("uuid", DataTypes.VARCHAR(20)),// record key DataTypes.FIELD("name", DataTypes.VARCHAR(10)), DataTypes.FIELD("age", DataTypes.INT()), - DataTypes.FIELD("ts", DataTypes.TIMESTAMP(3)), // precombine field + DataTypes.FIELD("ts", DataTypes.TIMESTAMP(3)), // ordering field DataTypes.FIELD("partition", DataTypes.VARCHAR(10))) .notNull(); @@ -68,7 +68,7 @@ private QuickstartConfigurations() { DataTypes.FIELD("name", DataTypes.VARCHAR(10)), DataTypes.FIELD("age", DataTypes.INT()), DataTypes.FIELD("salary", DataTypes.DOUBLE()), - DataTypes.FIELD("ts", DataTypes.TIMESTAMP(3)), // precombine field + DataTypes.FIELD("ts", DataTypes.TIMESTAMP(3)), // ordering field DataTypes.FIELD("partition", DataTypes.VARCHAR(10))) .notNull(); diff --git a/hudi-examples/hudi-examples-k8s/src/main/java/org/apache/hudi/examples/k8s/quickstart/HudiDataStreamWriter.java b/hudi-examples/hudi-examples-k8s/src/main/java/org/apache/hudi/examples/k8s/quickstart/HudiDataStreamWriter.java index fc3fd5bb642fc..410d7a6f385d6 100644 --- a/hudi-examples/hudi-examples-k8s/src/main/java/org/apache/hudi/examples/k8s/quickstart/HudiDataStreamWriter.java +++ b/hudi-examples/hudi-examples-k8s/src/main/java/org/apache/hudi/examples/k8s/quickstart/HudiDataStreamWriter.java @@ -53,7 +53,7 @@ public class HudiDataStreamWriter { public static DataType ROW_DATA_TYPE = DataTypes.ROW( - DataTypes.FIELD("ts", DataTypes.TIMESTAMP(3)), // precombine field + DataTypes.FIELD("ts", DataTypes.TIMESTAMP(3)), // ordering field DataTypes.FIELD("uuid", DataTypes.VARCHAR(40)),// record key DataTypes.FIELD("rider", DataTypes.VARCHAR(20)), DataTypes.FIELD("driver", DataTypes.VARCHAR(20)), @@ -107,7 +107,7 @@ private static Map createHudiOptions(String basePath) { options.put(FlinkOptions.PATH.key(), basePath); options.put(HoodieCommonConfig.HOODIE_FS_ATOMIC_CREATION_SUPPORT.key(), "s3a"); options.put(FlinkOptions.TABLE_TYPE.key(), HoodieTableType.MERGE_ON_READ.name()); - options.put(FlinkOptions.PRECOMBINE_FIELD.key(), "ts"); + options.put(FlinkOptions.ORDERING_FIELDS.key(), "ts"); options.put(FlinkOptions.RECORD_KEY_FIELD.key(), "uuid"); options.put(FlinkOptions.IGNORE_FAILED.key(), "true"); return options; diff --git a/hudi-examples/hudi-examples-spark/src/main/java/org/apache/hudi/examples/quickstart/HoodieSparkQuickstart.java b/hudi-examples/hudi-examples-spark/src/main/java/org/apache/hudi/examples/quickstart/HoodieSparkQuickstart.java index 2a22334989983..fd002d4177eb4 100644 --- a/hudi-examples/hudi-examples-spark/src/main/java/org/apache/hudi/examples/quickstart/HoodieSparkQuickstart.java +++ b/hudi-examples/hudi-examples-spark/src/main/java/org/apache/hudi/examples/quickstart/HoodieSparkQuickstart.java @@ -21,6 +21,7 @@ import org.apache.hudi.QuickstartUtils; import org.apache.hudi.common.model.HoodieAvroPayload; import org.apache.hudi.common.model.WriteOperationType; +import org.apache.hudi.common.table.HoodieTableConfig; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.examples.common.HoodieExampleDataGenerator; import org.apache.hudi.examples.common.HoodieExampleSparkUtils; @@ -115,7 +116,7 @@ public static Dataset insertData(SparkSession spark, JavaSparkContext jsc, df.write().format("hudi") .options(QuickstartUtils.getQuickstartWriteConfigs()) - .option(HoodieWriteConfig.PRECOMBINE_FIELD_NAME.key(), "ts") + .option(HoodieTableConfig.ORDERING_FIELDS.key(), "ts") .option(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key(), "uuid") .option(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key(), "partitionpath") .option(TBL_NAME.key(), tableName) @@ -136,7 +137,7 @@ public static Dataset insertOverwriteData(SparkSession spark, JavaSparkCont df.write().format("hudi") .options(QuickstartUtils.getQuickstartWriteConfigs()) .option("hoodie.datasource.write.operation", WriteOperationType.INSERT_OVERWRITE.name()) - .option(HoodieWriteConfig.PRECOMBINE_FIELD_NAME.key(), "ts") + .option(HoodieTableConfig.ORDERING_FIELDS.key(), "ts") .option(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key(), "uuid") .option(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key(), "partitionpath") .option(TBL_NAME.key(), tableName) @@ -183,7 +184,7 @@ public static Dataset updateData(SparkSession spark, JavaSparkContext jsc, Dataset df = spark.read().json(jsc.parallelize(updates, 1)); df.write().format("hudi") .options(QuickstartUtils.getQuickstartWriteConfigs()) - .option(HoodieWriteConfig.PRECOMBINE_FIELD_NAME.key(), "ts") + .option(HoodieTableConfig.ORDERING_FIELDS.key(), "ts") .option(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key(), "uuid") .option(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key(), "partitionpath") .option(TBL_NAME.key(), tableName) @@ -204,7 +205,7 @@ public static Dataset delete(SparkSession spark, String tablePath, String t df.write().format("hudi") .options(QuickstartUtils.getQuickstartWriteConfigs()) - .option(HoodieWriteConfig.PRECOMBINE_FIELD_NAME.key(), "ts") + .option(HoodieTableConfig.ORDERING_FIELDS.key(), "ts") .option(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key(), "uuid") .option(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key(), "partitionpath") .option(TBL_NAME.key(), tableName) @@ -221,7 +222,7 @@ public static void deleteByPartition(SparkSession spark, String tablePath, Strin Dataset df = spark.emptyDataFrame(); df.write().format("hudi") .options(QuickstartUtils.getQuickstartWriteConfigs()) - .option(HoodieWriteConfig.PRECOMBINE_FIELD_NAME.key(), "ts") + .option(HoodieTableConfig.ORDERING_FIELDS.key(), "ts") .option(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key(), "uuid") .option(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key(), "partitionpath") .option(TBL_NAME.key(), tableName) diff --git a/hudi-examples/hudi-examples-spark/src/main/java/org/apache/hudi/examples/spark/HoodieSparkBootstrapExample.java b/hudi-examples/hudi-examples-spark/src/main/java/org/apache/hudi/examples/spark/HoodieSparkBootstrapExample.java index 31f93601f9275..33e80319b3670 100644 --- a/hudi-examples/hudi-examples-spark/src/main/java/org/apache/hudi/examples/spark/HoodieSparkBootstrapExample.java +++ b/hudi-examples/hudi-examples-spark/src/main/java/org/apache/hudi/examples/spark/HoodieSparkBootstrapExample.java @@ -38,13 +38,13 @@ public class HoodieSparkBootstrapExample { public static void main(String[] args) throws Exception { if (args.length < 5) { - System.err.println("Usage: HoodieSparkBootstrapExample "); + System.err.println("Usage: HoodieSparkBootstrapExample "); System.exit(1); } String recordKey = args[0]; String tableName = args[1]; String partitionPath = args[2]; - String preCombineField = args[3]; + String orderingFields = args[3]; String basePath = args[4]; SparkConf sparkConf = HoodieExampleSparkUtils.defaultSparkConf("hoodie-client-example"); @@ -61,7 +61,7 @@ public static void main(String[] args) throws Exception { df.write().format("hudi").option(HoodieWriteConfig.TBL_NAME.key(), tableName) .option(DataSourceWriteOptions.OPERATION().key(), DataSourceWriteOptions.BOOTSTRAP_OPERATION_OPT_VAL()) .option(DataSourceWriteOptions.RECORDKEY_FIELD().key(), recordKey) - .option(DataSourceWriteOptions.PRECOMBINE_FIELD().key(), preCombineField) + .option(HoodieTableConfig.ORDERING_FIELDS.key(), orderingFields) .option(HoodieTableConfig.BASE_FILE_FORMAT.key(), HoodieFileFormat.ORC.name()) .option(HoodieBootstrapConfig.BASE_PATH.key(), basePath) .option(HoodieWriteConfig.KEYGENERATOR_CLASS_NAME.key(), NonpartitionedKeyGenerator.class.getCanonicalName()) diff --git a/hudi-examples/hudi-examples-spark/src/main/scala/org/apache/hudi/examples/spark/HoodieDataSourceExample.scala b/hudi-examples/hudi-examples-spark/src/main/scala/org/apache/hudi/examples/spark/HoodieDataSourceExample.scala index a319bc1812288..55251b9e2a001 100644 --- a/hudi-examples/hudi-examples-spark/src/main/scala/org/apache/hudi/examples/spark/HoodieDataSourceExample.scala +++ b/hudi-examples/hudi-examples-spark/src/main/scala/org/apache/hudi/examples/spark/HoodieDataSourceExample.scala @@ -19,12 +19,12 @@ package org.apache.hudi.examples.spark import org.apache.hudi.DataSourceReadOptions.{END_COMMIT, QUERY_TYPE, QUERY_TYPE_INCREMENTAL_OPT_VAL, START_COMMIT} -import org.apache.hudi.DataSourceWriteOptions.{DELETE_OPERATION_OPT_VAL, DELETE_PARTITION_OPERATION_OPT_VAL, OPERATION, PARTITIONPATH_FIELD, PARTITIONS_TO_DELETE, PRECOMBINE_FIELD, RECORDKEY_FIELD} +import org.apache.hudi.DataSourceWriteOptions.{DELETE_OPERATION_OPT_VAL, DELETE_PARTITION_OPERATION_OPT_VAL, OPERATION, PARTITIONPATH_FIELD, PARTITIONS_TO_DELETE, RECORDKEY_FIELD} import org.apache.hudi.QuickstartUtils.getQuickstartWriteConfigs import org.apache.hudi.common.model.HoodieAvroPayload +import org.apache.hudi.common.table.HoodieTableConfig import org.apache.hudi.config.HoodieWriteConfig.TBL_NAME import org.apache.hudi.examples.common.{HoodieExampleDataGenerator, HoodieExampleSparkUtils} - import org.apache.spark.sql.SaveMode.{Append, Overwrite} import org.apache.spark.sql.SparkSession @@ -78,7 +78,7 @@ object HoodieDataSourceExample { val df = spark.read.json(spark.sparkContext.parallelize(inserts, 1)) df.write.format("hudi"). options(getQuickstartWriteConfigs). - option(PRECOMBINE_FIELD.key, "ts"). + option(HoodieTableConfig.ORDERING_FIELDS.key, "ts"). option(RECORDKEY_FIELD.key, "uuid"). option(PARTITIONPATH_FIELD.key, "partitionpath"). option(TBL_NAME.key, tableName). @@ -120,7 +120,7 @@ object HoodieDataSourceExample { val df = spark.read.json(spark.sparkContext.parallelize(updates, 1)) df.write.format("hudi"). options(getQuickstartWriteConfigs). - option(PRECOMBINE_FIELD.key, "ts"). + option(HoodieTableConfig.ORDERING_FIELDS.key, "ts"). option(RECORDKEY_FIELD.key, "uuid"). option(PARTITIONPATH_FIELD.key, "partitionpath"). option(TBL_NAME.key, tableName). @@ -139,7 +139,7 @@ object HoodieDataSourceExample { df.write.format("hudi"). options(getQuickstartWriteConfigs). - option(PRECOMBINE_FIELD.key, "ts"). + option(HoodieTableConfig.ORDERING_FIELDS.key, "ts"). option(RECORDKEY_FIELD.key, "uuid"). option(PARTITIONPATH_FIELD.key, "partitionpath"). option(TBL_NAME.key, tableName). @@ -155,7 +155,7 @@ object HoodieDataSourceExample { val df = spark.emptyDataFrame df.write.format("hudi"). options(getQuickstartWriteConfigs). - option(PRECOMBINE_FIELD.key, "ts"). + option(HoodieTableConfig.ORDERING_FIELDS.key, "ts"). option(RECORDKEY_FIELD.key, "uuid"). option(PARTITIONPATH_FIELD.key, "partitionpath"). option(TBL_NAME.key, tableName). diff --git a/hudi-examples/hudi-examples-spark/src/main/scala/org/apache/hudi/examples/spark/HoodieMorCompactionJob.scala b/hudi-examples/hudi-examples-spark/src/main/scala/org/apache/hudi/examples/spark/HoodieMorCompactionJob.scala index 0fbb2a007a2c1..cbf587c3391bd 100644 --- a/hudi-examples/hudi-examples-spark/src/main/scala/org/apache/hudi/examples/spark/HoodieMorCompactionJob.scala +++ b/hudi-examples/hudi-examples-spark/src/main/scala/org/apache/hudi/examples/spark/HoodieMorCompactionJob.scala @@ -19,11 +19,12 @@ package org.apache.hudi.examples.spark -import org.apache.hudi.DataSourceWriteOptions.{PARTITIONPATH_FIELD, PRECOMBINE_FIELD, RECORDKEY_FIELD, TABLE_TYPE} +import org.apache.hudi.DataSourceWriteOptions.{PARTITIONPATH_FIELD, RECORDKEY_FIELD, TABLE_TYPE} import org.apache.hudi.QuickstartUtils.getQuickstartWriteConfigs import org.apache.hudi.client.SparkRDDWriteClient import org.apache.hudi.client.common.HoodieSparkEngineContext import org.apache.hudi.common.model.{HoodieAvroPayload, HoodieRecordPayload, HoodieTableType} +import org.apache.hudi.common.table.HoodieTableConfig import org.apache.hudi.common.util.Option import org.apache.hudi.config.HoodieWriteConfig.TBL_NAME import org.apache.hudi.config.{HoodieCompactionConfig, HoodieWriteConfig} @@ -89,7 +90,7 @@ object HoodieMorCompactionJob { val df = spark.read.json(spark.sparkContext.parallelize(inserts, 1)) df.write.format("hudi"). options(getQuickstartWriteConfigs). - option(PRECOMBINE_FIELD.key, "ts"). + option(HoodieTableConfig.ORDERING_FIELDS.key, "ts"). option(RECORDKEY_FIELD.key, "uuid"). option(PARTITIONPATH_FIELD.key, "partitionpath"). option(TBL_NAME.key, tableName). @@ -105,7 +106,7 @@ object HoodieMorCompactionJob { val df = spark.read.json(spark.sparkContext.parallelize(updates, 1)) df.write.format("hudi"). options(getQuickstartWriteConfigs). - option(PRECOMBINE_FIELD.key, "ts"). + option(HoodieTableConfig.ORDERING_FIELDS.key, "ts"). option(RECORDKEY_FIELD.key, "uuid"). option(PARTITIONPATH_FIELD.key, "partitionpath"). option(TBL_NAME.key, tableName). diff --git a/hudi-examples/hudi-examples-spark/src/test/python/HoodiePySparkQuickstart.py b/hudi-examples/hudi-examples-spark/src/test/python/HoodiePySparkQuickstart.py index 9505e3217850c..1613097b01976 100644 --- a/hudi-examples/hudi-examples-spark/src/test/python/HoodiePySparkQuickstart.py +++ b/hudi-examples/hudi-examples-spark/src/test/python/HoodiePySparkQuickstart.py @@ -35,7 +35,7 @@ def __init__(self, spark: sql.SparkSession, tableName: str, basePath: str): 'hoodie.datasource.write.recordkey.field': 'uuid', 'hoodie.datasource.write.partitionpath.field': 'partitionpath', 'hoodie.datasource.write.operation': 'upsert', - 'hoodie.datasource.write.precombine.field': 'ts', + 'hoodie.table.ordering.fields': 'ts', 'hoodie.upsert.shuffle.parallelism': 2, 'hoodie.insert.shuffle.parallelism': 2 } @@ -162,7 +162,7 @@ def softDeletes(self): 'hoodie.datasource.write.recordkey.field': 'uuid', 'hoodie.datasource.write.partitionpath.field': 'partitionpath', 'hoodie.datasource.write.operation': 'upsert', - 'hoodie.datasource.write.precombine.field': 'ts', + 'hoodie.table.ordering.fields': 'ts', 'hoodie.upsert.shuffle.parallelism': 2, 'hoodie.insert.shuffle.parallelism': 2 } @@ -196,7 +196,7 @@ def hardDeletes(self): 'hoodie.datasource.write.recordkey.field': 'uuid', 'hoodie.datasource.write.partitionpath.field': 'partitionpath', 'hoodie.datasource.write.operation': 'delete', - 'hoodie.datasource.write.precombine.field': 'ts', + 'hoodie.table.ordering.fields': 'ts', 'hoodie.upsert.shuffle.parallelism': 2, 'hoodie.insert.shuffle.parallelism': 2 } @@ -223,7 +223,7 @@ def insertOverwrite(self): 'hoodie.datasource.write.recordkey.field': 'uuid', 'hoodie.datasource.write.partitionpath.field': 'partitionpath', 'hoodie.datasource.write.operation': 'insert_overwrite', - 'hoodie.datasource.write.precombine.field': 'ts', + 'hoodie.table.ordering.fields': 'ts', 'hoodie.upsert.shuffle.parallelism': 2, 'hoodie.insert.shuffle.parallelism': 2 } diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/configuration/FlinkOptions.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/configuration/FlinkOptions.java index a2a43566231cb..2cfdac757217a 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/configuration/FlinkOptions.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/configuration/FlinkOptions.java @@ -110,14 +110,15 @@ private FlinkOptions() { .withDescription("Type of table to write. COPY_ON_WRITE (or) MERGE_ON_READ"); public static final String NO_PRE_COMBINE = "no_precombine"; - public static final ConfigOption PRECOMBINE_FIELD = ConfigOptions - .key("precombine.field") + public static final ConfigOption ORDERING_FIELDS = ConfigOptions + .key("ordering.fields") .stringType() .defaultValue("ts") - .withFallbackKeys("write.precombine.field", HoodieWriteConfig.PRECOMBINE_FIELD_NAME.key()) - .withDescription("Comma separated list of fields used in preCombining before actual write. When two records have the same\n" - + "key value, we will pick the one with the largest value for the precombine field,\n" - + "determined by Object.compareTo(..). For multiple fields if first key comparison is same, second key comparison is made and so on"); + .withFallbackKeys("precombine.field", "write.precombine.field", HoodieWriteConfig.PRECOMBINE_FIELD_NAME.key()) + .withDescription("Comma separated list of fields used in records merging. When two records have the same\n" + + "key value, we will pick the one with the largest value for the ordering field,\n" + + "determined by Object.compareTo(..). For multiple fields if first key comparison is same, second key comparison is made and so on.\n" + + "Config precombine.field is now deprecated, please use ordering.fields instead."); @AdvancedConfig public static final ConfigOption PAYLOAD_CLASS_NAME = ConfigOptions 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 aa453fe54cdb0..370ec1aff52d8 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 @@ -152,12 +152,12 @@ public static boolean isDefaultHoodieRecordPayloadClazz(Configuration conf) { } /** - * Returns the preCombine field + * Returns the ordering fields as comma separated string * or null if the value is set as {@link FlinkOptions#NO_PRE_COMBINE}. */ - public static String getPreCombineField(Configuration conf) { - final String preCombineField = conf.get(FlinkOptions.PRECOMBINE_FIELD); - return preCombineField.equals(FlinkOptions.NO_PRE_COMBINE) ? null : preCombineField; + public static String getOrderingFieldsStr(Configuration conf) { + final String orderingFields = conf.get(FlinkOptions.ORDERING_FIELDS); + return orderingFields.equals(FlinkOptions.NO_PRE_COMBINE) ? null : orderingFields; } /** diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/HoodieFlinkCompactor.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/HoodieFlinkCompactor.java index b686017996547..669195e46a339 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/HoodieFlinkCompactor.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/HoodieFlinkCompactor.java @@ -175,7 +175,7 @@ public AsyncCompactionService(FlinkCompactionConfig cfg, Configuration conf) thr // set table schema CompactionUtil.setAvroSchema(conf, metaClient); - CompactionUtil.setPreCombineField(conf, metaClient); + CompactionUtil.setOrderingFields(conf, metaClient); // infer changelog mode CompactionUtil.inferChangelogMode(conf, metaClient); diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/streamer/FlinkStreamerConfig.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/streamer/FlinkStreamerConfig.java index fc7444e2c2415..278daacb081e3 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/streamer/FlinkStreamerConfig.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/streamer/FlinkStreamerConfig.java @@ -114,9 +114,10 @@ public class FlinkStreamerConfig extends Configuration { + "By default `SIMPLE`.") public String keygenType = KeyGeneratorType.SIMPLE.name(); - @Parameter(names = {"--source-ordering-field"}, description = "Field within source record to decide how" - + " to break ties between records with same key in input data. Default: 'ts' holding unix timestamp of record.") - public String sourceOrderingField = "ts"; + @Parameter(names = {"--source-ordering-fields", "--source-ordering-field"}, description = "Field within source record to decide how" + + " to break ties between records with same key in input data. Default: 'ts' holding unix timestamp of record. " + + "Option --source-ordering-field is deprecated, please use --source-ordering-fields instead.") + public String sourceOrderingFields = "ts"; @Parameter(names = {"--write-table-version"}, description = "Version of table written") public Integer writeTableVersion = HoodieTableVersion.current().versionCode(); @@ -425,7 +426,7 @@ public static org.apache.flink.configuration.Configuration toFlinkConfig(FlinkSt conf.set(FlinkOptions.TABLE_TYPE, config.tableType.toUpperCase()); conf.set(FlinkOptions.INSERT_CLUSTER, config.insertCluster); conf.set(FlinkOptions.OPERATION, config.operation.value()); - conf.set(FlinkOptions.PRECOMBINE_FIELD, config.sourceOrderingField); + conf.set(FlinkOptions.ORDERING_FIELDS, config.sourceOrderingFields); conf.set(FlinkOptions.WRITE_TABLE_VERSION, config.writeTableVersion); conf.set(FlinkOptions.PAYLOAD_CLASS_NAME, config.payloadClassName); conf.set(FlinkOptions.RECORD_MERGER_IMPLS, config.recordMergerImpls); diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/HoodieTableFactory.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/HoodieTableFactory.java index cd10f8edb29a8..2ed7b80005e7c 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/HoodieTableFactory.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/HoodieTableFactory.java @@ -117,9 +117,9 @@ private void setupTableOptions(String basePath, Configuration conf) { && !conf.contains(FlinkOptions.RECORD_KEY_FIELD)) { conf.set(FlinkOptions.RECORD_KEY_FIELD, tableConfig.getString(HoodieTableConfig.RECORDKEY_FIELDS)); } - if (tableConfig.contains(HoodieTableConfig.PRECOMBINE_FIELDS) - && !conf.contains(FlinkOptions.PRECOMBINE_FIELD)) { - conf.set(FlinkOptions.PRECOMBINE_FIELD, tableConfig.getString(HoodieTableConfig.PRECOMBINE_FIELDS)); + if (tableConfig.contains(HoodieTableConfig.ORDERING_FIELDS) + && !conf.contains(FlinkOptions.ORDERING_FIELDS)) { + conf.set(FlinkOptions.ORDERING_FIELDS, tableConfig.getString(HoodieTableConfig.ORDERING_FIELDS)); } if (tableConfig.contains(HoodieTableConfig.HIVE_STYLE_PARTITIONING_ENABLE) && !conf.contains(FlinkOptions.HIVE_STYLE_PARTITIONING)) { @@ -177,7 +177,7 @@ private void sanityCheck(Configuration conf, ResolvedSchema schema) { if (!OptionsResolver.isAppendMode(conf)) { checkRecordKey(conf, schema); } - StreamerUtil.checkPreCombineKey(conf, schema.getColumnNames()); + StreamerUtil.checkOrderingFields(conf, schema.getColumnNames()); } /** diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/catalog/HoodieCatalog.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/catalog/HoodieCatalog.java index 641c74f20fbfc..e846ecfe8fb5b 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/catalog/HoodieCatalog.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/catalog/HoodieCatalog.java @@ -337,7 +337,7 @@ public void createTable(ObjectPath tablePath, CatalogBaseTable catalogTable, boo } // check preCombine - StreamerUtil.checkPreCombineKey(conf, resolvedSchema.getColumnNames()); + StreamerUtil.checkOrderingFields(conf, resolvedSchema.getColumnNames()); if (resolvedTable.isPartitioned()) { final String partitions = String.join(",", resolvedTable.getPartitionKeys()); diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/catalog/HoodieHiveCatalog.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/catalog/HoodieHiveCatalog.java index a30f3891f1a74..6569037e3fcca 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/catalog/HoodieHiveCatalog.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/catalog/HoodieHiveCatalog.java @@ -528,7 +528,7 @@ private HoodieTableMetaClient initTableIfNotExists(ObjectPath tablePath, Catalog List fields = new ArrayList<>(); catalogTable.getUnresolvedSchema().getColumns().forEach(column -> fields.add(column.getName())); - StreamerUtil.checkPreCombineKey(flinkConf, fields); + StreamerUtil.checkOrderingFields(flinkConf, fields); try { return StreamerUtil.initTableIfNotExists(flinkConf, hiveConf); diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/catalog/TableOptionProperties.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/catalog/TableOptionProperties.java index de6b6bc4111f4..a1d93972807d5 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/catalog/TableOptionProperties.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/catalog/TableOptionProperties.java @@ -90,13 +90,13 @@ public class TableOptionProperties { KEY_MAPPING.put("type", FlinkOptions.TABLE_TYPE.key()); KEY_MAPPING.put("primaryKey", FlinkOptions.RECORD_KEY_FIELD.key()); - KEY_MAPPING.put("preCombineField", FlinkOptions.PRECOMBINE_FIELD.key()); + KEY_MAPPING.put("orderingFields", FlinkOptions.ORDERING_FIELDS.key()); KEY_MAPPING.put("payloadClass", FlinkOptions.PAYLOAD_CLASS_NAME.key()); KEY_MAPPING.put(SPARK_SOURCE_PROVIDER, CONNECTOR.key()); KEY_MAPPING.put(FlinkOptions.KEYGEN_CLASS_NAME.key(), FlinkOptions.KEYGEN_CLASS_NAME.key()); KEY_MAPPING.put(FlinkOptions.TABLE_TYPE.key(), "type"); KEY_MAPPING.put(FlinkOptions.RECORD_KEY_FIELD.key(), "primaryKey"); - KEY_MAPPING.put(FlinkOptions.PRECOMBINE_FIELD.key(), "preCombineField"); + KEY_MAPPING.put(FlinkOptions.ORDERING_FIELDS.key(), "orderingFields"); KEY_MAPPING.put(FlinkOptions.PAYLOAD_CLASS_NAME.key(), "payloadClass"); } diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/CompactionUtil.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/CompactionUtil.java index 42df2ced0cdd4..4c482835d2035 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/CompactionUtil.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/CompactionUtil.java @@ -100,10 +100,10 @@ public static void setAvroSchema(HoodieWriteConfig writeConfig, HoodieTableMetaC * * @param conf The configuration */ - public static void setPreCombineField(Configuration conf, HoodieTableMetaClient metaClient) { - String preCombineField = metaClient.getTableConfig().getPreCombineFieldsStr().orElse(null); - if (preCombineField != null) { - conf.set(FlinkOptions.PRECOMBINE_FIELD, preCombineField); + public static void setOrderingFields(Configuration conf, HoodieTableMetaClient metaClient) { + String orderingFields = metaClient.getTableConfig().getOrderingFieldsStr().orElse(null); + if (orderingFields != null) { + conf.set(FlinkOptions.ORDERING_FIELDS, orderingFields); } } diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/StreamerUtil.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/StreamerUtil.java index f772dbb9c30f2..78d9df9221756 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/StreamerUtil.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/StreamerUtil.java @@ -175,8 +175,8 @@ public static DFSPropertiesConfiguration readConfig(org.apache.hadoop.conf.Confi public static HoodiePayloadConfig getPayloadConfig(Configuration conf) { return HoodiePayloadConfig.newBuilder() .withPayloadClass(conf.get(FlinkOptions.PAYLOAD_CLASS_NAME)) - .withPayloadOrderingFields(conf.get(FlinkOptions.PRECOMBINE_FIELD)) - .withPayloadEventTimeField(conf.get(FlinkOptions.PRECOMBINE_FIELD)) + .withPayloadOrderingFields(conf.get(FlinkOptions.ORDERING_FIELDS)) + .withPayloadEventTimeField(conf.get(FlinkOptions.ORDERING_FIELDS)) .build(); } @@ -296,7 +296,7 @@ public static HoodieTableMetaClient initTableIfNotExists( .setPayloadClassName(getPayloadClass(conf)) .setDatabaseName(conf.get(FlinkOptions.DATABASE_NAME)) .setRecordKeyFields(conf.getString(FlinkOptions.RECORD_KEY_FIELD.key(), null)) - .setPreCombineFields(OptionsResolver.getPreCombineField(conf)) + .setOrderingFields(OptionsResolver.getOrderingFieldsStr(conf)) .setArchiveLogFolder(TIMELINE_HISTORY_PATH.defaultValue()) .setPartitionFields(conf.getString(FlinkOptions.PARTITION_PATH_FIELD.key(), null)) .setKeyGeneratorClassProp( @@ -404,7 +404,7 @@ public static void addFlinkCheckpointIdIntoMetaData( */ public static Triple inferMergingBehavior(Configuration conf) { return HoodieTableConfig.inferMergingConfigsForWrites( - getMergeMode(conf), getPayloadClass(conf), getMergeStrategyId(conf), OptionsResolver.getPreCombineField(conf), HoodieTableVersion.EIGHT); + getMergeMode(conf), getPayloadClass(conf), getMergeStrategyId(conf), OptionsResolver.getOrderingFieldsStr(conf), HoodieTableVersion.EIGHT); } /** @@ -656,18 +656,18 @@ public static boolean isWriteCommit(HoodieTableType tableType, HoodieInstant ins /** * Validate pre_combine key. */ - public static void checkPreCombineKey(Configuration conf, List fields) { - String preCombineField = conf.get(FlinkOptions.PRECOMBINE_FIELD); - if (!fields.contains(preCombineField)) { + public static void checkOrderingFields(Configuration conf, List fields) { + String orderingFields = conf.get(FlinkOptions.ORDERING_FIELDS); + if (!fields.contains(orderingFields)) { if (OptionsResolver.isDefaultHoodieRecordPayloadClazz(conf)) { - throw new HoodieValidationException("Option '" + FlinkOptions.PRECOMBINE_FIELD.key() + throw new HoodieValidationException("Option '" + FlinkOptions.ORDERING_FIELDS.key() + "' is required for payload class: " + DefaultHoodieRecordPayload.class.getName()); } - if (preCombineField.equals(FlinkOptions.PRECOMBINE_FIELD.defaultValue())) { - conf.set(FlinkOptions.PRECOMBINE_FIELD, FlinkOptions.NO_PRE_COMBINE); - } else if (!preCombineField.equals(FlinkOptions.NO_PRE_COMBINE)) { - throw new HoodieValidationException("Field " + preCombineField + " does not exist in the table schema." - + "Please check '" + FlinkOptions.PRECOMBINE_FIELD.key() + "' option."); + if (orderingFields.equals(FlinkOptions.ORDERING_FIELDS.defaultValue())) { + conf.set(FlinkOptions.ORDERING_FIELDS, FlinkOptions.NO_PRE_COMBINE); + } else if (!orderingFields.equals(FlinkOptions.NO_PRE_COMBINE)) { + throw new HoodieValidationException("Field " + orderingFields + " does not exist in the table schema." + + "Please check '" + FlinkOptions.ORDERING_FIELDS.key() + "' option."); } } } diff --git a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/TestWriteMergeOnRead.java b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/TestWriteMergeOnRead.java index fc7dd4a3d2da9..6deef95901700 100644 --- a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/TestWriteMergeOnRead.java +++ b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/TestWriteMergeOnRead.java @@ -99,7 +99,7 @@ public void testEventTimeAvroPayloadMergeRead() throws Exception { conf.set(FlinkOptions.CHANGELOG_ENABLED, false); conf.set(FlinkOptions.COMPACTION_DELTA_COMMITS, 2); conf.set(FlinkOptions.PRE_COMBINE, true); - conf.set(FlinkOptions.PRECOMBINE_FIELD, "ts"); + conf.set(FlinkOptions.ORDERING_FIELDS, "ts"); conf.set(FlinkOptions.PAYLOAD_CLASS_NAME, EventTimeAvroPayload.class.getName()); HashMap mergedExpected = new HashMap<>(EXPECTED1); mergedExpected.put("par1", "[id1,par1,id1,Danny,22,4,par1, id2,par1,id2,Stephen,33,2,par1]"); diff --git a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/table/ITTestHoodieDataSource.java b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/table/ITTestHoodieDataSource.java index 5462ce3892ef8..0a317c5f40b66 100644 --- a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/table/ITTestHoodieDataSource.java +++ b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/table/ITTestHoodieDataSource.java @@ -689,7 +689,7 @@ void testWriteAndReadWithProctimeSequenceWithTsColumnExisting(HoodieTableType ta .option(FlinkOptions.PATH, tempFile.getAbsolutePath()) .option(FlinkOptions.TABLE_TYPE, tableType) .option(FlinkOptions.HIVE_STYLE_PARTITIONING, hiveStylePartitioning) - .option(FlinkOptions.PRECOMBINE_FIELD, FlinkOptions.NO_PRE_COMBINE) + .option(FlinkOptions.ORDERING_FIELDS, FlinkOptions.NO_PRE_COMBINE) .end(); tableEnv.executeSql(hoodieTableDDL); @@ -1541,7 +1541,7 @@ void testWriteReadDecimals(String operation) { .field("f3 decimal(38, 18)") .option(FlinkOptions.PATH, tempFile.getAbsolutePath()) .option(FlinkOptions.OPERATION, operation) - .option(FlinkOptions.PRECOMBINE_FIELD, "f1") + .option(FlinkOptions.ORDERING_FIELDS, "f1") .pkField("f0") .noPartition() .end(); @@ -2098,7 +2098,7 @@ void testBuiltinFunctionWithHMSCatalog() { .pkField("f_int") .partitionField("f_par") .option(FlinkOptions.RECORD_KEY_FIELD, "f_int") - .option(FlinkOptions.PRECOMBINE_FIELD, "f_date") + .option(FlinkOptions.ORDERING_FIELDS, "f_date") .end(); tableEnv.executeSql(hoodieTableDDL); @@ -2126,7 +2126,7 @@ void testWriteReadWithComputedColumns() { .field("f2 bigint") .field("f3 as f0 + f2") .option(FlinkOptions.PATH, tempFile.getAbsolutePath()) - .option(FlinkOptions.PRECOMBINE_FIELD, "f1") + .option(FlinkOptions.ORDERING_FIELDS, "f1") .pkField("f0") .noPartition() .end(); @@ -2154,7 +2154,7 @@ void testWriteReadWithComputedColumnsInTheMiddle() { .field("f2 as f0 + f1") .field("f3 varchar(10)") .option(FlinkOptions.PATH, tempFile.getAbsolutePath()) - .option(FlinkOptions.PRECOMBINE_FIELD, "f1") + .option(FlinkOptions.ORDERING_FIELDS, "f1") .pkField("f0") .noPartition() .end(); @@ -2184,7 +2184,7 @@ void testWriteReadWithLocalTimestamp(HoodieTableType tableType) { .field("f2 TIMESTAMP_LTZ(3)") .field("f4 TIMESTAMP_LTZ(6)") .option(FlinkOptions.PATH, tempFile.getAbsolutePath()) - .option(FlinkOptions.PRECOMBINE_FIELD, "f1") + .option(FlinkOptions.ORDERING_FIELDS, "f1") .option(FlinkOptions.TABLE_TYPE, tableType) .pkField("f0") .noPartition() @@ -2215,7 +2215,7 @@ void testWriteReadWithTimestampWithoutTZ(HoodieTableType tableType, boolean read .field("f2 TIMESTAMP(3)") .field("f3 TIMESTAMP(6)") .option(FlinkOptions.PATH, tempFile.getAbsolutePath()) - .option(FlinkOptions.PRECOMBINE_FIELD, "f1") + .option(FlinkOptions.ORDERING_FIELDS, "f1") .option(FlinkOptions.TABLE_TYPE, tableType) .option(FlinkOptions.WRITE_UTC_TIMEZONE, false) .option(FlinkOptions.READ_UTC_TIMEZONE, readUtcTimezone) diff --git a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/table/TestHoodieFileGroupReaderOnFlink.java b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/table/TestHoodieFileGroupReaderOnFlink.java index dc66a0e6a74c0..5c169f62cb9f6 100644 --- a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/table/TestHoodieFileGroupReaderOnFlink.java +++ b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/table/TestHoodieFileGroupReaderOnFlink.java @@ -33,6 +33,7 @@ import org.apache.hudi.common.testutils.HoodieTestDataGenerator; import org.apache.hudi.common.testutils.HoodieTestUtils; import org.apache.hudi.common.util.CollectionUtils; +import org.apache.hudi.common.util.ConfigUtils; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.OrderingValues; import org.apache.hudi.common.util.collection.ClosableIterator; @@ -140,7 +141,7 @@ protected void readWithFileGroupReader( @Override public void commitToTable(List recordList, String operation, boolean firstCommit, Map writeConfigs, String schemaStr) { writeConfigs.forEach((key, value) -> conf.setString(key, value)); - conf.set(FlinkOptions.PRECOMBINE_FIELD, writeConfigs.get("hoodie.datasource.write.precombine.field")); + conf.set(FlinkOptions.ORDERING_FIELDS, ConfigUtils.getOrderingFieldsStrDuringWrite(writeConfigs)); conf.set(FlinkOptions.OPERATION, operation); Schema localSchema = getRecordAvroSchema(schemaStr); conf.set(FlinkOptions.SOURCE_AVRO_SCHEMA, localSchema.toString()); diff --git a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/table/TestHoodieTableFactory.java b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/table/TestHoodieTableFactory.java index 2a4eb3d61d6a7..a4a69706e4fe2 100644 --- a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/table/TestHoodieTableFactory.java +++ b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/table/TestHoodieTableFactory.java @@ -112,7 +112,7 @@ void testRequiredOptions() { assertDoesNotThrow(() -> new HoodieTableFactory().createDynamicTableSink(sourceContext11)); //miss the pre combine key will be ok HoodieTableSink tableSink11 = (HoodieTableSink) new HoodieTableFactory().createDynamicTableSink(sourceContext11); - assertThat(tableSink11.getConf().get(FlinkOptions.PRECOMBINE_FIELD), is(FlinkOptions.NO_PRE_COMBINE)); + assertThat(tableSink11.getConf().get(FlinkOptions.ORDERING_FIELDS), is(FlinkOptions.NO_PRE_COMBINE)); this.conf.set(FlinkOptions.OPERATION, FlinkOptions.OPERATION.defaultValue()); // a non-exists precombine key will throw exception @@ -121,12 +121,12 @@ void testRequiredOptions() { .field("f1", DataTypes.VARCHAR(20)) .field("f2", DataTypes.TIMESTAMP(3)) .build(); - this.conf.set(FlinkOptions.PRECOMBINE_FIELD, "non_exist_field"); + this.conf.set(FlinkOptions.ORDERING_FIELDS, "non_exist_field"); final MockContext sourceContext2 = MockContext.getInstance(this.conf, schema2, "f2"); // createDynamicTableSource doesn't call sanity check, will not throw exception assertDoesNotThrow(() -> new HoodieTableFactory().createDynamicTableSource(sourceContext2)); assertThrows(HoodieValidationException.class, () -> new HoodieTableFactory().createDynamicTableSink(sourceContext2)); - this.conf.set(FlinkOptions.PRECOMBINE_FIELD, FlinkOptions.PRECOMBINE_FIELD.defaultValue()); + this.conf.set(FlinkOptions.ORDERING_FIELDS, FlinkOptions.ORDERING_FIELDS.defaultValue()); // given the pk but miss the pre combine key will be ok ResolvedSchema schema3 = SchemaBuilder.instance() @@ -139,7 +139,7 @@ void testRequiredOptions() { HoodieTableSource tableSource = (HoodieTableSource) new HoodieTableFactory().createDynamicTableSource(sourceContext3); HoodieTableSink tableSink = (HoodieTableSink) new HoodieTableFactory().createDynamicTableSink(sourceContext3); // the precombine field is overwritten - assertThat(tableSink.getConf().get(FlinkOptions.PRECOMBINE_FIELD), is(FlinkOptions.NO_PRE_COMBINE)); + assertThat(tableSink.getConf().get(FlinkOptions.ORDERING_FIELDS), is(FlinkOptions.NO_PRE_COMBINE)); // precombine field not specified, use the default payload clazz assertThat(tableSource.getConf().get(FlinkOptions.PAYLOAD_CLASS_NAME), is(FlinkOptions.PAYLOAD_CLASS_NAME.defaultValue())); assertThat(tableSink.getConf().get(FlinkOptions.PAYLOAD_CLASS_NAME), is(FlinkOptions.PAYLOAD_CLASS_NAME.defaultValue())); @@ -147,7 +147,7 @@ void testRequiredOptions() { // append mode given the pk but miss the pre combine key will be ok this.conf.set(FlinkOptions.OPERATION, "insert"); HoodieTableSink tableSink3 = (HoodieTableSink) new HoodieTableFactory().createDynamicTableSink(sourceContext3); - assertThat(tableSink3.getConf().get(FlinkOptions.PRECOMBINE_FIELD), is(FlinkOptions.NO_PRE_COMBINE)); + assertThat(tableSink3.getConf().get(FlinkOptions.ORDERING_FIELDS), is(FlinkOptions.NO_PRE_COMBINE)); this.conf.set(FlinkOptions.OPERATION, FlinkOptions.OPERATION.defaultValue()); this.conf.set(FlinkOptions.PAYLOAD_CLASS_NAME, DefaultHoodieRecordPayload.class.getName()); @@ -185,7 +185,7 @@ void testRequiredOptions() { .field("ts", DataTypes.TIMESTAMP(3)) .primaryKey("f0") .build(); - this.conf.set(FlinkOptions.PRECOMBINE_FIELD, FlinkOptions.NO_PRE_COMBINE); + this.conf.set(FlinkOptions.ORDERING_FIELDS, FlinkOptions.NO_PRE_COMBINE); final MockContext sourceContext6 = MockContext.getInstance(this.conf, schema5, "f2"); assertDoesNotThrow(() -> new HoodieTableFactory().createDynamicTableSource(sourceContext6)); @@ -266,7 +266,7 @@ void testSupplementTableConfig() throws Exception { tableConf.set(FlinkOptions.PATH, tablePath); tableConf.set(FlinkOptions.TABLE_NAME, "t2"); tableConf.set(FlinkOptions.RECORD_KEY_FIELD, "f0,f1"); - tableConf.set(FlinkOptions.PRECOMBINE_FIELD, "f2"); + tableConf.set(FlinkOptions.ORDERING_FIELDS, "f2"); tableConf.set(FlinkOptions.TABLE_TYPE, FlinkOptions.TABLE_TYPE_MERGE_ON_READ); tableConf.set(FlinkOptions.PAYLOAD_CLASS_NAME, "my_payload"); tableConf.set(FlinkOptions.PARTITION_PATH_FIELD, "partition"); @@ -292,9 +292,9 @@ void testSupplementTableConfig() throws Exception { assertThat("pk not provided, fallback to table config", sink1.getConf().get(FlinkOptions.RECORD_KEY_FIELD), is("f0,f1")); assertThat("pre-combine key not provided, fallback to table config", - source1.getConf().get(FlinkOptions.PRECOMBINE_FIELD), is("f2")); + source1.getConf().get(FlinkOptions.ORDERING_FIELDS), is("f2")); assertThat("pre-combine key not provided, fallback to table config", - sink1.getConf().get(FlinkOptions.PRECOMBINE_FIELD), is("f2")); + sink1.getConf().get(FlinkOptions.ORDERING_FIELDS), is("f2")); assertThat("table type not provided, fallback to table config", source1.getConf().get(FlinkOptions.TABLE_TYPE), is(FlinkOptions.TABLE_TYPE_MERGE_ON_READ)); assertThat("table type not provided, fallback to table config", @@ -307,7 +307,7 @@ void testSupplementTableConfig() throws Exception { // write config always has higher priority // set up a different primary key and pre_combine key with table config options writeConf.set(FlinkOptions.RECORD_KEY_FIELD, "f0"); - writeConf.set(FlinkOptions.PRECOMBINE_FIELD, "f1"); + writeConf.set(FlinkOptions.ORDERING_FIELDS, "f1"); final MockContext sourceContext2 = MockContext.getInstance(writeConf, schema1, "f2"); HoodieTableSource source2 = (HoodieTableSource) new HoodieTableFactory().createDynamicTableSource(sourceContext2); @@ -317,12 +317,12 @@ void testSupplementTableConfig() throws Exception { assertThat("choose pk from write config", sink2.getConf().get(FlinkOptions.RECORD_KEY_FIELD), is("f0")); assertThat("choose preCombine key from write config", - source2.getConf().get(FlinkOptions.PRECOMBINE_FIELD), is("f1")); + source2.getConf().get(FlinkOptions.ORDERING_FIELDS), is("f1")); assertThat("choose preCombine pk from write config", - sink2.getConf().get(FlinkOptions.PRECOMBINE_FIELD), is("f1")); + sink2.getConf().get(FlinkOptions.ORDERING_FIELDS), is("f1")); writeConf.removeConfig(FlinkOptions.RECORD_KEY_FIELD); - writeConf.removeConfig(FlinkOptions.PRECOMBINE_FIELD); + writeConf.removeConfig(FlinkOptions.ORDERING_FIELDS); // pk defined in table config but missing in schema will throw ResolvedSchema schema2 = SchemaBuilder.instance() diff --git a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/table/TestHoodieTableSource.java b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/table/TestHoodieTableSource.java index ae6320606b2b7..f12fc0b832f75 100644 --- a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/table/TestHoodieTableSource.java +++ b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/table/TestHoodieTableSource.java @@ -270,7 +270,7 @@ void testBucketPruningSpecialKeyDataType(boolean logicalTimestamp) throws Except final String f1 = "f_timestamp"; conf1.set(FlinkOptions.INDEX_TYPE, "BUCKET"); conf1.set(FlinkOptions.RECORD_KEY_FIELD, f1); - conf1.set(FlinkOptions.PRECOMBINE_FIELD, f1); + conf1.set(FlinkOptions.ORDERING_FIELDS, f1); conf1.removeConfig(FlinkOptions.PARTITION_PATH_FIELD); conf1.setString(KEYGENERATOR_CONSISTENT_LOGICAL_TIMESTAMP_ENABLED.key(), logicalTimestamp + ""); int numBuckets = (int)FlinkOptions.BUCKET_INDEX_NUM_BUCKETS.defaultValue(); @@ -292,7 +292,7 @@ void testBucketPruningSpecialKeyDataType(boolean logicalTimestamp) throws Except String tablePath2 = new Path(tempFile.getAbsolutePath(), "tbl2").toString(); conf2.set(FlinkOptions.PATH, tablePath2); conf2.set(FlinkOptions.RECORD_KEY_FIELD, f2); - conf2.set(FlinkOptions.PRECOMBINE_FIELD, f2); + conf2.set(FlinkOptions.ORDERING_FIELDS, f2); TestData.writeDataAsBatch(TestData.DATA_SET_INSERT_HOODIE_KEY_SPECIAL_DATA_TYPE, conf2); HoodieTableSource tableSource2 = createHoodieTableSource(conf2); tableSource2.applyFilters(Collections.singletonList( @@ -308,7 +308,7 @@ void testBucketPruningSpecialKeyDataType(boolean logicalTimestamp) throws Except String tablePath3 = new Path(tempFile.getAbsolutePath(), "tbl3").toString(); conf3.set(FlinkOptions.PATH, tablePath3); conf3.set(FlinkOptions.RECORD_KEY_FIELD, f3); - conf3.set(FlinkOptions.PRECOMBINE_FIELD, f3); + conf3.set(FlinkOptions.ORDERING_FIELDS, f3); TestData.writeDataAsBatch(TestData.DATA_SET_INSERT_HOODIE_KEY_SPECIAL_DATA_TYPE, conf3); HoodieTableSource tableSource3 = createHoodieTableSource(conf3); tableSource3.applyFilters(Collections.singletonList( diff --git a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/table/catalog/TestHoodieCatalog.java b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/table/catalog/TestHoodieCatalog.java index 3d202be87c4d0..61f7710b87751 100644 --- a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/table/catalog/TestHoodieCatalog.java +++ b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/table/catalog/TestHoodieCatalog.java @@ -371,7 +371,7 @@ void testCreateTableWithPartitionBucketIndex() throws TableAlreadyExistException } @Test - void testCreateTableWithoutPreCombineKey() { + void testCreateTableWithoutOrderingFields() { Map options = getDefaultCatalogOption(); options.put(FlinkOptions.PAYLOAD_CLASS_NAME.key(), DefaultHoodieRecordPayload.class.getName()); catalog = new HoodieCatalog("hudi", Configuration.fromMap(options)); @@ -383,7 +383,7 @@ void testCreateTableWithoutPreCombineKey() { + "org.apache.hudi.common.model.DefaultHoodieRecordPayload"); Map options2 = getDefaultCatalogOption(); - options2.put(FlinkOptions.PRECOMBINE_FIELD.key(), "not_exists"); + options2.put(FlinkOptions.ORDERING_FIELDS.key(), "not_exists"); catalog = new HoodieCatalog("hudi", Configuration.fromMap(options2)); catalog.open(); ObjectPath tablePath2 = new ObjectPath(TEST_DEFAULT_DATABASE, "tb2"); diff --git a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/table/catalog/TestHoodieHiveCatalog.java b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/table/catalog/TestHoodieHiveCatalog.java index a25cf3a84bb14..69e78920c2852 100644 --- a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/table/catalog/TestHoodieHiveCatalog.java +++ b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/table/catalog/TestHoodieHiveCatalog.java @@ -76,7 +76,7 @@ import java.util.stream.Collectors; import static org.apache.flink.table.factories.FactoryUtil.CONNECTOR; -import static org.apache.hudi.configuration.FlinkOptions.PRECOMBINE_FIELD; +import static org.apache.hudi.configuration.FlinkOptions.ORDERING_FIELDS; import static org.apache.hudi.keygen.constant.KeyGeneratorOptions.RECORDKEY_FIELD_NAME; import static org.apache.hudi.table.catalog.HoodieCatalogTestUtils.createStorageConf; import static org.hamcrest.CoreMatchers.containsString; @@ -208,7 +208,7 @@ public void testCreateAndGetHoodieTable(HoodieTableType tableType) throws Except assertEquals("hudi", table1.getOptions().get(CONNECTOR.key())); assertEquals(tableType.toString(), table1.getOptions().get(FlinkOptions.TABLE_TYPE.key())); assertEquals("uuid", table1.getOptions().get(FlinkOptions.RECORD_KEY_FIELD.key())); - assertNull(table1.getOptions().get(PRECOMBINE_FIELD.key()), "preCombine key is not declared"); + assertNull(table1.getOptions().get(ORDERING_FIELDS.key()), "preCombine key is not declared"); String tableSchema = table1.getUnresolvedSchema().getColumns().stream() .map(Schema.UnresolvedColumn::toString) .collect(Collectors.joining(",")); @@ -322,7 +322,7 @@ private void assertResult(Map params, String index) { } @Test - void testCreateTableWithoutPreCombineKey() throws TableAlreadyExistException, DatabaseNotExistException, IOException, TableNotExistException { + void testCreateTableWithoutOrderingFields() throws TableAlreadyExistException, DatabaseNotExistException, IOException, TableNotExistException { String db = "default"; hoodieCatalog = HoodieCatalogTestUtils.createHiveCatalog(); hoodieCatalog.open(); @@ -331,12 +331,12 @@ void testCreateTableWithoutPreCombineKey() throws TableAlreadyExistException, Da options.put(FactoryUtil.CONNECTOR.key(), "hudi"); TypedProperties props = createTableAndReturnTableProperties(options, new ObjectPath(db, "tmptb1")); - assertFalse(props.containsKey("hoodie.table.precombine.field")); + assertFalse(props.containsKey(HoodieTableConfig.ORDERING_FIELDS.key())); - options.put(PRECOMBINE_FIELD.key(), "ts_3"); + options.put(ORDERING_FIELDS.key(), "ts_3"); props = createTableAndReturnTableProperties(options, new ObjectPath(db, "tmptb2")); - assertTrue(props.containsKey("hoodie.table.precombine.field")); - assertEquals("ts_3", props.get("hoodie.table.precombine.field")); + assertTrue(props.containsKey(HoodieTableConfig.ORDERING_FIELDS.key())); + assertEquals("ts_3", props.get(HoodieTableConfig.ORDERING_FIELDS.key())); } private TypedProperties createTableAndReturnTableProperties(Map options, ObjectPath tablePath) diff --git a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/utils/TestStreamerUtil.java b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/utils/TestStreamerUtil.java index d3d9b46708baf..1635278dc08d3 100644 --- a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/utils/TestStreamerUtil.java +++ b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/utils/TestStreamerUtil.java @@ -77,7 +77,7 @@ void testInitTableIfNotExists() throws IOException { Configuration conf = TestConfigurations.getDefaultConf(tempFile.getAbsolutePath()); // Test for partitioned table. - conf.set(FlinkOptions.PRECOMBINE_FIELD, "ts"); + conf.set(FlinkOptions.ORDERING_FIELDS, "ts"); conf.set(FlinkOptions.PARTITION_PATH_FIELD, "p0,p1"); StreamerUtil.initTableIfNotExists(conf); @@ -86,7 +86,7 @@ void testInitTableIfNotExists() throws IOException { assertTrue(metaClient1.getTableConfig().getPartitionFields().isPresent(), "Missing partition columns in the hoodie.properties."); assertArrayEquals(metaClient1.getTableConfig().getPartitionFields().get(), new String[] {"p0", "p1"}); - assertEquals(metaClient1.getTableConfig().getPreCombineFieldsStr().get(), "ts"); + assertEquals(metaClient1.getTableConfig().getOrderingFieldsStr().get(), "ts"); assertEquals(metaClient1.getTableConfig().getKeyGeneratorClassName(), SimpleAvroKeyGenerator.class.getName()); assertEquals(HoodieTableVersion.current(), metaClient1.getTableConfig().getTableVersion()); diff --git a/hudi-hadoop-common/src/test/java/org/apache/hudi/common/table/TestHoodieTableConfig.java b/hudi-hadoop-common/src/test/java/org/apache/hudi/common/table/TestHoodieTableConfig.java index a25856ca3bd8a..35bae6f78058e 100644 --- a/hudi-hadoop-common/src/test/java/org/apache/hudi/common/table/TestHoodieTableConfig.java +++ b/hudi-hadoop-common/src/test/java/org/apache/hudi/common/table/TestHoodieTableConfig.java @@ -134,7 +134,7 @@ void testCreate() throws IOException { void testUpdate() throws IOException { Properties updatedProps = new Properties(); updatedProps.setProperty(HoodieTableConfig.NAME.key(), "test-table2"); - updatedProps.setProperty(HoodieTableConfig.PRECOMBINE_FIELDS.key(), "new_field"); + updatedProps.setProperty(HoodieTableConfig.ORDERING_FIELDS.key(), "new_field"); HoodieTableConfig.update(storage, metaPath, updatedProps); assertTrue(storage.exists(cfgPath)); @@ -142,8 +142,8 @@ void testUpdate() throws IOException { HoodieTableConfig config = new HoodieTableConfig(storage, metaPath); assertEquals(8, config.getProps().size()); assertEquals("test-table2", config.getTableName()); - assertEquals(Collections.singletonList("new_field"), config.getPreCombineFields()); - assertEquals(Option.of("new_field"), config.getPreCombineFieldsStr()); + assertEquals(Collections.singletonList("new_field"), config.getOrderingFields()); + assertEquals(Option.of("new_field"), config.getOrderingFieldsStr()); } @Test @@ -164,7 +164,7 @@ void testDelete() throws IOException { void testUpdateAndDelete() throws IOException { Properties updatedProps = new Properties(); updatedProps.setProperty(HoodieTableConfig.NAME.key(), "test-table2"); - updatedProps.setProperty(HoodieTableConfig.PRECOMBINE_FIELDS.key(), "new_field"); + updatedProps.setProperty(HoodieTableConfig.ORDERING_FIELDS.key(), "new_field"); updatedProps.setProperty(HoodieTableConfig.PARTITION_FIELDS.key(), "partition_path"); HoodieTableConfig.update(storage, metaPath, updatedProps); @@ -173,12 +173,12 @@ void testUpdateAndDelete() throws IOException { HoodieTableConfig config = new HoodieTableConfig(storage, metaPath); assertEquals(9, config.getProps().size()); assertEquals("test-table2", config.getTableName()); - assertEquals(Collections.singletonList("new_field"), config.getPreCombineFields()); + assertEquals(Collections.singletonList("new_field"), config.getOrderingFields()); assertEquals("partition_path", config.getPartitionFields().get()[0]); // update 1 property and delete 1 property updatedProps = new Properties(); - updatedProps.setProperty(HoodieTableConfig.PRECOMBINE_FIELDS.key(), "new_field2"); + updatedProps.setProperty(HoodieTableConfig.ORDERING_FIELDS.key(), "new_field2"); Set propsToDelete = new HashSet<>(); propsToDelete.add(HoodieTableConfig.PARTITION_FIELDS.key()); // delete a non existant property as well @@ -187,16 +187,16 @@ void testUpdateAndDelete() throws IOException { config = new HoodieTableConfig(storage, metaPath); assertEquals(8, config.getProps().size()); assertEquals("test-table2", config.getTableName()); - assertEquals(Collections.singletonList("new_field2"), config.getPreCombineFields()); + assertEquals(Collections.singletonList("new_field2"), config.getOrderingFields()); assertFalse(config.getPartitionFields().isPresent()); // just delete 1 property w/o updating anything. updatedProps = new Properties(); - HoodieTableConfig.updateAndDeleteProps(storage, metaPath, updatedProps, Collections.singleton(HoodieTableConfig.PRECOMBINE_FIELDS.key())); + HoodieTableConfig.updateAndDeleteProps(storage, metaPath, updatedProps, Collections.singleton(HoodieTableConfig.ORDERING_FIELDS.key())); config = new HoodieTableConfig(storage, metaPath); assertEquals(7, config.getProps().size()); assertEquals("test-table2", config.getTableName()); - assertTrue(config.getPreCombineFields().isEmpty()); + assertTrue(config.getOrderingFields().isEmpty()); assertFalse(config.getPartitionFields().isPresent()); } @@ -272,7 +272,7 @@ void testConcurrentlyUpdate() throws ExecutionException, InterruptedException { for (int i = 0; i < 100; i++) { Properties updatedProps = new Properties(); updatedProps.setProperty(HoodieTableConfig.NAME.key(), "test-table" + i); - updatedProps.setProperty(HoodieTableConfig.PRECOMBINE_FIELDS.key(), "new_field" + i); + updatedProps.setProperty(HoodieTableConfig.ORDERING_FIELDS.key(), "new_field" + i); HoodieTableConfig.update(storage, metaPath, updatedProps); } }); @@ -351,7 +351,7 @@ void testDropInvalidConfigs() { @Test void testDefinedTableConfigs() { List> configProperties = HoodieTableConfig.definedTableConfigs(); - assertEquals(41, configProperties.size()); + assertEquals(42, configProperties.size()); configProperties.forEach(c -> { assertNotNull(c); assertFalse(c.doc().isEmpty()); diff --git a/hudi-hadoop-common/src/test/java/org/apache/hudi/common/table/TestHoodieTableMetaClient.java b/hudi-hadoop-common/src/test/java/org/apache/hudi/common/table/TestHoodieTableMetaClient.java index dc7c022472c95..fd85ea2581788 100644 --- a/hudi-hadoop-common/src/test/java/org/apache/hudi/common/table/TestHoodieTableMetaClient.java +++ b/hudi-hadoop-common/src/test/java/org/apache/hudi/common/table/TestHoodieTableMetaClient.java @@ -204,7 +204,7 @@ void testCreateMetaClientFromProperties() throws IOException { Properties props = new Properties(); props.setProperty(HoodieTableConfig.NAME.key(), "test-table"); props.setProperty(HoodieTableConfig.TYPE.key(), HoodieTableType.COPY_ON_WRITE.name()); - props.setProperty(HoodieTableConfig.PRECOMBINE_FIELDS.key(), "timestamp"); + props.setProperty(HoodieTableConfig.ORDERING_FIELDS.key(), "timestamp"); HoodieTableMetaClient metaClient1 = HoodieTableMetaClient.newTableBuilder() .fromProperties(props) @@ -218,7 +218,7 @@ void testCreateMetaClientFromProperties() throws IOException { // test table name and type and precombine field also match assertEquals(metaClient1.getTableConfig().getTableName(), metaClient2.getTableConfig().getTableName()); assertEquals(metaClient1.getTableConfig().getTableType(), metaClient2.getTableConfig().getTableType()); - assertEquals(metaClient1.getTableConfig().getPreCombineFields(), metaClient2.getTableConfig().getPreCombineFields()); + assertEquals(metaClient1.getTableConfig().getOrderingFields(), metaClient2.getTableConfig().getOrderingFields()); // default table version should be current version assertEquals(HoodieTableVersion.current(), metaClient2.getTableConfig().getTableVersion()); } diff --git a/hudi-hadoop-common/src/test/java/org/apache/hudi/common/table/read/TestCustomRecordMerger.java b/hudi-hadoop-common/src/test/java/org/apache/hudi/common/table/read/TestCustomRecordMerger.java index 02eee5387621f..9793b1f77914d 100644 --- a/hudi-hadoop-common/src/test/java/org/apache/hudi/common/table/read/TestCustomRecordMerger.java +++ b/hudi-hadoop-common/src/test/java/org/apache/hudi/common/table/read/TestCustomRecordMerger.java @@ -53,7 +53,7 @@ import static org.apache.hudi.common.config.HoodieReaderConfig.RECORD_MERGE_IMPL_CLASSES_WRITE_CONFIG_KEY; import static org.apache.hudi.common.model.HoodieRecord.HoodieRecordType.AVRO; -import static org.apache.hudi.common.table.HoodieTableConfig.PRECOMBINE_FIELDS; +import static org.apache.hudi.common.table.HoodieTableConfig.ORDERING_FIELDS; import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.AVRO_SCHEMA; import static org.apache.hudi.common.testutils.reader.DataGenerationPlan.OperationType.DELETE; import static org.apache.hudi.common.testutils.reader.DataGenerationPlan.OperationType.INSERT; @@ -68,7 +68,7 @@ protected Properties getMetaProps() { Properties metaProps = super.getMetaProps(); metaProps.setProperty(HoodieTableConfig.RECORD_MERGE_MODE.key(), RecordMergeMode.CUSTOM.name()); metaProps.setProperty(HoodieTableConfig.RECORD_MERGE_STRATEGY_ID.key(), CustomAvroMerger.KEEP_CERTAIN_TIMESTAMP_VALUE_ONLY); - metaProps.setProperty(PRECOMBINE_FIELDS.key(), "timestamp"); + metaProps.setProperty(ORDERING_FIELDS.key(), "timestamp"); return metaProps; } diff --git a/hudi-hadoop-common/src/test/java/org/apache/hudi/common/table/read/TestEventTimeMerging.java b/hudi-hadoop-common/src/test/java/org/apache/hudi/common/table/read/TestEventTimeMerging.java index debbb57954398..0610dd4d8f837 100644 --- a/hudi-hadoop-common/src/test/java/org/apache/hudi/common/table/read/TestEventTimeMerging.java +++ b/hudi-hadoop-common/src/test/java/org/apache/hudi/common/table/read/TestEventTimeMerging.java @@ -44,7 +44,7 @@ import java.util.Properties; import java.util.stream.Stream; -import static org.apache.hudi.common.table.HoodieTableConfig.PRECOMBINE_FIELDS; +import static org.apache.hudi.common.table.HoodieTableConfig.ORDERING_FIELDS; import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.AVRO_SCHEMA; import static org.apache.hudi.common.testutils.reader.DataGenerationPlan.OperationType.DELETE; import static org.apache.hudi.common.testutils.reader.DataGenerationPlan.OperationType.INSERT; @@ -58,7 +58,7 @@ public class TestEventTimeMerging extends HoodieFileGroupReaderTestHarness { protected Properties getMetaProps() { Properties metaProps = super.getMetaProps(); metaProps.setProperty(HoodieTableConfig.RECORD_MERGE_MODE.key(), RecordMergeMode.EVENT_TIME_ORDERING.name()); - metaProps.setProperty(PRECOMBINE_FIELDS.key(), "timestamp"); + metaProps.setProperty(ORDERING_FIELDS.key(), "timestamp"); return metaProps; } diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/AbstractRealtimeRecordReader.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/AbstractRealtimeRecordReader.java index fc4795d987639..f09898cb25914 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/AbstractRealtimeRecordReader.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/AbstractRealtimeRecordReader.java @@ -90,8 +90,8 @@ public AbstractRealtimeRecordReader(RealtimeSplit split, JobConf job) { metaClient = HoodieTableMetaClient.builder() .setConf(HadoopFSUtils.getStorageConfWithCopy(jobConf)).setBasePath(split.getBasePath()).build(); payloadProps.putAll(metaClient.getTableConfig().getProps(true)); - if (metaClient.getTableConfig().getPreCombineFieldsStr().isPresent()) { - this.payloadProps.setProperty(HoodiePayloadProps.PAYLOAD_ORDERING_FIELD_PROP_KEY, metaClient.getTableConfig().getPreCombineFieldsStr().orElse(null)); + if (metaClient.getTableConfig().getOrderingFieldsStr().isPresent()) { + this.payloadProps.setProperty(HoodiePayloadProps.PAYLOAD_ORDERING_FIELD_PROP_KEY, metaClient.getTableConfig().getOrderingFieldsStr().orElse(null)); } this.usesCustomPayload = usesCustomPayload(metaClient); LOG.info("usesCustomPayload ==> " + this.usesCustomPayload); diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieParquetRealtimeInputFormat.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieParquetRealtimeInputFormat.java index cdeacaada6afe..a911bbab788b6 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieParquetRealtimeInputFormat.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieParquetRealtimeInputFormat.java @@ -76,7 +76,7 @@ public RecordReader getRecordReader(final InputSpli return super.getRecordReader(realtimeSplit, jobConf, reporter); } - // add preCombineKey + // add orderingFields HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder().setConf(getStorageConf(jobConf)).setBasePath(realtimeSplit.getBasePath()).build(); HoodieTableConfig tableConfig = metaClient.getTableConfig(); addProjectionToJobConf(realtimeSplit, jobConf, tableConfig); @@ -114,7 +114,7 @@ void addProjectionToJobConf(final RealtimeSplit realtimeSplit, final JobConf job List fieldsToAdd = new ArrayList<>(); if (!realtimeSplit.getDeltaLogPaths().isEmpty()) { HoodieRealtimeInputFormatUtils.addVirtualKeysProjection(jobConf, realtimeSplit.getVirtualKeyInfo()); - fieldsToAdd.addAll(tableConfig.getPreCombineFields()); + fieldsToAdd.addAll(tableConfig.getOrderingFields()); } Option partitions = tableConfig.getPartitionFields(); diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/SparkDataSourceContinuousIngestTool.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/SparkDataSourceContinuousIngestTool.java index a98415fa372cb..45e482cd9705a 100644 --- a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/SparkDataSourceContinuousIngestTool.java +++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/SparkDataSourceContinuousIngestTool.java @@ -61,7 +61,7 @@ * hoodie.datasource.write.recordkey.field=VendorID * hoodie.datasource.write.partitionpath.field=date_col * hoodie.datasource.write.operation=upsert - * hoodie.datasource.write.precombine.field=tpep_pickup_datetime + * hoodie.table.ordering.fields=tpep_pickup_datetime * hoodie.metadata.enable=false * hoodie.table.name=hudi_tbl */ diff --git a/hudi-integ-test/src/main/scala/org/apache/hudi/integ/testsuite/dag/nodes/SparkDeletePartitionNode.scala b/hudi-integ-test/src/main/scala/org/apache/hudi/integ/testsuite/dag/nodes/SparkDeletePartitionNode.scala index 8f586aed0605b..b2327717b8725 100644 --- a/hudi-integ-test/src/main/scala/org/apache/hudi/integ/testsuite/dag/nodes/SparkDeletePartitionNode.scala +++ b/hudi-integ-test/src/main/scala/org/apache/hudi/integ/testsuite/dag/nodes/SparkDeletePartitionNode.scala @@ -55,7 +55,7 @@ class SparkDeletePartitionNode(dagNodeConfig: Config) extends DagNode[RDD[WriteS context.getWriterContext.getSparkSession.emptyDataFrame.write.format("hudi") .options(DataSourceWriteOptions.mayBeDerivePartitionPath(context.getWriterContext.getProps.asScala.toMap)) - .option(DataSourceWriteOptions.PRECOMBINE_FIELD.key(), SchemaUtils.SOURCE_ORDERING_FIELD) + .option(DataSourceWriteOptions.ORDERING_FIELDS.key(), SchemaUtils.SOURCE_ORDERING_FIELD) .option(DataSourceWriteOptions.TABLE_NAME.key, context.getHoodieTestSuiteWriter.getCfg.targetTableName) .option(DataSourceWriteOptions.TABLE_TYPE.key, context.getHoodieTestSuiteWriter.getCfg.tableType) .option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.DELETE_PARTITION_OPERATION_OPT_VAL) diff --git a/hudi-integ-test/src/main/scala/org/apache/hudi/integ/testsuite/dag/nodes/SparkInsertNode.scala b/hudi-integ-test/src/main/scala/org/apache/hudi/integ/testsuite/dag/nodes/SparkInsertNode.scala index b4d028c79c85a..e5ada9363928f 100644 --- a/hudi-integ-test/src/main/scala/org/apache/hudi/integ/testsuite/dag/nodes/SparkInsertNode.scala +++ b/hudi-integ-test/src/main/scala/org/apache/hudi/integ/testsuite/dag/nodes/SparkInsertNode.scala @@ -20,6 +20,7 @@ package org.apache.hudi.integ.testsuite.dag.nodes import org.apache.hudi.{AvroConversionUtils, DataSourceWriteOptions, HoodieSparkUtils} import org.apache.hudi.client.WriteStatus +import org.apache.hudi.common.table.HoodieTableConfig import org.apache.hudi.common.util.collection.Pair import org.apache.hudi.config.{HoodieIndexConfig, HoodieWriteConfig} import org.apache.hudi.integ.testsuite.configuration.DeltaConfig.Config @@ -68,7 +69,7 @@ class SparkInsertNode(dagNodeConfig: Config) extends DagNode[RDD[WriteStatus]] { inputDF.write.format("hudi") .options(DataSourceWriteOptions.mayBeDerivePartitionPath(context.getWriterContext.getProps.asScala.toMap)) - .option(DataSourceWriteOptions.PRECOMBINE_FIELD.key(), "test_suite_source_ordering_field") + .option(HoodieTableConfig.ORDERING_FIELDS.key(), "test_suite_source_ordering_field") .option(DataSourceWriteOptions.TABLE_NAME.key, context.getHoodieTestSuiteWriter.getCfg.targetTableName) .option(DataSourceWriteOptions.TABLE_TYPE.key, context.getHoodieTestSuiteWriter.getCfg.tableType) .option(HoodieIndexConfig.INDEX_TYPE.key, context.getHoodieTestSuiteWriter.getCfg.indexType) diff --git a/hudi-integ-test/src/main/scala/org/apache/hudi/integ/testsuite/dag/nodes/SparkUpsertNode.scala b/hudi-integ-test/src/main/scala/org/apache/hudi/integ/testsuite/dag/nodes/SparkUpsertNode.scala index 9e85fc38d8e3c..229d39c4ed083 100644 --- a/hudi-integ-test/src/main/scala/org/apache/hudi/integ/testsuite/dag/nodes/SparkUpsertNode.scala +++ b/hudi-integ-test/src/main/scala/org/apache/hudi/integ/testsuite/dag/nodes/SparkUpsertNode.scala @@ -19,6 +19,7 @@ package org.apache.hudi.integ.testsuite.dag.nodes import org.apache.hudi.DataSourceWriteOptions +import org.apache.hudi.common.table.HoodieTableConfig import org.apache.hudi.common.util.collection.Pair import org.apache.hudi.integ.testsuite.configuration.DeltaConfig.Config import org.apache.hudi.integ.testsuite.dag.ExecutionContext @@ -68,7 +69,7 @@ class SparkUpsertNode(dagNodeConfig: Config) extends SparkInsertNode(dagNodeConf inputDF.write.format("hudi") .options(DataSourceWriteOptions.translateSqlOptions(context.getWriterContext.getProps.asScala.toMap)) - .option(DataSourceWriteOptions.PRECOMBINE_FIELD.key(), "test_suite_source_ordering_field") + .option(HoodieTableConfig.ORDERING_FIELDS.key(), "test_suite_source_ordering_field") .option(DataSourceWriteOptions.TABLE_NAME.key, context.getHoodieTestSuiteWriter.getCfg.targetTableName) .option(DataSourceWriteOptions.TABLE_TYPE.key, context.getHoodieTestSuiteWriter.getCfg.tableType) .option(DataSourceWriteOptions.OPERATION.key, getOperation()) diff --git a/hudi-integ-test/src/main/scala/org/apache/hudi/integ/testsuite/streaming/StructuredStreamingSinkTestWriter.scala b/hudi-integ-test/src/main/scala/org/apache/hudi/integ/testsuite/streaming/StructuredStreamingSinkTestWriter.scala index 60b6ed5ea4d1a..63e22b541c0d1 100644 --- a/hudi-integ-test/src/main/scala/org/apache/hudi/integ/testsuite/streaming/StructuredStreamingSinkTestWriter.scala +++ b/hudi-integ-test/src/main/scala/org/apache/hudi/integ/testsuite/streaming/StructuredStreamingSinkTestWriter.scala @@ -89,7 +89,7 @@ object StructuredStreamingSinkTestWriter { val writer = parquetdf.writeStream.format("org.apache.hudi"). option(TABLE_TYPE.key, tableType). - option(PRECOMBINE_FIELD.key, preCombineField). + option(ORDERING_FIELDS.key, preCombineField). option(RECORDKEY_FIELD.key, recordKeyField). option(PARTITIONPATH_FIELD.key, partitionPathField). option(FAIL_ON_TIMELINE_ARCHIVING_ENABLE.key, false). diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/DataSourceUtils.java b/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/DataSourceUtils.java index fd65233f1e36d..2ecfae013c20a 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/DataSourceUtils.java +++ b/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/DataSourceUtils.java @@ -34,6 +34,7 @@ import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.WriteOperationType; 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.ReflectionUtils; import org.apache.hudi.common.util.StringUtils; @@ -205,8 +206,7 @@ public static HoodieWriteConfig createHoodieConfig(String schemaStr, String base // to realize the SQL functionality, so the write config needs to be fetched first. .withPayloadClass(parameters.getOrDefault(DataSourceWriteOptions.PAYLOAD_CLASS_NAME().key(), parameters.getOrDefault(HoodieTableConfig.PAYLOAD_CLASS_NAME.key(), HoodieTableConfig.DEFAULT_PAYLOAD_CLASS_NAME))) - .withPayloadOrderingFields(parameters.getOrDefault(DataSourceWriteOptions.PRECOMBINE_FIELD().key(), - parameters.get(HoodieTableConfig.PRECOMBINE_FIELDS))) + .withPayloadOrderingFields(ConfigUtils.getOrderingFieldsStrDuringWrite(parameters)) .build()) // override above with Hoodie configs specified as options. .withProps(parameters).build(); diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/commit/BaseDatasetBulkInsertCommitActionExecutor.java b/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/commit/BaseDatasetBulkInsertCommitActionExecutor.java index 16041fdae5a52..7612ad95fe3c9 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/commit/BaseDatasetBulkInsertCommitActionExecutor.java +++ b/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/commit/BaseDatasetBulkInsertCommitActionExecutor.java @@ -105,7 +105,7 @@ public final HoodieWriteResult execute(Dataset records, boolean isTablePart preExecute(); BulkInsertPartitioner> bulkInsertPartitionerRows = getPartitioner(populateMetaFields, isTablePartitioned); - Dataset hoodieDF = HoodieDatasetBulkInsertHelper.prepareForBulkInsert(records, writeConfig, bulkInsertPartitionerRows, instantTime); + Dataset hoodieDF = HoodieDatasetBulkInsertHelper.prepareForBulkInsert(records, writeConfig, table.getMetaClient().getTableConfig(), bulkInsertPartitionerRows, instantTime); HoodieWriteMetadata> result = buildHoodieWriteMetadata(doExecute(hoodieDF, bulkInsertPartitionerRows.arePartitionRecordsSorted())); afterExecute(result); diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/AutoRecordKeyGenerationUtils.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/AutoRecordKeyGenerationUtils.scala index a240a9c3e484b..5d8bc1ee405ee 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/AutoRecordKeyGenerationUtils.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/AutoRecordKeyGenerationUtils.scala @@ -19,9 +19,10 @@ package org.apache.hudi -import org.apache.hudi.DataSourceWriteOptions.{INSERT_DROP_DUPS, PRECOMBINE_FIELD} +import org.apache.hudi.DataSourceWriteOptions.INSERT_DROP_DUPS import org.apache.hudi.common.config.HoodieConfig import org.apache.hudi.common.table.HoodieTableConfig +import org.apache.hudi.common.util.{ConfigUtils, StringUtils} import org.apache.hudi.config.HoodieWriteConfig import org.apache.hudi.exception.HoodieKeyGeneratorException import org.apache.hudi.keygen.constant.KeyGeneratorOptions @@ -46,8 +47,9 @@ object AutoRecordKeyGenerationUtils { if (!parameters.getOrElse(HoodieTableConfig.POPULATE_META_FIELDS.key(), HoodieTableConfig.POPULATE_META_FIELDS.defaultValue().toString).toBoolean) { throw new HoodieKeyGeneratorException("Disabling " + HoodieTableConfig.POPULATE_META_FIELDS.key() + " is not supported with auto generation of record keys") } - if (hoodieConfig.contains(PRECOMBINE_FIELD.key())) { - log.warn("Precombine field " + hoodieConfig.getString(PRECOMBINE_FIELD.key()) + " will be ignored with auto record key generation enabled") + val orderingFieldsStr = ConfigUtils.getOrderingFieldsStrDuringWrite(hoodieConfig.getProps) + if (StringUtils.nonEmpty(orderingFieldsStr)) { + log.warn("Ordering field " + orderingFieldsStr + " will be ignored with auto record key generation enabled") } } } diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DataSourceOptions.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DataSourceOptions.scala index 6e2177f539fb8..65334d3303db5 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DataSourceOptions.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DataSourceOptions.scala @@ -85,6 +85,7 @@ object DataSourceReadOptions { .markAdvanced() .withDocumentation("Comma separated list of file paths to read within a Hudi table.") + @Deprecated val READ_PRE_COMBINE_FIELD = HoodieWriteConfig.PRECOMBINE_FIELD_NAME val ENABLE_HOODIE_FILE_INDEX: ConfigProperty[Boolean] = ConfigProperty @@ -408,11 +409,11 @@ object DataSourceWriteOptions { .withDocumentation("Table name for the datasource write. Also used to register the table into meta stores.") /** - * Field used in preCombining before actual write. When two records have the same - * key value, we will pick the one with the largest value for the precombine field, - * determined by Object.compareTo(..) + * Field used in records merging comparison. When two records have the same + * key value, we will pick the one with the largest value for the ordering fields, + * determined by Object.compareTo(..). */ - val PRECOMBINE_FIELD = HoodieWriteConfig.PRECOMBINE_FIELD_NAME + val ORDERING_FIELDS = HoodieWriteConfig.PRECOMBINE_FIELD_NAME /** * Payload class used. Override this, if you like to roll your own merge logic, when upserting/inserting. @@ -861,7 +862,7 @@ object DataSourceWriteOptions { /** @deprecated Use {@link TABLE_NAME} and its methods instead */ @Deprecated val TABLE_NAME_OPT_KEY = TABLE_NAME.key() - /** @deprecated Use {@link PRECOMBINE_FIELD} and its methods instead */ + /** @deprecated Use {@link ORDERING_FIELDS} and its methods instead */ @Deprecated val PRECOMBINE_FIELD_OPT_KEY = HoodieWriteConfig.PRECOMBINE_FIELD_NAME.key() @@ -1070,7 +1071,8 @@ object DataSourceOptionsHelper { /** * Returns optional list of precombine fields from the provided parameteres. */ - def getPreCombineFields(params: Map[String, String]): Option[java.util.List[String]] = params.get(DataSourceWriteOptions.PRECOMBINE_FIELD.key) match { + @deprecated("Use ordering field key in table config", "1.1.0") + def getPreCombineFields(params: Map[String, String]): Option[java.util.List[String]] = params.get(DataSourceWriteOptions.ORDERING_FIELDS.key) match { // NOTE: This is required to compensate for cases when empty string is used to stub // property value to avoid it being set with the default value // TODO(HUDI-3456) cleanup diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DefaultSource.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DefaultSource.scala index 9269315de982d..54ca53794e346 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DefaultSource.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DefaultSource.scala @@ -135,8 +135,8 @@ class DefaultSource extends RelationProvider .setBasePath(tablePath).build() // Add preCombineField to options for buildReaderWithPartitionValues properly - val options = if (metaClient.getTableConfig.getPreCombineFieldsStr.isPresent) { - parameters ++ Map(HoodieTableConfig.PRECOMBINE_FIELDS.key -> metaClient.getTableConfig.getPreCombineFieldsStr.orElse(null)) + val options = if (metaClient.getTableConfig.getOrderingFieldsStr.isPresent) { + parameters ++ Map(HoodieTableConfig.ORDERING_FIELDS.key -> metaClient.getTableConfig.getOrderingFieldsStr.orElse(null)) } else { parameters } diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieBaseRelation.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieBaseRelation.scala index 54a3577f7f6c8..a4ad93c02fee2 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieBaseRelation.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieBaseRelation.scala @@ -81,7 +81,7 @@ case class HoodieTableSchema(structTypeSchema: StructType, avroSchemaStr: String case class HoodieTableState(tablePath: String, latestCommitTimestamp: Option[String], recordKeyField: String, - preCombineFields: List[String], + orderingFields: List[String], usesVirtualKeys: Boolean, recordPayloadClassName: String, metadataConfig: HoodieMetadataConfig, @@ -135,14 +135,14 @@ abstract class HoodieBaseRelation(val sqlContext: SQLContext, keyFields.head } - protected lazy val preCombineFields: List[String] = { - val tablePrecombineFields = tableConfig.getPreCombineFields - if (tablePrecombineFields.isEmpty) { + protected lazy val orderingFields: List[String] = { + val tableOrderingFields = tableConfig.getOrderingFields + if (tableOrderingFields.isEmpty) { DataSourceOptionsHelper.getPreCombineFields(optParams) .orElse(java.util.Collections.emptyList[String]) .asScala.toList } else { - tablePrecombineFields.asScala.toList + tableOrderingFields.asScala.toList } } @@ -268,7 +268,7 @@ abstract class HoodieBaseRelation(val sqlContext: SQLContext, HoodieTableState(tablePath = basePath.toString, latestCommitTimestamp = queryTimestamp, recordKeyField = recordKeyField, - preCombineFields = preCombineFields, + orderingFields = orderingFields, usesVirtualKeys = !tableConfig.populateMetaFields(), recordPayloadClassName = tableConfig.getPayloadClass, metadataConfig = fileIndex.metadataConfig, diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieBootstrapMORRelation.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieBootstrapMORRelation.scala index 1bf99898f3efb..3b5e51da33aa4 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieBootstrapMORRelation.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieBootstrapMORRelation.scala @@ -64,7 +64,7 @@ case class HoodieBootstrapMORRelation(override val sqlContext: SQLContext, override type Relation = HoodieBootstrapMORRelation protected lazy val mandatoryFieldsForMerging: Seq[String] = - Seq(recordKeyField) ++ preCombineFields + Seq(recordKeyField) ++ orderingFields override lazy val mandatoryFields: Seq[String] = mandatoryFieldsForMerging diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieCreateRecordUtils.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieCreateRecordUtils.scala index 1a39c8e083279..9ef9d7e14e7f8 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieCreateRecordUtils.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieCreateRecordUtils.scala @@ -18,12 +18,12 @@ package org.apache.hudi -import org.apache.hudi.DataSourceWriteOptions.{INSERT_DROP_DUPS, PRECOMBINE_FIELD} +import org.apache.hudi.DataSourceWriteOptions.{INSERT_DROP_DUPS} import org.apache.hudi.avro.{AvroSchemaCache, HoodieAvroUtils} import org.apache.hudi.common.config.TypedProperties import org.apache.hudi.common.fs.FSUtils import org.apache.hudi.common.model._ -import org.apache.hudi.common.util.{ConfigUtils, OrderingValues, StringUtils} +import org.apache.hudi.common.util.OrderingValues import org.apache.hudi.config.HoodieWriteConfig import org.apache.hudi.keygen.{BaseKeyGenerator, KeyGenUtils, SparkKeyGeneratorInterface} import org.apache.hudi.keygen.constant.KeyGeneratorOptions @@ -58,7 +58,8 @@ object HoodieCreateRecordUtils { instantTime: String, preppedSparkSqlWrites: Boolean, preppedSparkSqlMergeInto: Boolean, - preppedWriteOperation: Boolean) + preppedWriteOperation: Boolean, + orderingFields: java.util.List[String]) def createHoodieRecordRdd(args: createHoodieRecordRddArgs) = { val df = args.df @@ -73,6 +74,7 @@ object HoodieCreateRecordUtils { val preppedSparkSqlWrites = args.preppedSparkSqlWrites val preppedSparkSqlMergeInto = args.preppedSparkSqlMergeInto val preppedWriteOperation = args.preppedWriteOperation + val orderingFields = args.orderingFields val shouldDropPartitionColumns = config.getBoolean(DataSourceWriteOptions.DROP_PARTITION_COLUMNS) val recordType = config.getRecordMerger.getRecordType @@ -125,7 +127,6 @@ object HoodieCreateRecordUtils { val consistentLogicalTimestampEnabled = parameters.getOrElse( DataSourceWriteOptions.KEYGENERATOR_CONSISTENT_LOGICAL_TIMESTAMP_ENABLED.key(), DataSourceWriteOptions.KEYGENERATOR_CONSISTENT_LOGICAL_TIMESTAMP_ENABLED.defaultValue()).toBoolean - val precombineFields = ConfigUtils.getOrderingFields(config.getProps) // handle dropping partition columns it.map { avroRec => @@ -143,9 +144,9 @@ object HoodieCreateRecordUtils { avroRecWithoutMeta } - val hoodieRecord = if (shouldCombine && precombineFields != null && precombineFields.nonEmpty) { + val hoodieRecord = if (shouldCombine && !orderingFields.isEmpty) { val orderingVal = OrderingValues.create( - precombineFields, + orderingFields, JFunction.toJavaFunction[String, Comparable[_]]( field => HoodieAvroUtils.getNestedFieldVal(avroRec, field, false, consistentLogicalTimestampEnabled).asInstanceOf[Comparable[_]])) diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieFileIndex.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieFileIndex.scala index 135ade4bf4cc6..46a75b9ac958f 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieFileIndex.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieFileIndex.scala @@ -18,7 +18,7 @@ package org.apache.hudi import org.apache.hudi.BaseHoodieTableFileIndex.PartitionPath -import org.apache.hudi.DataSourceWriteOptions.{PARTITIONPATH_FIELD, PRECOMBINE_FIELD, RECORDKEY_FIELD} +import org.apache.hudi.DataSourceWriteOptions.{PARTITIONPATH_FIELD, RECORDKEY_FIELD} import org.apache.hudi.HoodieFileIndex.{collectReferencedColumns, convertFilterForTimestampKeyGenerator, getConfigProperties, DataSkippingFailureMode} import org.apache.hudi.HoodieSparkConfUtils.getConfigValue import org.apache.hudi.common.config.{HoodieMetadataConfig, HoodieStorageConfig, TypedProperties} @@ -529,7 +529,6 @@ object HoodieFileIndex extends Logging { if (tableConfig != null) { properties.setProperty(RECORDKEY_FIELD.key, tableConfig.getRecordKeyFields.orElse(Array.empty).mkString(",")) - properties.setProperty(PRECOMBINE_FIELD.key, tableConfig.getPreCombineFieldsStr.orElse("")) properties.setProperty(PARTITIONPATH_FIELD.key, HoodieTableConfig.getPartitionFieldPropForKeyGenerator(tableConfig).orElse("")) // for simple bucket index, we need to set the INDEX_TYPE, BUCKET_INDEX_HASH_FIELD, BUCKET_INDEX_NUM_BUCKETS diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieHadoopFsRelationFactory.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieHadoopFsRelationFactory.scala index 0a9bef1ab30c1..9221e4f0ff78c 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieHadoopFsRelationFactory.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieHadoopFsRelationFactory.scala @@ -79,7 +79,7 @@ abstract class HoodieBaseHadoopFsRelationFactory(val sqlContext: SQLContext, private lazy val partitionColumns: Array[String] = tableConfig.getPartitionFields.orElse(Array.empty) // very much not recommended to use a partition column as the precombine - private lazy val partitionColumnsHasPrecombine = preCombineFields.nonEmpty && partitionColumns.exists(col => preCombineFields.contains(col)) + private lazy val partitionColumnsHasPrecombine = orderingFields.nonEmpty && partitionColumns.exists(col => orderingFields.contains(col)) private lazy val keygenTypeHasVariablePartitionCols = isTimestampKeygen || isCustomKeygen @@ -110,19 +110,19 @@ abstract class HoodieBaseHadoopFsRelationFactory(val sqlContext: SQLContext, if (shouldExtractPartitionValuesFromPartitionPath) { Seq.empty } else if (partitionColumnsHasPrecombine) { - logWarning(s"Not recommended for field '${preCombineFields}' to be both precombine and partition") + logWarning(s"Not recommended for field '${orderingFields}' to be both precombine and partition") if (keygenTypeHasVariablePartitionCols) { // still need to read any timestamp/custom keygen timestamp columns - if (variableTimestampKeygenPartitionCols.exists(col => preCombineFields.contains(col))) { + if (variableTimestampKeygenPartitionCols.exists(col => orderingFields.contains(col))) { // precombine is already included in the list variableTimestampKeygenPartitionCols } else { // precombine is not included in the list so we append it - variableTimestampKeygenPartitionCols ++ preCombineFields + variableTimestampKeygenPartitionCols ++ orderingFields } } else { // not timestamp/custom keygen so just need to read precombine - preCombineFields + orderingFields } } else if (keygenTypeHasVariablePartitionCols) { variableTimestampKeygenPartitionCols @@ -185,8 +185,8 @@ abstract class HoodieBaseHadoopFsRelationFactory(val sqlContext: SQLContext, }) } - protected lazy val preCombineFields: List[String] = { - val tablePrecombineFields = tableConfig.getPreCombineFields + protected lazy val orderingFields: List[String] = { + val tablePrecombineFields = tableConfig.getOrderingFields if (tablePrecombineFields.isEmpty) { DataSourceOptionsHelper.getPreCombineFields(optParams) .orElse(java.util.Collections.emptyList[String]) @@ -393,7 +393,7 @@ abstract class HoodieBaseCopyOnWriteIncrementalHadoopFsRelationFactory(override extends HoodieBaseHadoopFsRelationFactory(sqlContext, metaClient, options, schemaSpec, isBootstrap) { override protected def getMandatoryFields(): Seq[String] = Seq(HoodieRecord.RECORD_KEY_METADATA_FIELD, HoodieRecord.COMMIT_TIME_METADATA_FIELD) ++ - preCombineFields ++ partitionColumnsToRead + orderingFields ++ partitionColumnsToRead override protected def isMOR: Boolean = false diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala index f2d938ceeaedd..f36cd4b1437c7 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala @@ -40,7 +40,7 @@ import org.apache.hudi.common.model.HoodieTableType.{COPY_ON_WRITE, MERGE_ON_REA import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient, HoodieTableVersion, TableSchemaResolver} import org.apache.hudi.common.table.log.block.HoodieLogBlock.HoodieLogBlockType import org.apache.hudi.common.table.timeline.HoodieInstantTimeGenerator -import org.apache.hudi.common.util.{CommitUtils, Option => HOption, StringUtils} +import org.apache.hudi.common.util.{CommitUtils, ConfigUtils, Option => HOption, StringUtils} import org.apache.hudi.common.util.ConfigUtils.getAllConfigKeys import org.apache.hudi.config.{HoodieCompactionConfig, HoodieIndexConfig, HoodieInternalConfig, HoodieWriteConfig} import org.apache.hudi.config.HoodieBootstrapConfig.{BASE_PATH, INDEX_CLASS_NAME} @@ -310,7 +310,7 @@ class HoodieSparkSqlWriterInternal { .setArchiveLogFolder(archiveLogFolder) // we can't fetch preCombine field from hoodieConfig object, since it falls back to "ts" as default value, // but we are interested in what user has set, hence fetching from optParams. - .setPreCombineFields(optParams.getOrElse(PRECOMBINE_FIELD.key(), null)) + .setOrderingFields(ConfigUtils.getOrderingFieldsStrDuringWrite(optParams.asJava)) .setPartitionFields(partitionColumnsForKeyGenerator) .setPopulateMetaFields(populateMetaFields) .setRecordKeyFields(hoodieConfig.getString(RECORDKEY_FIELD)) @@ -517,7 +517,7 @@ class HoodieSparkSqlWriterInternal { val hoodieRecords = Try(HoodieCreateRecordUtils.createHoodieRecordRdd( HoodieCreateRecordUtils.createHoodieRecordRddArgs(df, writeConfig, parameters, avroRecordName, avroRecordNamespace, writerSchema, processedDataSchema, operation, instantTime, preppedSparkSqlWrites, - preppedSparkSqlMergeInto, preppedWriteOperation))) match { + preppedSparkSqlMergeInto, preppedWriteOperation, tableConfig.getOrderingFields))) match { case Success(recs) => recs case Failure(e) => throw new HoodieRecordCreationException("Failed to create Hoodie Spark Record", e) } @@ -772,7 +772,7 @@ class HoodieSparkSqlWriterInternal { .setPayloadClassName(payloadClass) .setRecordMergeMode(RecordMergeMode.getValue(hoodieConfig.getString(HoodieWriteConfig.RECORD_MERGE_MODE))) .setRecordMergeStrategyId(recordMergerStrategy) - .setPreCombineFields(hoodieConfig.getStringOrDefault(PRECOMBINE_FIELD, null)) + .setOrderingFields(ConfigUtils.getOrderingFieldsStrDuringWrite(hoodieConfig.getProps)) .setBootstrapIndexClass(bootstrapIndexClass) .setBaseFileFormat(baseFileFormat) .setBootstrapBasePath(bootstrapBasePath) @@ -1098,8 +1098,9 @@ class HoodieSparkSqlWriterInternal { mergedParams(HoodieTableConfig.KEY_GENERATOR_TYPE.key) = KeyGeneratorType.fromClassName(mergedParams(DataSourceWriteOptions.KEYGENERATOR_CLASS_NAME.key)).name } // use preCombineField to fill in PAYLOAD_ORDERING_FIELD_PROP_KEY - if (mergedParams.contains(PRECOMBINE_FIELD.key())) { - mergedParams.put(HoodiePayloadProps.PAYLOAD_ORDERING_FIELD_PROP_KEY, mergedParams(PRECOMBINE_FIELD.key())) + val orderingFieldsStr = ConfigUtils.getOrderingFieldsStrDuringWrite(mergedParams.asJava) + if (!StringUtils.isNullOrEmpty(orderingFieldsStr)) { + mergedParams.put(HoodiePayloadProps.PAYLOAD_ORDERING_FIELD_PROP_KEY, orderingFieldsStr) } if (mergedParams.get(OPERATION.key()).get == INSERT_OPERATION_OPT_VAL && mergedParams.contains(DataSourceWriteOptions.INSERT_DUP_POLICY.key()) && mergedParams.get(DataSourceWriteOptions.INSERT_DUP_POLICY.key()).get != FAIL_INSERT_DUP_POLICY) { @@ -1133,7 +1134,7 @@ class HoodieSparkSqlWriterInternal { RecordMergeMode.getValue(mergedParams.getOrElse(DataSourceWriteOptions.RECORD_MERGE_MODE.key(), null)), mergedParams.getOrElse(DataSourceWriteOptions.PAYLOAD_CLASS_NAME.key(), ""), mergedParams.getOrElse(DataSourceWriteOptions.RECORD_MERGE_STRATEGY_ID.key(), ""), - optParams.getOrElse(PRECOMBINE_FIELD.key(), null), + ConfigUtils.getOrderingFieldsStrDuringWrite(optParams.asJava), tableVersion) mergedParams.put(DataSourceWriteOptions.RECORD_MERGE_MODE.key(), inferredMergeConfigs.getLeft.name()) mergedParams.put(HoodieTableConfig.RECORD_MERGE_MODE.key(), inferredMergeConfigs.getLeft.name()) diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieWriterUtils.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieWriterUtils.scala index 669642a644e21..c70aef2a9e0ac 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieWriterUtils.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieWriterUtils.scala @@ -56,7 +56,6 @@ object HoodieWriterUtils { val hoodieConfig: HoodieConfig = new HoodieConfig(props) hoodieConfig.setDefaultValue(OPERATION) hoodieConfig.setDefaultValue(TABLE_TYPE) - hoodieConfig.setDefaultValue(PRECOMBINE_FIELD) hoodieConfig.setDefaultValue(KEYGENERATOR_CLASS_NAME) hoodieConfig.setDefaultValue(ENABLE) hoodieConfig.setDefaultValue(COMMIT_METADATA_KEYPREFIX) @@ -280,10 +279,10 @@ object HoodieWriterUtils { } } - val datasourcePreCombineKey = params.getOrElse(PRECOMBINE_FIELD.key(), null) - val tableConfigPreCombineKey = tableConfig.getString(HoodieTableConfig.PRECOMBINE_FIELDS) - if (null != datasourcePreCombineKey && null != tableConfigPreCombineKey && datasourcePreCombineKey != tableConfigPreCombineKey) { - diffConfigs.append(s"PreCombineKey:\t$datasourcePreCombineKey\t$tableConfigPreCombineKey\n") + val datasourceOrderingFields = params.getOrElse(ORDERING_FIELDS.key(), null) + val tableConfigOrderingKey = tableConfig.getString(HoodieTableConfig.ORDERING_FIELDS) + if (null != datasourceOrderingFields && null != tableConfigOrderingKey && datasourceOrderingFields != tableConfigOrderingKey) { + diffConfigs.append(s"OrderingFields:\t$datasourceOrderingFields\t$tableConfigOrderingKey\n") } val datasourceKeyGen = getOriginKeyGenerator(params) @@ -373,7 +372,7 @@ object HoodieWriterUtils { private val sparkDatasourceConfigsToTableConfigsMap = Map( TABLE_NAME -> HoodieTableConfig.NAME, TABLE_TYPE -> HoodieTableConfig.TYPE, - PRECOMBINE_FIELD -> HoodieTableConfig.PRECOMBINE_FIELDS, + ORDERING_FIELDS -> HoodieTableConfig.ORDERING_FIELDS, PARTITIONPATH_FIELD -> HoodieTableConfig.PARTITION_FIELDS, RECORDKEY_FIELD -> HoodieTableConfig.RECORDKEY_FIELDS, PAYLOAD_CLASS_NAME -> HoodieTableConfig.PAYLOAD_CLASS_NAME, diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/Iterators.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/Iterators.scala index 09375160a8340..0d38e01bbbd01 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/Iterators.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/Iterators.scala @@ -86,9 +86,9 @@ class LogFileIterator(logFiles: List[HoodieLogFile], } private val maxCompactionMemoryInBytes: Long = getMaxCompactionMemoryInBytes(new JobConf(config)) - protected val payloadProps: TypedProperties = if (tableState.preCombineFields.nonEmpty) { + protected val payloadProps: TypedProperties = if (tableState.orderingFields.nonEmpty) { HoodiePayloadConfig.newBuilder - .withPayloadOrderingFields(String.join(",", tableState.preCombineFields: _*)) + .withPayloadOrderingFields(String.join(",", tableState.orderingFields: _*)) .build .getProps } else { diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/MergeOnReadIncrementalRelationV1.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/MergeOnReadIncrementalRelationV1.scala index 7689f9a465784..41bb465d8d52d 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/MergeOnReadIncrementalRelationV1.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/MergeOnReadIncrementalRelationV1.scala @@ -250,7 +250,7 @@ trait HoodieIncrementalRelationV1Trait extends HoodieBaseRelation { override lazy val mandatoryFields: Seq[String] = { // NOTE: This columns are required for Incremental flow to be able to handle the rows properly, even in // cases when no columns are requested to be fetched (for ex, when using {@code count()} API) - Seq(HoodieRecord.RECORD_KEY_METADATA_FIELD, HoodieRecord.COMMIT_TIME_METADATA_FIELD) ++ preCombineFields + Seq(HoodieRecord.RECORD_KEY_METADATA_FIELD, HoodieRecord.COMMIT_TIME_METADATA_FIELD) ++ orderingFields } protected def validate(): Unit = { diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/MergeOnReadIncrementalRelationV2.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/MergeOnReadIncrementalRelationV2.scala index a71fbc4248417..3cd941e6fbf29 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/MergeOnReadIncrementalRelationV2.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/MergeOnReadIncrementalRelationV2.scala @@ -242,7 +242,7 @@ trait HoodieIncrementalRelationV2Trait extends HoodieBaseRelation { override lazy val mandatoryFields: Seq[String] = { // NOTE: These columns are required for Incremental flow to be able to handle the rows properly, even in // cases when no columns are requested to be fetched (for ex, when using {@code count()} API) - Seq(HoodieRecord.RECORD_KEY_METADATA_FIELD, HoodieRecord.COMMIT_TIME_METADATA_FIELD) ++ preCombineFields + Seq(HoodieRecord.RECORD_KEY_METADATA_FIELD, HoodieRecord.COMMIT_TIME_METADATA_FIELD) ++ orderingFields } protected def validate(): Unit = { diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/MergeOnReadSnapshotRelation.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/MergeOnReadSnapshotRelation.scala index 5c6012de63337..8bd50881e8fe1 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/MergeOnReadSnapshotRelation.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/MergeOnReadSnapshotRelation.scala @@ -92,7 +92,7 @@ abstract class BaseMergeOnReadSnapshotRelation(sqlContext: SQLContext, * by the query), therefore saving on throughput */ protected lazy val mandatoryFieldsForMerging: Seq[String] = - Seq(recordKeyField) ++ preCombineFields + Seq(recordKeyField) ++ orderingFields override lazy val mandatoryFields: Seq[String] = mandatoryFieldsForMerging diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/cdc/CDCFileGroupIterator.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/cdc/CDCFileGroupIterator.scala index c0872b9f7fbe8..96087fc207bae 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/cdc/CDCFileGroupIterator.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/cdc/CDCFileGroupIterator.scala @@ -512,7 +512,7 @@ class CDCFileGroupIterator(split: HoodieCDCFileGroupSplit, val stats = new HoodieReadStats keyBasedFileGroupRecordBuffer.ifPresent(k => k.close()) keyBasedFileGroupRecordBuffer = Option.of(new KeyBasedFileGroupRecordBuffer[InternalRow](readerContext, metaClient, readerContext.getMergeMode, - metaClient.getTableConfig.getPartialUpdateMode, readerProperties, metaClient.getTableConfig.getPreCombineFields, + metaClient.getTableConfig.getPartialUpdateMode, readerProperties, metaClient.getTableConfig.getOrderingFields, UpdateProcessor.create(stats, readerContext, true, Option.empty(), props))) HoodieMergedLogRecordReader.newBuilder[InternalRow] diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/cdc/HoodieCDCRDD.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/cdc/HoodieCDCRDD.scala index edb5ea7ff10a5..65d50ef82c333 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/cdc/HoodieCDCRDD.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/cdc/HoodieCDCRDD.scala @@ -92,7 +92,7 @@ class HoodieCDCRDD( private val props = HoodieFileIndex.getConfigProperties(spark, Map.empty, metaClient.getTableConfig) - protected val payloadProps: Properties = metaClient.getTableConfig.getPreCombineFieldsStr + protected val payloadProps: Properties = metaClient.getTableConfig.getOrderingFieldsStr .map[TypedProperties](JFunction.toJavaFunction(preCombineFields => HoodiePayloadConfig.newBuilder .withPayloadOrderingFields(preCombineFields) @@ -138,7 +138,7 @@ class HoodieCDCRDD( keyFields.head } - private lazy val preCombineFields: List[String] = metaClient.getTableConfig.getPreCombineFields.asScala.toList + private lazy val orderingFields: List[String] = metaClient.getTableConfig.getOrderingFields.asScala.toList private lazy val tableState = { val metadataConfig = HoodieMetadataConfig.newBuilder() @@ -148,7 +148,7 @@ class HoodieCDCRDD( basePath.toUri.toString, Some(split.changes.last.getInstant), recordKeyField, - preCombineFields, + orderingFields, usesVirtualKeys = !populateMetaFields, metaClient.getTableConfig.getPayloadClass, metadataConfig, diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/catalyst/catalog/HoodieCatalogTable.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/catalyst/catalog/HoodieCatalogTable.scala index ceb55b9fd94c9..f3fdc5db225d0 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/catalyst/catalog/HoodieCatalogTable.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/catalyst/catalog/HoodieCatalogTable.scala @@ -130,7 +130,7 @@ class HoodieCatalogTable(val spark: SparkSession, var table: CatalogTable) exten /** * Comparables Field */ - lazy val preCombineKeys: java.util.List[String] = tableConfig.getPreCombineFields + lazy val orderingFields: java.util.List[String] = tableConfig.getOrderingFields /** * Partition Fields diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/HoodieOptionConfig.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/HoodieOptionConfig.scala index 89472466a2db7..1947e61575da9 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/HoodieOptionConfig.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/HoodieOptionConfig.scala @@ -58,10 +58,11 @@ object HoodieOptionConfig { .defaultValue(SQL_VALUE_TABLE_TYPE_COW) .build() - val SQL_KEY_PRECOMBINE_FIELD: HoodieSQLOption[String] = buildConf() - .withSqlKey("preCombineField") - .withHoodieKey(DataSourceWriteOptions.PRECOMBINE_FIELD.key) - .withTableConfigKey(HoodieTableConfig.PRECOMBINE_FIELDS.key) + val SQL_KEY_ORDERING_FIELDS: HoodieSQLOption[String] = buildConf() + .withSqlKey("orderingFields") + .withAlternatives(List("preCombineField")) + .withHoodieKey(DataSourceWriteOptions.ORDERING_FIELDS.key()) + .withTableConfigKey(HoodieTableConfig.ORDERING_FIELDS.key) .build() val SQL_PAYLOAD_CLASS: HoodieSQLOption[String] = buildConf() @@ -82,6 +83,14 @@ object HoodieOptionConfig { .withTableConfigKey(HoodieTableConfig.RECORD_MERGE_STRATEGY_ID.key) .build() + private lazy val legacySqlOptionKeys: Set[String] = { + HoodieOptionConfig.getClass.getDeclaredFields + .filter(f => f.getType == classOf[HoodieSQLOption[_]]) + .map(f => {f.setAccessible(true); f.get(HoodieOptionConfig).asInstanceOf[HoodieSQLOption[_]]}) + .filter(_.alternatives.nonEmpty).flatMap(option => option.alternatives.toStream) + .toSet + } + /** * The mapping of the sql short name key to the hoodie's config key. */ @@ -89,12 +98,18 @@ object HoodieOptionConfig { HoodieOptionConfig.getClass.getDeclaredFields .filter(f => f.getType == classOf[HoodieSQLOption[_]]) .map(f => {f.setAccessible(true); f.get(HoodieOptionConfig).asInstanceOf[HoodieSQLOption[_]]}) - .map(option => option.sqlKeyName -> option.hoodieKeyName) + .flatMap(option => { + if (option.alternatives.isEmpty) { + Map(option.sqlKeyName -> option.hoodieKeyName).toStream + } else { + (List(option.sqlKeyName) ++ option.alternatives).map(k => k -> option.hoodieKeyName).toStream + } + }) .toMap } private lazy val writeConfigKeyToSqlOptionKey: Map[String, String] = - sqlOptionKeyToWriteConfigKey.map(f => f._2 -> f._1) + (sqlOptionKeyToWriteConfigKey -- legacySqlOptionKeys).map(f => f._2 -> f._1) /** * The mapping of the sql short name key to the hoodie table config key @@ -105,12 +120,18 @@ object HoodieOptionConfig { .filter(f => f.getType == classOf[HoodieSQLOption[_]]) .map(f => {f.setAccessible(true); f.get(HoodieOptionConfig).asInstanceOf[HoodieSQLOption[_]]}) .filter(_.tableConfigKey.isDefined) - .map(option => option.sqlKeyName -> option.tableConfigKey.get) + .flatMap(option => { + if (option.alternatives.isEmpty) { + Map(option.sqlKeyName -> option.tableConfigKey.get).toStream + } else { + (List(option.sqlKeyName) ++ option.alternatives).map(k => k -> option.tableConfigKey.get).toStream + } + }) .toMap } private lazy val tableConfigKeyToSqlOptionKey: Map[String, String] = - sqlOptionKeyToTableConfigKey.map(f => f._2 -> f._1) + (sqlOptionKeyToTableConfigKey -- legacySqlOptionKeys).map(f => f._2 -> f._1) /** * Mapping of the short sql value to the hoodie's config value @@ -181,11 +202,6 @@ object HoodieOptionConfig { DataSourceWriteOptions.TABLE_TYPE.defaultValue) } - def getPreCombineField(options: Map[String, String]): Option[String] = { - val params = mapSqlOptionsToDataSourceWriteConfigs(options) - params.get(DataSourceWriteOptions.PRECOMBINE_FIELD.key).filter(_.nonEmpty) - } - def deleteHoodieOptions(options: Map[String, String]): Map[String, String] = { options.filterNot(_._1.startsWith("hoodie.")).filterNot(kv => sqlOptionKeyToWriteConfigKey.contains(kv._1)) } @@ -198,19 +214,19 @@ object HoodieOptionConfig { options.filter(_._1.startsWith("hoodie.")) ++ extractSqlOptions(options) } - // extract primaryKey, preCombineField, type options + // extract primaryKey, orderingFields, type options def extractSqlOptions(options: Map[String, String]): Map[String, String] = { val sqlOptions = mapHoodieConfigsToSqlOptions(options) val targetOptions = sqlOptionKeyToWriteConfigKey.keySet -- Set(SQL_PAYLOAD_CLASS.sqlKeyName, SQL_RECORD_MERGE_STRATEGY_ID.sqlKeyName, SQL_RECORD_MERGE_MODE.sqlKeyName) sqlOptions.filterKeys(targetOptions.contains).toMap } - // validate primaryKey, preCombineField and type options + // validate primaryKey, orderingFields and type options def validateTable(spark: SparkSession, schema: StructType, sqlOptions: Map[String, String]): Unit = { val resolver = spark.sessionState.conf.resolver // validate primary key val primaryKeys = sqlOptions.get(SQL_KEY_TABLE_PRIMARY_KEY.sqlKeyName) - .map(_.split(",").filter(_.length > 0)) + .map(_.split(",").filter(_.nonEmpty)) if (primaryKeys.isDefined) { primaryKeys.get.foreach { primaryKey => ValidationUtils.checkArgument(schema.exists(f => resolver(f.name, getRootLevelFieldName(primaryKey))), @@ -219,10 +235,10 @@ object HoodieOptionConfig { } // validate preCombine key - val preCombineKey = sqlOptions.get(SQL_KEY_PRECOMBINE_FIELD.sqlKeyName) - if (preCombineKey.isDefined && preCombineKey.get.nonEmpty) { - ValidationUtils.checkArgument(schema.exists(f => resolver(f.name, getRootLevelFieldName(preCombineKey.get))), - s"Can't find preCombineKey `${preCombineKey.get}` in ${schema.treeString}.") + val orderingFields = getSqlOptionWithAlternatives(SQL_KEY_ORDERING_FIELDS, sqlOptions) + if (orderingFields.isDefined && orderingFields.get.nonEmpty) { + ValidationUtils.checkArgument(schema.exists(f => resolver(f.name, getRootLevelFieldName(orderingFields.get))), + s"Can't find ordering fields `${orderingFields.get}` in ${schema.treeString}.") } // validate table type @@ -246,8 +262,9 @@ object HoodieOptionConfig { def makeOptionsCaseInsensitive(sqlOptions: Map[String, String]): Map[String, String] = { // Make Keys Case Insensitive - val standardOptions = Seq(SQL_KEY_TABLE_PRIMARY_KEY, SQL_KEY_PRECOMBINE_FIELD, - SQL_KEY_TABLE_TYPE, SQL_PAYLOAD_CLASS, SQL_RECORD_MERGE_STRATEGY_ID, SQL_RECORD_MERGE_MODE).map(key => key.sqlKeyName) + val standardOptions = Seq(SQL_KEY_TABLE_PRIMARY_KEY, SQL_KEY_ORDERING_FIELDS, + SQL_KEY_TABLE_TYPE, SQL_PAYLOAD_CLASS, SQL_RECORD_MERGE_STRATEGY_ID, SQL_RECORD_MERGE_MODE) + .flatMap(key => (List(key.sqlKeyName) ++ key.alternatives).toStream) sqlOptions.map(option => { standardOptions.find(x => x.toLowerCase().contains(option._1.toLowerCase())) match { @@ -256,10 +273,16 @@ object HoodieOptionConfig { } }) } + + private def getSqlOptionWithAlternatives(sqlOption: HoodieSQLOption[String], options: Map[String, String]): Option[String] = { + options.get(sqlOption.sqlKeyName) + .orElse(sqlOption.alternatives.map(alternative => options.get(alternative)).filter(_.isDefined).collectFirst({case opt => opt.get})) + } } case class HoodieSQLOption[T]( sqlKeyName: String, + alternatives: List[String], hoodieKeyName: String, tableConfigKey: Option[String], defaultValue: Option[T] @@ -268,6 +291,7 @@ case class HoodieSQLOption[T]( class HoodieSQLOptionBuilder[T] { private var sqlKeyName: String = _ + private var alternatives: List[String] = List.empty private var hoodieKeyName: String =_ private var tableConfigKey: String =_ private var defaultValue: T =_ @@ -277,6 +301,11 @@ class HoodieSQLOptionBuilder[T] { this } + def withAlternatives(alternatives: List[String]): HoodieSQLOptionBuilder[T] = { + this.alternatives = alternatives + this + } + def withHoodieKey(hoodieKeyName: String): HoodieSQLOptionBuilder[T] = { this.hoodieKeyName = hoodieKeyName this @@ -293,6 +322,6 @@ class HoodieSQLOptionBuilder[T] { } def build(): HoodieSQLOption[T] = { - HoodieSQLOption(sqlKeyName, hoodieKeyName, Option(tableConfigKey), Option(defaultValue)) + HoodieSQLOption(sqlKeyName, alternatives, hoodieKeyName, Option(tableConfigKey), Option(defaultValue)) } } diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/ProvidesHoodieConfig.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/ProvidesHoodieConfig.scala index b087e711dd7a5..9736b3fd76ee1 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/ProvidesHoodieConfig.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/ProvidesHoodieConfig.scala @@ -25,7 +25,7 @@ import org.apache.hudi.common.config.{DFSPropertiesConfiguration, HoodieCommonCo import org.apache.hudi.common.model.WriteOperationType import org.apache.hudi.common.table.HoodieTableConfig import org.apache.hudi.common.table.HoodieTableConfig.DATABASE_NAME -import org.apache.hudi.common.util.{ReflectionUtils, StringUtils} +import org.apache.hudi.common.util.{ConfigUtils, ReflectionUtils, StringUtils} import org.apache.hudi.config.{HoodieIndexConfig, HoodieInternalConfig, HoodieWriteConfig} import org.apache.hudi.config.HoodieWriteConfig.TBL_NAME import org.apache.hudi.hive.{HiveSyncConfig, HiveSyncConfigHolder, MultiPartKeysValueExtractor} @@ -53,6 +53,7 @@ import org.slf4j.LoggerFactory import java.util.Locale +import scala.collection.JavaConverters import scala.collection.JavaConverters._ trait ProvidesHoodieConfig extends Logging { @@ -64,7 +65,7 @@ trait ProvidesHoodieConfig extends Logging { // NOTE: Here we fallback to "" to make sure that null value is not overridden with // default value ("ts") // TODO(HUDI-3456) clean up - val preCombineFields = tableConfig.getPreCombineFieldsStr.orElse("") + val orderingFields = tableConfig.getOrderingFieldsStr.orElse("") val hiveSyncConfig = buildHiveSyncConfig(sparkSession, hoodieCatalogTable, tableConfig) val defaultOpts = Map[String, String]( @@ -82,7 +83,7 @@ trait ProvidesHoodieConfig extends Logging { HiveSyncConfigHolder.HIVE_SUPPORT_TIMESTAMP_TYPE.key -> hiveSyncConfig.getBoolean(HiveSyncConfigHolder.HIVE_SUPPORT_TIMESTAMP_TYPE).toString ) - val overridingOpts = buildOverridingOpts(hoodieCatalogTable, preCombineFields) + val overridingOpts = buildOverridingOpts(hoodieCatalogTable, orderingFields) combineOptions(hoodieCatalogTable, tableConfig, sparkSession.sqlContext.conf, defaultOpts = defaultOpts, overridingOpts = overridingOpts) } @@ -90,7 +91,7 @@ trait ProvidesHoodieConfig extends Logging { def buildBucketRescaleHoodieConfig(hoodieCatalogTable: HoodieCatalogTable): Map[String, String] = { val sparkSession: SparkSession = hoodieCatalogTable.spark val tableConfig = hoodieCatalogTable.tableConfig - val preCombineFields = tableConfig.getPreCombineFieldsStr.orElse("") + val orderingFields = tableConfig.getOrderingFieldsStr.orElse("") val hiveSyncConfig = buildHiveSyncConfig(sparkSession, hoodieCatalogTable, tableConfig) val defaultOpts = Map[String, String]( @@ -103,7 +104,7 @@ trait ProvidesHoodieConfig extends Logging { HiveSyncConfigHolder.HIVE_SYNC_ENABLED.key -> "false" ) - val overridingOpts = buildOverridingOpts(hoodieCatalogTable, preCombineFields) + val overridingOpts = buildOverridingOpts(hoodieCatalogTable, orderingFields) combineOptions(hoodieCatalogTable, tableConfig, sparkSession.sqlContext.conf, defaultOpts = defaultOpts, overridingOpts = overridingOpts) } @@ -147,7 +148,7 @@ trait ProvidesHoodieConfig extends Logging { case (false, false, true, _, _, _, _) => INSERT_OVERWRITE_TABLE_OPERATION_OPT_VAL // insert overwrite partition case (false, true, false, _, _, true, _) => INSERT_OVERWRITE_OPERATION_OPT_VAL - // disable dropDuplicate, and provide preCombineKey, use the upsert operation for strict and upsert mode. + // disable dropDuplicate, and provide orderingFields, use the upsert operation for strict and upsert mode. case (false, false, false, false, false, _, _) if combineBeforeInsert => UPSERT_OPERATION_OPT_VAL // if table is pk table and has enableBulkInsert use bulk insert for non-strict mode. case (true, false, false, _, true, _, _) => BULK_INSERT_OPERATION_OPT_VAL @@ -190,8 +191,7 @@ trait ProvidesHoodieConfig extends Logging { // NOTE: Here we fallback to "" to make sure that null value is not overridden with // default value ("ts") // TODO(HUDI-3456) clean up - val preCombineField = combinedOpts.getOrElse(HoodieTableConfig.PRECOMBINE_FIELDS.key, - combinedOpts.getOrElse(PRECOMBINE_FIELD.key, "")) + val orderingFieldsStr = Option.apply(ConfigUtils.getOrderingFieldsStrDuringWrite(combinedOpts.asJava)).getOrElse("") val hiveStylePartitioningEnable = Option(tableConfig.getHiveStylePartitioningEnable).getOrElse("true") val urlEncodePartitioning = Option(tableConfig.getUrlEncodePartitioning).getOrElse("false") @@ -215,7 +215,7 @@ trait ProvidesHoodieConfig extends Logging { val insertDupPolicy = combinedOpts.getOrElse(INSERT_DUP_POLICY.key(), INSERT_DUP_POLICY.defaultValue()) val isNonStrictMode = insertMode == InsertMode.NON_STRICT val isPartitionedTable = hoodieCatalogTable.partitionFields.nonEmpty - val combineBeforeInsert = !hoodieCatalogTable.preCombineKeys.isEmpty && hoodieCatalogTable.primaryKeys.nonEmpty + val combineBeforeInsert = !hoodieCatalogTable.orderingFields.isEmpty && hoodieCatalogTable.primaryKeys.nonEmpty /* * The sql write operation has higher precedence than the legacy insert mode. @@ -232,7 +232,7 @@ trait ProvidesHoodieConfig extends Logging { isNonStrictMode, isPartitionedTable, combineBeforeInsert, insertMode, shouldAutoKeyGen) } else { deduceSparkSqlInsertIntoWriteOperation(isOverwritePartition, isOverwriteTable, - shouldAutoKeyGen, preCombineField, sparkSqlInsertIntoOperationSet, sparkSqlInsertIntoOperation) + shouldAutoKeyGen, orderingFieldsStr, sparkSqlInsertIntoOperationSet, sparkSqlInsertIntoOperation) } ) @@ -301,7 +301,7 @@ trait ProvidesHoodieConfig extends Logging { HIVE_STYLE_PARTITIONING.key -> hiveStylePartitioningEnable, URL_ENCODE_PARTITIONING.key -> urlEncodePartitioning, RECORDKEY_FIELD.key -> recordKeyConfigValue, - PRECOMBINE_FIELD.key -> preCombineField, + HoodieTableConfig.ORDERING_FIELDS.key -> orderingFieldsStr, PARTITIONPATH_FIELD.key -> getPartitionPathFieldWriteConfig( keyGeneratorClassName, partitionFieldsStr, hoodieCatalogTable) ) ++ overwriteTableOpts ++ getDropDupsConfig(useLegacyInsertModeFlow, combinedOpts) ++ staticOverwritePartitionPathOptions @@ -398,7 +398,6 @@ trait ProvidesHoodieConfig extends Logging { OPERATION.key -> DataSourceWriteOptions.DELETE_PARTITION_OPERATION_OPT_VAL, PARTITIONS_TO_DELETE.key -> partitionsToDrop, RECORDKEY_FIELD.key -> hoodieCatalogTable.primaryKeys.mkString(","), - PRECOMBINE_FIELD.key -> String.join(",", hoodieCatalogTable.preCombineKeys), PARTITIONPATH_FIELD.key -> getPartitionPathFieldWriteConfig( tableConfig.getKeyGeneratorClassName, partitionFields, hoodieCatalogTable), HoodieSyncConfig.META_SYNC_ENABLED.key -> hiveSyncConfig.getString(HoodieSyncConfig.META_SYNC_ENABLED.key), @@ -560,9 +559,9 @@ object ProvidesHoodieConfig { opts.filter { case (_, v) => v != null } private def buildOverridingOpts(hoodieCatalogTable: HoodieCatalogTable, - preCombineFields: String): Map[String, String] = { + orderingFields: String): Map[String, String] = { buildCommonOverridingOpts(hoodieCatalogTable) ++ Map( - PRECOMBINE_FIELD.key -> preCombineFields + HoodieTableConfig.ORDERING_FIELDS.key -> orderingFields ) } diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/AlterTableCommand.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/AlterTableCommand.scala index 65a25b6ab0194..d8c823b532429 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/AlterTableCommand.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/AlterTableCommand.scala @@ -228,12 +228,12 @@ case class AlterTableCommand(table: CatalogTable, changes: Seq[TableChange], cha def checkSchemaChange(colNames: Seq[String], catalogTable: CatalogTable): Unit = { val primaryKeys = catalogTable.storage.properties.getOrElse("primaryKey", catalogTable.properties.getOrElse("primaryKey", "keyid")).split(",").map(_.trim) - val preCombineKey = Seq(catalogTable.storage.properties.getOrElse("preCombineField", catalogTable.properties.getOrElse("preCombineField", "ts"))).map(_.trim) + val orderingFields = Seq(catalogTable.storage.properties.getOrElse("orderingFields", catalogTable.properties.getOrElse("orderingFields", "ts"))).map(_.trim) val partitionKey = catalogTable.partitionColumnNames.map(_.trim) - val checkNames = primaryKeys ++ preCombineKey ++ partitionKey + val checkNames = primaryKeys ++ orderingFields ++ partitionKey colNames.foreach { col => if (checkNames.contains(col)) { - throw new UnsupportedOperationException("cannot support apply changes for primaryKey/CombineKey/partitionKey") + throw new UnsupportedOperationException("cannot support apply changes for primaryKey/orderingFields/partitionKey") } } } diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/CreateHoodieTableCommand.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/CreateHoodieTableCommand.scala index 69e09598f7c0e..fa4102d7e55f4 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/CreateHoodieTableCommand.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/CreateHoodieTableCommand.scala @@ -125,7 +125,8 @@ object CreateHoodieTableCommand { val originTableConfig = hoodieCatalogTable.tableConfig.getProps.asScala.toMap val tableOptions = hoodieCatalogTable.catalogProperties - checkTableConfigEqual(originTableConfig, tableOptions, HoodieTableConfig.PRECOMBINE_FIELDS.key) + checkTableConfigEqual(originTableConfig, tableOptions, HoodieTableConfig.ORDERING_FIELDS.key) + checkTableConfigEqual(originTableConfig, tableOptions, HoodieTableConfig.PRECOMBINE_FIELD.key()) checkTableConfigEqual(originTableConfig, tableOptions, HoodieTableConfig.PARTITION_FIELDS.key) checkTableConfigEqual(originTableConfig, tableOptions, HoodieTableConfig.RECORDKEY_FIELDS.key) checkTableConfigEqual(originTableConfig, tableOptions, HoodieTableConfig.KEY_GENERATOR_CLASS_NAME.key) @@ -173,7 +174,7 @@ object CreateHoodieTableCommand { .copy(table = tableName, database = Some(newDatabaseName)) val partitionColumnNames = hoodieCatalogTable.partitionSchema.map(_.name) - // Remove some properties should not be used;append pk, preCombineKey, type to the properties of table + // Remove some properties should not be used;append pk, orderingFields, type to the properties of table var newTblProperties = hoodieCatalogTable.catalogProperties.--(needFilterProps) ++ HoodieOptionConfig.extractSqlOptions(properties) diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/ShowHoodieCreateTableCommand.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/ShowHoodieCreateTableCommand.scala index 48ca799479cb1..9adc27ad7ff01 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/ShowHoodieCreateTableCommand.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/ShowHoodieCreateTableCommand.scala @@ -27,7 +27,7 @@ import org.apache.spark.sql.catalyst.analysis.NoSuchTableException import org.apache.spark.sql.catalyst.catalog.HoodieCatalogTable import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference} import org.apache.spark.sql.catalyst.util.escapeSingleQuotedString -import org.apache.spark.sql.hudi.HoodieOptionConfig.{SQL_KEY_PRECOMBINE_FIELD, SQL_KEY_TABLE_PRIMARY_KEY, SQL_KEY_TABLE_TYPE, SQL_PAYLOAD_CLASS, SQL_RECORD_MERGE_STRATEGY_ID} +import org.apache.spark.sql.hudi.HoodieOptionConfig.{SQL_KEY_ORDERING_FIELDS, SQL_KEY_TABLE_PRIMARY_KEY, SQL_KEY_TABLE_TYPE, SQL_PAYLOAD_CLASS, SQL_RECORD_MERGE_STRATEGY_ID} import org.apache.spark.sql.types.StringType case class ShowHoodieCreateTableCommand(table: TableIdentifier) @@ -80,8 +80,9 @@ case class ShowHoodieCreateTableCommand(table: TableIdentifier) } private def showHoodieTableProperties(metadata: HoodieCatalogTable, builder: StringBuilder): Unit = { - val standardOptions = Seq(SQL_KEY_TABLE_PRIMARY_KEY, SQL_KEY_PRECOMBINE_FIELD, - SQL_KEY_TABLE_TYPE, SQL_PAYLOAD_CLASS, SQL_RECORD_MERGE_STRATEGY_ID).map(key => key.sqlKeyName) + val standardOptions = Seq(SQL_KEY_TABLE_PRIMARY_KEY, SQL_KEY_ORDERING_FIELDS, + SQL_KEY_TABLE_TYPE, SQL_PAYLOAD_CLASS, SQL_RECORD_MERGE_STRATEGY_ID) + .flatMap(key => (List(key.sqlKeyName) ++ key.alternatives).toStream) val props = metadata.catalogProperties.filter(key => standardOptions.contains(key._1)).map { case (key, value) => s"$key='${escapeSingleQuotedString(value)}'" } ++ metadata.catalogProperties.filterNot(_._1.equals(ConfigUtils.IS_QUERY_AS_RO_TABLE)).map { diff --git a/hudi-spark-datasource/hudi-spark-common/src/test/scala/org/apache/spark/sql/hudi/TestProvidesHoodieConfig.scala b/hudi-spark-datasource/hudi-spark-common/src/test/scala/org/apache/spark/sql/hudi/TestProvidesHoodieConfig.scala index 670d96b6b9c10..e7ef57f3856c7 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/test/scala/org/apache/spark/sql/hudi/TestProvidesHoodieConfig.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/test/scala/org/apache/spark/sql/hudi/TestProvidesHoodieConfig.scala @@ -87,14 +87,14 @@ class TestProvidesHoodieConfig { } @Test - def testInferPrecombineFieldFromTableConfig(): Unit = { + def testInferOrderingFieldsFromTableConfig(): Unit = { // ProvidesHoodieConfig should be able to infer precombine field from table config // mock catalogTable val mockCatalog = mock(classOf[HoodieCatalogTable]) // catalogProperties won't be passed in correctly, because they were not synced properly when(mockCatalog.catalogProperties).thenReturn(Map.empty[String, String]) when(mockCatalog.partitionFields).thenReturn(Array("partition")) - when(mockCatalog.preCombineKeys).thenCallRealMethod() + when(mockCatalog.orderingFields).thenCallRealMethod() when(mockCatalog.partitionSchema).thenReturn(StructType(Nil)) when(mockCatalog.primaryKeys).thenReturn(Array("key")) when(mockCatalog.table).thenReturn(CatalogTable.apply( @@ -103,7 +103,7 @@ class TestProvidesHoodieConfig { CatalogStorageFormat.empty, StructType(Nil))) val props = new TypedProperties() - props.setProperty(HoodieTableConfig.PRECOMBINE_FIELDS.key, "segment") + props.setProperty(HoodieTableConfig.ORDERING_FIELDS.key, "segment") val mockTableConfig = spy(classOf[HoodieTableConfig]) when(mockTableConfig.getProps).thenReturn(props) when(mockCatalog.tableConfig).thenReturn(mockTableConfig) @@ -140,13 +140,7 @@ class TestProvidesHoodieConfig { assertEquals( "segment", - combinedConfig.getOrElse(HoodieTableConfig.PRECOMBINE_FIELDS.key, "") - ) - - // write config precombine field should be inferred from table config - assertEquals( - "segment", - combinedConfig.getOrElse(DataSourceWriteOptions.PRECOMBINE_FIELD.key, "") + combinedConfig.getOrElse(HoodieTableConfig.ORDERING_FIELDS.key, "") ) } diff --git a/hudi-spark-datasource/hudi-spark/src/main/java/org/apache/hudi/bootstrap/SparkFullBootstrapDataProviderBase.java b/hudi-spark-datasource/hudi-spark/src/main/java/org/apache/hudi/bootstrap/SparkFullBootstrapDataProviderBase.java index 65af0c3543409..bccebeb2db469 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/java/org/apache/hudi/bootstrap/SparkFullBootstrapDataProviderBase.java +++ b/hudi-spark-datasource/hudi-spark/src/main/java/org/apache/hudi/bootstrap/SparkFullBootstrapDataProviderBase.java @@ -72,7 +72,7 @@ public JavaRDD generateInputRecords(String tableName, String sourc HoodieRecordType recordType = config.getRecordMerger().getRecordType(); Dataset inputDataset = sparkSession.read().format(getFormat()).option("basePath", sourceBasePath).load(filePaths); KeyGenerator keyGenerator = HoodieSparkKeyGeneratorFactory.createKeyGenerator(props); - String precombineKey = props.getString("hoodie.datasource.write.precombine.field"); + String orderingFieldsStr = ConfigUtils.getOrderingFieldsStrDuringWrite(props); String structName = tableName + "_record"; String namespace = "hoodie." + tableName; if (recordType == HoodieRecordType.AVRO) { @@ -80,7 +80,7 @@ public JavaRDD generateInputRecords(String tableName, String sourc Option.empty()); return genericRecords.toJavaRDD().map(gr -> { String orderingVal = HoodieAvroUtils.getNestedFieldValAsString( - gr, precombineKey, false, props.getBoolean( + gr, orderingFieldsStr, false, props.getBoolean( KeyGeneratorOptions.KEYGENERATOR_CONSISTENT_LOGICAL_TIMESTAMP_ENABLED.key(), Boolean.parseBoolean(KeyGeneratorOptions.KEYGENERATOR_CONSISTENT_LOGICAL_TIMESTAMP_ENABLED.defaultValue()))); try { diff --git a/hudi-spark-datasource/hudi-spark/src/main/java/org/apache/hudi/cli/BootstrapExecutorUtils.java b/hudi-spark-datasource/hudi-spark/src/main/java/org/apache/hudi/cli/BootstrapExecutorUtils.java index bd2c44e4370f5..a3be757c377b1 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/java/org/apache/hudi/cli/BootstrapExecutorUtils.java +++ b/hudi-spark-datasource/hudi-spark/src/main/java/org/apache/hudi/cli/BootstrapExecutorUtils.java @@ -25,6 +25,7 @@ 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.util.ConfigUtils; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.ReflectionUtils; import org.apache.hudi.common.util.StringUtils; @@ -67,7 +68,6 @@ import static org.apache.hudi.common.util.ConfigUtils.filterProperties; import static org.apache.hudi.config.HoodieIndexConfig.BUCKET_INDEX_HASH_FIELD; import static org.apache.hudi.config.HoodieIndexConfig.BUCKET_INDEX_NUM_BUCKETS; -import static org.apache.hudi.config.HoodieWriteConfig.PRECOMBINE_FIELD_NAME; import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_SYNC_BUCKET_SYNC; import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_SYNC_BUCKET_SYNC_SPEC; import static org.apache.hudi.keygen.constant.KeyGeneratorOptions.HIVE_STYLE_PARTITIONING_ENABLE; @@ -237,7 +237,7 @@ private void initializeTable() throws IOException { .setTableName(cfg.tableName) .setTableVersion(bootstrapConfig.getWriteVersion()) .setRecordKeyFields(props.getString(RECORDKEY_FIELD_NAME.key())) - .setPreCombineFields(props.getString(PRECOMBINE_FIELD_NAME.key(), null)) + .setOrderingFields(ConfigUtils.getOrderingFieldsStrDuringWrite(props)) .setPopulateMetaFields(props.getBoolean( POPULATE_META_FIELDS.key(), POPULATE_META_FIELDS.defaultValue())) .setArchiveLogFolder(props.getString( diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/MergeIntoHoodieTableCommand.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/MergeIntoHoodieTableCommand.scala index 6aaa9cfaf3182..6f5ce6558f2cb 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/MergeIntoHoodieTableCommand.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/MergeIntoHoodieTableCommand.scala @@ -25,8 +25,8 @@ import org.apache.hudi.avro.HoodieAvroUtils import org.apache.hudi.common.config.RecordMergeMode import org.apache.hudi.common.model.{HoodieAvroRecordMerger, HoodieRecordMerger} import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableVersion, PartialUpdateMode} +import org.apache.hudi.common.util.{ConfigUtils, StringUtils} import org.apache.hudi.common.util.ConfigUtils.getStringWithAltKeys -import org.apache.hudi.common.util.StringUtils import org.apache.hudi.config.{HoodieIndexConfig, HoodieWriteConfig} import org.apache.hudi.config.HoodieWriteConfig.{AVRO_SCHEMA_VALIDATE_ENABLE, SCHEMA_ALLOW_AUTO_EVOLUTION_COLUMN_DROP, TBL_NAME, WRITE_PARTIAL_UPDATE_SCHEMA} import org.apache.hudi.exception.{HoodieException, HoodieNotSupportedException} @@ -87,7 +87,7 @@ class MergeIntoFieldTypeMismatchException(message: String) * *
    *
  1. Incoming batch ([[sourceTable]]) is reshaped such that it bears correspondingly: - * a) (required) "primary-key" column as well as b) (optional) "precombine" column; this is + * a) (required) "primary-key" column as well as b) (optional) "ordering" columns; this is * required since MIT statements does not restrict [[sourceTable]]s schema to be aligned w/ the * [[targetTable]]s one, while Hudi's upserting flow expects such columns to be present
  2. * @@ -153,7 +153,7 @@ case class MergeIntoHoodieTableCommand(mergeInto: MergeIntoTable, * * To be able to leverage Hudi's engine to merge an incoming dataset against the existing table * we will have to make sure that both [[source]] and [[target]] tables have the *same* - * "primary-key" and "precombine" columns. Since actual MIT condition might be leveraging an arbitrary + * "primary-key" and "ordering" columns. Since actual MIT condition might be leveraging an arbitrary * expression involving [[source]] column(s), we will have to add "phony" column matching the * primary-key one of the target table. */ @@ -263,13 +263,13 @@ case class MergeIntoHoodieTableCommand(mergeInto: MergeIntoTable, /** * Please check description for [[primaryKeyAttributeToConditionExpression]] */ - private lazy val preCombineAttributeAssociatedExpressions: Seq[(Attribute, Expression)] = + private lazy val orderingFieldsAssociatedExpressions: Seq[(Attribute, Expression)] = resolveFieldAssociationsBetweenSourceAndTarget( sparkSession.sessionState.conf.resolver, mergeInto.targetTable, mergeInto.sourceTable, - hoodieCatalogTable.preCombineKeys.asScala.toSeq, - "precombine field", + hoodieCatalogTable.orderingFields.asScala.toSeq, + "ordering fields", updatingActions.flatMap(_.assignments)) override def run(sparkSession: SparkSession, inputPlan: SparkPlan): Seq[Row] = { @@ -313,7 +313,7 @@ case class MergeIntoHoodieTableCommand(mergeInto: MergeIntoTable, * *
      *
    1. Contains "primary-key" column (as defined by target table's config)
    2. - *
    3. Contains "precombine" column (as defined by target table's config, if any)
    4. + *
    5. Contains "ordering" columns (as defined by target table's config, if any)
    6. *
    * * In cases when [[sourceTable]] doesn't contain aforementioned columns, following heuristic @@ -326,12 +326,12 @@ case class MergeIntoHoodieTableCommand(mergeInto: MergeIntoTable, * leveraging matching side of such conditional expression (containing [[sourceTable]] attribute) * interpreting it as a primary-key column in the [[sourceTable]] * - *
  3. Expression for the "precombine" column (optional) is extracted from the matching update + *
  4. Expression for the "ordering" columns (optional) is extracted from the matching update * clause ({@code WHEN MATCHED ... THEN UPDATE ...}) as right-hand side of the expression referencing - * precombine attribute of the target column
  5. + * ordering attribute of the target column *
      * - * For example, w/ the following statement (primary-key column is [[id]], while precombine column is [[ts]]) + * For example, w/ the following statement (primary-key column is [[id]], while ordering column is [[ts]]) *
          * MERGE INTO target
          * USING (SELECT 1 AS sid, 'A1' AS sname, 1000 AS sts) source
      @@ -366,7 +366,7 @@ case class MergeIntoHoodieTableCommand(mergeInto: MergeIntoTable,
       
           val inputPlanAttributes = inputPlan.output
       
      -    val requiredAttributesMap = recordKeyAttributeToConditionExpression ++ preCombineAttributeAssociatedExpressions
      +    val requiredAttributesMap = recordKeyAttributeToConditionExpression ++ orderingFieldsAssociatedExpressions
       
           val (existingAttributesMap, missingAttributesMap) = requiredAttributesMap.partition {
             case (keyAttr, _) => inputPlanAttributes.exists(attr => resolver(keyAttr.name, attr.name))
      @@ -374,7 +374,7 @@ case class MergeIntoHoodieTableCommand(mergeInto: MergeIntoTable,
       
           // This is to handle the situation where condition is something like "s0.s_id = t0.id" so In the source table
           // we add an additional column that is an alias of "s0.s_id" named "id"
      -    // NOTE: Record key attribute (required) as well as precombine one (optional) defined
      +    // NOTE: Record key attribute (required) as well as ordering attributes (optional) defined
           //       in the [[targetTable]] schema has to be present in the incoming [[sourceTable]] dataset.
           //       In cases when [[sourceTable]] doesn't bear such attributes (which, for ex, could happen
           //       in case of it having different schema), we will be adding additional columns (while setting
      @@ -525,7 +525,7 @@ case class MergeIntoHoodieTableCommand(mergeInto: MergeIntoTable,
         }
       
         private def getOperationType(parameters: Map[String, String]) = {
      -    if (StringUtils.isNullOrEmpty(parameters.getOrElse(PRECOMBINE_FIELD.key, "")) && updatingActions.isEmpty) {
      +    if (StringUtils.isNullOrEmpty(ConfigUtils.getOrderingFieldsStrDuringWrite(parameters.asJava)) && updatingActions.isEmpty) {
             INSERT_OPERATION_OPT_VAL
           } else {
             UPSERT_OPERATION_OPT_VAL
      @@ -727,11 +727,11 @@ case class MergeIntoHoodieTableCommand(mergeInto: MergeIntoTable,
       
         /**
          * Output of the expected (left) join of the a) [[sourceTable]] dataset (potentially amended w/ primary-key,
      -   * precombine columns) with b) existing [[targetTable]]
      +   * ordering columns) with b) existing [[targetTable]]
          */
         private def joinedExpectedOutput: Seq[Attribute] = {
           // NOTE: We're relying on [[sourceDataset]] here instead of [[mergeInto.sourceTable]],
      -    //       as it could be amended to add missing primary-key and/or precombine columns.
      +    //       as it could be amended to add missing primary-key and/or ordering columns.
           //       Please check [[sourceDataset]] scala-doc for more details
           (query.output ++ mergeInto.targetTable.output).filterNot(a => isMetaField(a.name))
         }
      @@ -763,7 +763,7 @@ case class MergeIntoHoodieTableCommand(mergeInto: MergeIntoTable,
           // NOTE: Here we fallback to "" to make sure that null value is not overridden with
           // default value ("ts")
           // TODO(HUDI-3456) clean up
      -    val preCombineFieldsAsString = String.join(",", hoodieCatalogTable.preCombineKeys)
      +    val orderingFieldsAsString = String.join(",", hoodieCatalogTable.orderingFields)
           val hiveSyncConfig = buildHiveSyncConfig(sparkSession, hoodieCatalogTable, tableConfig)
           // for pkless tables, we need to enable optimized merge
           val isPrimaryKeylessTable = !hasPrimaryKey()
      @@ -778,7 +778,7 @@ case class MergeIntoHoodieTableCommand(mergeInto: MergeIntoTable,
               tableConfig.getRecordMergeMode,
               tableConfig.getPayloadClass,
               tableConfig.getRecordMergeStrategyId,
      -        tableConfig.getPreCombineFieldsStr.orElse(null),
      +        tableConfig.getOrderingFieldsStr.orElse(null),
               tableConfig.getTableVersion)
             inferredMergeConfigs.getLeft.name()
           } else {
      @@ -787,7 +787,7 @@ case class MergeIntoHoodieTableCommand(mergeInto: MergeIntoTable,
           val overridingOpts = Map(
             "path" -> path,
             RECORDKEY_FIELD.key -> tableConfig.getRawRecordKeyFieldProp,
      -      PRECOMBINE_FIELD.key -> preCombineFieldsAsString,
      +      HoodieTableConfig.ORDERING_FIELDS.key -> orderingFieldsAsString,
             TBL_NAME.key -> hoodieCatalogTable.tableName,
             PARTITIONPATH_FIELD.key -> getPartitionPathFieldWriteConfig(
               tableConfig.getKeyGeneratorClassName, tableConfig.getPartitionFieldProp, hoodieCatalogTable),
      @@ -821,7 +821,7 @@ case class MergeIntoHoodieTableCommand(mergeInto: MergeIntoTable,
             HoodieSparkSqlWriter.SQL_MERGE_INTO_WRITES.key -> "true",
             // Only primary keyless table requires prepped keys and upsert
             HoodieWriteConfig.SPARK_SQL_MERGE_INTO_PREPPED_KEY -> isPrimaryKeylessTable.toString,
      -      HoodieWriteConfig.COMBINE_BEFORE_UPSERT.key() -> (!StringUtils.isNullOrEmpty(preCombineFieldsAsString)).toString
      +      HoodieWriteConfig.COMBINE_BEFORE_UPSERT.key() -> (!StringUtils.isNullOrEmpty(orderingFieldsAsString)).toString
           )
       
           combineOptions(hoodieCatalogTable, tableConfig, sparkSession.sqlContext.conf,
      @@ -845,7 +845,7 @@ case class MergeIntoHoodieTableCommand(mergeInto: MergeIntoTable,
             assert(insert.assignments.length <= targetTableSchema.length,
               s"The number of insert assignments[${insert.assignments.length}] must be less than or equal to the " +
                 s"targetTable field size[${targetTableSchema.length}]"))
      -    // Precombine field and record key field must be present in the assignment clause of all insert actions for event time ordering mode.
      +    // Ordering field and record key field must be present in the assignment clause of all insert actions for event time ordering mode.
           // Check has no effect if we don't have such fields in target table or we don't have insert actions
           // Please note we are relying on merge mode in the table config as writer merge mode is always "CUSTOM" for MIT.
           if (isEventTimeOrdering(props)) {
      @@ -853,8 +853,8 @@ case class MergeIntoHoodieTableCommand(mergeInto: MergeIntoTable,
               validateTargetTableAttrExistsInAssignments(
                 sparkSession.sessionState.conf.resolver,
                 mergeInto.targetTable,
      -          hoodieCatalogTable.preCombineKeys.asScala.toSeq,
      -          "precombine field",
      +          hoodieCatalogTable.orderingFields.asScala.toSeq,
      +          "ordering field",
                 action.assignments)
             )
           }
      @@ -881,7 +881,7 @@ case class MergeIntoHoodieTableCommand(mergeInto: MergeIntoTable,
           * The merge into schema compatibility requires data type matching for the following fields:
           * 1. Partition key
           * 2. Primary key
      -    * 3. Precombine key
      +    * 3. Ordering Fields
           *
           * @param assignments the assignment clause of the insert/update statement for figuring out
           *                    the mapping between the target table and the source table.
      @@ -923,10 +923,10 @@ case class MergeIntoHoodieTableCommand(mergeInto: MergeIntoTable,
                   sparkSession.sessionState.conf.resolver,
                   mergeInto.targetTable,
                   mergeInto.sourceTable,
      -            hoodieCatalogTable.preCombineKeys.asScala.toSeq,
      -            "precombine field",
      +            hoodieCatalogTable.orderingFields.asScala.toSeq,
      +            "ordering field",
                   assignments)
      -          associations.foreach(association => validateDataTypes(association._1, association._2, "Precombine field"))
      +          associations.foreach(association => validateDataTypes(association._1, association._2, "Ordering field"))
               } catch {
                 // Only catch AnalysisException from resolveFieldAssociationsBetweenSourceAndTarget
                 case _: MergeIntoFieldResolutionException =>
      @@ -936,7 +936,7 @@ case class MergeIntoHoodieTableCommand(mergeInto: MergeIntoTable,
         }
       
         private def checkUpdatingActions(updateActions: Seq[UpdateAction], props: Map[String, String]): Unit = {
      -    if (hoodieCatalogTable.preCombineKeys.isEmpty && updateActions.nonEmpty) {
      +    if (hoodieCatalogTable.orderingFields.isEmpty && updateActions.nonEmpty) {
             logWarning(s"Updates without precombine can have nondeterministic behavior")
           }
           updateActions.foreach(update =>
      diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/HoodieJavaApp.java b/hudi-spark-datasource/hudi-spark/src/test/java/HoodieJavaApp.java
      index 55a986a80c912..c250d15adcee2 100644
      --- a/hudi-spark-datasource/hudi-spark/src/test/java/HoodieJavaApp.java
      +++ b/hudi-spark-datasource/hudi-spark/src/test/java/HoodieJavaApp.java
      @@ -21,6 +21,7 @@
       import org.apache.hudi.HoodieDataSourceHelpers;
       import org.apache.hudi.common.model.HoodieRecord;
       import org.apache.hudi.common.model.HoodieTableType;
      +import org.apache.hudi.common.table.HoodieTableConfig;
       import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
       import org.apache.hudi.config.HoodieCompactionConfig;
       import org.apache.hudi.config.HoodieWriteConfig;
      @@ -155,7 +156,7 @@ public void run() throws Exception {
               // this is the partition to place it into
               .option(DataSourceWriteOptions.PARTITIONPATH_FIELD().key(), "partition")
               // use to combine duplicate records in input/with disk val
      -        .option(DataSourceWriteOptions.PRECOMBINE_FIELD().key(), "timestamp")
      +        .option(HoodieTableConfig.ORDERING_FIELDS.key(), "timestamp")
               // Used by hive sync and queries
               .option(HoodieWriteConfig.TBL_NAME.key(), tableName)
               .option(DataSourceWriteOptions.ASYNC_COMPACT_ENABLE().key(), "false")
      @@ -183,7 +184,7 @@ public void run() throws Exception {
               .option(DataSourceWriteOptions.TABLE_TYPE().key(), tableType) // Hoodie Table Type
               .option(DataSourceWriteOptions.RECORDKEY_FIELD().key(), "_row_key")
               .option(DataSourceWriteOptions.PARTITIONPATH_FIELD().key(), "partition")
      -        .option(DataSourceWriteOptions.PRECOMBINE_FIELD().key(), "timestamp")
      +        .option(HoodieTableConfig.ORDERING_FIELDS.key(), "timestamp")
               .option(HoodieCompactionConfig.INLINE_COMPACT_NUM_DELTA_COMMITS.key(), "1")
               .option(DataSourceWriteOptions.ASYNC_COMPACT_ENABLE().key(), "false")
               .option(DataSourceWriteOptions.ASYNC_CLUSTERING_ENABLE().key(), "true")
      @@ -208,7 +209,7 @@ public void run() throws Exception {
               .option(DataSourceWriteOptions.OPERATION().key(), "delete")
               .option(DataSourceWriteOptions.RECORDKEY_FIELD().key(), "_row_key")
               .option(DataSourceWriteOptions.PARTITIONPATH_FIELD().key(), "partition")
      -        .option(DataSourceWriteOptions.PRECOMBINE_FIELD().key(), "timestamp")
      +        .option(HoodieTableConfig.ORDERING_FIELDS.key(), "timestamp")
               .option(HoodieCompactionConfig.INLINE_COMPACT_NUM_DELTA_COMMITS.key(), "1")
               .option(DataSourceWriteOptions.ASYNC_COMPACT_ENABLE().key(), "false")
               .option(DataSourceWriteOptions.ASYNC_CLUSTERING_ENABLE().key(), "true")
      diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/HoodieJavaGenerateApp.java b/hudi-spark-datasource/hudi-spark/src/test/java/HoodieJavaGenerateApp.java
      index a41db540ec40f..0f325a0f2f743 100644
      --- a/hudi-spark-datasource/hudi-spark/src/test/java/HoodieJavaGenerateApp.java
      +++ b/hudi-spark-datasource/hudi-spark/src/test/java/HoodieJavaGenerateApp.java
      @@ -20,6 +20,7 @@
       import org.apache.hudi.HoodieDataSourceHelpers;
       import org.apache.hudi.common.model.HoodieRecord;
       import org.apache.hudi.common.model.HoodieTableType;
      +import org.apache.hudi.common.table.HoodieTableConfig;
       import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
       import org.apache.hudi.common.testutils.InProcessTimeGenerator;
       import org.apache.hudi.config.HoodieWriteConfig;
      @@ -183,7 +184,7 @@ private void insert(SparkSession spark) throws IOException {
               // this is the partition to place it into
               .option(DataSourceWriteOptions.PARTITIONPATH_FIELD().key(), "partition")
               // use to combine duplicate records in input/with disk val
      -        .option(DataSourceWriteOptions.PRECOMBINE_FIELD().key(), "timestamp")
      +        .option(HoodieTableConfig.ORDERING_FIELDS.key(), "timestamp")
               // Used by hive sync and queries
               .option(HoodieWriteConfig.TBL_NAME.key(), tableName)
               .mode(commitType);
      diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/HoodieJavaStreamingApp.java b/hudi-spark-datasource/hudi-spark/src/test/java/HoodieJavaStreamingApp.java
      index 54a64895c6de7..b18a47dcc443e 100644
      --- a/hudi-spark-datasource/hudi-spark/src/test/java/HoodieJavaStreamingApp.java
      +++ b/hudi-spark-datasource/hudi-spark/src/test/java/HoodieJavaStreamingApp.java
      @@ -20,6 +20,7 @@
       import org.apache.hudi.DataSourceWriteOptions;
       import org.apache.hudi.HoodieDataSourceHelpers;
       import org.apache.hudi.common.model.HoodieTableType;
      +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.testutils.HoodieTestDataGenerator;
      @@ -372,7 +373,7 @@ public void stream(Dataset streamingInput, String operationType, String che
               .option(DataSourceWriteOptions.TABLE_TYPE().key(), tableType)
               .option(DataSourceWriteOptions.RECORDKEY_FIELD().key(), "_row_key")
               .option(DataSourceWriteOptions.PARTITIONPATH_FIELD().key(), "partition")
      -        .option(DataSourceWriteOptions.PRECOMBINE_FIELD().key(), "timestamp")
      +        .option(HoodieTableConfig.ORDERING_FIELDS.key(), "timestamp")
               .option(HoodieCompactionConfig.INLINE_COMPACT_NUM_DELTA_COMMITS.key(), "1")
               .option(DataSourceWriteOptions.ASYNC_COMPACT_ENABLE().key(), "true")
               .option(DataSourceWriteOptions.ASYNC_CLUSTERING_ENABLE().key(), "true")
      diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/TestDecimalTypeDataWorkflow.scala b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/TestDecimalTypeDataWorkflow.scala
      index 8913e2a2df653..fee69a855c201 100644
      --- a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/TestDecimalTypeDataWorkflow.scala
      +++ b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/TestDecimalTypeDataWorkflow.scala
      @@ -21,6 +21,7 @@ package org.apache.hudi
       
       import org.apache.hudi.DataSourceWriteOptions._
       import org.apache.hudi.common.config.{HoodieReaderConfig, HoodieStorageConfig}
      +import org.apache.hudi.common.table.HoodieTableConfig
       import org.apache.hudi.config.HoodieWriteConfig
       import org.apache.hudi.testutils.SparkClientFunctionalTestHarness
       import org.apache.spark.sql.types.{Decimal, DecimalType, IntegerType, StructField, StructType}
      @@ -65,7 +66,7 @@ class TestDecimalTypeDataWorkflow extends SparkClientFunctionalTestHarness{
             .toDF("id", "decimal_col").sort("id")
           insertDf.write.format("hudi")
             .option(RECORDKEY_FIELD.key(), "id")
      -      .option(PRECOMBINE_FIELD.key(), "decimal_col")
      +      .option(HoodieTableConfig.ORDERING_FIELDS.key(), "decimal_col")
             .option(TABLE_TYPE.key, "MERGE_ON_READ")
             .option(TABLE_NAME.key, "test_table")
             .options(opts)
      diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/TestDefaultSparkRecordMerger.java b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/TestDefaultSparkRecordMerger.java
      index ccdcd1878055d..eaa31036a8726 100644
      --- a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/TestDefaultSparkRecordMerger.java
      +++ b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/TestDefaultSparkRecordMerger.java
      @@ -42,7 +42,7 @@
       import java.util.List;
       
       import static org.apache.hudi.common.model.HoodieRecord.HoodieRecordType.SPARK;
      -import static org.apache.hudi.common.table.HoodieTableConfig.PRECOMBINE_FIELDS;
      +import static org.apache.hudi.common.table.HoodieTableConfig.ORDERING_FIELDS;
       import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA;
       import static org.junit.jupiter.api.Assertions.assertEquals;
       import static org.junit.jupiter.api.Assertions.assertThrows;
      @@ -100,7 +100,7 @@ void testMergerWithNewRecordAccepted() throws IOException {
       
           DefaultSparkRecordMerger merger = new DefaultSparkRecordMerger();
           TypedProperties props = new TypedProperties();
      -    props.setProperty(PRECOMBINE_FIELDS.key(), INT_COLUMN_NAME);
      +    props.setProperty(ORDERING_FIELDS.key(), INT_COLUMN_NAME);
           Schema avroSchema = AvroConversionUtils.convertStructTypeToAvroSchema(
               SPARK_SCHEMA, ANY_NAME, ANY_NAMESPACE);
           Option> merged =
      @@ -127,7 +127,7 @@ void testMergerWithOldRecordAccepted() throws IOException {
       
           DefaultSparkRecordMerger merger = new DefaultSparkRecordMerger();
           TypedProperties props = new TypedProperties();
      -    props.setProperty(PRECOMBINE_FIELDS.key(), INT_COLUMN_NAME);
      +    props.setProperty(ORDERING_FIELDS.key(), INT_COLUMN_NAME);
           Schema avroSchema = AvroConversionUtils.convertStructTypeToAvroSchema(
               SPARK_SCHEMA, ANY_NAME, ANY_NAMESPACE);
           Option> r =
      @@ -151,7 +151,7 @@ void testMergerWithNewRecordAsDelete() throws IOException {
       
           DefaultSparkRecordMerger merger = new DefaultSparkRecordMerger();
           TypedProperties props = new TypedProperties();
      -    props.setProperty(PRECOMBINE_FIELDS.key(), INT_COLUMN_NAME);
      +    props.setProperty(ORDERING_FIELDS.key(), INT_COLUMN_NAME);
           Schema avroSchema = AvroConversionUtils.convertStructTypeToAvroSchema(
               SPARK_SCHEMA, ANY_NAME, ANY_NAMESPACE);
           Option> r =
      @@ -172,7 +172,7 @@ void testMergerWithOldRecordAsDelete() throws IOException {
       
           DefaultSparkRecordMerger merger = new DefaultSparkRecordMerger();
           TypedProperties props = new TypedProperties();
      -    props.setProperty(PRECOMBINE_FIELDS.key(), INT_COLUMN_NAME);
      +    props.setProperty(ORDERING_FIELDS.key(), INT_COLUMN_NAME);
           Schema avroSchema = AvroConversionUtils.convertStructTypeToAvroSchema(
               SPARK_SCHEMA, ANY_NAME, ANY_NAMESPACE);
           Option> r =
      diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/TestHoodieMergeHandleWithSparkMerger.java b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/TestHoodieMergeHandleWithSparkMerger.java
      index 1b15f52654268..4cac28119ae1f 100644
      --- a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/TestHoodieMergeHandleWithSparkMerger.java
      +++ b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/TestHoodieMergeHandleWithSparkMerger.java
      @@ -88,7 +88,7 @@ public void setUp() throws IOException {
           properties.put(
               HoodieTableConfig.BASE_FILE_FORMAT.key(),
               HoodieTableConfig.BASE_FILE_FORMAT.defaultValue().toString());
      -    properties.put(HoodieTableConfig.PRECOMBINE_FIELDS.key(), "record_key");
      +    properties.put(HoodieTableConfig.ORDERING_FIELDS.key(), "record_key");
           properties.put(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key(),"partition_path");
           properties.put(HoodieTableConfig.PARTITION_FIELDS.key(), "partition_path");
         }
      diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/TestPositionBasedFileGroupRecordBuffer.java b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/TestPositionBasedFileGroupRecordBuffer.java
      index 44c13eed32b99..0065fc71c4ec7 100644
      --- a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/TestPositionBasedFileGroupRecordBuffer.java
      +++ b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/TestPositionBasedFileGroupRecordBuffer.java
      @@ -95,7 +95,7 @@ private void prepareBuffer(RecordMergeMode mergeMode, String baseFileInstantTime
           writeConfigs.put(HoodieStorageConfig.LOGFILE_DATA_BLOCK_FORMAT.key(), "parquet");
           writeConfigs.put(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key(), "_row_key");
           writeConfigs.put(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key(), "partition_path");
      -    writeConfigs.put("hoodie.datasource.write.precombine.field", mergeMode.equals(RecordMergeMode.COMMIT_TIME_ORDERING) ? "" : "timestamp");
      +    writeConfigs.put(HoodieTableConfig.ORDERING_FIELDS.key(), mergeMode.equals(RecordMergeMode.COMMIT_TIME_ORDERING) ? "" : "timestamp");
           writeConfigs.put("hoodie.payload.ordering.field", "timestamp");
           writeConfigs.put(HoodieTableConfig.HOODIE_TABLE_NAME_KEY, "hoodie_test");
           writeConfigs.put("hoodie.insert.shuffle.parallelism", "4");
      diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/client/functional/TestMetadataUtilRLIandSIRecordGeneration.java b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/client/functional/TestMetadataUtilRLIandSIRecordGeneration.java
      index 3f2ffcb36704b..cd8b2c9cdc274 100644
      --- a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/client/functional/TestMetadataUtilRLIandSIRecordGeneration.java
      +++ b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/client/functional/TestMetadataUtilRLIandSIRecordGeneration.java
      @@ -209,7 +209,7 @@ public void testRecordGenerationAPIsForMOR() throws IOException {
           HoodieTableType tableType = HoodieTableType.MERGE_ON_READ;
           cleanupClients();
           Properties props = new Properties();
      -    props.put(HoodieTableConfig.PRECOMBINE_FIELDS.key(), "timestamp");
      +    props.put(HoodieTableConfig.ORDERING_FIELDS.key(), "timestamp");
           initMetaClient(tableType, props);
           cleanupTimelineService();
           initTimelineService();
      @@ -307,7 +307,7 @@ public void testSecondaryIndexRecordGenerationForMOR() throws IOException {
           HoodieTableType tableType = HoodieTableType.MERGE_ON_READ;
           cleanupClients();
           Properties props = new Properties();
      -    props.put(HoodieTableConfig.PRECOMBINE_FIELDS.key(), "timestamp");
      +    props.put(HoodieTableConfig.ORDERING_FIELDS.key(), "timestamp");
           initMetaClient(tableType, props);
           cleanupTimelineService();
           initTimelineService();
      diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestBootstrap.java b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestBootstrap.java
      index e833919c30ac4..50c8c551f161a 100644
      --- a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestBootstrap.java
      +++ b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestBootstrap.java
      @@ -35,6 +35,7 @@
       import org.apache.hudi.common.model.HoodieKey;
       import org.apache.hudi.common.model.HoodieRecord;
       import org.apache.hudi.common.model.HoodieTableType;
      +import org.apache.hudi.common.table.HoodieTableConfig;
       import org.apache.hudi.common.table.timeline.HoodieInstant.State;
       import org.apache.hudi.common.table.timeline.HoodieTimeline;
       import org.apache.hudi.common.table.timeline.TimelineUtils;
      @@ -255,7 +256,6 @@ private void testBootstrapCommon(boolean partitioned, boolean deltaCommit, Effec
           long timestamp = Instant.now().toEpochMilli();
           Schema schema = generateNewDataSetAndReturnSchema(timestamp, totalRecords, partitions, bootstrapBasePath);
           HoodieWriteConfig config = getConfigBuilder(schema.toString())
      -        .withPreCombineField("timestamp")
               .withSchema(schema.toString())
               .withKeyGenerator(keyGeneratorClass)
               .withCompactionConfig(HoodieCompactionConfig.newBuilder()
      @@ -270,6 +270,7 @@ private void testBootstrapCommon(boolean partitioned, boolean deltaCommit, Effec
               .withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(true).withMaxNumDeltaCommitsBeforeCompaction(3)
                   .withMetadataIndexColumnStats(false).build()) // HUDI-8774
               .build();
      +    config.setValue(HoodieTableConfig.ORDERING_FIELDS, "timestamp");
       
           SparkRDDWriteClient client = new SparkRDDWriteClient(context, config);
           client.bootstrap(Option.empty());
      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 e32ddc698a398..33011276c9bdd 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
      @@ -25,6 +25,7 @@
       import org.apache.hudi.client.bootstrap.translator.DecodedBootstrapPartitionPathTranslator;
       import org.apache.hudi.common.config.HoodieMetadataConfig;
       import org.apache.hudi.common.model.HoodieTableType;
      +import org.apache.hudi.common.table.HoodieTableConfig;
       import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
       import org.apache.hudi.config.HoodieBootstrapConfig;
       import org.apache.hudi.config.HoodieCompactionConfig;
      @@ -105,7 +106,7 @@ protected Map basicOptions() {
               options.put(HoodieWriteConfig.KEYGENERATOR_CLASS_NAME.key(), ComplexKeyGenerator.class.getName());
             }
           }
      -    options.put(DataSourceWriteOptions.PRECOMBINE_FIELD().key(), "timestamp");
      +    options.put(HoodieTableConfig.ORDERING_FIELDS.key(), "timestamp");
           if (tableType.equals(MERGE_ON_READ)) {
             options.put(HoodieCompactionConfig.INLINE_COMPACT.key(), "true");
           }
      diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestBufferedRecordMerger.java b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestBufferedRecordMerger.java
      index cddfb1f6d4b48..a0c90bd802ba6 100644
      --- a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestBufferedRecordMerger.java
      +++ b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestBufferedRecordMerger.java
      @@ -76,7 +76,7 @@
       import static org.apache.hudi.common.config.RecordMergeMode.COMMIT_TIME_ORDERING;
       import static org.apache.hudi.common.config.RecordMergeMode.EVENT_TIME_ORDERING;
       import static org.apache.hudi.common.table.HoodieTableConfig.PARTIAL_UPDATE_UNAVAILABLE_VALUE;
      -import static org.apache.hudi.common.table.HoodieTableConfig.PRECOMBINE_FIELDS;
      +import static org.apache.hudi.common.table.HoodieTableConfig.ORDERING_FIELDS;
       import static org.junit.jupiter.api.Assertions.assertEquals;
       import static org.junit.jupiter.api.Assertions.assertFalse;
       import static org.junit.jupiter.api.Assertions.assertNotNull;
      @@ -113,7 +113,7 @@ void setUp() throws IOException {
           when(tableConfig.getRecordKeyFields()).thenReturn(Option.of(new String[]{"id"}));
           // Create reader context.
           props = new TypedProperties();
      -    props.put(PRECOMBINE_FIELDS.key(), "precombine");
      +    props.put(ORDERING_FIELDS.key(), "precombine");
           readerContext = new DummyInternalRowReaderContext(
               storageConfig, tableConfig, Option.empty(), Option.empty(), new DummyRecordContext(tableConfig));
         }
      diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestColStatsRecordWithMetadataRecord.java b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestColStatsRecordWithMetadataRecord.java
      index 967508927e430..43bae063ba0ac 100644
      --- a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestColStatsRecordWithMetadataRecord.java
      +++ b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestColStatsRecordWithMetadataRecord.java
      @@ -220,7 +220,7 @@ public int compare(HoodieRecord o1, HoodieRecord mdtRec = SpillableMapUtils.convertToHoodieRecordPayload(genericRecord,
      -          mdtWriteConfig.getPayloadClass(), mdtWriteConfig.getPreCombineFields().toArray(new String[0]),
      +          mdtWriteConfig.getPayloadClass(), new String[0],
                 Pair.of(mdtMetaClient.getTableConfig().getRecordKeyFieldProp(), mdtMetaClient.getTableConfig().getPartitionFieldProp()),
                 false, Option.of(COLUMN_STATS.getPartitionPath()), Option.empty());
             allRecords.add(mdtRec);
      diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestDataSkippingWithMORColstats.java b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestDataSkippingWithMORColstats.java
      index beb7141470904..e16ab9fb96c51 100644
      --- a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestDataSkippingWithMORColstats.java
      +++ b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestDataSkippingWithMORColstats.java
      @@ -25,6 +25,7 @@
       import org.apache.hudi.common.config.HoodieMetadataConfig;
       import org.apache.hudi.common.fs.FSUtils;
       import org.apache.hudi.common.model.HoodieTableType;
      +import org.apache.hudi.common.table.HoodieTableConfig;
       import org.apache.hudi.common.table.HoodieTableMetaClient;
       import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
       import org.apache.hudi.common.util.Option;
      @@ -333,7 +334,7 @@ private Map getOptions() {
           options.put(DataSourceReadOptions.ENABLE_DATA_SKIPPING().key(), "true");
           options.put(DataSourceWriteOptions.TABLE_TYPE().key(), DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL());
           options.put(HoodieWriteConfig.TBL_NAME.key(), "testTable");
      -    options.put(DataSourceWriteOptions.PRECOMBINE_FIELD().key(), "timestamp");
      +    options.put(HoodieTableConfig.ORDERING_FIELDS.key(), "timestamp");
           options.put(DataSourceWriteOptions.RECORDKEY_FIELD().key(), "_row_key");
           options.put("hoodie.datasource.write.keygenerator.class", "org.apache.hudi.keygen.NonpartitionedKeyGenerator");
           options.put(HoodieCompactionConfig.PARQUET_SMALL_FILE_LIMIT.key(), "0");
      diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestHiveTableSchemaEvolution.java b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestHiveTableSchemaEvolution.java
      index ecc0a2afbf5c4..d6cf641cfe369 100644
      --- a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestHiveTableSchemaEvolution.java
      +++ b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestHiveTableSchemaEvolution.java
      @@ -102,7 +102,7 @@ public void testHiveReadTimestampColumnAsTimestampWritable() throws Exception {
           spark.sql("set hoodie.datasource.write.schema.allow.auto.evolution.column.drop=true");
       
           spark.sql(String.format("create table %s (col0 int, col1 float, col2 string, col3 timestamp) using hudi "
      -            + "tblproperties (type='mor', primaryKey='col0', preCombineField='col1', "
      +            + "tblproperties (type='mor', primaryKey='col0', orderingFields='col1', "
                   + "hoodie.compaction.payload.class='org.apache.hudi.common.model.OverwriteWithLatestAvroPayload') location '%s'",
               tableName, path));
           spark.sql(String.format("insert into %s values(1, 1.1, 'text', timestamp('2021-12-25 12:01:01'))", tableName));
      @@ -148,7 +148,7 @@ public void testHiveReadSchemaEvolutionTable(String tableType) throws Exception
           spark.sql("set hoodie.schema.on.read.enable=true");
           spark.sql("set hoodie.datasource.write.schema.allow.auto.evolution.column.drop=true");
           spark.sql(String.format("create table %s (col0 int, col1 float, col2 string) using hudi "
      -            + "tblproperties (type='%s', primaryKey='col0', preCombineField='col1', "
      +            + "tblproperties (type='%s', primaryKey='col0', orderingFields='col1', "
                   + "hoodie.compaction.payload.class='org.apache.hudi.common.model.OverwriteWithLatestAvroPayload') location '%s'",
               tableName, tableType, path));
           spark.sql(String.format("insert into %s values(1, 1.1, 'text')", tableName));
      @@ -212,7 +212,7 @@ public void testHiveReadSchemaEvolutionWithAddingColumns(String tableType) throw
           spark.sql("set hoodie.schema.on.read.enable=true");
       
           spark.sql(String.format("create table %s (col0 int, col1 float, col2 string, col3 timestamp) using hudi "
      -                    + "tblproperties (type='%s', primaryKey='col0', preCombineField='col1', "
      +                    + "tblproperties (type='%s', primaryKey='col0', orderingFields='col1', "
                           + "hoodie.compaction.payload.class='org.apache.hudi.common.model.OverwriteWithLatestAvroPayload') location '%s'",
                   tableName, tableType, path));
           spark.sql(String.format("insert into %s values(1, 1.1, 'text', timestamp('2021-12-25 12:01:01'))", tableName));
      diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestHoodieDatasetBulkInsertHelper.java b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestHoodieDatasetBulkInsertHelper.java
      index 21b1a9be870ea..6f08491f61702 100644
      --- a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestHoodieDatasetBulkInsertHelper.java
      +++ b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestHoodieDatasetBulkInsertHelper.java
      @@ -23,6 +23,7 @@
       import org.apache.hudi.SparkAdapterSupport$;
       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.FileIOUtils;
       import org.apache.hudi.config.HoodieWriteConfig;
       import org.apache.hudi.execution.bulkinsert.NonSortPartitionerWithRows;
      @@ -132,7 +133,7 @@ private void testBulkInsertHelperFor(String keyGenClass, String recordKeyField)
           List rows = DataSourceTestUtils.generateRandomRows(10);
           Dataset dataset = sqlContext.createDataFrame(rows, structType);
           Dataset result = HoodieDatasetBulkInsertHelper.prepareForBulkInsert(dataset, config,
      -        new NonSortPartitionerWithRows(), "0000000001");
      +        new HoodieTableConfig(), new NonSortPartitionerWithRows(), "0000000001");
           StructType resultSchema = result.schema();
       
           assertEquals(result.count(), 10);
      @@ -176,7 +177,7 @@ public void testBulkInsertHelperNoMetaFields() {
               .build();
           Dataset dataset = sqlContext.createDataFrame(rows, structType);
           Dataset result = HoodieDatasetBulkInsertHelper.prepareForBulkInsert(dataset, config,
      -        new NonSortPartitionerWithRows(), "000001111");
      +        new HoodieTableConfig(), new NonSortPartitionerWithRows(), "000001111");
           StructType resultSchema = result.schema();
       
           assertEquals(result.count(), 10);
      @@ -204,7 +205,10 @@ public void testBulkInsertHelperNoMetaFields() {
         public void testBulkInsertPreCombine(boolean enablePreCombine) {
           HoodieWriteConfig config = getConfigBuilder(schemaStr).withProps(getPropsAllSet("_row_key"))
                   .combineInput(enablePreCombine, enablePreCombine)
      -            .withPreCombineField("ts").build();
      +            .build();
      +    config.setValue(HoodieTableConfig.ORDERING_FIELDS, "ts");
      +    HoodieTableConfig tableConfig = new HoodieTableConfig();
      +    tableConfig.setValue(HoodieTableConfig.ORDERING_FIELDS, "ts");
           List inserts = DataSourceTestUtils.generateRandomRows(10);
           Dataset toUpdateDataset = sqlContext.createDataFrame(inserts.subList(0, 5), structType);
           List updates = DataSourceTestUtils.updateRowsWithUpdatedTs(toUpdateDataset);
      @@ -213,7 +217,7 @@ public void testBulkInsertPreCombine(boolean enablePreCombine) {
           rows.addAll(updates);
           Dataset dataset = sqlContext.createDataFrame(rows, structType);
           Dataset result = HoodieDatasetBulkInsertHelper.prepareForBulkInsert(dataset, config,
      -        new NonSortPartitionerWithRows(), "000001111");
      +        tableConfig, new NonSortPartitionerWithRows(), "000001111");
           StructType resultSchema = result.schema();
       
           assertEquals(result.count(), enablePreCombine ? 10 : 15);
      @@ -317,7 +321,7 @@ public void testNoPropsSet() {
           Dataset dataset = sqlContext.createDataFrame(rows, structType);
           try {
             Dataset preparedDF = HoodieDatasetBulkInsertHelper.prepareForBulkInsert(dataset, config,
      -          new NonSortPartitionerWithRows(), "000001111");
      +          new HoodieTableConfig(), new NonSortPartitionerWithRows(), "000001111");
             preparedDF.count();
             fail("Should have thrown exception");
           } catch (Exception e) {
      @@ -329,7 +333,7 @@ public void testNoPropsSet() {
           dataset = sqlContext.createDataFrame(rows, structType);
           try {
             Dataset preparedDF = HoodieDatasetBulkInsertHelper.prepareForBulkInsert(dataset, config,
      -          new NonSortPartitionerWithRows(), "000001111");
      +          new HoodieTableConfig(), new NonSortPartitionerWithRows(), "000001111");
             preparedDF.count();
             fail("Should have thrown exception");
           } catch (Exception e) {
      @@ -341,7 +345,7 @@ public void testNoPropsSet() {
           dataset = sqlContext.createDataFrame(rows, structType);
           try {
             Dataset preparedDF = HoodieDatasetBulkInsertHelper.prepareForBulkInsert(dataset, config,
      -          new NonSortPartitionerWithRows(), "000001111");
      +          new HoodieTableConfig(), new NonSortPartitionerWithRows(), "000001111");
             preparedDF.count();
             fail("Should have thrown exception");
           } catch (Exception e) {
      @@ -357,7 +361,10 @@ private ExpressionEncoder getEncoder(StructType schema) {
         public void testBulkInsertParallelismParam() {
           HoodieWriteConfig config = getConfigBuilder(schemaStr).withProps(getPropsAllSet("_row_key"))
               .combineInput(true, true)
      -        .withPreCombineField("ts").build();
      +        .build();
      +    config.setValue(HoodieTableConfig.ORDERING_FIELDS, "ts");
      +    HoodieTableConfig tableConfig = new HoodieTableConfig();
      +    tableConfig.setValue(HoodieTableConfig.ORDERING_FIELDS, "ts");
           int checkParallelism = 7;
           config.setValue("hoodie.bulkinsert.shuffle.parallelism", String.valueOf(checkParallelism));
           StageCheckBulkParallelismListener stageCheckBulkParallelismListener =
      @@ -368,7 +375,7 @@ public void testBulkInsertParallelismParam() {
           assertNotEquals(checkParallelism, HoodieUnsafeUtils.getNumPartitions(dataset));
           assertNotEquals(checkParallelism, sqlContext.sparkContext().defaultParallelism());
           Dataset result = HoodieDatasetBulkInsertHelper.prepareForBulkInsert(dataset, config,
      -        new NonSortPartitionerWithRows(), "000001111");
      +        tableConfig, new NonSortPartitionerWithRows(), "000001111");
           // trigger job
           result.count();
           assertEquals(checkParallelism, stageCheckBulkParallelismListener.getParallelism());
      diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestHoodieIndex.java b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestHoodieIndex.java
      index c0fc5846f4b25..4b343e33051cf 100644
      --- a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestHoodieIndex.java
      +++ b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestHoodieIndex.java
      @@ -182,9 +182,9 @@ private Properties getPropsForKeyGen(IndexType indexType, boolean populateMetaFi
             properties.put(HoodieTableConfig.RECORDKEY_FIELDS.key(), "_row_key");
             properties.put("hoodie.datasource.write.keygenerator.class", RawTripTestPayloadKeyGenerator.class.getName());
             properties.put("hoodie.datasource.write.partitionpath.field", "time");
      -      properties.put("hoodie.datasource.write.precombine.field", "number");
      +      properties.put(HoodieTableConfig.ORDERING_FIELDS.key(), "number");
             properties.put(HoodieTableConfig.PARTITION_FIELDS.key(), "time");
      -      properties.put(HoodieTableConfig.PRECOMBINE_FIELDS.key(), "number");
      +      properties.put(HoodieTableConfig.ORDERING_FIELDS.key(), "number");
           }
           return properties;
         }
      diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/io/TestMergeHandle.java b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/io/TestMergeHandle.java
      index 1ef3bce83e3c4..db9481e436199 100644
      --- a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/io/TestMergeHandle.java
      +++ b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/io/TestMergeHandle.java
      @@ -292,7 +292,7 @@ private void testFGReaderBasedMergeHandleInsertUpsertDeleteInternal(String merge
       
           properties.put(HoodieTableConfig.RECORDKEY_FIELDS.key(), "_row_key");
           properties.put(HoodieTableConfig.PARTITION_FIELDS.key(), "partition_path");
      -    properties.put(HoodieTableConfig.PRECOMBINE_FIELDS.key(), ORDERING_FIELD);
      +    properties.put(HoodieTableConfig.ORDERING_FIELDS.key(), ORDERING_FIELD);
           properties.put(HoodieTableConfig.RECORD_MERGE_MODE.key(), mergeMode);
           if (mergeMode.equals("CUSTOM_MERGER")) {
             config.setValue(HoodieWriteConfig.RECORD_MERGE_IMPL_CLASSES, CustomMerger.class.getName());
      diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/table/functional/TestHoodieSparkMergeOnReadTableInsertUpdateDelete.java b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/table/functional/TestHoodieSparkMergeOnReadTableInsertUpdateDelete.java
      index 0ed059fe20fb8..f8f67c14f80dc 100644
      --- a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/table/functional/TestHoodieSparkMergeOnReadTableInsertUpdateDelete.java
      +++ b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/table/functional/TestHoodieSparkMergeOnReadTableInsertUpdateDelete.java
      @@ -270,7 +270,7 @@ public void testRepeatedRollbackOfCompaction() throws Exception {
         @ValueSource(booleans = {true, false})
         public void testSimpleInsertUpdateAndDelete(boolean populateMetaFields) throws Exception {
           Properties properties = populateMetaFields ? new Properties() : getPropertiesForKeyGen();
      -    properties.setProperty(HoodieTableConfig.PRECOMBINE_FIELDS.key(), "timestamp");
      +    properties.setProperty(HoodieTableConfig.ORDERING_FIELDS.key(), "timestamp");
           properties.setProperty(HoodieTableConfig.BASE_FILE_FORMAT.key(), HoodieTableConfig.BASE_FILE_FORMAT.defaultValue().toString());
           HoodieTableMetaClient metaClient = getHoodieMetaClient(HoodieTableType.MERGE_ON_READ, properties);
       
      diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/table/functional/TestSparkNonBlockingConcurrencyControl.java b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/table/functional/TestSparkNonBlockingConcurrencyControl.java
      index 855c6510f97d0..035e33480be84 100644
      --- a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/table/functional/TestSparkNonBlockingConcurrencyControl.java
      +++ b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/table/functional/TestSparkNonBlockingConcurrencyControl.java
      @@ -617,7 +617,7 @@ private HoodieWriteConfig createHoodieWriteConfig(boolean fullUpdate) {
           props.put(ENABLE_SCHEMA_CONFLICT_RESOLUTION.key(), "false");
           String basePath = basePath();
           return HoodieWriteConfig.newBuilder()
      -        .withProps(Collections.singletonMap(HoodieTableConfig.PRECOMBINE_FIELDS.key(), "ts"))
      +        .withProps(Collections.singletonMap(HoodieTableConfig.ORDERING_FIELDS.key(), "ts"))
               .forTable("test")
               .withPath(basePath)
               .withSchema(jsonSchema)
      diff --git a/hudi-spark-datasource/hudi-spark/src/test/resources/sql-statements.sql b/hudi-spark-datasource/hudi-spark/src/test/resources/sql-statements.sql
      index 4aacb206a591e..c9ae556513de5 100644
      --- a/hudi-spark-datasource/hudi-spark/src/test/resources/sql-statements.sql
      +++ b/hudi-spark-datasource/hudi-spark/src/test/resources/sql-statements.sql
      @@ -71,7 +71,7 @@ create table h1 (
       options (
         type = '${tableType}',
         primaryKey = 'id',
      -  preCombineField = 'ts'
      +  orderingFields = 'ts'
       )
       location '${tmpDir}/h1';
       +----------+
      @@ -89,7 +89,7 @@ partitioned by (dt)
       options (
         type = '${tableType}',
         primaryKey = 'id',
      -  preCombineField = 'ts'
      +  orderingFields = 'ts'
       )
       location '${tmpDir}/h1_p';
       +----------+
      diff --git a/hudi-spark-datasource/hudi-spark/src/test/resources/upgrade-downgrade-fixtures/scala-templates/generate-fixture.scala b/hudi-spark-datasource/hudi-spark/src/test/resources/upgrade-downgrade-fixtures/scala-templates/generate-fixture.scala
      index fbefed9e05dfc..46f4eb43ced42 100644
      --- a/hudi-spark-datasource/hudi-spark/src/test/resources/upgrade-downgrade-fixtures/scala-templates/generate-fixture.scala
      +++ b/hudi-spark-datasource/hudi-spark/src/test/resources/upgrade-downgrade-fixtures/scala-templates/generate-fixture.scala
      @@ -16,6 +16,7 @@
        */
       
       import org.apache.spark.sql.SaveMode
      +import org.apache.hudi.common.table.HoodieTableConfig
       import org.apache.hudi.DataSourceWriteOptions._
       import spark.implicits._
       
      @@ -35,7 +36,7 @@ val df = testData.toDF("id", "name", "ts", "partition")
       
       // Write initial batch (creates base files)
       df.write.format("hudi").
      -  option(PRECOMBINE_FIELD.key, "ts").
      +  option(HoodieTableConfig.ORDERING_FIELDS.key, "ts").
         option(RECORDKEY_FIELD.key, "id").
         option(PARTITIONPATH_FIELD.key, "partition").
         option("hoodie.table.name", tableName).
      @@ -61,7 +62,7 @@ val updateData = Seq(
       val updateDf = updateData.toDF("id", "name", "ts", "partition")
       
       updateDf.write.format("hudi").
      -  option(PRECOMBINE_FIELD.key, "ts").
      +  option(HoodieTableConfig.ORDERING_FIELDS.key, "ts").
         option(RECORDKEY_FIELD.key, "id").
         option(PARTITIONPATH_FIELD.key, "partition").
         option("hoodie.table.name", tableName).
      @@ -86,7 +87,7 @@ val insertData = Seq(
       val insertDf = insertData.toDF("id", "name", "ts", "partition")
       
       insertDf.write.format("hudi").
      -  option(PRECOMBINE_FIELD.key, "ts").
      +  option(HoodieTableConfig.ORDERING_FIELDS.key, "ts").
         option(RECORDKEY_FIELD.key, "id").
         option(PARTITIONPATH_FIELD.key, "partition").
         option("hoodie.table.name", tableName).
      diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestAvroSchemaResolutionSupport.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestAvroSchemaResolutionSupport.scala
      index 2929534ffa745..7b8f9b737e0f1 100644
      --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestAvroSchemaResolutionSupport.scala
      +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestAvroSchemaResolutionSupport.scala
      @@ -20,6 +20,7 @@ package org.apache.hudi
       
       import org.apache.hudi.common.config.{HoodieMetadataConfig, HoodieReaderConfig}
       import org.apache.hudi.common.model.HoodieTableType
      +import org.apache.hudi.common.table.HoodieTableConfig
       import org.apache.hudi.config.HoodieWriteConfig
       import org.apache.hudi.exception.SchemaCompatibilityException
       import org.apache.hudi.testutils.HoodieClientTestBase
      @@ -45,7 +46,7 @@ class TestAvroSchemaResolutionSupport extends HoodieClientTestBase with ScalaAss
           "hoodie.insert.shuffle.parallelism" -> "1",
           "hoodie.upsert.shuffle.parallelism" -> "1",
           DataSourceWriteOptions.RECORDKEY_FIELD.key -> "id",
      -    DataSourceWriteOptions.PRECOMBINE_FIELD.key -> "id",
      +    HoodieTableConfig.ORDERING_FIELDS.key() -> "id",
           DataSourceWriteOptions.PARTITIONPATH_FIELD.key -> "name",
           DataSourceWriteOptions.KEYGENERATOR_CLASS_NAME.key -> "org.apache.hudi.keygen.SimpleKeyGenerator",
           HoodieMetadataConfig.ENABLE.key -> "false"
      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 f8b265a96464f..0abbf94c7eb33 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
      @@ -20,7 +20,7 @@ package org.apache.hudi
       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.{PreCombineTestUtils, SchemaTestUtil}
      +import org.apache.hudi.common.testutils.{OrderingFieldsTestUtils, SchemaTestUtil}
       import org.apache.hudi.common.util.Option
       import org.apache.hudi.common.util.PartitionPathEncodeUtils.DEFAULT_PARTITION_PATH
       import org.apache.hudi.config.HoodiePayloadConfig
      @@ -556,10 +556,10 @@ class TestDataSourceDefaults extends ScalaAssertionSupport {
         }
       
         @ParameterizedTest
      -  @MethodSource(Array("org.apache.hudi.common.testutils.PreCombineTestUtils#configurePreCombine"))
      +  @MethodSource(Array("org.apache.hudi.common.testutils.OrderingFieldsTestUtils#configureOrderingFields"))
         def testOverwriteWithLatestAvroPayloadCombineAndGetUpdateValue(key: String): Unit = {
           val props = new TypedProperties()
      -    PreCombineTestUtils.setPreCombineConfig(props, key, "favoriteIntNumber")
      +    OrderingFieldsTestUtils.setOrderingFieldsConfig(props, key, "favoriteIntNumber")
           val baseOrderingVal: Object = baseRecord.get("favoriteIntNumber")
           val fieldSchema: Schema = baseRecord.getSchema().getField("favoriteIntNumber").schema()
       
      diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestGenericRecordAndRowConsistency.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestGenericRecordAndRowConsistency.scala
      index 54ccfd0fd4b19..f7e132e23636f 100644
      --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestGenericRecordAndRowConsistency.scala
      +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestGenericRecordAndRowConsistency.scala
      @@ -17,6 +17,7 @@
        */
       package org.apache.hudi
       
      +import org.apache.hudi.common.table.HoodieTableConfig
       import org.apache.hudi.config.HoodieWriteConfig
       import org.apache.hudi.testutils.HoodieSparkClientTestBase
       
      @@ -35,7 +36,7 @@ class TestGenericRecordAndRowConsistency extends HoodieSparkClientTestBase {
           "hoodie.upsert.shuffle.parallelism" -> "1",
           DataSourceWriteOptions.TABLE_TYPE.key -> "COPY_ON_WRITE",
           DataSourceWriteOptions.RECORDKEY_FIELD.key -> "str,eventTime",
      -    DataSourceWriteOptions.PRECOMBINE_FIELD.key -> "typeId",
      +    HoodieTableConfig.ORDERING_FIELDS.key() -> "typeId",
           DataSourceWriteOptions.PARTITIONPATH_FIELD.key -> "typeId",
           DataSourceWriteOptions.KEYGENERATOR_CLASS_NAME.key -> "org.apache.hudi.keygen.ComplexKeyGenerator",
           DataSourceWriteOptions.KEYGENERATOR_CONSISTENT_LOGICAL_TIMESTAMP_ENABLED.key -> "true"
      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 1071cdd8438a7..7fc9b07185d87 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
      @@ -76,7 +76,7 @@ class TestHoodieFileIndex extends HoodieSparkClientTestBase with ScalaAssertionS
           "hoodie.upsert.shuffle.parallelism" -> "4",
           DataSourceWriteOptions.RECORDKEY_FIELD.key -> "_row_key",
           DataSourceWriteOptions.PARTITIONPATH_FIELD.key -> "partition",
      -    DataSourceWriteOptions.PRECOMBINE_FIELD.key -> "timestamp",
      +    HoodieTableConfig.ORDERING_FIELDS.key() -> "timestamp",
           HoodieWriteConfig.TBL_NAME.key -> "hoodie_test"
         )
       
      @@ -207,7 +207,7 @@ class TestHoodieFileIndex extends HoodieSparkClientTestBase with ScalaAssertionS
             "hoodie.upsert.shuffle.parallelism" -> "4",
             DataSourceWriteOptions.RECORDKEY_FIELD.key -> "_row_key",
             DataSourceWriteOptions.PARTITIONPATH_FIELD.key -> "partition_path",
      -      DataSourceWriteOptions.PRECOMBINE_FIELD.key -> "timestamp",
      +      HoodieTableConfig.ORDERING_FIELDS.key() -> "timestamp",
             HoodieWriteConfig.TBL_NAME.key -> "hoodie_test",
             DataSourceWriteOptions.OPERATION.key -> DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL
           )
      @@ -353,7 +353,7 @@ class TestHoodieFileIndex extends HoodieSparkClientTestBase with ScalaAssertionS
           val writerOpts: Map[String, String] = commonOpts ++ Map(
             DataSourceWriteOptions.OPERATION.key -> DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL,
             RECORDKEY_FIELD.key -> "id",
      -      PRECOMBINE_FIELD.key -> "version",
      +      HoodieTableConfig.ORDERING_FIELDS.key() -> "version",
             PARTITIONPATH_FIELD.key -> "dt,hh",
             HoodieMetadataConfig.ENABLE.key -> useMetadataTable.toString
           )
      @@ -496,7 +496,7 @@ class TestHoodieFileIndex extends HoodieSparkClientTestBase with ScalaAssertionS
           val writerOpts: Map[String, String] = commonOpts ++ Map(
             DataSourceWriteOptions.OPERATION.key -> DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL,
             RECORDKEY_FIELD.key -> "id",
      -      PRECOMBINE_FIELD.key -> "version",
      +      HoodieTableConfig.ORDERING_FIELDS.key() -> "version",
             PARTITIONPATH_FIELD.key -> partitionNames.mkString(","),
             HoodieMetadataConfig.ENABLE.key -> useMetadataTable.toString
           )
      @@ -551,7 +551,7 @@ class TestHoodieFileIndex extends HoodieSparkClientTestBase with ScalaAssertionS
             val writerOpts: Map[String, String] = commonOpts ++ Map(
               DataSourceWriteOptions.OPERATION.key -> DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL,
               RECORDKEY_FIELD.key -> "id",
      -        PRECOMBINE_FIELD.key -> "version",
      +        HoodieTableConfig.ORDERING_FIELDS.key() -> "version",
               PARTITIONPATH_FIELD.key -> "dt,hh"
             )
       
      @@ -613,7 +613,7 @@ 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"
      +      HoodieTableConfig.ORDERING_FIELDS.key() -> "price"
           )
       
           val readerOpts: Map[String, String] = queryOpts ++ Map(
      @@ -734,7 +734,7 @@ class TestHoodieFileIndex extends HoodieSparkClientTestBase with ScalaAssertionS
             .options(commonOpts)
             .option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
             .option(RECORDKEY_FIELD.key, "id")
      -      .option(PRECOMBINE_FIELD.key, "id")
      +      .option(HoodieTableConfig.ORDERING_FIELDS.key, "id")
             .option(PARTITIONPATH_FIELD.key, partitionBy)
             .option(HoodieMetadataConfig.ENABLE.key(), useMetaFileList)
             .mode(SaveMode.Overwrite)
      @@ -765,7 +765,7 @@ class TestHoodieFileIndex extends HoodieSparkClientTestBase with ScalaAssertionS
             "hoodie.upsert.shuffle.parallelism" -> "4",
             HoodieWriteConfig.TBL_NAME.key -> "hoodie_test",
             RECORDKEY_FIELD.key -> "id",
      -      PRECOMBINE_FIELD.key -> "id",
      +      HoodieTableConfig.ORDERING_FIELDS.key() -> "id",
             HoodieTableConfig.POPULATE_META_FIELDS.key -> "true"
           ) ++ writeMetadataOpts
       
      diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieSparkSqlWriter.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieSparkSqlWriter.scala
      index 793c7ef657e1a..37fdd03be341c 100644
      --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieSparkSqlWriter.scala
      +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieSparkSqlWriter.scala
      @@ -307,7 +307,7 @@ def testBulkInsertForDropPartitionColumn(): Unit = {
           option(DataSourceWriteOptions.PARTITIONPATH_FIELD.key(), "city").
           option(HoodieWriteConfig.TBL_NAME.key(), hoodieFooTableName).
           option("hoodie.datasource.write.recordkey.field", "uuid").
      -    option("hoodie.datasource.write.precombine.field", "rider").
      +    option(HoodieTableConfig.ORDERING_FIELDS.key(), "rider").
           option("hoodie.datasource.write.operation", "bulk_insert").
           option("hoodie.datasource.write.hive_style_partitioning", "true").
           option("hoodie.populate.meta.fields", "false").
      @@ -390,10 +390,10 @@ def testBulkInsertForDropPartitionColumn(): Unit = {
         }
       
         /**
      -   * Test case for insert dataset without precombine field.
      +   * Test case for insert dataset without ordering fields.
          */
         @Test
      -  def testInsertDatasetWithoutPrecombineField(): Unit = {
      +  def testInsertDatasetWithoutOrderingField(): Unit = {
       
           val fooTableModifier = commonTableModifier.updated(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
             .updated(DataSourceWriteOptions.INSERT_DROP_DUPS.key, "false")
      @@ -405,7 +405,7 @@ def testBulkInsertForDropPartitionColumn(): Unit = {
           val recordsSeq = convertRowListToSeq(records)
           val df = spark.createDataFrame(sc.parallelize(recordsSeq), structType)
           // write to Hudi
      -    HoodieSparkSqlWriter.write(sqlContext, SaveMode.Append, fooTableModifier - DataSourceWriteOptions.PRECOMBINE_FIELD.key, df)
      +    HoodieSparkSqlWriter.write(sqlContext, SaveMode.Append, fooTableModifier - HoodieTableConfig.ORDERING_FIELDS.key, df)
       
           // collect all partition paths to issue read of parquet files
           val partitions = Seq(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, HoodieTestDataGenerator.DEFAULT_SECOND_PARTITION_PATH,
      @@ -614,7 +614,7 @@ def testBulkInsertForDropPartitionColumn(): Unit = {
             .setBaseFileFormat(fooTableParams.getOrElse(HoodieWriteConfig.BASE_FILE_FORMAT.key,
               HoodieTableConfig.BASE_FILE_FORMAT.defaultValue().name))
             .setArchiveLogFolder(HoodieTableConfig.TIMELINE_HISTORY_PATH.defaultValue())
      -      .setPreCombineFields(fooTableParams.getOrElse(DataSourceWriteOptions.PRECOMBINE_FIELD.key, null))
      +      .setOrderingFields(fooTableParams.getOrElse(HoodieTableConfig.ORDERING_FIELDS.key, null))
             .setPartitionFields(fooTableParams(DataSourceWriteOptions.PARTITIONPATH_FIELD.key))
             .setKeyGeneratorClassProp(fooTableParams.getOrElse(DataSourceWriteOptions.KEYGENERATOR_CLASS_NAME.key,
               DataSourceWriteOptions.KEYGENERATOR_CLASS_NAME.defaultValue()))
      @@ -763,7 +763,7 @@ def testBulkInsertForDropPartitionColumn(): Unit = {
           List(DataSourceWriteOptions.COW_TABLE_TYPE_OPT_VAL, DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL).foreach { tableType =>
             val baseBootStrapPath = tempBootStrapPath.toAbsolutePath.toString
             val options = Map(DataSourceWriteOptions.TABLE_TYPE.key -> tableType,
      -        DataSourceWriteOptions.PRECOMBINE_FIELD.key -> "col3",
      +        HoodieTableConfig.ORDERING_FIELDS.key -> "col3",
               DataSourceWriteOptions.RECORDKEY_FIELD.key -> "keyid",
               DataSourceWriteOptions.PARTITIONPATH_FIELD.key -> "",
               DataSourceWriteOptions.KEYGENERATOR_CLASS_NAME.key -> "org.apache.hudi.keygen.NonpartitionedKeyGenerator",
      @@ -963,7 +963,7 @@ def testBulkInsertForDropPartitionColumn(): Unit = {
         @EnumSource(value = classOf[HoodieTableType])
         def testNonPartitionTableWithMetatableSupport(tableType: HoodieTableType): Unit = {
           val options = Map(DataSourceWriteOptions.TABLE_TYPE.key -> tableType.name,
      -      DataSourceWriteOptions.PRECOMBINE_FIELD.key -> "col3",
      +      HoodieTableConfig.ORDERING_FIELDS.key -> "col3",
             DataSourceWriteOptions.RECORDKEY_FIELD.key -> "keyid",
             DataSourceWriteOptions.PARTITIONPATH_FIELD.key -> "",
             DataSourceWriteOptions.KEYGENERATOR_CLASS_NAME.key -> "org.apache.hudi.keygen.NonpartitionedKeyGenerator",
      @@ -989,10 +989,10 @@ def testBulkInsertForDropPartitionColumn(): Unit = {
         }
       
         /**
      -   * Test upsert for CoW table without precombine field and combine before upsert disabled.
      +   * Test upsert for CoW table without ordering fields and combine before upsert disabled.
          */
         @Test
      -  def testUpsertWithoutPrecombineFieldAndCombineBeforeUpsertDisabled(): Unit = {
      +  def testUpsertWithoutOrderingFieldsAndCombineBeforeUpsertDisabled(): Unit = {
           val options = Map(DataSourceWriteOptions.TABLE_TYPE.key -> HoodieTableType.COPY_ON_WRITE.name(),
             DataSourceWriteOptions.RECORDKEY_FIELD.key -> "keyid",
             DataSourceWriteOptions.PARTITIONPATH_FIELD.key -> "",
      @@ -1049,7 +1049,7 @@ def testBulkInsertForDropPartitionColumn(): Unit = {
         def testUpsertWithCombineBeforeUpsertDisabled(tableType: HoodieTableType): Unit = {
           val options = Map(
             DataSourceWriteOptions.TABLE_TYPE.key -> tableType.name,
      -      DataSourceWriteOptions.PRECOMBINE_FIELD.key -> "col3",
      +      HoodieTableConfig.ORDERING_FIELDS.key -> "col3",
             DataSourceWriteOptions.RECORDKEY_FIELD.key -> "keyid",
             DataSourceWriteOptions.PARTITIONPATH_FIELD.key -> "",
             DataSourceWriteOptions.KEYGENERATOR_CLASS_NAME.key -> "org.apache.hudi.keygen.NonpartitionedKeyGenerator",
      @@ -1080,7 +1080,7 @@ def testBulkInsertForDropPartitionColumn(): Unit = {
           val df = Seq((1, "a1", 10, 1000L, "2021-10-16")).toDF("id", "name", "value", "ts", "dt")
           val options = Map(
             DataSourceWriteOptions.RECORDKEY_FIELD.key -> "id",
      -      DataSourceWriteOptions.PRECOMBINE_FIELD.key -> "ts",
      +      HoodieTableConfig.ORDERING_FIELDS.key -> "ts",
             DataSourceWriteOptions.PARTITIONPATH_FIELD.key -> "dt"
           )
       
      @@ -1098,7 +1098,7 @@ def testBulkInsertForDropPartitionColumn(): Unit = {
                | partitioned by (dt)
                | options (
                |  primaryKey = 'id',
      -         |  preCombineField = 'ts'
      +         |  orderingFields = 'ts'
                | )
                | location '$tablePath1'
              """.stripMargin)
      @@ -1155,7 +1155,7 @@ def testBulkInsertForDropPartitionColumn(): Unit = {
           val df = Seq((1, "a1", 10, 1000, "2021-10-16")).toDF("id", "name", "value", "ts", "dt")
           val options = Map(
             DataSourceWriteOptions.RECORDKEY_FIELD.key -> "id",
      -      DataSourceWriteOptions.PRECOMBINE_FIELD.key -> "ts"
      +      HoodieTableConfig.ORDERING_FIELDS.key -> "ts"
           )
       
           // case 1: When commit C1 specifies a key generator and commit C2 does not specify key generator
      @@ -1183,7 +1183,7 @@ def testBulkInsertForDropPartitionColumn(): Unit = {
           val df = Seq((1, "a1", 10, 1000, "2021-10-16")).toDF("id", "name", "value", "ts", "dt")
           val options = Map(
             DataSourceWriteOptions.RECORDKEY_FIELD.key -> "id",
      -      DataSourceWriteOptions.PRECOMBINE_FIELD.key -> "ts",
      +      HoodieTableConfig.ORDERING_FIELDS.key -> "ts",
             DataSourceWriteOptions.PARTITIONPATH_FIELD.key -> "dt"
           )
       
      @@ -1215,7 +1215,7 @@ def testBulkInsertForDropPartitionColumn(): Unit = {
           val df = Seq((1, "a1", 10, 1000, "2021-10-16")).toDF("id", "name", "value", "ts", "dt")
           val options = Map(
             DataSourceWriteOptions.RECORDKEY_FIELD.key -> "id",
      -      DataSourceWriteOptions.PRECOMBINE_FIELD.key -> "ts",
      +      HoodieTableConfig.ORDERING_FIELDS.key -> "ts",
             DataSourceWriteOptions.PARTITIONPATH_FIELD.key -> "dt"
           )
       
      @@ -1247,7 +1247,7 @@ def testBulkInsertForDropPartitionColumn(): Unit = {
           val df = Seq((1, "a1", 10, 1000, "2021-10-16")).toDF("id", "name", "value", "ts", "dt")
           val options = Map(
             DataSourceWriteOptions.RECORDKEY_FIELD.key -> "id",
      -      DataSourceWriteOptions.PRECOMBINE_FIELD.key -> "ts",
      +      HoodieTableConfig.ORDERING_FIELDS.key -> "ts",
             DataSourceWriteOptions.PARTITIONPATH_FIELD.key -> "dt"
           )
       
      @@ -1302,7 +1302,7 @@ def testBulkInsertForDropPartitionColumn(): Unit = {
           val df = Seq((1, "a1", 10, 1000, "2021-10-16")).toDF("id", "name", "value", "ts", "dt")
           val options = Map(
             DataSourceWriteOptions.RECORDKEY_FIELD.key -> "id",
      -      DataSourceWriteOptions.PRECOMBINE_FIELD.key -> "ts",
      +      HoodieTableConfig.ORDERING_FIELDS.key -> "ts",
             HoodieIndexConfig.BUCKET_INDEX_ENGINE_TYPE.key -> "CONSISTENT_HASHING",
             HoodieIndexConfig.INDEX_TYPE.key -> "BUCKET"
           )
      diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieSparkSqlWriterWithTestFormat.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieSparkSqlWriterWithTestFormat.scala
      index 312fc6ec24140..d5cafe06d5cf5 100644
      --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieSparkSqlWriterWithTestFormat.scala
      +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieSparkSqlWriterWithTestFormat.scala
      @@ -261,7 +261,7 @@ class TestHoodieSparkSqlWriterWithTestFormat extends HoodieSparkWriterTestBase {
          * Test case for insert dataset without precombine field.
          */
         @Test
      -  def testInsertDatasetWithoutPrecombineField(): Unit = {
      +  def testInsertDatasetWithoutOrderingFields(): Unit = {
       
           val fooTableModifier = commonTableModifier.updated(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
             .updated(DataSourceWriteOptions.INSERT_DROP_DUPS.key, "false")
      @@ -274,7 +274,7 @@ class TestHoodieSparkSqlWriterWithTestFormat extends HoodieSparkWriterTestBase {
           val recordsSeq = convertRowListToSeq(records)
           val df = spark.createDataFrame(sc.parallelize(recordsSeq), structType)
           // write to Hudi
      -    HoodieSparkSqlWriter.write(sqlContext, SaveMode.Append, fooTableModifier - DataSourceWriteOptions.PRECOMBINE_FIELD.key, df)
      +    HoodieSparkSqlWriter.write(sqlContext, SaveMode.Append, fooTableModifier - HoodieTableConfig.ORDERING_FIELDS.key, df)
       
           // collect all partition paths to issue read of parquet files
           val partitions = Seq(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, HoodieTestDataGenerator.DEFAULT_SECOND_PARTITION_PATH,
      @@ -485,7 +485,7 @@ class TestHoodieSparkSqlWriterWithTestFormat extends HoodieSparkWriterTestBase {
             .setBaseFileFormat(fooTableParams.getOrElse(HoodieWriteConfig.BASE_FILE_FORMAT.key,
               HoodieTableConfig.BASE_FILE_FORMAT.defaultValue().name))
             .setArchiveLogFolder(HoodieTableConfig.TIMELINE_HISTORY_PATH.defaultValue())
      -      .setPreCombineFields(fooTableParams.getOrElse(DataSourceWriteOptions.PRECOMBINE_FIELD.key, null))
      +      .setOrderingFields(fooTableParams.getOrElse(HoodieTableConfig.ORDERING_FIELDS.key, null))
             .setPartitionFields(fooTableParams(DataSourceWriteOptions.PARTITIONPATH_FIELD.key))
             .setKeyGeneratorClassProp(fooTableParams.getOrElse(DataSourceWriteOptions.KEYGENERATOR_CLASS_NAME.key,
               DataSourceWriteOptions.KEYGENERATOR_CLASS_NAME.defaultValue()))
      @@ -505,7 +505,7 @@ class TestHoodieSparkSqlWriterWithTestFormat extends HoodieSparkWriterTestBase {
           val df = Seq((1, "a1", 10, 1000, "2021-10-16")).toDF("id", "name", "value", "ts", "dt")
           val options = Map(
             DataSourceWriteOptions.RECORDKEY_FIELD.key -> "id",
      -      DataSourceWriteOptions.PRECOMBINE_FIELD.key -> "ts",
      +      HoodieTableConfig.ORDERING_FIELDS.key -> "ts",
             HoodieTableConfig.TABLE_FORMAT.key -> "test-format"
           )
       
      @@ -534,7 +534,7 @@ class TestHoodieSparkSqlWriterWithTestFormat extends HoodieSparkWriterTestBase {
           val df = Seq((1, "a1", 10, 1000, "2021-10-16")).toDF("id", "name", "value", "ts", "dt")
           val options = Map(
             DataSourceWriteOptions.RECORDKEY_FIELD.key -> "id",
      -      DataSourceWriteOptions.PRECOMBINE_FIELD.key -> "ts",
      +      HoodieTableConfig.ORDERING_FIELDS.key -> "ts",
             DataSourceWriteOptions.PARTITIONPATH_FIELD.key -> "dt",
             HoodieTableConfig.TABLE_FORMAT.key -> "test-format"
           )
      @@ -567,7 +567,7 @@ class TestHoodieSparkSqlWriterWithTestFormat extends HoodieSparkWriterTestBase {
           val df = Seq((1, "a1", 10, 1000, "2021-10-16")).toDF("id", "name", "value", "ts", "dt")
           val options = Map(
             DataSourceWriteOptions.RECORDKEY_FIELD.key -> "id",
      -      DataSourceWriteOptions.PRECOMBINE_FIELD.key -> "ts",
      +      HoodieTableConfig.ORDERING_FIELDS.key -> "ts",
             DataSourceWriteOptions.PARTITIONPATH_FIELD.key -> "dt",
             HoodieTableConfig.TABLE_FORMAT.key -> "test-format"
           )
      @@ -600,7 +600,7 @@ class TestHoodieSparkSqlWriterWithTestFormat extends HoodieSparkWriterTestBase {
           val df = Seq((1, "a1", 10, 1000, "2021-10-16")).toDF("id", "name", "value", "ts", "dt")
           val options = Map(
             DataSourceWriteOptions.RECORDKEY_FIELD.key -> "id",
      -      DataSourceWriteOptions.PRECOMBINE_FIELD.key -> "ts",
      +      HoodieTableConfig.ORDERING_FIELDS.key -> "ts",
             DataSourceWriteOptions.PARTITIONPATH_FIELD.key -> "dt",
             HoodieTableConfig.TABLE_FORMAT.key -> "test-format"
           )
      @@ -658,7 +658,7 @@ class TestHoodieSparkSqlWriterWithTestFormat extends HoodieSparkWriterTestBase {
           val df = Seq((1, "a1", 10, 1000, "2021-10-16")).toDF("id", "name", "value", "ts", "dt")
           val options = Map(
             DataSourceWriteOptions.RECORDKEY_FIELD.key -> "id",
      -      DataSourceWriteOptions.PRECOMBINE_FIELD.key -> "ts",
      +      HoodieTableConfig.ORDERING_FIELDS.key -> "ts",
             HoodieIndexConfig.BUCKET_INDEX_ENGINE_TYPE.key -> "CONSISTENT_HASHING",
             HoodieIndexConfig.INDEX_TYPE.key -> "BUCKET",
             HoodieTableConfig.TABLE_FORMAT.key -> "test-format"
      diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestInsertDedupPolicy.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestInsertDedupPolicy.scala
      index 62b02815e4901..376b38d6812da 100644
      --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestInsertDedupPolicy.scala
      +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestInsertDedupPolicy.scala
      @@ -21,6 +21,7 @@ package org.apache.hudi
       
       import org.apache.hudi.DataSourceWriteOptions._
       import org.apache.hudi.common.config.{HoodieMetadataConfig, HoodieReaderConfig}
      +import org.apache.hudi.common.table.HoodieTableConfig
       import org.apache.hudi.config.{HoodieCompactionConfig, HoodieWriteConfig}
       import org.apache.hudi.exception.HoodieUpsertException
       import org.apache.hudi.testutils.SparkClientFunctionalTestHarness
      @@ -70,7 +71,7 @@ class TestInsertDedupPolicy extends SparkClientFunctionalTestHarness {
           val inserts = spark.createDataFrame(firstInsertData).toDF(columns: _*)
           inserts.write.format("hudi").
             option(RECORDKEY_FIELD.key, "key").
      -      option(PRECOMBINE_FIELD.key, "ts").
      +      option(HoodieTableConfig.ORDERING_FIELDS.key, "ts").
             option(TABLE_TYPE.key, tableType).
             option(DataSourceWriteOptions.TABLE_NAME.key, "test_table").
             option(HoodieCompactionConfig.INLINE_COMPACT.key, "false").
      diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestParquetReaderCompatibility.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestParquetReaderCompatibility.scala
      index f9cf1a398d5e8..780e80b9e217f 100644
      --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestParquetReaderCompatibility.scala
      +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestParquetReaderCompatibility.scala
      @@ -23,7 +23,7 @@ import org.apache.hudi.TestParquetReaderCompatibility.NullabilityEnum.{NotNullab
       import org.apache.hudi.client.common.HoodieSparkEngineContext
       import org.apache.hudi.common.config.HoodieMetadataConfig
       import org.apache.hudi.common.model.HoodieRecord.HoodieRecordType
      -import org.apache.hudi.common.table.ParquetTableSchemaResolver
      +import org.apache.hudi.common.table.{HoodieTableConfig, ParquetTableSchemaResolver}
       import org.apache.hudi.common.testutils.HoodieTestUtils
       import org.apache.hudi.common.util.ConfigUtils.DEFAULT_HUDI_CONFIG_FOR_READER
       import org.apache.hudi.config.HoodieWriteConfig
      @@ -202,7 +202,7 @@ class TestParquetReaderCompatibility extends HoodieSparkWriterTestBase {
           val path = tempBasePath + "_avro_list_update"
           val options = Map(
             DataSourceWriteOptions.RECORDKEY_FIELD.key -> "key",
      -      DataSourceWriteOptions.PRECOMBINE_FIELD.key -> "ts",
      +      HoodieTableConfig.ORDERING_FIELDS.key -> "ts",
             DataSourceWriteOptions.PARTITIONPATH_FIELD.key -> "partition",
             HoodieWriteConfig.TBL_NAME.key -> hoodieFooTableName,
             "path" -> path
      diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/common/table/read/TestHoodieFileGroupReaderOnSpark.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/common/table/read/TestHoodieFileGroupReaderOnSpark.scala
      index abcb98014f78a..88841bdb6d882 100644
      --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/common/table/read/TestHoodieFileGroupReaderOnSpark.scala
      +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/common/table/read/TestHoodieFileGroupReaderOnSpark.scala
      @@ -20,7 +20,7 @@
       package org.apache.hudi.common.table.read
       
       import org.apache.hudi.{AvroConversionUtils, DataSourceWriteOptions, SparkAdapterSupport, SparkFileFormatInternalRowReaderContext}
      -import org.apache.hudi.DataSourceWriteOptions.{OPERATION, PRECOMBINE_FIELD, RECORDKEY_FIELD, TABLE_TYPE}
      +import org.apache.hudi.DataSourceWriteOptions.{OPERATION, ORDERING_FIELDS, RECORDKEY_FIELD, TABLE_TYPE}
       import org.apache.hudi.common.config.{HoodieReaderConfig, RecordMergeMode, TypedProperties}
       import org.apache.hudi.common.engine.HoodieReaderContext
       import org.apache.hudi.common.fs.FSUtils
      @@ -239,7 +239,7 @@ class TestHoodieFileGroupReaderOnSpark extends TestHoodieFileGroupReaderBase[Int
           val inserts = spark.createDataFrame(data).toDF(columns: _*)
           inserts.write.format("hudi").
             option(RECORDKEY_FIELD.key(), "key").
      -      option(PRECOMBINE_FIELD.key(), "ts").
      +      option(HoodieTableConfig.ORDERING_FIELDS.key(), "ts").
             option(TABLE_TYPE.key(), tableType).
             option(DataSourceWriteOptions.TABLE_NAME.key(), "test_table").
             option(HoodieCompactionConfig.INLINE_COMPACT.key(), "false").
      diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/common/table/read/TestSpark35RecordPositionMetadataColumn.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/common/table/read/TestSpark35RecordPositionMetadataColumn.scala
      index fd06f0382e1e2..cbf37e0c6281c 100644
      --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/common/table/read/TestSpark35RecordPositionMetadataColumn.scala
      +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/common/table/read/TestSpark35RecordPositionMetadataColumn.scala
      @@ -23,6 +23,7 @@ import org.apache.hudi.{DataSourceWriteOptions, HoodieSparkUtils, SparkFileForma
       import org.apache.hudi.SparkAdapterSupport.sparkAdapter
       import org.apache.hudi.common.config.{HoodieReaderConfig, HoodieStorageConfig}
       import org.apache.hudi.common.model.HoodieTableType
      +import org.apache.hudi.common.table.HoodieTableConfig
       import org.apache.hudi.common.testutils.HoodieTestTable
       import org.apache.hudi.common.util
       import org.apache.hudi.config.HoodieWriteConfig
      @@ -56,7 +57,7 @@ class TestSpark35RecordPositionMetadataColumn extends SparkClientFunctionalTestH
           // Create the file with record positions.
           userToCountryDF.write.format("hudi")
             .option(DataSourceWriteOptions.RECORDKEY_FIELD.key, "userid")
      -      .option(DataSourceWriteOptions.PRECOMBINE_FIELD.key, "ts")
      +      .option(HoodieTableConfig.ORDERING_FIELDS.key, "ts")
             .option(HoodieWriteConfig.TBL_NAME.key, "user_to_country")
             .option(HoodieWriteConfig.RECORD_MERGE_IMPL_CLASSES.key, SPARK_MERGER)
             .option(HoodieWriteConfig.WRITE_RECORD_POSITIONS.key, "true")
      diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/CommonOptionUtils.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/CommonOptionUtils.scala
      index f88fb4ba2efb2..8510a2028ca21 100644
      --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/CommonOptionUtils.scala
      +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/CommonOptionUtils.scala
      @@ -35,7 +35,7 @@ object CommonOptionUtils {
           HoodieTableConfig.PARTITION_METAFILE_USE_BASE_FORMAT.key() -> "true",
           DataSourceWriteOptions.RECORDKEY_FIELD.key -> "_row_key",
           DataSourceWriteOptions.PARTITIONPATH_FIELD.key -> "partition",
      -    DataSourceWriteOptions.PRECOMBINE_FIELD.key -> "timestamp",
      +    HoodieTableConfig.ORDERING_FIELDS.key -> "timestamp",
           HoodieWriteConfig.TBL_NAME.key -> "hoodie_test",
           HoodieMetadataConfig.COMPACT_NUM_DELTA_COMMITS.key -> "1"
         )
      diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/HoodieStatsIndexTestBase.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/HoodieStatsIndexTestBase.scala
      index bf790e216ac05..a0dbcbedf8168 100644
      --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/HoodieStatsIndexTestBase.scala
      +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/HoodieStatsIndexTestBase.scala
      @@ -18,12 +18,12 @@
       
       package org.apache.hudi.functional
       
      -import org.apache.hudi.DataSourceWriteOptions.{INSERT_OVERWRITE_OPERATION_OPT_VAL, INSERT_OVERWRITE_TABLE_OPERATION_OPT_VAL, PARTITIONPATH_FIELD, PRECOMBINE_FIELD, RECORDKEY_FIELD}
      +import org.apache.hudi.DataSourceWriteOptions.{INSERT_OVERWRITE_OPERATION_OPT_VAL, INSERT_OVERWRITE_TABLE_OPERATION_OPT_VAL, ORDERING_FIELDS, PARTITIONPATH_FIELD, RECORDKEY_FIELD}
       import org.apache.hudi.client.SparkRDDWriteClient
       import org.apache.hudi.client.common.HoodieSparkEngineContext
       import org.apache.hudi.common.config.TypedProperties
       import org.apache.hudi.common.model.{ActionType, HoodieCommitMetadata, WriteOperationType}
      -import org.apache.hudi.common.table.HoodieTableMetaClient
      +import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient}
       import org.apache.hudi.common.table.timeline.{HoodieInstant, HoodieInstantTimeGenerator, MetadataConversionUtils}
       import org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_FILE_NAME_GENERATOR
       import org.apache.hudi.config.HoodieWriteConfig
      @@ -53,7 +53,7 @@ class HoodieStatsIndexTestBase extends HoodieSparkClientTestBase {
           "hoodie.upsert.shuffle.parallelism" -> "4",
           HoodieWriteConfig.TBL_NAME.key -> "hoodie_test",
           RECORDKEY_FIELD.key -> "_row_key",
      -    PRECOMBINE_FIELD.key -> "timestamp",
      +    HoodieTableConfig.ORDERING_FIELDS.key -> "timestamp",
           HoodieWriteConfig.MARKERS_TIMELINE_SERVER_BASED_BATCH_INTERVAL_MS.key -> "10"
         )
       
      diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/RecordLevelIndexTestBase.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/RecordLevelIndexTestBase.scala
      index 0a20ec0001751..5805ec7a1bf55 100644
      --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/RecordLevelIndexTestBase.scala
      +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/RecordLevelIndexTestBase.scala
      @@ -60,7 +60,7 @@ class RecordLevelIndexTestBase extends HoodieStatsIndexTestBase {
           RECORDKEY_FIELD.key -> "uuid",
           SECONDARYKEY_COLUMN_NAME.key -> "city",
           PARTITIONPATH_FIELD.key -> "state",
      -    PRECOMBINE_FIELD.key -> "ts",
      +    HoodieTableConfig.ORDERING_FIELDS.key -> "ts",
           HoodieTableConfig.POPULATE_META_FIELDS.key -> "true"
         ) ++ metadataOpts
       
      diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestBasicSchemaEvolution.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestBasicSchemaEvolution.scala
      index 5a0ade3dc0bc3..ced770865878a 100644
      --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestBasicSchemaEvolution.scala
      +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestBasicSchemaEvolution.scala
      @@ -54,7 +54,7 @@ class TestBasicSchemaEvolution extends HoodieSparkClientTestBase with ScalaAsser
           HoodieWriteConfig.RECORD_MERGE_MODE.key() -> RecordMergeMode.COMMIT_TIME_ORDERING.name(),
           DataSourceWriteOptions.RECORDKEY_FIELD.key -> "_row_key",
           DataSourceWriteOptions.PARTITIONPATH_FIELD.key -> "partition",
      -    DataSourceWriteOptions.PRECOMBINE_FIELD.key -> "timestamp",
      +    HoodieTableConfig.ORDERING_FIELDS.key -> "timestamp",
           HoodieWriteConfig.TBL_NAME.key -> "hoodie_test",
           HoodieMetadataConfig.ENABLE_METADATA_INDEX_PARTITION_STATS.key -> "true"
         )
      diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestBloomFiltersIndexSupport.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestBloomFiltersIndexSupport.scala
      index fefc9f603ccf3..902071a411582 100644
      --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestBloomFiltersIndexSupport.scala
      +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestBloomFiltersIndexSupport.scala
      @@ -64,7 +64,7 @@ class TestBloomFiltersIndexSupport extends HoodieSparkClientTestBase {
           HoodieWriteConfig.TBL_NAME.key -> "hoodie_test",
           RECORDKEY_FIELD.key -> "_row_key",
           PARTITIONPATH_FIELD.key -> "partition",
      -    PRECOMBINE_FIELD.key -> "timestamp",
      +    HoodieTableConfig.ORDERING_FIELDS.key -> "timestamp",
           HoodieTableConfig.POPULATE_META_FIELDS.key -> "true"
         ) ++ metadataOpts
         var mergedDfList: List[DataFrame] = List.empty
      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 468f2f11e97cc..c2611631f6a28 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
      @@ -191,7 +191,7 @@ class TestCOWDataSource extends HoodieSparkClientTestBase with ScalaAssertionSup
             "hoodie.upsert.shuffle.parallelism" -> "4",
             DataSourceWriteOptions.RECORDKEY_FIELD.key -> "_row_key",
             DataSourceWriteOptions.PARTITIONPATH_FIELD.key -> "partition",
      -      DataSourceWriteOptions.PRECOMBINE_FIELD.key() -> "timestamp,rider",
      +      HoodieTableConfig.ORDERING_FIELDS.key() -> "timestamp,rider",
             DataSourceWriteOptions.RECORD_MERGE_MODE.key() -> RecordMergeMode.EVENT_TIME_ORDERING.name(),
             HoodieWriteConfig.TBL_NAME.key -> "hoodie_test"
           )
      @@ -443,7 +443,7 @@ class TestCOWDataSource extends HoodieSparkClientTestBase with ScalaAssertionSup
       
         @ParameterizedTest
         @CsvSource(Array("hoodie.datasource.write.recordkey.field,begin_lat", "hoodie.datasource.write.partitionpath.field,end_lon",
      -    "hoodie.datasource.write.keygenerator.class,org.apache.hudi.keygen.NonpartitionedKeyGenerator", "hoodie.datasource.write.precombine.field,fare"))
      +    "hoodie.datasource.write.keygenerator.class,org.apache.hudi.keygen.NonpartitionedKeyGenerator", "hoodie.table.ordering.fields,fare"))
         def testAlteringRecordKeyConfig(configKey: String, configValue: String) {
           val recordType = HoodieRecordType.AVRO
           val (writeOpts, readOpts) = getWriterReaderOpts(recordType, Map(
      @@ -451,7 +451,7 @@ class TestCOWDataSource extends HoodieSparkClientTestBase with ScalaAssertionSup
             "hoodie.upsert.shuffle.parallelism" -> "4",
             "hoodie.bulkinsert.shuffle.parallelism" -> "2",
             "hoodie.delete.shuffle.parallelism" -> "1",
      -      "hoodie.datasource.write.precombine.field" -> "timestamp",
      +      HoodieTableConfig.ORDERING_FIELDS.key() -> "timestamp",
             HoodieMetadataConfig.ENABLE.key -> "false" // this is testing table configs and write configs. disabling metadata to save on test run time.
           ))
       
      @@ -1602,7 +1602,7 @@ class TestCOWDataSource extends HoodieSparkClientTestBase with ScalaAssertionSup
             .option("hoodie.bulkinsert.shuffle.parallelism", "2")
             .option(DataSourceWriteOptions.RECORDKEY_FIELD.key, "id")
             .option(DataSourceWriteOptions.PARTITIONPATH_FIELD.key, "data_date")
      -      .option(DataSourceWriteOptions.PRECOMBINE_FIELD.key, "ts")
      +      .option(HoodieTableConfig.ORDERING_FIELDS.key, "ts")
             .option(DataSourceWriteOptions.KEYGENERATOR_CLASS_NAME.key, "org.apache.hudi.keygen.TimestampBasedKeyGenerator")
             .option(TIMESTAMP_TYPE_FIELD.key, "DATE_STRING")
             .option(TIMESTAMP_INPUT_DATE_FORMAT.key, "yyyy-MM-dd")
      @@ -1730,7 +1730,7 @@ class TestCOWDataSource extends HoodieSparkClientTestBase with ScalaAssertionSup
             .options(writeOpts)
             .option(DataSourceWriteOptions.RECORDKEY_FIELD.key, "uuid")
             .option(DataSourceWriteOptions.PARTITIONPATH_FIELD.key, "partitionpath")
      -      .option(DataSourceWriteOptions.PRECOMBINE_FIELD.key, "ts")
      +      .option(HoodieTableConfig.ORDERING_FIELDS.key, "ts")
             .option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
             .option(HoodieWriteConfig.TBL_NAME.key, "hoodie_test")
             .option(HoodieMetricsConfig.TURN_METRICS_ON.key(), "true")
      @@ -1859,7 +1859,7 @@ class TestCOWDataSource extends HoodieSparkClientTestBase with ScalaAssertionSup
             DataSourceWriteOptions.TABLE_TYPE.key() -> "COPY_ON_WRITE",
             DataSourceWriteOptions.RECORDKEY_FIELD.key() -> "id",
             DataSourceWriteOptions.PARTITIONPATH_FIELD.key() -> "partition",
      -      DataSourceWriteOptions.PRECOMBINE_FIELD.key() -> "precombine",
      +      HoodieTableConfig.ORDERING_FIELDS.key() -> "precombine",
             DataSourceWriteOptions.HIVE_STYLE_PARTITIONING.key() -> "true"
           )
       
      @@ -1871,7 +1871,7 @@ class TestCOWDataSource extends HoodieSparkClientTestBase with ScalaAssertionSup
             DataSourceWriteOptions.TABLE_TYPE.key() -> "COPY_ON_WRITE",
             DataSourceWriteOptions.RECORDKEY_FIELD.key() -> "id",
             DataSourceWriteOptions.PARTITIONPATH_FIELD.key() -> "partition",
      -      DataSourceWriteOptions.PRECOMBINE_FIELD.key() -> "precombine"
      +      HoodieTableConfig.ORDERING_FIELDS.key() -> "precombine"
           )
       
           df.filter(df("id") === 1).
      diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestCOWDataSourceStorage.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestCOWDataSourceStorage.scala
      index cd77a5f6f674f..dfbd07cabe4a4 100644
      --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestCOWDataSourceStorage.scala
      +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestCOWDataSourceStorage.scala
      @@ -24,6 +24,7 @@ import org.apache.hudi.client.validator.{SqlQueryEqualityPreCommitValidator, Sql
       import org.apache.hudi.common.config.{HoodieMetadataConfig, HoodieReaderConfig}
       import org.apache.hudi.common.config.TimestampKeyGeneratorConfig.{TIMESTAMP_INPUT_DATE_FORMAT, TIMESTAMP_OUTPUT_DATE_FORMAT, TIMESTAMP_TYPE_FIELD}
       import org.apache.hudi.common.model.WriteOperationType
      +import org.apache.hudi.common.table.HoodieTableConfig
       import org.apache.hudi.common.table.timeline.{HoodieInstant, HoodieTimeline}
       import org.apache.hudi.common.testutils.HoodieTestDataGenerator
       import org.apache.hudi.common.testutils.RawTripTestPayload.recordsToStrings
      @@ -59,7 +60,7 @@ class TestCOWDataSourceStorage extends SparkClientFunctionalTestHarness {
           "hoodie.delete.shuffle.parallelism" -> "1",
           DataSourceWriteOptions.RECORDKEY_FIELD.key -> "_row_key",
           DataSourceWriteOptions.PARTITIONPATH_FIELD.key -> "partition",
      -    DataSourceWriteOptions.PRECOMBINE_FIELD.key -> "timestamp",
      +    HoodieTableConfig.ORDERING_FIELDS.key -> "timestamp",
           HoodieWriteConfig.TBL_NAME.key -> "hoodie_test",
           DataSourceWriteOptions.HIVE_STYLE_PARTITIONING.key -> "false"
         )
      diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestColumnStatsIndex.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestColumnStatsIndex.scala
      index 21dedc23d8636..db80c93dff6bd 100644
      --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestColumnStatsIndex.scala
      +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestColumnStatsIndex.scala
      @@ -20,7 +20,7 @@ package org.apache.hudi.functional
       
       import org.apache.hudi.{ColumnStatsIndexSupport, DataSourceWriteOptions}
       import org.apache.hudi.ColumnStatsIndexSupport.composeIndexSchema
      -import org.apache.hudi.DataSourceWriteOptions.{PARTITIONPATH_FIELD, PRECOMBINE_FIELD, RECORDKEY_FIELD}
      +import org.apache.hudi.DataSourceWriteOptions.{ORDERING_FIELDS, PARTITIONPATH_FIELD, RECORDKEY_FIELD}
       import org.apache.hudi.HoodieConversionUtils.toProperties
       import org.apache.hudi.avro.model.DecimalWrapper
       import org.apache.hudi.client.common.HoodieSparkEngineContext
      @@ -87,7 +87,7 @@ class TestColumnStatsIndex extends ColumnStatIndexTestBase {
               HoodieWriteConfig.TBL_NAME.key -> "hoodie_test",
               DataSourceWriteOptions.TABLE_TYPE.key -> testCase.tableType.toString,
               RECORDKEY_FIELD.key -> "c1",
      -        PRECOMBINE_FIELD.key -> "c1",
      +        HoodieTableConfig.ORDERING_FIELDS.key -> "c1",
               HoodieTableConfig.POPULATE_META_FIELDS.key -> "true",
               "hoodie.compact.inline.max.delta.commits" -> "10",
               HoodieWriteConfig.WRITE_TABLE_VERSION.key() -> testCase.tableVersion.toString
      @@ -212,7 +212,7 @@ class TestColumnStatsIndex extends ColumnStatIndexTestBase {
             HoodieWriteConfig.TBL_NAME.key -> "hoodie_test",
             DataSourceWriteOptions.TABLE_TYPE.key -> testCase.tableType.toString,
             RECORDKEY_FIELD.key -> "c1",
      -      PRECOMBINE_FIELD.key -> "c1",
      +      HoodieTableConfig.ORDERING_FIELDS.key -> "c1",
             HoodieTableConfig.POPULATE_META_FIELDS.key -> "true",
             HoodieWriteConfig.WRITE_TABLE_VERSION.key() -> testCase.tableVersion.toString
           ) ++ metadataOpts
      @@ -272,7 +272,7 @@ class TestColumnStatsIndex extends ColumnStatIndexTestBase {
             HoodieWriteConfig.TBL_NAME.key -> "hoodie_test",
             DataSourceWriteOptions.TABLE_TYPE.key -> testCase.tableType.toString,
             RECORDKEY_FIELD.key -> "c1",
      -      PRECOMBINE_FIELD.key -> "c1",
      +      HoodieTableConfig.ORDERING_FIELDS.key -> "c1",
             PARTITIONPATH_FIELD.key -> partitionCol,
             HoodieTableConfig.POPULATE_META_FIELDS.key -> "true",
             HoodieCompactionConfig.INLINE_COMPACT_NUM_DELTA_COMMITS.key -> "5",
      @@ -389,7 +389,7 @@ class TestColumnStatsIndex extends ColumnStatIndexTestBase {
             HoodieWriteConfig.TBL_NAME.key -> "hoodie_test",
             DataSourceWriteOptions.TABLE_TYPE.key -> testCase.tableType.toString,
             RECORDKEY_FIELD.key -> "c1",
      -      PRECOMBINE_FIELD.key -> "c1",
      +      HoodieTableConfig.ORDERING_FIELDS.key -> "c1",
             PARTITIONPATH_FIELD.key() -> partitionCol,
             "hoodie.write.markers.type" -> "DIRECT",
             HoodieTableConfig.POPULATE_META_FIELDS.key -> "true",
      @@ -520,7 +520,7 @@ class TestColumnStatsIndex extends ColumnStatIndexTestBase {
             HoodieWriteConfig.TBL_NAME.key -> "hoodie_test",
             DataSourceWriteOptions.TABLE_TYPE.key -> testCase.tableType.toString,
             RECORDKEY_FIELD.key -> "c1",
      -      PRECOMBINE_FIELD.key -> "c1",
      +      HoodieTableConfig.ORDERING_FIELDS.key -> "c1",
             PARTITIONPATH_FIELD.key() -> partitionCol,
             HoodieTableConfig.POPULATE_META_FIELDS.key -> "true",
             HoodieCompactionConfig.INLINE_COMPACT_NUM_DELTA_COMMITS.key() -> "5",
      @@ -577,7 +577,7 @@ class TestColumnStatsIndex extends ColumnStatIndexTestBase {
             HoodieWriteConfig.TBL_NAME.key -> "hoodie_test",
             DataSourceWriteOptions.TABLE_TYPE.key -> testCase.tableType.toString,
             RECORDKEY_FIELD.key -> "c1",
      -      PRECOMBINE_FIELD.key -> "c1",
      +      HoodieTableConfig.ORDERING_FIELDS.key -> "c1",
             PARTITIONPATH_FIELD.key() -> partitionCol,
             HoodieTableConfig.POPULATE_META_FIELDS.key -> "true",
             HoodieCleanConfig.CLEANER_COMMITS_RETAINED.key() -> "1",
      @@ -643,7 +643,7 @@ class TestColumnStatsIndex extends ColumnStatIndexTestBase {
             HoodieWriteConfig.TBL_NAME.key -> "hoodie_test",
             DataSourceWriteOptions.TABLE_TYPE.key -> testCase.tableType.toString,
             RECORDKEY_FIELD.key -> "c1",
      -      PRECOMBINE_FIELD.key -> "c1",
      +      HoodieTableConfig.ORDERING_FIELDS.key -> "c1",
             PARTITIONPATH_FIELD.key() -> partitionCol,
             HoodieTableConfig.POPULATE_META_FIELDS.key -> "true",
             HoodieCleanConfig.CLEANER_COMMITS_RETAINED.key() -> "1",
      @@ -712,7 +712,7 @@ class TestColumnStatsIndex extends ColumnStatIndexTestBase {
             HoodieWriteConfig.TBL_NAME.key -> "hoodie_test",
             DataSourceWriteOptions.TABLE_TYPE.key -> tableType.name(),
             RECORDKEY_FIELD.key -> "c1",
      -      PRECOMBINE_FIELD.key -> "c1",
      +      HoodieTableConfig.ORDERING_FIELDS.key -> "c1",
             HoodieTableConfig.POPULATE_META_FIELDS.key -> "true",
             HoodieWriteConfig.WRITE_TABLE_VERSION.key() -> tableVersion.toString
           ) ++ metadataOpts
      @@ -763,7 +763,7 @@ class TestColumnStatsIndex extends ColumnStatIndexTestBase {
             "hoodie.upsert.shuffle.parallelism" -> "4",
             HoodieWriteConfig.TBL_NAME.key -> "hoodie_test",
             RECORDKEY_FIELD.key -> "c1",
      -      PRECOMBINE_FIELD.key -> "c1",
      +      HoodieTableConfig.ORDERING_FIELDS.key -> "c1",
             PARTITIONPATH_FIELD.key() -> "c8",
             HoodieTableConfig.POPULATE_META_FIELDS.key -> "true",
             HoodieCommonConfig.RECONCILE_SCHEMA.key -> "true",
      @@ -860,7 +860,7 @@ class TestColumnStatsIndex extends ColumnStatIndexTestBase {
             "hoodie.upsert.shuffle.parallelism" -> "4",
             HoodieWriteConfig.TBL_NAME.key -> "hoodie_test",
             RECORDKEY_FIELD.key -> "c1",
      -      PRECOMBINE_FIELD.key -> "c1",
      +      HoodieTableConfig.ORDERING_FIELDS.key -> "c1",
             HoodieTableConfig.POPULATE_META_FIELDS.key -> "true",
             HoodieCommonConfig.RECONCILE_SCHEMA.key -> "true",
             HoodieWriteConfig.WRITE_TABLE_VERSION.key() -> tableVersion.toString
      @@ -975,7 +975,7 @@ class TestColumnStatsIndex extends ColumnStatIndexTestBase {
             "hoodie.upsert.shuffle.parallelism" -> "4",
             HoodieWriteConfig.TBL_NAME.key -> "hoodie_test",
             RECORDKEY_FIELD.key -> "c1",
      -      PRECOMBINE_FIELD.key -> "c1",
      +      HoodieTableConfig.ORDERING_FIELDS.key -> "c1",
             HoodieTableConfig.POPULATE_META_FIELDS.key -> "true",
             HoodieWriteConfig.WRITE_TABLE_VERSION.key() -> tableVersion.toString
           ) ++ metadataOpts
      diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestColumnStatsIndexWithSQL.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestColumnStatsIndexWithSQL.scala
      index f05c008361004..93d805932c68c 100644
      --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestColumnStatsIndexWithSQL.scala
      +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestColumnStatsIndexWithSQL.scala
      @@ -19,7 +19,7 @@
       package org.apache.hudi.functional
       
       import org.apache.hudi.{ColumnStatsIndexSupport, DataSourceReadOptions, DataSourceWriteOptions, HoodieFileIndex}
      -import org.apache.hudi.DataSourceWriteOptions.{DELETE_OPERATION_OPT_VAL, PRECOMBINE_FIELD, RECORDKEY_FIELD}
      +import org.apache.hudi.DataSourceWriteOptions.{DELETE_OPERATION_OPT_VAL, ORDERING_FIELDS, RECORDKEY_FIELD}
       import org.apache.hudi.client.SparkRDDWriteClient
       import org.apache.hudi.client.common.HoodieSparkEngineContext
       import org.apache.hudi.common.config.{HoodieMetadataConfig, TypedProperties}
      @@ -67,7 +67,7 @@ class TestColumnStatsIndexWithSQL extends ColumnStatIndexTestBase {
             HoodieWriteConfig.TBL_NAME.key -> "hoodie_test",
             DataSourceWriteOptions.TABLE_TYPE.key -> testCase.tableType.toString,
             RECORDKEY_FIELD.key -> "c1",
      -      PRECOMBINE_FIELD.key -> "c1",
      +      HoodieTableConfig.ORDERING_FIELDS.key -> "c1",
             HoodieTableConfig.POPULATE_META_FIELDS.key -> "true",
             DataSourceReadOptions.ENABLE_DATA_SKIPPING.key -> "true",
             DataSourceReadOptions.QUERY_TYPE.key -> DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL
      @@ -91,7 +91,7 @@ class TestColumnStatsIndexWithSQL extends ColumnStatIndexTestBase {
             HoodieWriteConfig.TBL_NAME.key -> "hoodie_test",
             DataSourceWriteOptions.TABLE_TYPE.key -> testCase.tableType.toString,
             RECORDKEY_FIELD.key -> "c1",
      -      PRECOMBINE_FIELD.key -> "c1",
      +      HoodieTableConfig.ORDERING_FIELDS.key -> "c1",
             HoodieTableConfig.POPULATE_META_FIELDS.key -> "true",
             DataSourceReadOptions.ENABLE_DATA_SKIPPING.key -> "true",
             DataSourceReadOptions.QUERY_TYPE.key -> DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL
      @@ -328,7 +328,7 @@ class TestColumnStatsIndexWithSQL extends ColumnStatIndexTestBase {
             HoodieWriteConfig.TBL_NAME.key -> "hoodie_test",
             DataSourceWriteOptions.TABLE_TYPE.key -> testCase.tableType.toString,
             RECORDKEY_FIELD.key -> "c1",
      -      PRECOMBINE_FIELD.key -> "c1",
      +      HoodieTableConfig.ORDERING_FIELDS.key -> "c1",
             HoodieTableConfig.POPULATE_META_FIELDS.key -> "true",
             DataSourceReadOptions.ENABLE_DATA_SKIPPING.key -> "true",
             DataSourceReadOptions.QUERY_TYPE.key -> DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL,
      @@ -364,7 +364,7 @@ class TestColumnStatsIndexWithSQL extends ColumnStatIndexTestBase {
             HoodieWriteConfig.TBL_NAME.key -> "hoodie_test",
             DataSourceWriteOptions.TABLE_TYPE.key -> testCase.tableType.toString,
             RECORDKEY_FIELD.key -> "c1",
      -      PRECOMBINE_FIELD.key -> "c1",
      +      HoodieTableConfig.ORDERING_FIELDS.key -> "c1",
             HoodieTableConfig.POPULATE_META_FIELDS.key -> "true",
             DataSourceReadOptions.ENABLE_DATA_SKIPPING.key -> "true",
             DataSourceReadOptions.QUERY_TYPE.key -> DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL,
      @@ -404,7 +404,7 @@ class TestColumnStatsIndexWithSQL extends ColumnStatIndexTestBase {
             HoodieWriteConfig.TBL_NAME.key -> "hoodie_test",
             DataSourceWriteOptions.TABLE_TYPE.key -> testCase.tableType.toString,
             RECORDKEY_FIELD.key -> "c1",
      -      PRECOMBINE_FIELD.key -> "c1",
      +      HoodieTableConfig.ORDERING_FIELDS.key -> "c1",
             HoodieTableConfig.POPULATE_META_FIELDS.key -> "true",
             DataSourceReadOptions.ENABLE_DATA_SKIPPING.key -> "true",
             DataSourceReadOptions.QUERY_TYPE.key -> DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL,
      @@ -431,7 +431,7 @@ class TestColumnStatsIndexWithSQL extends ColumnStatIndexTestBase {
             HoodieWriteConfig.TBL_NAME.key -> "hoodie_test",
             DataSourceWriteOptions.TABLE_TYPE.key -> testCase.tableType.toString,
             RECORDKEY_FIELD.key -> "c1",
      -      PRECOMBINE_FIELD.key -> "c1",
      +      HoodieTableConfig.ORDERING_FIELDS.key -> "c1",
             HoodieTableConfig.POPULATE_META_FIELDS.key -> "true",
             DataSourceReadOptions.ENABLE_DATA_SKIPPING.key -> "true",
             DataSourceReadOptions.QUERY_TYPE.key -> DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL,
      @@ -466,7 +466,7 @@ class TestColumnStatsIndexWithSQL extends ColumnStatIndexTestBase {
             HoodieWriteConfig.TBL_NAME.key -> "hoodie_test",
             DataSourceWriteOptions.TABLE_TYPE.key -> testCase.tableType.toString,
             RECORDKEY_FIELD.key -> "c1",
      -      PRECOMBINE_FIELD.key -> "c1",
      +      HoodieTableConfig.ORDERING_FIELDS.key -> "c1",
             HoodieTableConfig.POPULATE_META_FIELDS.key -> "true",
             DataSourceReadOptions.ENABLE_DATA_SKIPPING.key -> "true",
             DataSourceReadOptions.QUERY_TYPE.key -> DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL,
      @@ -511,7 +511,7 @@ class TestColumnStatsIndexWithSQL extends ColumnStatIndexTestBase {
             HoodieWriteConfig.TBL_NAME.key -> tableName,
             DataSourceWriteOptions.TABLE_TYPE.key -> "mor",
             RECORDKEY_FIELD.key -> "id",
      -      PRECOMBINE_FIELD.key -> "ts",
      +      HoodieTableConfig.ORDERING_FIELDS.key -> "ts",
             HoodieTableConfig.POPULATE_META_FIELDS.key -> "true",
             DataSourceReadOptions.ENABLE_DATA_SKIPPING.key -> "true",
             HoodieCompactionConfig.INLINE_COMPACT_NUM_DELTA_COMMITS.key() -> "20",
      @@ -530,7 +530,7 @@ class TestColumnStatsIndexWithSQL extends ColumnStatIndexTestBase {
                | options (
                |  primaryKey ='id',
                |  type = 'mor',
      -         |  preCombineField = 'name',
      +         |  orderingFields = 'name',
                |  hoodie.metadata.enable = 'true',
                |  hoodie.metadata.index.column.stats.enable = 'true'
                | )
      diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestDataSourceForBootstrap.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestDataSourceForBootstrap.scala
      index 7d6bbdbb6b7fb..7fe676065af87 100644
      --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestDataSourceForBootstrap.scala
      +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestDataSourceForBootstrap.scala
      @@ -23,6 +23,7 @@ import org.apache.hudi.client.bootstrap.selector.{FullRecordBootstrapModeSelecto
       import org.apache.hudi.common.config.{HoodieMetadataConfig, HoodieStorageConfig}
       import org.apache.hudi.common.model.HoodieRecord
       import org.apache.hudi.common.model.HoodieRecord.HoodieRecordType
      +import org.apache.hudi.common.table.HoodieTableConfig
       import org.apache.hudi.common.table.timeline.{HoodieInstantTimeGenerator, HoodieTimeline}
       import org.apache.hudi.config.{HoodieBootstrapConfig, HoodieClusteringConfig, HoodieCompactionConfig, HoodieWriteConfig}
       import org.apache.hudi.functional.TestDataSourceForBootstrap.{dropMetaCols, sort}
      @@ -57,7 +58,7 @@ class TestDataSourceForBootstrap {
           HoodieBootstrapConfig.PARALLELISM_VALUE.key -> "4",
           DataSourceWriteOptions.RECORDKEY_FIELD.key -> "_row_key",
           DataSourceWriteOptions.PARTITIONPATH_FIELD.key -> "partition",
      -    DataSourceWriteOptions.PRECOMBINE_FIELD.key -> "timestamp",
      +    HoodieTableConfig.ORDERING_FIELDS.key -> "timestamp",
           HoodieWriteConfig.TBL_NAME.key -> "hoodie_test",
           HoodieMetadataConfig.ENABLE_METADATA_INDEX_COLUMN_STATS.key -> "false"
         )
      diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestEightToNineUpgrade.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestEightToNineUpgrade.scala
      index 2202a63f203e1..1cf6d7d5d8326 100644
      --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestEightToNineUpgrade.scala
      +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestEightToNineUpgrade.scala
      @@ -28,6 +28,7 @@ import org.apache.hudi.common.model.debezium.{DebeziumConstants, MySqlDebeziumAv
       import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient, HoodieTableVersion, PartialUpdateMode}
       import org.apache.hudi.common.table.HoodieTableConfig.{DEBEZIUM_UNAVAILABLE_VALUE, PARTIAL_UPDATE_UNAVAILABLE_VALUE, RECORD_MERGE_PROPERTY_PREFIX}
       import org.apache.hudi.common.testutils.HoodieTestDataGenerator
      +import org.apache.hudi.common.util.StringUtils
       import org.apache.hudi.config.{HoodieCompactionConfig, HoodieWriteConfig}
       import org.apache.hudi.table.upgrade.{SparkUpgradeDowngradeHelper, UpgradeDowngrade}
       
      @@ -38,6 +39,8 @@ import org.junit.jupiter.api.Test
       import org.junit.jupiter.params.ParameterizedTest
       import org.junit.jupiter.params.provider.{Arguments, MethodSource}
       
      +import java.util.{Collections, Properties}
      +
       class TestEightToNineUpgrade extends RecordLevelIndexTestBase {
       
         @ParameterizedTest
      @@ -56,6 +59,8 @@ class TestEightToNineUpgrade extends RecordLevelIndexTestBase {
             HoodieWriteConfig.WRITE_TABLE_VERSION.key -> "8",
             HoodieStorageConfig.LOGFILE_DATA_BLOCK_FORMAT.key -> "parquet"
           )
      +    hudiOpts = hudiOpts ++ Map(HoodieTableConfig.PRECOMBINE_FIELD.key() -> hudiOpts(HoodieTableConfig.ORDERING_FIELDS.key())) - HoodieTableConfig.ORDERING_FIELDS.key()
      +    val orderingValue = "timestamp"
       
           // Create a table in table version 8.
           doWriteAndValidateDataAndRecordIndex(hudiOpts,
      @@ -63,15 +68,16 @@ class TestEightToNineUpgrade extends RecordLevelIndexTestBase {
             saveMode = SaveMode.Overwrite,
             schemaStr = HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA_WITH_PAYLOAD_SPECIFIC_COLS)
           metaClient = getLatestMetaClient(true)
      +    setupV8OrderingFields(hudiOpts)
           // Assert table version is 8.
      -    checkResultForVersion8(payloadClass)
      +    checkResultForVersion8(payloadClass, orderingValue)
           // Add an extra commit.
           doWriteAndValidateDataAndRecordIndex(hudiOpts,
             operation = INSERT_OVERWRITE_OPERATION_OPT_VAL,
             saveMode = SaveMode.Append,
             schemaStr = HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA_WITH_PAYLOAD_SPECIFIC_COLS)
           // Do validations.
      -    checkResultForVersion8(payloadClass)
      +    checkResultForVersion8(payloadClass, orderingValue)
       
           // Upgrade to version 9.
           // Remove the write table version config, such that an upgrade could be triggered.
      @@ -82,14 +88,14 @@ class TestEightToNineUpgrade extends RecordLevelIndexTestBase {
             schemaStr = HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA_WITH_PAYLOAD_SPECIFIC_COLS)
           // Table should be automatically upgraded to version 9.
           // Do validations for table version 9.
      -    checkResultForVersion9(partitionFields, payloadClass)
      +    checkResultForVersion9(partitionFields, payloadClass, orderingValue)
           // Add an extra commit.
           doWriteAndValidateDataAndRecordIndex(hudiOpts,
             operation = INSERT_OVERWRITE_OPERATION_OPT_VAL,
             saveMode = SaveMode.Append,
             schemaStr = HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA_WITH_PAYLOAD_SPECIFIC_COLS)
           // Do validations for table version 9.
      -    checkResultForVersion9(partitionFields, payloadClass)
      +    checkResultForVersion9(partitionFields, payloadClass, orderingValue)
       
           // Downgrade to table version 8 explicitly.
           // Note that downgrade is NOT automatic.
      @@ -101,14 +107,21 @@ class TestEightToNineUpgrade extends RecordLevelIndexTestBase {
             operation = INSERT_OVERWRITE_OPERATION_OPT_VAL,
             saveMode = SaveMode.Append,
             schemaStr = HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA_WITH_PAYLOAD_SPECIFIC_COLS)
      -    checkResultForVersion8(payloadClass)
      +    checkResultForVersion8(payloadClass, orderingValue)
           // Add an extra commit.
           doWriteAndValidateDataAndRecordIndex(hudiOpts,
             operation = INSERT_OVERWRITE_OPERATION_OPT_VAL,
             saveMode = SaveMode.Append,
             schemaStr = HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA_WITH_PAYLOAD_SPECIFIC_COLS)
           // Do validations.
      -    checkResultForVersion8(payloadClass)
      +    checkResultForVersion8(payloadClass, orderingValue)
      +  }
      +
      +  private def setupV8OrderingFields(hudiOpts: Map[String, String]): Unit = {
      +    val props = new Properties()
      +    props.put(HoodieTableConfig.PRECOMBINE_FIELD.key(), hudiOpts(HoodieTableConfig.PRECOMBINE_FIELD.key()))
      +    HoodieTableConfig.updateAndDeleteProps(metaClient.getStorage, metaClient.getMetaPath, props, Collections.singleton(HoodieTableConfig.ORDERING_FIELDS.key()))
      +    metaClient = getLatestMetaClient(true)
         }
       
         @Test
      @@ -129,17 +142,19 @@ class TestEightToNineUpgrade extends RecordLevelIndexTestBase {
             (10, "4", "rider-D", "driver-D", 4, 8, "4.8"),
             (10, "5", "rider-E", "driver-E", 5, 4, "5.4"))
           val inserts = spark.createDataFrame(data).toDF(columns: _*)
      +    var orderingValue: String = DebeziumConstants.ADDED_SEQ_COL_NAME
           inserts.write.format("hudi").
             option(RECORDKEY_FIELD.key(), "key").
             option(TABLE_TYPE.key(), HoodieTableType.MERGE_ON_READ.name()).
      -      option(DataSourceWriteOptions.PRECOMBINE_FIELD.key(), DebeziumConstants.ADDED_SEQ_COL_NAME).
      +      option(DataSourceWriteOptions.ORDERING_FIELDS.key(), orderingValue).
             option(DataSourceWriteOptions.TABLE_NAME.key(), "test_table").
             option(HoodieCompactionConfig.INLINE_COMPACT.key(), "false").
             option(HoodieWriteConfig.WRITE_TABLE_VERSION.key(), "8").
             options(opts).
             mode(SaveMode.Overwrite).
             save(basePath)
      -    checkResultForVersion8(payloadClass)
      +    setupV8OrderingFields(opts ++ Map(HoodieTableConfig.PRECOMBINE_FIELD.key() -> orderingValue))
      +    checkResultForVersion8(payloadClass, orderingValue)
       
           // 2. Add an update and upgrade the table to v9
           // first two records with larger ordering values based on debezium payload
      @@ -156,7 +171,8 @@ class TestEightToNineUpgrade extends RecordLevelIndexTestBase {
             option(HoodieCompactionConfig.INLINE_COMPACT.key(), "false").
             mode(SaveMode.Append).
             save(basePath)
      -    checkResultForVersion9("", payloadClass)
      +    orderingValue = DebeziumConstants.FLATTENED_FILE_COL_NAME + "," + DebeziumConstants.FLATTENED_POS_COL_NAME
      +    checkResultForVersion9("", payloadClass, orderingValue)
       
           // Downgrade to table version 8 explicitly.
           // Note that downgrade is NOT automatic.
      @@ -181,7 +197,8 @@ class TestEightToNineUpgrade extends RecordLevelIndexTestBase {
             option(HoodieWriteConfig.WRITE_TABLE_VERSION.key(), "8").
             mode(SaveMode.Append).
             save(basePath)
      -    checkResultForVersion8(payloadClass)
      +    orderingValue = DebeziumConstants.ADDED_SEQ_COL_NAME
      +    checkResultForVersion8(payloadClass, orderingValue)
       
           tableName = "testUpgradeDowngradeMySqlDebeziumPayload"
           spark.sql(s"create table testUpgradeDowngradeMySqlDebeziumPayload using hudi location '$basePath'")
      @@ -197,9 +214,12 @@ class TestEightToNineUpgrade extends RecordLevelIndexTestBase {
           spark.sql(s"drop table default.$tableName")
         }
       
      -  def checkResultForVersion8(payloadClass: String): Unit = {
      +  def checkResultForVersion8(payloadClass: String, orderingValue: String): Unit = {
           metaClient = HoodieTableMetaClient.reload(metaClient)
           assertEquals(HoodieTableVersion.EIGHT, metaClient.getTableConfig.getTableVersion)
      +    // Check ordering fields
      +    assertEquals(orderingValue, metaClient.getTableConfig.getString(HoodieTableConfig.PRECOMBINE_FIELD.key()))
      +    assertTrue(StringUtils.isNullOrEmpty(metaClient.getTableConfig.getString(HoodieTableConfig.ORDERING_FIELDS.key())))
           // The payload class should be maintained.
           assertEquals(payloadClass, metaClient.getTableConfig.getPayloadClass)
           // The partial update mode should not be present
      @@ -211,17 +231,21 @@ class TestEightToNineUpgrade extends RecordLevelIndexTestBase {
             assertEquals(HoodieRecordMerger.PAYLOAD_BASED_MERGE_STRATEGY_UUID, metaClient.getTableConfig.getRecordMergeStrategyId)
           }
           if (payloadClass.equals(classOf[MySqlDebeziumAvroPayload].getName)) {
      -      assertFalse(metaClient.getTableConfig.getPreCombineFieldsStr.isEmpty)
      -      assertEquals(DebeziumConstants.ADDED_SEQ_COL_NAME, metaClient.getTableConfig.getPreCombineFieldsStr.get())
      +      assertFalse(metaClient.getTableConfig.getOrderingFieldsStr.isEmpty)
      +      assertEquals(DebeziumConstants.ADDED_SEQ_COL_NAME, metaClient.getTableConfig.getOrderingFieldsStr.get())
           }
         }
       
      -  def checkResultForVersion9(partitionFields: String, payloadClass: String): Unit = {
      +  def checkResultForVersion9(partitionFields: String, payloadClass: String, orderingValue: String): Unit = {
           metaClient = HoodieTableMetaClient.reload(metaClient)
           assertEquals(HoodieTableVersion.NINE, metaClient.getTableConfig.getTableVersion)
           assertEquals(
             partitionFields,
             HoodieTableConfig.getPartitionFieldPropForKeyGenerator(metaClient.getTableConfig).get())
      +    // Check ordering fields
      +    assertEquals(orderingValue, metaClient.getTableConfig.getString(HoodieTableConfig.ORDERING_FIELDS.key()))
      +    assertTrue(StringUtils.isNullOrEmpty(metaClient.getTableConfig.getString(HoodieTableConfig.PRECOMBINE_FIELD.key())))
      +
           assertEquals(payloadClass, metaClient.getTableConfig.getLegacyPayloadClass)
           // Based on the payload and table type, the merge mode is updated accordingly.
           if (payloadClass.equals(classOf[PartialUpdateAvroPayload].getName)) {
      @@ -261,7 +285,7 @@ class TestEightToNineUpgrade extends RecordLevelIndexTestBase {
             assertEquals(RecordMergeMode.EVENT_TIME_ORDERING, metaClient.getTableConfig.getRecordMergeMode)
             assertTrue(metaClient.getTableConfig.getPartialUpdateMode.isEmpty)
             assertEquals(DebeziumConstants.FLATTENED_FILE_COL_NAME + "," + DebeziumConstants.FLATTENED_POS_COL_NAME,
      -        metaClient.getTableConfig.getPreCombineFieldsStr.get())
      +        metaClient.getTableConfig.getOrderingFieldsStr.get())
           } else {
             assertTrue(metaClient.getTableConfig.getPartialUpdateMode.isEmpty)
           }
      diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestEmptyCommit.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestEmptyCommit.scala
      index c9e1c970f98c4..e85e12e632646 100644
      --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestEmptyCommit.scala
      +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestEmptyCommit.scala
      @@ -18,6 +18,7 @@
       package org.apache.hudi.functional
       
       import org.apache.hudi.{DataSourceWriteOptions, HoodieDataSourceHelpers}
      +import org.apache.hudi.common.table.HoodieTableConfig
       import org.apache.hudi.config.HoodieWriteConfig
       import org.apache.hudi.testutils.HoodieSparkClientTestBase
       
      @@ -34,7 +35,7 @@ class TestEmptyCommit extends HoodieSparkClientTestBase {
           "hoodie.upsert.shuffle.parallelism" -> "4",
           DataSourceWriteOptions.RECORDKEY_FIELD.key -> "_row_key",
           DataSourceWriteOptions.PARTITIONPATH_FIELD.key -> "partition",
      -    DataSourceWriteOptions.PRECOMBINE_FIELD.key -> "timestamp",
      +    HoodieTableConfig.ORDERING_FIELDS.key -> "timestamp",
           HoodieWriteConfig.TBL_NAME.key -> "hoodie_test"
         )
       
      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 9b6feacca0f1c..4f5e14c092777 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
      @@ -65,7 +65,7 @@ class TestGetPartitionValuesFromPath extends HoodieSparkSqlTestBase {
                  |tblproperties (
                  | primaryKey = 'id',
                  | type = 'mor',
      -           | preCombineField = 'ts',
      +           | orderingFields = 'ts',
                  | hoodie.datasource.write.drop.partition.columns = 'true'
                  |)
                  |partitioned by (region, dt)""".stripMargin)
      diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestHoodieActiveTimeline.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestHoodieActiveTimeline.scala
      index 0e581fdbfbf1a..cf1a402079038 100644
      --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestHoodieActiveTimeline.scala
      +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestHoodieActiveTimeline.scala
      @@ -20,7 +20,7 @@ package org.apache.hudi.functional
       import org.apache.hudi.{DataSourceWriteOptions, HoodieDataSourceHelpers}
       import org.apache.hudi.client.WriteClientTestUtils
       import org.apache.hudi.common.model.HoodieFileFormat
      -import org.apache.hudi.common.table.HoodieTableMetaClient
      +import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient}
       import org.apache.hudi.common.table.timeline.{HoodieInstant, HoodieTimeline}
       import org.apache.hudi.common.testutils.HoodieTestUtils
       import org.apache.hudi.common.testutils.RawTripTestPayload.recordsToStrings
      @@ -46,7 +46,7 @@ class TestHoodieActiveTimeline extends HoodieSparkClientTestBase {
           "hoodie.upsert.shuffle.parallelism" -> "4",
           DataSourceWriteOptions.RECORDKEY_FIELD.key -> "_row_key",
           DataSourceWriteOptions.PARTITIONPATH_FIELD.key -> "partition",
      -    DataSourceWriteOptions.PRECOMBINE_FIELD.key -> "timestamp",
      +    HoodieTableConfig.ORDERING_FIELDS.key -> "timestamp",
           HoodieWriteConfig.TBL_NAME.key -> "hoodie_test"
         )
       
      diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestHoodieMultipleBaseFileFormat.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestHoodieMultipleBaseFileFormat.scala
      index bf34735786508..1dab17c13d4cc 100644
      --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestHoodieMultipleBaseFileFormat.scala
      +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestHoodieMultipleBaseFileFormat.scala
      @@ -50,7 +50,7 @@ class TestHoodieMultipleBaseFileFormat extends HoodieSparkClientTestBase with Sp
           HoodieTableConfig.MULTIPLE_BASE_FILE_FORMATS_ENABLE.key -> "true",
           DataSourceWriteOptions.RECORDKEY_FIELD.key -> "_row_key",
           DataSourceWriteOptions.PARTITIONPATH_FIELD.key -> "partition",
      -    DataSourceWriteOptions.PRECOMBINE_FIELD.key -> "timestamp",
      +    HoodieTableConfig.ORDERING_FIELDS.key -> "timestamp",
           HoodieWriteConfig.TBL_NAME.key -> "hoodie_test"
         )
         val sparkOpts = Map(
      diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestIncrementalReadByStateTransitionTime.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestIncrementalReadByStateTransitionTime.scala
      index 703138f0c7c15..498c5d3c6c214 100644
      --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestIncrementalReadByStateTransitionTime.scala
      +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestIncrementalReadByStateTransitionTime.scala
      @@ -20,6 +20,7 @@ package org.apache.hudi.functional
       import org.apache.hudi.{DataSourceReadOptions, DataSourceWriteOptions}
       import org.apache.hudi.common.config.HoodieMetadataConfig
       import org.apache.hudi.common.model.HoodieTableType
      +import org.apache.hudi.common.table.HoodieTableConfig
       import org.apache.hudi.common.table.timeline.TimelineUtils.HollowCommitHandling.USE_TRANSITION_TIME
       import org.apache.hudi.common.testutils.RawTripTestPayload.recordsToStrings
       import org.apache.hudi.config.HoodieWriteConfig
      @@ -41,7 +42,7 @@ class TestIncrementalReadByStateTransitionTime extends HoodieSparkClientTestBase
           "hoodie.upsert.shuffle.parallelism" -> "4",
           DataSourceWriteOptions.RECORDKEY_FIELD.key -> "_row_key",
           DataSourceWriteOptions.PARTITIONPATH_FIELD.key -> "partition",
      -    DataSourceWriteOptions.PRECOMBINE_FIELD.key -> "timestamp",
      +    HoodieTableConfig.ORDERING_FIELDS.key -> "timestamp",
           HoodieWriteConfig.TBL_NAME.key -> "hoodie_test",
           HoodieMetadataConfig.COMPACT_NUM_DELTA_COMMITS.key -> "1"
         )
      diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestIncrementalReadWithFullTableScan.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestIncrementalReadWithFullTableScan.scala
      index 985fd1f8de03b..615dd88257297 100644
      --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestIncrementalReadWithFullTableScan.scala
      +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestIncrementalReadWithFullTableScan.scala
      @@ -20,6 +20,7 @@ package org.apache.hudi.functional
       import org.apache.hudi.{DataSourceReadOptions, DataSourceWriteOptions}
       import org.apache.hudi.common.config.HoodieMetadataConfig
       import org.apache.hudi.common.model.HoodieTableType
      +import org.apache.hudi.common.table.HoodieTableConfig
       import org.apache.hudi.common.table.timeline.{HoodieInstant, HoodieInstantTimeGenerator, InstantComparison}
       import org.apache.hudi.common.table.timeline.HoodieInstantTimeGenerator.instantTimeMinusMillis
       import org.apache.hudi.common.table.timeline.InstantComparison.compareTimestamps
      @@ -70,7 +71,7 @@ class TestIncrementalReadWithFullTableScan extends HoodieSparkClientTestBase {
             "hoodie.upsert.shuffle.parallelism" -> "4",
             DataSourceWriteOptions.RECORDKEY_FIELD.key -> "_row_key",
             DataSourceWriteOptions.PARTITIONPATH_FIELD.key -> "partition",
      -      DataSourceWriteOptions.PRECOMBINE_FIELD.key -> "timestamp",
      +      HoodieTableConfig.ORDERING_FIELDS.key -> "timestamp",
             HoodieWriteConfig.TBL_NAME.key -> "hoodie_test",
             HoodieMetadataConfig.COMPACT_NUM_DELTA_COMMITS.key -> "1"
           )
      diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestLayoutOptimization.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestLayoutOptimization.scala
      index 20db00f7d1bdd..00fc5a4dc82c2 100644
      --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestLayoutOptimization.scala
      +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestLayoutOptimization.scala
      @@ -21,6 +21,7 @@ package org.apache.hudi.functional
       import org.apache.hudi.{DataSourceReadOptions, DataSourceWriteOptions}
       import org.apache.hudi.HoodieFileIndex.DataSkippingFailureMode
       import org.apache.hudi.common.config.HoodieMetadataConfig
      +import org.apache.hudi.common.table.HoodieTableConfig
       import org.apache.hudi.common.table.timeline.{HoodieInstant, HoodieTimeline}
       import org.apache.hudi.common.testutils.RawTripTestPayload.recordsToStrings
       import org.apache.hudi.config.{HoodieClusteringConfig, HoodieWriteConfig}
      @@ -62,7 +63,7 @@ class TestLayoutOptimization extends HoodieSparkClientTestBase {
           "hoodie.bulkinsert.shuffle.parallelism" -> "4",
           DataSourceWriteOptions.RECORDKEY_FIELD.key() -> "_row_key",
           DataSourceWriteOptions.PARTITIONPATH_FIELD.key() -> "partition",
      -    DataSourceWriteOptions.PRECOMBINE_FIELD.key() -> "timestamp",
      +    HoodieTableConfig.ORDERING_FIELDS.key() -> "timestamp",
           HoodieWriteConfig.TBL_NAME.key -> "hoodie_test"
         ) ++ metadataOpts
       
      diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestMORDataSource.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestMORDataSource.scala
      index 5b215b82c4c0f..0727ff63c93a3 100644
      --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestMORDataSource.scala
      +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestMORDataSource.scala
      @@ -69,7 +69,7 @@ class TestMORDataSource extends HoodieSparkClientTestBase with SparkDatasetMixin
           "hoodie.upsert.shuffle.parallelism" -> "4",
           DataSourceWriteOptions.RECORDKEY_FIELD.key -> "_row_key",
           DataSourceWriteOptions.PARTITIONPATH_FIELD.key -> "partition",
      -    DataSourceWriteOptions.PRECOMBINE_FIELD.key -> "timestamp",
      +    HoodieTableConfig.ORDERING_FIELDS.key -> "timestamp",
           HoodieWriteConfig.TBL_NAME.key -> "hoodie_test"
         )
         val sparkOpts = Map(
      @@ -142,11 +142,11 @@ class TestMORDataSource extends HoodieSparkClientTestBase with SparkDatasetMixin
           var (writeOpts, _) = getWriterReaderOpts(writeType)
           readOpts = readOpts ++ Map(HoodieStorageConfig.LOGFILE_DATA_BLOCK_FORMAT.key -> logType)
           writeOpts = writeOpts ++ Map(HoodieStorageConfig.LOGFILE_DATA_BLOCK_FORMAT.key -> logType)
      -    readOpts = readOpts - DataSourceWriteOptions.PRECOMBINE_FIELD.key
      +    readOpts = readOpts - HoodieTableConfig.ORDERING_FIELDS.key
           if (!hasPreCombineField) {
      -      writeOpts = writeOpts - DataSourceWriteOptions.PRECOMBINE_FIELD.key
      +      writeOpts = writeOpts - HoodieTableConfig.ORDERING_FIELDS.key
           } else {
      -      writeOpts = writeOpts ++ Map(DataSourceWriteOptions.PRECOMBINE_FIELD.key ->
      +      writeOpts = writeOpts ++ Map(HoodieTableConfig.ORDERING_FIELDS.key ->
               (if (isNullOrEmpty(precombineField)) "" else precombineField))
           }
           if (!isNullOrEmpty(recordMergeMode)) {
      @@ -206,14 +206,14 @@ class TestMORDataSource extends HoodieSparkClientTestBase with SparkDatasetMixin
               Map(HoodieTableConfig.VERSION.key -> HoodieTableVersion.SIX.versionCode().toString)
             } ++
             (if (hasPreCombineField && !isNullOrEmpty(precombineField)) {
      -        Map(HoodieTableConfig.PRECOMBINE_FIELDS.key -> precombineField)
      +        Map(HoodieTableConfig.ORDERING_FIELDS.key -> precombineField)
             } else {
               Map()
             })).asJava
           val nonExistentConfigs: java.util.List[String] = (if (hasPreCombineField) {
             Seq[String]()
           } else {
      -      Seq(HoodieTableConfig.PRECOMBINE_FIELDS.key)
      +      Seq(HoodieTableConfig.ORDERING_FIELDS.key)
           }).asJava
           HoodieTestUtils.validateTableConfig(storage, basePath, expectedConfigs, nonExistentConfigs)
           val commit1CompletionTime = if (tableVersion.greaterThanOrEquals(HoodieTableVersion.EIGHT)) {
      @@ -772,7 +772,7 @@ class TestMORDataSource extends HoodieSparkClientTestBase with SparkDatasetMixin
             .option(PAYLOAD_CLASS_NAME.key, classOf[DefaultHoodieRecordPayload].getCanonicalName)
             .option(DataSourceWriteOptions.TABLE_TYPE.key, DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL)
             .option(RECORDKEY_FIELD.key, "id")
      -      .option(PRECOMBINE_FIELD.key, "version")
      +      .option(HoodieTableConfig.ORDERING_FIELDS.key, "version")
             .option(PARTITIONPATH_FIELD.key, "")
             .mode(SaveMode.Append)
             .save(basePath)
      @@ -1143,7 +1143,7 @@ class TestMORDataSource extends HoodieSparkClientTestBase with SparkDatasetMixin
             "hoodie.upsert.shuffle.parallelism" -> "4",
             DataSourceWriteOptions.RECORDKEY_FIELD.key -> "_row_key",
             DataSourceWriteOptions.PARTITIONPATH_FIELD.key -> "partition",
      -      DataSourceWriteOptions.PRECOMBINE_FIELD.key -> "timestamp",
      +      HoodieTableConfig.ORDERING_FIELDS.key -> "timestamp",
             HoodieWriteConfig.TBL_NAME.key -> "hoodie_test",
             DataSourceWriteOptions.OPERATION.key() -> DataSourceWriteOptions.UPSERT_OPERATION_OPT_VAL,
             DataSourceWriteOptions.TABLE_TYPE.key()-> DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL,
      @@ -1185,7 +1185,7 @@ class TestMORDataSource extends HoodieSparkClientTestBase with SparkDatasetMixin
             "hoodie.upsert.shuffle.parallelism" -> "4",
             DataSourceWriteOptions.RECORDKEY_FIELD.key -> "_row_key",
             DataSourceWriteOptions.PARTITIONPATH_FIELD.key -> "partition",
      -      DataSourceWriteOptions.PRECOMBINE_FIELD.key -> "timestamp",
      +      HoodieTableConfig.ORDERING_FIELDS.key -> "timestamp",
             HoodieWriteConfig.TBL_NAME.key -> "hoodie_test",
             DataSourceWriteOptions.OPERATION.key() -> DataSourceWriteOptions.UPSERT_OPERATION_OPT_VAL,
             DataSourceWriteOptions.TABLE_TYPE.key() -> DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL,
      @@ -1322,14 +1322,14 @@ class TestMORDataSource extends HoodieSparkClientTestBase with SparkDatasetMixin
         @ValueSource(booleans = Array(true, false))
         def testReadOptimizedQueryAfterInflightCompactionAndCompletedDeltaCommit(enableFileIndex: Boolean): Unit = {
           val (tableName, tablePath) = ("hoodie_mor_ro_read_test_table", s"${basePath}_mor_test_table")
      -    val precombineField = "col3"
      +    val orderingFields = "col3"
           val recordKeyField = "key"
           val dataField = "age"
       
           val options = Map[String, String](
             DataSourceWriteOptions.TABLE_TYPE.key -> HoodieTableType.MERGE_ON_READ.name,
             DataSourceWriteOptions.OPERATION.key -> UPSERT_OPERATION_OPT_VAL,
      -      DataSourceWriteOptions.PRECOMBINE_FIELD.key -> precombineField,
      +      HoodieTableConfig.ORDERING_FIELDS.key -> orderingFields,
             DataSourceWriteOptions.RECORDKEY_FIELD.key -> recordKeyField,
             DataSourceWriteOptions.PARTITIONPATH_FIELD.key -> "",
             DataSourceWriteOptions.KEYGENERATOR_CLASS_NAME.key -> "org.apache.hudi.keygen.NonpartitionedKeyGenerator",
      @@ -1345,7 +1345,7 @@ class TestMORDataSource extends HoodieSparkClientTestBase with SparkDatasetMixin
           // fg1_dc1.parquet written to storage
           // For record key "0", the row is (0, 0, 1000)
           val firstDf = spark.range(0, 10).toDF(recordKeyField)
      -      .withColumn(precombineField, expr(recordKeyField))
      +      .withColumn(orderingFields, expr(recordKeyField))
             .withColumn(dataField, expr(recordKeyField + " + 1000"))
       
           firstDf.write.format("hudi")
      @@ -1358,7 +1358,7 @@ class TestMORDataSource extends HoodieSparkClientTestBase with SparkDatasetMixin
           // .fg1_dc1.log (from DC2) written to storage
           // For record key "0", the row is (0, 0, 2000)
           val secondDf = spark.range(0, 10).toDF(recordKeyField)
      -      .withColumn(precombineField, expr(recordKeyField))
      +      .withColumn(orderingFields, expr(recordKeyField))
             .withColumn(dataField, expr(recordKeyField + " + 2000"))
       
           secondDf.write.format("hudi")
      @@ -1391,7 +1391,7 @@ class TestMORDataSource extends HoodieSparkClientTestBase with SparkDatasetMixin
           // .fg1_c3.log (from DC4) is written to storage
           // For record key "0", the row is (0, 0, 3000)
           val thirdDf = spark.range(0, 10).toDF(recordKeyField)
      -      .withColumn(precombineField, expr(recordKeyField))
      +      .withColumn(orderingFields, expr(recordKeyField))
             .withColumn(dataField, expr(recordKeyField + " + 3000"))
       
           thirdDf.write.format("hudi")
      @@ -1420,14 +1420,14 @@ class TestMORDataSource extends HoodieSparkClientTestBase with SparkDatasetMixin
       //  @ValueSource(booleans = Array(true, false))
         def testSnapshotQueryAfterInflightDeltaCommit(enableFileIndex: Boolean, tableVersion: Int): Unit = {
           val (tableName, tablePath) = ("hoodie_mor_snapshot_read_test_table", s"${basePath}_mor_test_table")
      -    val precombineField = "col3"
      +    val orderingFields = "col3"
           val recordKeyField = "key"
           val dataField = "age"
       
           val options = Map[String, String](
             DataSourceWriteOptions.TABLE_TYPE.key -> HoodieTableType.MERGE_ON_READ.name,
             DataSourceWriteOptions.OPERATION.key -> UPSERT_OPERATION_OPT_VAL,
      -      DataSourceWriteOptions.PRECOMBINE_FIELD.key -> precombineField,
      +      HoodieTableConfig.ORDERING_FIELDS.key -> orderingFields,
             DataSourceWriteOptions.RECORDKEY_FIELD.key -> recordKeyField,
             DataSourceWriteOptions.PARTITIONPATH_FIELD.key -> "",
             DataSourceWriteOptions.KEYGENERATOR_CLASS_NAME.key -> "org.apache.hudi.keygen.NonpartitionedKeyGenerator",
      @@ -1442,7 +1442,7 @@ class TestMORDataSource extends HoodieSparkClientTestBase with SparkDatasetMixin
             HoodieWriteConfig.WRITE_TABLE_VERSION.key() -> tableVersion.toString)
       
           val firstDf = spark.range(0, 10).toDF(recordKeyField)
      -      .withColumn(precombineField, expr(recordKeyField))
      +      .withColumn(orderingFields, expr(recordKeyField))
             .withColumn(dataField, expr(recordKeyField + " + 1000"))
           firstDf.write.format("hudi")
             .options(writeOpts)
      @@ -1450,7 +1450,7 @@ class TestMORDataSource extends HoodieSparkClientTestBase with SparkDatasetMixin
             .save(tablePath)
       
           val secondDf = spark.range(0, 10).toDF(recordKeyField)
      -      .withColumn(precombineField, expr(recordKeyField))
      +      .withColumn(orderingFields, expr(recordKeyField))
             .withColumn(dataField, expr(recordKeyField + " + 2000"))
           secondDf.write.format("hudi")
             .options(writeOpts)
      @@ -1742,7 +1742,7 @@ class TestMORDataSource extends HoodieSparkClientTestBase with SparkDatasetMixin
             "hoodie.upsert.shuffle.parallelism" -> "4",
             DataSourceWriteOptions.RECORDKEY_FIELD.key -> "_row_key",
             DataSourceWriteOptions.PARTITIONPATH_FIELD.key -> "partition",
      -      DataSourceWriteOptions.PRECOMBINE_FIELD.key() -> "timestamp,rider",
      +      HoodieTableConfig.ORDERING_FIELDS.key() -> "timestamp,rider",
             DataSourceWriteOptions.RECORD_MERGE_MODE.key() -> RecordMergeMode.EVENT_TIME_ORDERING.name(),
             HoodieWriteConfig.TBL_NAME.key -> "hoodie_test"
           )
      @@ -1935,7 +1935,7 @@ class TestMORDataSource extends HoodieSparkClientTestBase with SparkDatasetMixin
           Map(
             // Don't override table name - let fixture table configuration take precedence
             DataSourceWriteOptions.RECORDKEY_FIELD.key -> "id",         // Fixture uses 'id' as record key
      -      DataSourceWriteOptions.PRECOMBINE_FIELD.key -> "ts",        // Fixture uses 'ts' as precombine field
      +      HoodieTableConfig.ORDERING_FIELDS.key -> "ts",        // Fixture uses 'ts' as precombine field
             DataSourceWriteOptions.PARTITIONPATH_FIELD.key -> "partition", // Fixture uses 'partition' as partition field
             "hoodie.upsert.shuffle.parallelism" -> "2",
             "hoodie.insert.shuffle.parallelism" -> "2"
      diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestMORDataSourceStorage.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestMORDataSourceStorage.scala
      index 487ec0f1418cf..fa4fd82f8868f 100644
      --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestMORDataSourceStorage.scala
      +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestMORDataSourceStorage.scala
      @@ -61,7 +61,7 @@ class TestMORDataSourceStorage extends SparkClientFunctionalTestHarness {
           "false,,false",
           "false,fare.currency,true"
         ))
      -  def testMergeOnReadStorage(isMetadataEnabled: Boolean, preCombineField: String, useFileGroupReader: Boolean): Unit = {
      +  def testMergeOnReadStorage(isMetadataEnabled: Boolean, orderingFields: String, useFileGroupReader: Boolean): Unit = {
           val commonOpts = Map(
             "hoodie.insert.shuffle.parallelism" -> "4",
             "hoodie.upsert.shuffle.parallelism" -> "4",
      @@ -69,7 +69,7 @@ class TestMORDataSourceStorage extends SparkClientFunctionalTestHarness {
             "hoodie.delete.shuffle.parallelism" -> "1",
             DataSourceWriteOptions.RECORDKEY_FIELD.key -> "_row_key",
             DataSourceWriteOptions.PARTITIONPATH_FIELD.key -> "partition_path",
      -      DataSourceWriteOptions.PRECOMBINE_FIELD.key -> "timestamp",
      +      HoodieTableConfig.ORDERING_FIELDS.key -> "timestamp",
             HoodieWriteConfig.TBL_NAME.key -> "hoodie_test",
             HoodieWriteConfig.MARKERS_TIMELINE_SERVER_BASED_BATCH_INTERVAL_MS.key -> "10"
           )
      @@ -78,8 +78,8 @@ class TestMORDataSourceStorage extends SparkClientFunctionalTestHarness {
       
           var options: Map[String, String] = commonOpts +
             (HoodieMetadataConfig.ENABLE.key -> String.valueOf(isMetadataEnabled))
      -    if (!StringUtils.isNullOrEmpty(preCombineField)) {
      -      options += (DataSourceWriteOptions.PRECOMBINE_FIELD.key() -> preCombineField)
      +    if (!StringUtils.isNullOrEmpty(orderingFields)) {
      +      options += (HoodieTableConfig.ORDERING_FIELDS.key() -> orderingFields)
           }
           if (useFileGroupReader) {
             options += (HoodieReaderConfig.FILE_GROUP_READER_ENABLED.key() -> String.valueOf(useFileGroupReader))
      @@ -155,7 +155,7 @@ class TestMORDataSourceStorage extends SparkClientFunctionalTestHarness {
             HoodieWriteConfig.WRITE_RECORD_POSITIONS.key -> "true",
             DataSourceWriteOptions.RECORDKEY_FIELD.key -> "_row_key",
             DataSourceWriteOptions.PARTITIONPATH_FIELD.key -> "partition_path",
      -      DataSourceWriteOptions.PRECOMBINE_FIELD.key -> "timestamp",
      +      HoodieTableConfig.ORDERING_FIELDS.key -> "timestamp",
             HoodieWriteConfig.TBL_NAME.key -> "hoodie_test"
           )
       
      @@ -207,7 +207,7 @@ class TestMORDataSourceStorage extends SparkClientFunctionalTestHarness {
             HoodieWriteConfig.WRITE_RECORD_POSITIONS.key -> writeRecordPosition.toString,
             DataSourceWriteOptions.RECORDKEY_FIELD.key -> "_row_key",
             DataSourceWriteOptions.KEYGENERATOR_CLASS_NAME.key -> classOf[NonpartitionedKeyGenerator].getName,
      -      DataSourceWriteOptions.PRECOMBINE_FIELD.key -> "timestamp",
      +      HoodieTableConfig.ORDERING_FIELDS.key -> "timestamp",
             HoodieWriteConfig.TBL_NAME.key -> "hoodie_test",
             HoodieIndexConfig.INDEX_TYPE.key -> (if (enableNBCC) BUCKET.name else SIMPLE.name),
             HoodieLockConfig.LOCK_PROVIDER_CLASS_NAME.key() -> classOf[InProcessLockProvider].getName,
      diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestMORDataSourceWithBucketIndex.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestMORDataSourceWithBucketIndex.scala
      index e88cc35b36ce3..718a12518f65a 100644
      --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestMORDataSourceWithBucketIndex.scala
      +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestMORDataSourceWithBucketIndex.scala
      @@ -18,6 +18,7 @@
       package org.apache.hudi.functional
       
       import org.apache.hudi.{DataSourceReadOptions, DataSourceWriteOptions, HoodieDataSourceHelpers}
      +import org.apache.hudi.common.table.HoodieTableConfig
       import org.apache.hudi.common.testutils.HoodieTestDataGenerator
       import org.apache.hudi.common.testutils.RawTripTestPayload.recordsToStrings
       import org.apache.hudi.config.{HoodieIndexConfig, HoodieLayoutConfig, HoodieWriteConfig}
      @@ -45,7 +46,7 @@ class TestMORDataSourceWithBucketIndex extends HoodieSparkClientTestBase {
           "hoodie.bulkinsert.shuffle.parallelism" -> "2",
           DataSourceWriteOptions.RECORDKEY_FIELD.key -> "_row_key",
           DataSourceWriteOptions.PARTITIONPATH_FIELD.key -> "partition",
      -    DataSourceWriteOptions.PRECOMBINE_FIELD.key -> "timestamp",
      +    HoodieTableConfig.ORDERING_FIELDS.key -> "timestamp",
           HoodieWriteConfig.TBL_NAME.key -> "hoodie_test",
           HoodieIndexConfig.INDEX_TYPE.key -> IndexType.BUCKET.name,
           HoodieIndexConfig.BUCKET_INDEX_NUM_BUCKETS.key -> "8",
      diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestMetadataRecordIndex.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestMetadataRecordIndex.scala
      index e1ece6d88ecb2..3e386e7f029a9 100644
      --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestMetadataRecordIndex.scala
      +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestMetadataRecordIndex.scala
      @@ -56,7 +56,7 @@ class TestMetadataRecordIndex extends HoodieSparkClientTestBase {
           HoodieWriteConfig.TBL_NAME.key -> "hoodie_test",
           RECORDKEY_FIELD.key -> "_row_key",
           PARTITIONPATH_FIELD.key -> "partition",
      -    PRECOMBINE_FIELD.key -> "timestamp",
      +    HoodieTableConfig.ORDERING_FIELDS.key -> "timestamp",
           HoodieTableConfig.POPULATE_META_FIELDS.key -> "true"
         ) ++ metadataOpts
         var mergedDfList: List[DataFrame] = List.empty
      diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestMetadataTableWithSparkDataSource.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestMetadataTableWithSparkDataSource.scala
      index 43b6b6511297c..d01be74eeae67 100644
      --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestMetadataTableWithSparkDataSource.scala
      +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestMetadataTableWithSparkDataSource.scala
      @@ -23,7 +23,7 @@ import org.apache.hudi.avro.HoodieAvroWrapperUtils.unwrapAvroValueWrapper
       import org.apache.hudi.client.common.HoodieSparkEngineContext
       import org.apache.hudi.common.config.HoodieMetadataConfig
       import org.apache.hudi.common.model.HoodieColumnRangeMetadata
      -import org.apache.hudi.common.table.HoodieTableMetaClient
      +import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient}
       import org.apache.hudi.common.table.timeline.HoodieTimeline
       import org.apache.hudi.common.table.view.FileSystemViewManager
       import org.apache.hudi.common.testutils.HoodieTestDataGenerator
      @@ -62,7 +62,7 @@ class TestMetadataTableWithSparkDataSource extends SparkClientFunctionalTestHarn
           "hoodie.bulkinsert.shuffle.parallelism" -> "2",
           "hoodie.delete.shuffle.parallelism" -> "1",
           DataSourceWriteOptions.RECORDKEY_FIELD.key -> "_row_key",
      -    DataSourceWriteOptions.PRECOMBINE_FIELD.key -> "timestamp",
      +    HoodieTableConfig.ORDERING_FIELDS.key -> "timestamp",
           HoodieWriteConfig.TBL_NAME.key -> "hoodie_test"
         )
       
      diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestMetricsReporter.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestMetricsReporter.scala
      index 5f6b86662af34..a41fe883b8466 100644
      --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestMetricsReporter.scala
      +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestMetricsReporter.scala
      @@ -20,6 +20,7 @@ package org.apache.hudi.functional
       import org.apache.hudi.{DataSourceWriteOptions, SparkDatasetMixin}
       import org.apache.hudi.HoodieConversionUtils.toJavaOption
       import org.apache.hudi.common.config.HoodieMetadataConfig
      +import org.apache.hudi.common.table.HoodieTableConfig
       import org.apache.hudi.common.testutils.RawTripTestPayload.recordsToStrings
       import org.apache.hudi.common.util.Option
       import org.apache.hudi.config.HoodieWriteConfig
      @@ -48,7 +49,7 @@ class TestMetricsReporter extends HoodieSparkClientTestBase with SparkDatasetMix
           "hoodie.upsert.shuffle.parallelism" -> "4",
           DataSourceWriteOptions.RECORDKEY_FIELD.key -> "_row_key",
           DataSourceWriteOptions.PARTITIONPATH_FIELD.key -> "partition",
      -    DataSourceWriteOptions.PRECOMBINE_FIELD.key -> "timestamp",
      +    HoodieTableConfig.ORDERING_FIELDS.key -> "timestamp",
           HoodieWriteConfig.TBL_NAME.key -> "hoodie_test"
         )
       
      diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestParquetColumnProjection.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestParquetColumnProjection.scala
      index 3be899b43e4ac..8784e254ea5d0 100644
      --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestParquetColumnProjection.scala
      +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestParquetColumnProjection.scala
      @@ -50,7 +50,7 @@ class TestParquetColumnProjection extends SparkClientFunctionalTestHarness with
           "hoodie.bulkinsert.shuffle.parallelism" -> "2",
           "hoodie.delete.shuffle.parallelism" -> "1",
           DataSourceWriteOptions.RECORDKEY_FIELD.key -> "_row_key",
      -    DataSourceWriteOptions.PRECOMBINE_FIELD.key -> "timestamp",
      +    HoodieTableConfig.ORDERING_FIELDS.key -> "timestamp",
           HoodieWriteConfig.TBL_NAME.key -> "hoodie_test",
           HoodieMetadataConfig.ENABLE.key -> "true"
           // NOTE: It's critical that we use non-partitioned table, since the way we track amount of bytes read
      diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestPartialUpdateAvroPayload.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestPartialUpdateAvroPayload.scala
      index 98ff000262239..ec5dd8dabf03d 100644
      --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestPartialUpdateAvroPayload.scala
      +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestPartialUpdateAvroPayload.scala
      @@ -23,6 +23,7 @@ import org.apache.hudi.HoodieConversionUtils.toJavaOption
       import org.apache.hudi.QuickstartUtils.{convertToStringList, getQuickstartWriteConfigs}
       import org.apache.hudi.common.config.{HoodieReaderConfig, RecordMergeMode}
       import org.apache.hudi.common.model.HoodieTableType
      +import org.apache.hudi.common.table.HoodieTableConfig
       import org.apache.hudi.common.util.Option
       import org.apache.hudi.config.HoodieWriteConfig
       import org.apache.hudi.testutils.HoodieClientTestBase
      @@ -85,7 +86,7 @@ class TestPartialUpdateAvroPayload extends HoodieClientTestBase {
             .option(DataSourceWriteOptions.TABLE_TYPE.key(), hoodieTableType.name())
             .option(DataSourceWriteOptions.RECORDKEY_FIELD.key, "uuid")
             .option(DataSourceWriteOptions.PARTITIONPATH_FIELD.key, "partitionpath")
      -      .option(DataSourceWriteOptions.PRECOMBINE_FIELD.key, "ts")
      +      .option(HoodieTableConfig.ORDERING_FIELDS.key, "ts")
             .option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
             .option(HoodieWriteConfig.WRITE_PAYLOAD_CLASS_NAME.key, "org.apache.hudi.common.model.PartialUpdateAvroPayload")
             .option(HoodieWriteConfig.RECORD_MERGE_MODE.key(), RecordMergeMode.CUSTOM.name())
      @@ -118,7 +119,7 @@ class TestPartialUpdateAvroPayload extends HoodieClientTestBase {
             .option(DataSourceWriteOptions.TABLE_TYPE.key(), hoodieTableType.name())
             .option(DataSourceWriteOptions.RECORDKEY_FIELD.key, "uuid")
             .option(DataSourceWriteOptions.PARTITIONPATH_FIELD.key, "partitionpath")
      -      .option(DataSourceWriteOptions.PRECOMBINE_FIELD.key, "ts")
      +      .option(HoodieTableConfig.ORDERING_FIELDS.key, "ts")
             .option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.UPSERT_OPERATION_OPT_VAL)
             .option(HoodieWriteConfig.TBL_NAME.key, "hoodie_test")
             .option(HoodieWriteConfig.WRITE_PAYLOAD_CLASS_NAME.key, "org.apache.hudi.common.model.PartialUpdateAvroPayload")
      diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestPartitionStatsIndexWithSql.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestPartitionStatsIndexWithSql.scala
      index d58b5d81e8c2e..ebe95bcecc3e9 100644
      --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestPartitionStatsIndexWithSql.scala
      +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestPartitionStatsIndexWithSql.scala
      @@ -62,7 +62,7 @@ class TestPartitionStatsIndexWithSql extends HoodieSparkSqlTestBase {
                    | tblproperties(
                    |    type = '$tableType',
                    |    primaryKey = 'id',
      -             |    preCombineField = 'ts',
      +             |    orderingFields = 'ts',
                    |    'hoodie.metadata.index.partition.stats.enable' = 'true',
                    |    'hoodie.metadata.index.column.stats.enable' = 'true',
                    |    'hoodie.metadata.index.column.stats.column.list' = 'name'
      @@ -118,7 +118,7 @@ class TestPartitionStatsIndexWithSql extends HoodieSparkSqlTestBase {
                    | tblproperties(
                    |    type = '$tableType',
                    |    primaryKey = 'id',
      -             |    preCombineField = 'ts',
      +             |    orderingFields = 'ts',
                    |    'hoodie.metadata.index.partition.stats.enable' = 'true',
                    |    'hoodie.metadata.index.column.stats.enable' = 'true',
                    |    'hoodie.metadata.index.column.stats.column.list' = 'name'
      @@ -146,7 +146,7 @@ class TestPartitionStatsIndexWithSql extends HoodieSparkSqlTestBase {
               val properties = metaClient.getTableConfig.getProps.asScala.toMap
               assertResult(true)(properties.contains(HoodieTableConfig.CREATE_SCHEMA.key))
               assertResult("dt")(properties(HoodieTableConfig.PARTITION_FIELDS.key))
      -        assertResult("ts")(properties(HoodieTableConfig.PRECOMBINE_FIELDS.key))
      +        assertResult("ts")(properties(HoodieTableConfig.ORDERING_FIELDS.key))
               assertResult(tableName)(metaClient.getTableConfig.getTableName)
               // Validate partition_stats index exists
               assertTrue(metaClient.getTableConfig.getMetadataPartitions.contains(PARTITION_STATS.getPartitionPath))
      @@ -203,7 +203,7 @@ class TestPartitionStatsIndexWithSql extends HoodieSparkSqlTestBase {
                    | options(
                    |    type = '$tableType',
                    |    primaryKey ='uuid',
      -             |    preCombineField = 'ts',
      +             |    orderingFields = 'ts',
                    |    hoodie.metadata.index.partition.stats.enable = 'true',
                    |    hoodie.metadata.index.column.stats.enable = 'true',
                    |    hoodie.metadata.index.column.stats.column.list = 'rider'
      @@ -349,7 +349,7 @@ class TestPartitionStatsIndexWithSql extends HoodieSparkSqlTestBase {
                    |tblproperties (
                    |  type = '$tableType',
                    |  primaryKey = 'id',
      -             |  preCombineField = 'price',
      +             |  orderingFields = 'price',
                    |  hoodie.metadata.index.partition.stats.enable = 'true',
                    |  hoodie.metadata.index.column.stats.enable = 'true'
                    |)
      @@ -401,7 +401,7 @@ class TestPartitionStatsIndexWithSql extends HoodieSparkSqlTestBase {
                    |tblproperties (
                    |  type = '$tableType',
                    |  primaryKey = 'id',
      -             |  preCombineField = 'price',
      +             |  orderingFields = 'price',
                    |  hoodie.metadata.index.partition.stats.enable = 'true',
                    |  hoodie.metadata.index.column.stats.enable = 'true',
                    |  hoodie.metadata.index.column.stats.column.list = 'price'
      @@ -488,7 +488,7 @@ class TestPartitionStatsIndexWithSql extends HoodieSparkSqlTestBase {
                  |tblproperties (
                  |  type = 'mor',
                  |  primaryKey = 'id',
      -           |  preCombineField = 'price',
      +           |  orderingFields = 'price',
                  |  hoodie.metadata.index.partition.stats.enable = 'true',
                  |  hoodie.metadata.index.column.stats.enable = 'true',
                  |  hoodie.metadata.index.column.stats.column.list = 'price'
      @@ -563,7 +563,7 @@ class TestPartitionStatsIndexWithSql extends HoodieSparkSqlTestBase {
                      |tblproperties (
                      |  type = '$tableType',
                      |  primaryKey = 'id',
      -               |  preCombineField = 'price',
      +               |  orderingFields = 'price',
                      |  hoodie.metadata.index.partition.stats.enable = 'true',
                      |  hoodie.metadata.index.column.stats.enable = 'true',
                      |  hoodie.metadata.index.column.stats.column.list = 'price'
      diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestPartitionStatsPruning.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestPartitionStatsPruning.scala
      index 26f9aa71ab311..fd8d4ddc1035c 100644
      --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestPartitionStatsPruning.scala
      +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestPartitionStatsPruning.scala
      @@ -20,7 +20,7 @@
       package org.apache.hudi.functional
       
       import org.apache.hudi.DataSourceWriteOptions
      -import org.apache.hudi.DataSourceWriteOptions.{PARTITIONPATH_FIELD, PRECOMBINE_FIELD, RECORDKEY_FIELD}
      +import org.apache.hudi.DataSourceWriteOptions.{ORDERING_FIELDS, PARTITIONPATH_FIELD, RECORDKEY_FIELD}
       import org.apache.hudi.common.config.HoodieMetadataConfig
       import org.apache.hudi.common.model.{HoodieRecord, HoodieTableType}
       import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient}
      @@ -58,7 +58,7 @@ class TestPartitionStatsPruning extends ColumnStatIndexTestBase {
             HoodieWriteConfig.TBL_NAME.key -> "hoodie_test",
             DataSourceWriteOptions.TABLE_TYPE.key -> testCase.tableType.toString,
             RECORDKEY_FIELD.key -> "c1",
      -      PRECOMBINE_FIELD.key -> "c1",
      +      HoodieTableConfig.ORDERING_FIELDS.key -> "c1",
             PARTITIONPATH_FIELD.key() -> "c8",
             HoodieTableConfig.POPULATE_META_FIELDS.key -> "true"
           ) ++ metadataOpts
      @@ -87,7 +87,7 @@ class TestPartitionStatsPruning extends ColumnStatIndexTestBase {
             HoodieWriteConfig.TBL_NAME.key -> "hoodie_test",
             DataSourceWriteOptions.TABLE_TYPE.key -> testCase.tableType.toString,
             RECORDKEY_FIELD.key -> "c1",
      -      PRECOMBINE_FIELD.key -> "c1",
      +      HoodieTableConfig.ORDERING_FIELDS.key -> "c1",
             PARTITIONPATH_FIELD.key() -> "c8",
             HoodieTableConfig.POPULATE_META_FIELDS.key -> "true",
             "hoodie.compact.inline.max.delta.commits" -> "10"
      @@ -195,7 +195,7 @@ class TestPartitionStatsPruning extends ColumnStatIndexTestBase {
             HoodieWriteConfig.TBL_NAME.key -> "hoodie_test",
             DataSourceWriteOptions.TABLE_TYPE.key -> testCase.tableType.toString,
             RECORDKEY_FIELD.key -> "c1",
      -      PRECOMBINE_FIELD.key -> "c1",
      +      HoodieTableConfig.ORDERING_FIELDS.key -> "c1",
             PARTITIONPATH_FIELD.key -> partitionCol,
             HoodieTableConfig.POPULATE_META_FIELDS.key -> "true",
             HoodieCompactionConfig.INLINE_COMPACT_NUM_DELTA_COMMITS.key -> "3"
      @@ -306,7 +306,7 @@ class TestPartitionStatsPruning extends ColumnStatIndexTestBase {
             HoodieWriteConfig.TBL_NAME.key -> "hoodie_test",
             DataSourceWriteOptions.TABLE_TYPE.key -> testCase.tableType.toString,
             RECORDKEY_FIELD.key -> "c1",
      -      PRECOMBINE_FIELD.key -> "c1",
      +      HoodieTableConfig.ORDERING_FIELDS.key -> "c1",
             PARTITIONPATH_FIELD.key() -> partitionCol,
             "hoodie.write.markers.type" -> "DIRECT",
             HoodieTableConfig.POPULATE_META_FIELDS.key -> "true"
      diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestPartitionedRecordLevelIndex.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestPartitionedRecordLevelIndex.scala
      index d3a02ab4debbc..a9ec4d984ad64 100644
      --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestPartitionedRecordLevelIndex.scala
      +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestPartitionedRecordLevelIndex.scala
      @@ -27,7 +27,7 @@ import org.apache.hudi.common.config.{HoodieMetadataConfig, TypedProperties}
       import org.apache.hudi.common.data.HoodieListData
       import org.apache.hudi.common.fs.FSUtils
       import org.apache.hudi.common.model.{HoodieRecordGlobalLocation, HoodieTableType}
      -import org.apache.hudi.common.table.TableSchemaResolver
      +import org.apache.hudi.common.table.{HoodieTableConfig, TableSchemaResolver}
       import org.apache.hudi.common.testutils.HoodieTestDataGenerator
       import org.apache.hudi.common.testutils.RawTripTestPayload.recordsToStrings
       import org.apache.hudi.common.util.{Option => HOption}
      @@ -69,7 +69,7 @@ class TestPartitionedRecordLevelIndex extends RecordLevelIndexTestBase {
             DataSourceWriteOptions.TABLE_TYPE.key -> tableType.name(),
             RECORDKEY_FIELD.key -> "_row_key",
             PARTITIONPATH_FIELD.key -> "data_partition_path",
      -      PRECOMBINE_FIELD.key -> "timestamp",
      +      HoodieTableConfig.ORDERING_FIELDS.key -> "timestamp",
             HoodieMetadataConfig.RECORD_INDEX_ENABLE_PROP.key()-> "false",
             HoodieMetadataConfig.PARTITIONED_RECORD_INDEX_ENABLE_PROP.key() -> "true",
             HoodieMetadataConfig.STREAMING_WRITE_ENABLED.key() -> streamingWriteEnabled.toString,
      @@ -342,7 +342,7 @@ class TestPartitionedRecordLevelIndex extends RecordLevelIndexTestBase {
             DataSourceWriteOptions.TABLE_TYPE.key -> tableType.name(),
             RECORDKEY_FIELD.key -> "_row_key",
             PARTITIONPATH_FIELD.key -> "data_partition_path",
      -      PRECOMBINE_FIELD.key -> "timestamp",
      +      HoodieTableConfig.ORDERING_FIELDS.key -> "timestamp",
             HoodieMetadataConfig.RECORD_INDEX_ENABLE_PROP.key()-> "false",
             HoodieMetadataConfig.SECONDARY_INDEX_ENABLE_PROP.key() -> "false",
             HoodieMetadataConfig.STREAMING_WRITE_ENABLED.key() -> streamingWriteEnabled.toString,
      diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestPayloadDeprecationFlow.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestPayloadDeprecationFlow.scala
      index 376d8f069af8c..b3de7692c65e1 100644
      --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestPayloadDeprecationFlow.scala
      +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestPayloadDeprecationFlow.scala
      @@ -20,7 +20,7 @@
       package org.apache.hudi.functional
       
       import org.apache.hudi.DataSourceWriteOptions
      -import org.apache.hudi.DataSourceWriteOptions.{OPERATION, PRECOMBINE_FIELD, RECORDKEY_FIELD, TABLE_TYPE}
      +import org.apache.hudi.DataSourceWriteOptions.{OPERATION, ORDERING_FIELDS, RECORDKEY_FIELD, TABLE_TYPE}
       import org.apache.hudi.common.config.TypedProperties
       import org.apache.hudi.common.model.{AWSDmsAvroPayload, DefaultHoodieRecordPayload, EventTimeAvroPayload, HoodieRecordMerger, OverwriteNonDefaultsWithLatestAvroPayload, OverwriteWithLatestAvroPayload, PartialUpdateAvroPayload}
       import org.apache.hudi.common.model.debezium.{DebeziumConstants, MySqlDebeziumAvroPayload, PostgresDebeziumAvroPayload}
      @@ -57,14 +57,14 @@ class TestPayloadDeprecationFlow extends SparkClientFunctionalTestHarness {
             (10, 4L, "rider-D", "driver-D", 34.15, "i", "10.1", 10, 1),
             (10, 5L, "rider-E", "driver-E", 17.85, "i", "10.1", 10, 1))
           val inserts = spark.createDataFrame(data).toDF(columns: _*)
      -    val precombineField = if (payloadClazz.equals(classOf[MySqlDebeziumAvroPayload].getName)) {
      +    val orderingFields = if (payloadClazz.equals(classOf[MySqlDebeziumAvroPayload].getName)) {
             "_event_seq"
           } else {
             "ts"
           }
           inserts.write.format("hudi").
             option(RECORDKEY_FIELD.key(), "_event_lsn").
      -      option(PRECOMBINE_FIELD.key(), precombineField).
      +      option(HoodieTableConfig.ORDERING_FIELDS.key(), orderingFields).
             option(TABLE_TYPE.key(), tableType).
             option(DataSourceWriteOptions.TABLE_NAME.key(), "test_table").
             option(HoodieCompactionConfig.INLINE_COMPACT.key(), "false").
      @@ -269,7 +269,7 @@ object TestPayloadDeprecationFlow {
                 HoodieTableConfig.RECORD_MERGE_MODE.key() -> "EVENT_TIME_ORDERING",
                 HoodieTableConfig.LEGACY_PAYLOAD_CLASS_NAME.key() -> classOf[MySqlDebeziumAvroPayload].getName,
                 HoodieTableConfig.RECORD_MERGE_STRATEGY_ID.key() -> HoodieRecordMerger.EVENT_TIME_BASED_MERGE_STRATEGY_UUID,
      -          HoodieTableConfig.PRECOMBINE_FIELDS.key() -> (DebeziumConstants.FLATTENED_FILE_COL_NAME + "," + DebeziumConstants.FLATTENED_POS_COL_NAME))),
      +          HoodieTableConfig.ORDERING_FIELDS.key() -> (DebeziumConstants.FLATTENED_FILE_COL_NAME + "," + DebeziumConstants.FLATTENED_POS_COL_NAME))),
             Arguments.of(
               "COPY_ON_WRITE",
               classOf[AWSDmsAvroPayload].getName,
      @@ -339,7 +339,7 @@ object TestPayloadDeprecationFlow {
                 HoodieTableConfig.RECORD_MERGE_MODE.key() -> "EVENT_TIME_ORDERING",
                 HoodieTableConfig.LEGACY_PAYLOAD_CLASS_NAME.key() -> classOf[MySqlDebeziumAvroPayload].getName,
                 HoodieTableConfig.RECORD_MERGE_STRATEGY_ID.key() -> HoodieRecordMerger.EVENT_TIME_BASED_MERGE_STRATEGY_UUID,
      -          HoodieTableConfig.PRECOMBINE_FIELDS.key() -> (DebeziumConstants.FLATTENED_FILE_COL_NAME + "," + DebeziumConstants.FLATTENED_POS_COL_NAME))),
      +          HoodieTableConfig.ORDERING_FIELDS.key() -> (DebeziumConstants.FLATTENED_FILE_COL_NAME + "," + DebeziumConstants.FLATTENED_POS_COL_NAME))),
             Arguments.of(
               "MERGE_ON_READ",
               classOf[AWSDmsAvroPayload].getName,
      diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestSecondaryIndexDataTypes.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestSecondaryIndexDataTypes.scala
      index 85c547981b076..1a0ca62704299 100644
      --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestSecondaryIndexDataTypes.scala
      +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestSecondaryIndexDataTypes.scala
      @@ -73,7 +73,7 @@ class TestSecondaryIndexDataTypes extends HoodieSparkSqlTestBase {
                  |) USING HUDI
                  |OPTIONS (
                  |  primaryKey = 'id',
      -           |  preCombineField = 'ts',
      +           |  orderingFields = 'ts',
                  |  'hoodie.metadata.enable' = 'true',
                  |  'hoodie.metadata.record.index.enable' = 'true',
                  |  'hoodie.datasource.write.partitionpath.field' = 'partition_col'
      @@ -284,7 +284,7 @@ class TestSecondaryIndexDataTypes extends HoodieSparkSqlTestBase {
                  |) USING HUDI
                  |OPTIONS (
                  |  primaryKey = 'id',
      -           |  preCombineField = 'ts',
      +           |  orderingFields = 'ts',
                  |  'hoodie.metadata.enable' = 'true',
                  |  'hoodie.metadata.record.index.enable' = 'true',
                  |  'hoodie.datasource.write.partitionpath.field' = 'partition_col'
      diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestSecondaryIndexPruning.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestSecondaryIndexPruning.scala
      index f412b696174cf..b63bf6198cafb 100644
      --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestSecondaryIndexPruning.scala
      +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestSecondaryIndexPruning.scala
      @@ -27,7 +27,7 @@ import org.apache.hudi.client.transaction.SimpleConcurrentFileWritesConflictReso
       import org.apache.hudi.client.transaction.lock.InProcessLockProvider
       import org.apache.hudi.common.config.{HoodieMetadataConfig, RecordMergeMode, TypedProperties}
       import org.apache.hudi.common.model._
      -import org.apache.hudi.common.table.HoodieTableMetaClient
      +import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient}
       import org.apache.hudi.common.table.timeline.HoodieInstant
       import org.apache.hudi.common.table.view.HoodieTableFileSystemView
       import org.apache.hudi.common.testutils.HoodieTestUtils
      @@ -76,7 +76,7 @@ class TestSecondaryIndexPruning extends SparkClientFunctionalTestHarness {
           RECORDKEY_FIELD.key -> "record_key_col",
           PARTITIONPATH_FIELD.key -> "partition_key_col",
           HIVE_STYLE_PARTITIONING.key -> "true",
      -    PRECOMBINE_FIELD.key -> "ts",
      +    HoodieTableConfig.ORDERING_FIELDS.key -> "ts",
           HoodieWriteConfig.WRITE_PAYLOAD_CLASS_NAME.key() -> "org.apache.hudi.common.model.OverwriteWithLatestAvroPayload",
           DataSourceWriteOptions.RECORD_MERGE_MODE.key() -> RecordMergeMode.COMMIT_TIME_ORDERING.name()
         ) ++ metadataOpts
      @@ -473,7 +473,7 @@ class TestSecondaryIndexPruning extends SparkClientFunctionalTestHarness {
                |) USING hudi
                | OPTIONS (
                |  primaryKey = 'record_key_col',
      -         |  preCombineField = 'ts',
      +         |  orderingFields = 'ts',
                |  hoodie.metadata.enable = 'true',
                |  hoodie.metadata.record.index.enable = 'true',
                |  hoodie.datasource.write.recordkey.field = 'record_key_col',
      @@ -729,7 +729,7 @@ class TestSecondaryIndexPruning extends SparkClientFunctionalTestHarness {
                | OPTIONS (
                |  primaryKey = 'record_key_col',
                |  type = 'mor',
      -         |  preCombineField = 'ts',
      +         |  orderingFields = 'ts',
                |  hoodie.metadata.enable = 'true',
                |  hoodie.metadata.record.index.enable = 'true',
                |  hoodie.datasource.write.recordkey.field = 'record_key_col',
      @@ -804,7 +804,7 @@ class TestSecondaryIndexPruning extends SparkClientFunctionalTestHarness {
                | OPTIONS (
                |  primaryKey = 'record_key_col',
                |  type = 'mor',
      -         |  preCombineField = 'replicadmstimestamp',
      +         |  orderingFields = 'replicadmstimestamp',
                |  hoodie.metadata.enable = 'true',
                |  hoodie.metadata.record.index.enable = 'true',
                |  hoodie.datasource.write.recordkey.field = 'record_key_col',
      diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestSevenToEightUpgrade.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestSevenToEightUpgrade.scala
      index 92d920f90e094..55f0a68bf2d6b 100644
      --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestSevenToEightUpgrade.scala
      +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestSevenToEightUpgrade.scala
      @@ -19,7 +19,7 @@
       package org.apache.hudi.functional
       
       import org.apache.hudi.DataSourceUtils
      -import org.apache.hudi.DataSourceWriteOptions.{INSERT_OPERATION_OPT_VAL, KEYGENERATOR_CLASS_NAME, OPERATION, PARTITIONPATH_FIELD, PAYLOAD_CLASS_NAME, PRECOMBINE_FIELD, RECORD_MERGE_MODE, RECORDKEY_FIELD, TABLE_TYPE, UPSERT_OPERATION_OPT_VAL}
      +import org.apache.hudi.DataSourceWriteOptions.{INSERT_OPERATION_OPT_VAL, KEYGENERATOR_CLASS_NAME, OPERATION, ORDERING_FIELDS, PARTITIONPATH_FIELD, PAYLOAD_CLASS_NAME, RECORD_MERGE_MODE, RECORDKEY_FIELD, TABLE_TYPE, UPSERT_OPERATION_OPT_VAL}
       import org.apache.hudi.client.SparkRDDWriteClient
       import org.apache.hudi.common.config.{HoodieMetadataConfig, RecordMergeMode, TypedProperties}
       import org.apache.hudi.common.model.{DefaultHoodieRecordPayload, HoodieRecordMerger, HoodieRecordPayload, HoodieTableType, OverwriteWithLatestAvroPayload, TableServiceType}
      @@ -107,7 +107,7 @@ class TestSevenToEightUpgrade extends RecordLevelIndexTestBase {
             assertEquals(RecordMergeMode.COMMIT_TIME_ORDERING.name, metaClient.getTableConfig.getRecordMergeMode.name)
             assertEquals(HoodieRecordMerger.COMMIT_TIME_BASED_MERGE_STRATEGY_UUID, metaClient.getTableConfig.getRecordMergeStrategyId)
           } else {
      -      if (metaClient.getTableConfig.getPreCombineFieldsStr.isPresent && StringUtils.isNullOrEmpty(metaClient.getTableConfig.getPreCombineFieldsStr.get())) {
      +      if (metaClient.getTableConfig.getOrderingFieldsStr.isPresent && StringUtils.isNullOrEmpty(metaClient.getTableConfig.getOrderingFieldsStr.get())) {
               assertEquals(classOf[OverwriteWithLatestAvroPayload].getName, metaClient.getTableConfig.getPayloadClass)
               assertEquals(RecordMergeMode.COMMIT_TIME_ORDERING.name, metaClient.getTableConfig.getRecordMergeMode.name)
               assertEquals(HoodieRecordMerger.COMMIT_TIME_BASED_MERGE_STRATEGY_UUID, metaClient.getTableConfig.getRecordMergeStrategyId)
      @@ -133,7 +133,7 @@ class TestSevenToEightUpgrade extends RecordLevelIndexTestBase {
           val hudiOptions = Map[String, String](
             "hoodie.table.name" -> tableName,
             RECORDKEY_FIELD.key -> "id",
      -      PRECOMBINE_FIELD.key -> "ts",
      +      HoodieTableConfig.ORDERING_FIELDS.key -> "ts",
             TABLE_TYPE.key -> HoodieTableType.MERGE_ON_READ.name(),
             OPERATION.key -> UPSERT_OPERATION_OPT_VAL,
             KEYGENERATOR_CLASS_NAME.key -> classOf[NonpartitionedKeyGenerator].getName,
      diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestSparkDataSource.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestSparkDataSource.scala
      index d891ed7c302b2..4dcbede1ca716 100644
      --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestSparkDataSource.scala
      +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestSparkDataSource.scala
      @@ -23,6 +23,7 @@ import org.apache.hudi.{AvroConversionUtils, DataSourceReadOptions, DataSourceWr
       import org.apache.hudi.common.config.{HoodieMetadataConfig, RecordMergeMode}
       import org.apache.hudi.common.model.{HoodieRecord, HoodieTableType}
       import org.apache.hudi.common.model.HoodieRecord.HoodieRecordType
      +import org.apache.hudi.common.table.HoodieTableConfig
       import org.apache.hudi.common.testutils.HoodieTestDataGenerator
       import org.apache.hudi.common.testutils.RawTripTestPayload.recordsToStrings
       import org.apache.hudi.config.{HoodieCompactionConfig, HoodieIndexConfig, HoodieWriteConfig}
      @@ -55,7 +56,7 @@ class TestSparkDataSource extends SparkClientFunctionalTestHarness {
           "hoodie.bulkinsert.shuffle.parallelism" -> s"$parallelism",
           "hoodie.delete.shuffle.parallelism" -> s"$parallelism",
           DataSourceWriteOptions.RECORDKEY_FIELD.key -> "_row_key",
      -    DataSourceWriteOptions.PRECOMBINE_FIELD.key -> "timestamp",
      +    HoodieTableConfig.ORDERING_FIELDS.key -> "timestamp",
           HoodieWriteConfig.TBL_NAME.key -> "hoodie_test"
         )
       
      @@ -312,7 +313,7 @@ class TestSparkDataSource extends SparkClientFunctionalTestHarness {
           var (writeOpts, readOpts) = getWriterReaderOpts(HoodieRecordType.AVRO)
           writeOpts = writeOpts ++ Map("hoodie.write.table.version" -> tableVersion.toString,
             "hoodie.datasource.write.table.type" -> tableType.name(),
      -      "hoodie.datasource.write.precombine.field" -> "ts",
      +      HoodieTableConfig.ORDERING_FIELDS.key() -> "ts",
             "hoodie.write.record.merge.mode" -> mergeMode.name(),
             "hoodie.index.type" -> indexType.name(),
             "hoodie.metadata.record.index.enable" -> "true",
      diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestSparkDataSourceDAGExecution.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestSparkDataSourceDAGExecution.scala
      index 0255b22c77e92..2410dd8df505f 100644
      --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestSparkDataSourceDAGExecution.scala
      +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestSparkDataSourceDAGExecution.scala
      @@ -56,7 +56,7 @@ class TestSparkDataSourceDAGExecution extends HoodieSparkClientTestBase with Sca
           HoodieTableConfig.PARTITION_METAFILE_USE_BASE_FORMAT.key() -> "true",
           DataSourceWriteOptions.RECORDKEY_FIELD.key -> "_row_key",
           DataSourceWriteOptions.PARTITIONPATH_FIELD.key -> "partition",
      -    DataSourceWriteOptions.PRECOMBINE_FIELD.key -> "timestamp",
      +    HoodieTableConfig.ORDERING_FIELDS.key -> "timestamp",
           HoodieWriteConfig.TBL_NAME.key -> "hoodie_test",
           HoodieMetadataConfig.ENABLE.key -> "false"
         )
      diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestSparkSqlCoreFlow.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestSparkSqlCoreFlow.scala
      index bbd0b8be07d31..cddfb3e549e94 100644
      --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestSparkSqlCoreFlow.scala
      +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestSparkSqlCoreFlow.scala
      @@ -224,7 +224,7 @@ class TestSparkSqlCoreFlow extends HoodieSparkSqlTestBase {
              |tblproperties (
              |  type = '$typeString',
              |  primaryKey = '_row_key',
      -       |  preCombineField = 'timestamp',
      +       |  orderingFields = 'timestamp',
              |  hoodie.bulkinsert.shuffle.parallelism = 4,
              |  hoodie.database.name = "databaseName",
              |  hoodie.table.keygenerator.class = '$keyGenClass',
      diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestSparkSqlWithCustomKeyGenerator.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestSparkSqlWithCustomKeyGenerator.scala
      index 712b66c84cbc1..5672d4e95f11d 100644
      --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestSparkSqlWithCustomKeyGenerator.scala
      +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestSparkSqlWithCustomKeyGenerator.scala
      @@ -625,7 +625,7 @@ class TestSparkSqlWithCustomKeyGenerator extends HoodieSparkSqlTestBase {
             .option("hoodie.datasource.write.keygenerator.class", keyGenClassName)
             .option("hoodie.datasource.write.partitionpath.field", writePartitionFields)
             .option("hoodie.datasource.write.recordkey.field", "id")
      -      .option("hoodie.datasource.write.precombine.field", "name")
      +      .option(HoodieTableConfig.ORDERING_FIELDS.key(), "name")
             .option("hoodie.table.name", tableName)
             .option("hoodie.insert.shuffle.parallelism", "1")
             .option("hoodie.upsert.shuffle.parallelism", "1")
      diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestSparkSqlWithTimestampKeyGenerator.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestSparkSqlWithTimestampKeyGenerator.scala
      index d741d5e262c03..0ebcc2d275e2f 100644
      --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestSparkSqlWithTimestampKeyGenerator.scala
      +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestSparkSqlWithTimestampKeyGenerator.scala
      @@ -61,7 +61,7 @@ class TestSparkSqlWithTimestampKeyGenerator extends HoodieSparkSqlTestBase {
                        | TBLPROPERTIES (
                        |   type = '$tableType',
                        |   primaryKey = 'id',
      -                 |   preCombineField = 'precomb',
      +                 |   orderingFields = 'precomb',
                        |   hoodie.datasource.write.partitionpath.field = 'ts',
                        |   hoodie.datasource.write.hive_style_partitioning = 'false',
                        |   hoodie.file.group.reader.enabled = '$shouldUseFileGroupReader',
      @@ -118,7 +118,7 @@ class TestSparkSqlWithTimestampKeyGenerator extends HoodieSparkSqlTestBase {
                  | TBLPROPERTIES (
                  |   type = 'COPY_ON_WRITE',
                  |   primaryKey = 'id',
      -           |   preCombineField = 'precomb'
      +           |   orderingFields = 'precomb'
                  | )
                  |""".stripMargin)
             val dataBatches =   Array(
      @@ -151,7 +151,7 @@ class TestSparkSqlWithTimestampKeyGenerator extends HoodieSparkSqlTestBase {
                  | TBLPROPERTIES (
                  |   type = 'COPY_ON_WRITE',
                  |   primaryKey = 'id',
      -           |   preCombineField = 'precomb',
      +           |   orderingFields = 'precomb',
                  |   hoodie.table.keygenerator.class = 'org.apache.hudi.keygen.TimestampBasedKeyGenerator',
                  |   ${timestampKeyGeneratorSettings.head}
                  | )
      diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestStreamSourceReadByStateTransitionTime.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestStreamSourceReadByStateTransitionTime.scala
      index d61963f365b80..aeefff50f2eb1 100644
      --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestStreamSourceReadByStateTransitionTime.scala
      +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestStreamSourceReadByStateTransitionTime.scala
      @@ -18,12 +18,12 @@
       package org.apache.hudi.functional
       
       import org.apache.hudi.DataSourceWriteOptions
      -import org.apache.hudi.DataSourceWriteOptions.{PRECOMBINE_FIELD, RECORDKEY_FIELD}
      +import org.apache.hudi.DataSourceWriteOptions.{ORDERING_FIELDS, RECORDKEY_FIELD}
       import org.apache.hudi.client.{SparkRDDWriteClient, WriteClientTestUtils}
       import org.apache.hudi.client.common.HoodieSparkEngineContext
       import org.apache.hudi.common.engine.EngineType
       import org.apache.hudi.common.model.{HoodieFailedWritesCleaningPolicy, HoodieRecord, HoodieTableType}
      -import org.apache.hudi.common.table.HoodieTableMetaClient
      +import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient}
       import org.apache.hudi.common.testutils.HoodieTestDataGenerator
       import org.apache.hudi.common.testutils.HoodieTestTable.makeNewCommitTime
       import org.apache.hudi.config.{HoodieCleanConfig, HoodieWriteConfig}
      @@ -39,7 +39,7 @@ class TestStreamSourceReadByStateTransitionTime extends StreamTest  {
       
         protected val commonOptions: Map[String, String] = Map(
           RECORDKEY_FIELD.key -> "id",
      -    PRECOMBINE_FIELD.key -> "ts",
      +    HoodieTableConfig.ORDERING_FIELDS.key -> "ts",
           INSERT_PARALLELISM_VALUE.key -> "4",
           UPSERT_PARALLELISM_VALUE.key -> "4",
           DELETE_PARALLELISM_VALUE.key -> "4",
      @@ -64,7 +64,7 @@ class TestStreamSourceReadByStateTransitionTime extends StreamTest  {
               HoodieTableMetaClient.newTableBuilder()
                 .setTableType(tableType)
                 .setTableName(s"test_stream_${tableType.name()}")
      -          .setPreCombineFields("timestamp")
      +          .setOrderingFields("timestamp")
                 .setPartitionFields("partition_path")
                 .initTable(HadoopFSUtils.getStorageConf(spark.sessionState.newHadoopConf()), tablePath)
       
      diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestStreamingSource.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestStreamingSource.scala
      index 5915f93f1a0d2..a35d49993bd58 100644
      --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestStreamingSource.scala
      +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestStreamingSource.scala
      @@ -19,10 +19,10 @@ package org.apache.hudi.functional
       
       import org.apache.hudi.DataSourceReadOptions
       import org.apache.hudi.DataSourceReadOptions.{START_OFFSET, STREAMING_READ_TABLE_VERSION}
      -import org.apache.hudi.DataSourceWriteOptions.{PRECOMBINE_FIELD, RECORDKEY_FIELD}
      +import org.apache.hudi.DataSourceWriteOptions.{ORDERING_FIELDS, RECORDKEY_FIELD}
       import org.apache.hudi.common.model.HoodieTableType
       import org.apache.hudi.common.model.HoodieTableType.{COPY_ON_WRITE, MERGE_ON_READ}
      -import org.apache.hudi.common.table.{HoodieTableMetaClient, HoodieTableVersion}
      +import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient, HoodieTableVersion}
       import org.apache.hudi.common.table.timeline.HoodieTimeline
       import org.apache.hudi.config.{HoodieClusteringConfig, HoodieCompactionConfig}
       import org.apache.hudi.config.HoodieWriteConfig.{DELETE_PARALLELISM_VALUE, INSERT_PARALLELISM_VALUE, TBL_NAME, UPSERT_PARALLELISM_VALUE, WRITE_TABLE_VERSION}
      @@ -38,7 +38,7 @@ class TestStreamingSource extends StreamTest {
         import testImplicits._
         protected val commonOptions: Map[String, String] = Map(
           RECORDKEY_FIELD.key -> "id",
      -    PRECOMBINE_FIELD.key -> "ts",
      +    HoodieTableConfig.ORDERING_FIELDS.key -> "ts",
           INSERT_PARALLELISM_VALUE.key -> "4",
           UPSERT_PARALLELISM_VALUE.key -> "4",
           DELETE_PARALLELISM_VALUE.key -> "4"
      @@ -61,7 +61,7 @@ class TestStreamingSource extends StreamTest {
               .setTableType(COPY_ON_WRITE)
               .setTableName(getTableName(tablePath))
               .setRecordKeyFields("id")
      -        .setPreCombineFields("ts")
      +        .setOrderingFields("ts")
               .initTable(HadoopFSUtils.getStorageConf(spark.sessionState.newHadoopConf()), tablePath)
       
             addData(tablePath, Seq(("1", "a1", "10", "000")))
      @@ -112,7 +112,7 @@ class TestStreamingSource extends StreamTest {
               .setTableType(MERGE_ON_READ)
               .setTableName(getTableName(tablePath))
               .setRecordKeyFields("id")
      -        .setPreCombineFields("ts")
      +        .setOrderingFields("ts")
               .initTable(HadoopFSUtils.getStorageConf(spark.sessionState.newHadoopConf()), tablePath)
       
             addData(tablePath, Seq(("1", "a1", "10", "000")))
      @@ -157,7 +157,7 @@ class TestStreamingSource extends StreamTest {
               .setTableType(COPY_ON_WRITE)
               .setTableName(getTableName(tablePath))
               .setRecordKeyFields("id")
      -        .setPreCombineFields("ts")
      +        .setOrderingFields("ts")
               .initTable(HadoopFSUtils.getStorageConf(spark.sessionState.newHadoopConf()), tablePath)
       
             addData(tablePath, Seq(("1", "a1", "10", "000")))
      @@ -188,7 +188,7 @@ class TestStreamingSource extends StreamTest {
               .setTableType(COPY_ON_WRITE)
               .setTableName(getTableName(tablePath))
               .setRecordKeyFields("id")
      -        .setPreCombineFields("ts")
      +        .setOrderingFields("ts")
               .initTable(HadoopFSUtils.getStorageConf(spark.sessionState.newHadoopConf()), tablePath)
       
             addData(tablePath, Seq(("1", "a1", "10", "000")))
      @@ -220,7 +220,7 @@ class TestStreamingSource extends StreamTest {
               .setTableType(MERGE_ON_READ)
               .setTableName(getTableName(tablePath))
               .setRecordKeyFields("id")
      -        .setPreCombineFields("ts")
      +        .setOrderingFields("ts")
               .initTable(HadoopFSUtils.getStorageConf(spark.sessionState.newHadoopConf()), tablePath)
       
             addData(tablePath, Seq(("1", "a1", "10", "000")))
      @@ -262,7 +262,7 @@ class TestStreamingSource extends StreamTest {
                 .setTableType(MERGE_ON_READ)
                 .setTableName(getTableName(tablePath))
                 .setRecordKeyFields("id")
      -          .setPreCombineFields("ts")
      +          .setOrderingFields("ts")
                 .initTable(HadoopFSUtils.getStorageConf(spark.sessionState.newHadoopConf()), tablePath)
       
               addData(tablePath, Seq(("1", "a1", "10", "000")))
      @@ -305,7 +305,7 @@ class TestStreamingSource extends StreamTest {
               .setTableName(getTableName(tablePath))
               .setTableVersion(writeTableVersion)
               .setRecordKeyFields("id")
      -        .setPreCombineFields("ts")
      +        .setOrderingFields("ts")
               .initTable(HadoopFSUtils.getStorageConf(spark.sessionState.newHadoopConf()), tablePath)
       
             // Add initial data
      diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestStructuredStreaming.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestStructuredStreaming.scala
      index 56b77919bd224..1025e43b9bdfb 100644
      --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestStructuredStreaming.scala
      +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestStructuredStreaming.scala
      @@ -23,7 +23,7 @@ import org.apache.hudi.HoodieStreamingSink.SINK_CHECKPOINT_KEY
       import org.apache.hudi.client.transaction.lock.InProcessLockProvider
       import org.apache.hudi.common.config.HoodieStorageConfig
       import org.apache.hudi.common.model.{FileSlice, HoodieTableType, WriteConcurrencyMode}
      -import org.apache.hudi.common.table.HoodieTableMetaClient
      +import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient}
       import org.apache.hudi.common.table.timeline.HoodieTimeline
       import org.apache.hudi.common.testutils.{HoodieTestDataGenerator, HoodieTestTable, HoodieTestUtils}
       import org.apache.hudi.common.testutils.RawTripTestPayload.recordsToStrings
      @@ -60,7 +60,7 @@ class TestStructuredStreaming extends HoodieSparkClientTestBase {
           "hoodie.upsert.shuffle.parallelism" -> "4",
           DataSourceWriteOptions.RECORDKEY_FIELD.key -> "_row_key",
           DataSourceWriteOptions.PARTITIONPATH_FIELD.key -> "partition",
      -    DataSourceWriteOptions.PRECOMBINE_FIELD.key -> "timestamp",
      +    HoodieTableConfig.ORDERING_FIELDS.key -> "timestamp",
           HoodieWriteConfig.TBL_NAME.key -> "hoodie_test"
         )
       
      @@ -526,7 +526,7 @@ class TestStructuredStreaming extends HoodieSparkClientTestBase {
             DataSourceWriteOptions.TABLE_TYPE.key() -> tableType,
             DataSourceWriteOptions.RECORD_MERGE_MODE.key() -> mergeMode,
             HoodieStorageConfig.LOGFILE_DATA_BLOCK_FORMAT.key() -> "parquet",
      -      DataSourceWriteOptions.PRECOMBINE_FIELD.key -> "weight")
      +      HoodieTableConfig.ORDERING_FIELDS.key -> "weight")
           if (mergeMode == "CUSTOM") {
             opts = opts ++ Map(DataSourceWriteOptions.RECORD_MERGE_STRATEGY_ID.key() -> HoodieSparkDeleteRecordMerger.DELETE_MERGER_STRATEGY,
               DataSourceWriteOptions.RECORD_MERGE_IMPL_CLASSES.key() -> classOf[HoodieSparkDeleteRecordMerger].getName)
      diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestTimeTravelQuery.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestTimeTravelQuery.scala
      index 7f95db48d8bd2..93c80f9479206 100644
      --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestTimeTravelQuery.scala
      +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestTimeTravelQuery.scala
      @@ -21,7 +21,7 @@ import org.apache.hudi.{DataSourceReadOptions, DataSourceWriteOptions, ScalaAsse
       import org.apache.hudi.common.config.HoodieMetadataConfig
       import org.apache.hudi.common.model.{HoodieCleaningPolicy, HoodieTableType}
       import org.apache.hudi.common.model.HoodieTableType.{COPY_ON_WRITE, MERGE_ON_READ}
      -import org.apache.hudi.common.table.TableSchemaResolver
      +import org.apache.hudi.common.table.{HoodieTableConfig, TableSchemaResolver}
       import org.apache.hudi.common.table.timeline.TimelineUtils
       import org.apache.hudi.common.testutils.HoodieTestTable
       import org.apache.hudi.config.{HoodieArchivalConfig, HoodieCleanConfig, HoodieCompactionConfig, HoodieWriteConfig}
      @@ -46,7 +46,7 @@ class TestTimeTravelQuery extends HoodieSparkClientTestBase with ScalaAssertionS
           "hoodie.bulkinsert.shuffle.parallelism" -> "2",
           "hoodie.delete.shuffle.parallelism" -> "1",
           DataSourceWriteOptions.RECORDKEY_FIELD.key -> "id",
      -    DataSourceWriteOptions.PRECOMBINE_FIELD.key -> "version",
      +    HoodieTableConfig.ORDERING_FIELDS.key -> "version",
           HoodieWriteConfig.TBL_NAME.key -> "hoodie_test"
         )
       
      @@ -177,7 +177,7 @@ class TestTimeTravelQuery extends HoodieSparkClientTestBase with ScalaAssertionS
           val opts = commonOpts ++ Map(
             DataSourceWriteOptions.TABLE_TYPE.key -> tableType.name,
             DataSourceWriteOptions.RECORDKEY_FIELD.key -> "id",
      -      DataSourceWriteOptions.PRECOMBINE_FIELD.key -> "version",
      +      HoodieTableConfig.ORDERING_FIELDS.key -> "version",
             DataSourceWriteOptions.PARTITIONPATH_FIELD.key -> "dt"
           )
       
      diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestWriteTableVersionConfig.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestWriteTableVersionConfig.scala
      index 034a91009ce4e..640dee1fe396b 100644
      --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestWriteTableVersionConfig.scala
      +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestWriteTableVersionConfig.scala
      @@ -41,7 +41,7 @@ class TestWriteTableVersionConfig extends HoodieSparkSqlTestBase {
                    | tblproperties (
                    |  type = 'mor',
                    |  primaryKey = 'id',
      -             |  preCombineField = 'ts',
      +             |  orderingFields = 'ts',
                    |  hoodie.write.table.version = $tableVersion
                    | )
                    | partitioned by(dt)
      diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/cdc/HoodieCDCTestBase.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/cdc/HoodieCDCTestBase.scala
      index 6f13c9257bd3e..0694f582f9037 100644
      --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/cdc/HoodieCDCTestBase.scala
      +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/cdc/HoodieCDCTestBase.scala
      @@ -54,7 +54,7 @@ abstract class HoodieCDCTestBase extends HoodieSparkClientTestBase {
           "hoodie.bulkinsert.shuffle.parallelism" -> "2",
           "hoodie.delete.shuffle.parallelism" -> "1",
           RECORDKEY_FIELD.key -> "_row_key",
      -    PRECOMBINE_FIELD.key -> "timestamp",
      +    HoodieTableConfig.ORDERING_FIELDS.key -> "timestamp",
           HoodieWriteConfig.TBL_NAME.key -> "hoodie_test",
           HoodieMetadataConfig.COMPACT_NUM_DELTA_COMMITS.key -> "1",
           HoodieCleanConfig.AUTO_CLEAN.key -> "false",
      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 e9107a90e6ca0..4b30ddd2da3d1 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
      @@ -649,7 +649,7 @@ class TestCDCDataFrameSuite extends HoodieCDCTestBase {
           val options = Map(
             "hoodie.table.name" -> "test",
             "hoodie.datasource.write.recordkey.field" -> "id",
      -      "hoodie.datasource.write.precombine.field" -> "replicadmstimestamp",
      +      HoodieTableConfig.ORDERING_FIELDS.key() -> "replicadmstimestamp",
             "hoodie.datasource.write.keygenerator.class" -> "org.apache.hudi.keygen.NonpartitionedKeyGenerator",
             "hoodie.datasource.write.partitionpath.field" -> "",
             "hoodie.datasource.write.payload.class" -> "org.apache.hudi.common.model.AWSDmsAvroPayload",
      @@ -708,7 +708,7 @@ class TestCDCDataFrameSuite extends HoodieCDCTestBase {
             "hoodie.bulkinsert.shuffle.parallelism" -> "2",
             "hoodie.delete.shuffle.parallelism" -> "1",
             "hoodie.datasource.write.recordkey.field" -> "_row_key",
      -      "hoodie.datasource.write.precombine.field" -> "timestamp",
      +      HoodieTableConfig.ORDERING_FIELDS.key() -> "timestamp",
             "hoodie.table.name" -> ("hoodie_test" + loggingMode.name()),
             "hoodie.clean.automatic" -> "true",
             "hoodie.clean.commits.retained" -> "1"
      diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/cdc/TestCDCStreamingSuite.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/cdc/TestCDCStreamingSuite.scala
      index 4ba1eb76b739e..d39ba79c9f0db 100644
      --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/cdc/TestCDCStreamingSuite.scala
      +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/cdc/TestCDCStreamingSuite.scala
      @@ -70,7 +70,7 @@ class TestCDCStreamingSuite extends HoodieCDCTestBase {
             .option(HoodieTableConfig.CDC_ENABLED.key, "true")
             .option(HoodieTableConfig.CDC_SUPPLEMENTAL_LOGGING_MODE.key, loggingMode.name())
             .option(DataSourceWriteOptions.RECORDKEY_FIELD.key, "userid")
      -      .option(DataSourceWriteOptions.PRECOMBINE_FIELD.key, "ts")
      +      .option(HoodieTableConfig.ORDERING_FIELDS.key, "ts")
             .option(HoodieWriteConfig.TBL_NAME.key, "user_to_country")
             .save(userToCountryTblPath)
       
      @@ -82,7 +82,7 @@ class TestCDCStreamingSuite extends HoodieCDCTestBase {
           countryToPopulationDF.write.format("hudi")
             .options(commonOptions)
             .option(DataSourceWriteOptions.RECORDKEY_FIELD.key, "country")
      -      .option(DataSourceWriteOptions.PRECOMBINE_FIELD.key, "ts")
      +      .option(HoodieTableConfig.ORDERING_FIELDS.key, "ts")
             .option(HoodieWriteConfig.TBL_NAME.key, "country_to_population")
             .save(countryToPopulationTblPath)
       
      @@ -98,7 +98,7 @@ class TestCDCStreamingSuite extends HoodieCDCTestBase {
                 .options(commonOptions)
                 .option(HoodieTableConfig.CDC_ENABLED.key, "true")
                 .option(DataSourceWriteOptions.RECORDKEY_FIELD.key, "userid")
      -          .option(DataSourceWriteOptions.PRECOMBINE_FIELD.key, "ts")
      +          .option(HoodieTableConfig.ORDERING_FIELDS.key, "ts")
                 .option(HoodieWriteConfig.TBL_NAME.key, "user_to_country")
                 .mode(SaveMode.Append)
                 .save(userToCountryTblPath)
      @@ -147,7 +147,7 @@ class TestCDCStreamingSuite extends HoodieCDCTestBase {
                 .write.format("hudi")
                 .options(commonOptions)
                 .option(DataSourceWriteOptions.RECORDKEY_FIELD.key, "country")
      -          .option(DataSourceWriteOptions.PRECOMBINE_FIELD.key, "ts")
      +          .option(HoodieTableConfig.ORDERING_FIELDS.key, "ts")
                 .option(HoodieWriteConfig.TBL_NAME.key, "country_to_population")
                 .mode(SaveMode.Append)
                 .save(countryToPopulationTblPath)
      diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/analysis/TestHoodiePruneFileSourcePartitions.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/analysis/TestHoodiePruneFileSourcePartitions.scala
      index e20a486def000..b91b52c8aa836 100644
      --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/analysis/TestHoodiePruneFileSourcePartitions.scala
      +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/analysis/TestHoodiePruneFileSourcePartitions.scala
      @@ -72,7 +72,7 @@ class TestHoodiePruneFileSourcePartitions extends HoodieClientTestBase with Scal
                |TBLPROPERTIES (
                |  type = '$tableType',
                |  primaryKey = 'id',
      -         |  preCombineField = 'ts'
      +         |  orderingFields = 'ts'
                |)
                |LOCATION '$basePath/$tableName'
                """.stripMargin)
      @@ -171,7 +171,7 @@ class TestHoodiePruneFileSourcePartitions extends HoodieClientTestBase with Scal
                |TBLPROPERTIES (
                |  type = '$tableType',
                |  primaryKey = 'id',
      -         |  preCombineField = 'ts'
      +         |  orderingFields = 'ts'
                |)
                |LOCATION '$basePath/$tableName'
                """.stripMargin)
      diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/common/TestHoodieOptionConfig.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/common/TestHoodieOptionConfig.scala
      index daadfb6a37679..aa37b30865c55 100644
      --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/common/TestHoodieOptionConfig.scala
      +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/common/TestHoodieOptionConfig.scala
      @@ -38,7 +38,7 @@ class TestHoodieOptionConfig extends SparkClientFunctionalTestHarness {
           assertEquals("cow", with1("type"))
       
           val ops2 = Map("primaryKey" -> "id",
      -      "preCombineField" -> "timestamp",
      +      "orderingFields" -> "timestamp",
             "type" -> "mor",
             "payloadClass" -> classOf[OverwriteWithLatestAvroPayload].getName,
             "recordMergeStrategyId" -> HoodieRecordMerger.EVENT_TIME_BASED_MERGE_STRATEGY_UUID
      @@ -50,7 +50,7 @@ class TestHoodieOptionConfig extends SparkClientFunctionalTestHarness {
         @Test
         def testMappingSqlOptionToTableConfig(): Unit = {
           val sqlOptions = Map("primaryKey" -> "id,addr",
      -      "preCombineField" -> "timestamp",
      +      "orderingFields" -> "timestamp",
             "type" -> "mor",
             "hoodie.index.type" -> "INMEMORY",
             "hoodie.compact.inline" -> "true"
      @@ -59,7 +59,7 @@ class TestHoodieOptionConfig extends SparkClientFunctionalTestHarness {
       
           assertTrue(tableConfigs.size == 5)
           assertTrue(tableConfigs(HoodieTableConfig.RECORDKEY_FIELDS.key) == "id,addr")
      -    assertTrue(tableConfigs(HoodieTableConfig.PRECOMBINE_FIELDS.key) == "timestamp")
      +    assertTrue(tableConfigs(HoodieTableConfig.ORDERING_FIELDS.key) == "timestamp")
           assertTrue(tableConfigs(HoodieTableConfig.TYPE.key) == "MERGE_ON_READ")
           assertTrue(tableConfigs("hoodie.index.type") == "INMEMORY")
           assertTrue(tableConfigs("hoodie.compact.inline") == "true")
      @@ -68,7 +68,7 @@ class TestHoodieOptionConfig extends SparkClientFunctionalTestHarness {
         @Test
         def testDeleteHoodieOptions(): Unit = {
           val sqlOptions = Map("primaryKey" -> "id,addr",
      -      "preCombineField" -> "timestamp",
      +      "orderingFields" -> "timestamp",
             "type" -> "mor",
             "hoodie.index.type" -> "INMEMORY",
             "hoodie.compact.inline" -> "true",
      @@ -82,7 +82,7 @@ class TestHoodieOptionConfig extends SparkClientFunctionalTestHarness {
         @Test
         def testExtractSqlOptions(): Unit = {
           val sqlOptions = Map("primaryKey" -> "id,addr",
      -      "preCombineField" -> "timestamp",
      +      "orderingFields" -> "timestamp",
             "type" -> "mor",
             "hoodie.index.type" -> "INMEMORY",
             "hoodie.compact.inline" -> "true",
      @@ -90,7 +90,7 @@ class TestHoodieOptionConfig extends SparkClientFunctionalTestHarness {
           )
           val tableConfigs = HoodieOptionConfig.extractSqlOptions(sqlOptions)
           assertTrue(tableConfigs.size == 3)
      -    assertTrue(tableConfigs.keySet == Set("primaryKey", "preCombineField", "type"))
      +    assertTrue(tableConfigs.keySet == Set("primaryKey", "orderingFields", "type"))
         }
       
         @Test
      @@ -121,18 +121,18 @@ class TestHoodieOptionConfig extends SparkClientFunctionalTestHarness {
           // preCombine field not found
           val sqlOptions3 = baseSqlOptions ++ Map(
             "primaryKey" -> "id",
      -      "preCombineField" -> "ts",
      +      "orderingFields" -> "ts",
             "type" -> "mor"
           )
           val e3 = intercept[IllegalArgumentException] {
             HoodieOptionConfig.validateTable(spark, schema, sqlOptions3)
           }
      -    assertTrue(e3.getMessage.contains("Can't find preCombineKey"))
      +    assertTrue(e3.getMessage.contains("Can't find ordering fields"))
       
           // miss type parameter
           val sqlOptions4 = baseSqlOptions ++ Map(
             "primaryKey" -> "id",
      -      "preCombineField" -> "timestamp"
      +      "orderingFields" -> "timestamp"
           )
           val e4 = intercept[IllegalArgumentException] {
             HoodieOptionConfig.validateTable(spark, schema, sqlOptions4)
      @@ -142,7 +142,7 @@ class TestHoodieOptionConfig extends SparkClientFunctionalTestHarness {
           // type is invalid
           val sqlOptions5 = baseSqlOptions ++ Map(
             "primaryKey" -> "id",
      -      "preCombineField" -> "timestamp",
      +      "orderingFields" -> "timestamp",
             "type" -> "abc"
           )
           val e5 = intercept[IllegalArgumentException] {
      @@ -153,7 +153,7 @@ class TestHoodieOptionConfig extends SparkClientFunctionalTestHarness {
           // right options and schema
           val sqlOptions6 = baseSqlOptions ++ Map(
             "primaryKey" -> "id",
      -      "preCombineField" -> "timestamp",
      +      "orderingFields" -> "timestamp",
             "type" -> "cow"
           )
           HoodieOptionConfig.validateTable(spark, schema, sqlOptions6)
      diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/common/TestLazyPartitionPathFetching.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/common/TestLazyPartitionPathFetching.scala
      index aa6cd64fcb3e2..dc168a79f26ef 100644
      --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/common/TestLazyPartitionPathFetching.scala
      +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/common/TestLazyPartitionPathFetching.scala
      @@ -35,7 +35,7 @@ class TestLazyPartitionPathFetching extends HoodieSparkSqlTestBase {
                  | tblproperties (
                  |  primaryKey ='id',
                  |  type = 'cow',
      -           |  preCombineField = 'ts'
      +           |  orderingFields = 'ts'
                  | )
                  | PARTITIONED BY (date_par)
              """.stripMargin)
      @@ -71,7 +71,7 @@ class TestLazyPartitionPathFetching extends HoodieSparkSqlTestBase {
                  | tblproperties (
                  |  primaryKey ='id',
                  |  type = 'cow',
      -           |  preCombineField = 'ts'
      +           |  orderingFields = 'ts'
                  | )
                  | PARTITIONED BY (grass_date)
                """.stripMargin)
      @@ -102,7 +102,7 @@ class TestLazyPartitionPathFetching extends HoodieSparkSqlTestBase {
                  | tblproperties (
                  |  primaryKey ='id',
                  |  type = 'cow',
      -           |  preCombineField = 'ts'
      +           |  orderingFields = 'ts'
                  | )
                  | PARTITIONED BY (country, date_par)
                """.stripMargin)
      diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/common/TestNestedSchemaPruningOptimization.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/common/TestNestedSchemaPruningOptimization.scala
      index 3eaa20111ad3b..25afe03b12e03 100644
      --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/common/TestNestedSchemaPruningOptimization.scala
      +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/common/TestNestedSchemaPruningOptimization.scala
      @@ -176,7 +176,7 @@ class TestNestedSchemaPruningOptimization extends HoodieSparkSqlTestBase with Sp
                |CREATE TABLE $tableName USING HUDI TBLPROPERTIES (
                |  type = '$tableType',
                |  primaryKey = 'id',
      -         |  preCombineField = 'ts',
      +         |  orderingFields = 'ts',
                |  hoodie.populate.meta.fields = 'false'
                |  ${if (opts.nonEmpty) "," + opts.map{ case (k, v) => s"'$k' = '$v'" }.mkString(",") else ""}
                |)
      diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/common/TestPartitionPushDownWhenListingPaths.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/common/TestPartitionPushDownWhenListingPaths.scala
      index 7740da5e664c9..5ced3b8af7ef6 100644
      --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/common/TestPartitionPushDownWhenListingPaths.scala
      +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/common/TestPartitionPushDownWhenListingPaths.scala
      @@ -43,7 +43,7 @@ class TestPartitionPushDownWhenListingPaths extends HoodieSparkSqlTestBase {
                        | tblproperties (
                        |  primaryKey ='id',
                        |  type = '$tableType',
      -                 |  preCombineField = 'ts',
      +                 |  orderingFields = 'ts',
                        |  hoodie.datasource.write.hive_style_partitioning = 'true',
                        |  hoodie.datasource.write.partitionpath.urlencode = 'true'
                        | )
      diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/common/TestSqlConf.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/common/TestSqlConf.scala
      index d918ba2bcfb09..0bfaa580af724 100644
      --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/common/TestSqlConf.scala
      +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/common/TestSqlConf.scala
      @@ -58,7 +58,7 @@ class TestSqlConf extends HoodieSparkSqlTestBase with BeforeAndAfter {
                  | location '$tablePath'
                  | options (
                  |  primaryKey ='id',
      -           |  preCombineField = 'ts'
      +           |  orderingFields = 'ts'
                  | )
              """.stripMargin)
       
      diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/common/TestTableColumnTypeMismatch.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/common/TestTableColumnTypeMismatch.scala
      index 60fb1c1bd4949..8d0565aaefcc0 100644
      --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/common/TestTableColumnTypeMismatch.scala
      +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/common/TestTableColumnTypeMismatch.scala
      @@ -92,7 +92,7 @@ class TestTableColumnTypeMismatch extends HoodieSparkSqlTestBase with ScalaAsser
                  |location '${tmp.getCanonicalPath}/$tableName'
                  |tblproperties (
                  |  primaryKey = 'id',
      -           |  preCombineField = 'ts'
      +           |  orderingFields = 'ts'
                  |)
                """.stripMargin)
       
      @@ -165,7 +165,7 @@ class TestTableColumnTypeMismatch extends HoodieSparkSqlTestBase with ScalaAsser
                    |location '${tmp.getCanonicalPath}/$tableName'
                    |tblproperties (
                    |  primaryKey = 'id',
      -             |  preCombineField = 'ts'
      +             |  orderingFields = 'ts'
                    |)
                """.stripMargin)
       
      @@ -251,7 +251,7 @@ class TestTableColumnTypeMismatch extends HoodieSparkSqlTestBase with ScalaAsser
                    |tblproperties (
                    |  type = '$tableType',
                    |  primaryKey = 'id',
      -             |  preCombineField = 'ts'
      +             |  orderingFields = 'ts'
                    |)
                """.stripMargin)
       
      @@ -267,7 +267,7 @@ class TestTableColumnTypeMismatch extends HoodieSparkSqlTestBase with ScalaAsser
                    |tblproperties (
                    |  type = '$tableType',
                    |  primaryKey = 'id',
      -             |  preCombineField = 'ts'
      +             |  orderingFields = 'ts'
                    |)
                """.stripMargin)
       
      @@ -352,7 +352,7 @@ class TestTableColumnTypeMismatch extends HoodieSparkSqlTestBase with ScalaAsser
                      |tblproperties (
                      |  type = '$tableType',
                      |  primaryKey = 'id',
      -               |  preCombineField = 'ts'
      +               |  orderingFields = 'ts'
                      |)
                """.stripMargin)
                 targetTable
      @@ -509,7 +509,7 @@ class TestTableColumnTypeMismatch extends HoodieSparkSqlTestBase with ScalaAsser
                                            sourceSchema: Seq[(String, String)],
                                            partitionCols: Seq[String],
                                            primaryKey: String,
      -                                     preCombineField: String,
      +                                     orderingFields: String,
                                            tableType: String, // COW or MOR
                                            expectedErrorPattern: String
                                          )
      @@ -531,7 +531,7 @@ class TestTableColumnTypeMismatch extends HoodieSparkSqlTestBase with ScalaAsser
               ),
               partitionCols = Seq("name", "price"),
               primaryKey = "id",
      -        preCombineField = "ts",
      +        orderingFields = "ts",
               tableType = "cow",
               expectedErrorPattern = "Partition key data type mismatch between source table and target table. Target table uses StringType for column 'name', source table uses IntegerType for 's0.name'"
             ),
      @@ -551,7 +551,7 @@ class TestTableColumnTypeMismatch extends HoodieSparkSqlTestBase with ScalaAsser
               ),
               partitionCols = Seq("name", "price"),
               primaryKey = "id",
      -        preCombineField = "ts",
      +        orderingFields = "ts",
               tableType = "mor",
               expectedErrorPattern = "Primary key data type mismatch between source table and target table. Target table uses IntegerType for column 'id', source table uses LongType for 's0.id'"
             ),
      @@ -571,14 +571,14 @@ class TestTableColumnTypeMismatch extends HoodieSparkSqlTestBase with ScalaAsser
               ),
               partitionCols = Seq("name", "price"),
               primaryKey = "id",
      -        preCombineField = "ts",
      +        orderingFields = "ts",
               tableType = "cow",
               expectedErrorPattern = "Precombine field data type mismatch between source table and target table. Target table uses LongType for column 'ts', source table uses IntegerType for 's0.ts'"
             )
           )
       
           def createTable(tableName: String, schema: Seq[(String, String)], partitionCols: Seq[String],
      -                    primaryKey: String, preCombineField: String, tableType: String, location: String): Unit = {
      +                    primaryKey: String, orderingFields: String, tableType: String, location: String): Unit = {
             val schemaStr = schema.map { case (name, dataType) => s"$name $dataType" }.mkString(",\n  ")
             val partitionColsStr = if (partitionCols.nonEmpty) s"partitioned by (${partitionCols.mkString(", ")})" else ""
       
      @@ -592,7 +592,7 @@ class TestTableColumnTypeMismatch extends HoodieSparkSqlTestBase with ScalaAsser
                  |tblproperties (
                  |  type = '$tableType',
                  |  primaryKey = '$primaryKey',
      -           |  preCombineField = '$preCombineField'
      +           |  orderingFields = '$orderingFields'
                  |)
              """.stripMargin)
           }
      @@ -609,7 +609,7 @@ class TestTableColumnTypeMismatch extends HoodieSparkSqlTestBase with ScalaAsser
                   testCase.targetSchema,
                   testCase.partitionCols,
                   testCase.primaryKey,
      -            testCase.preCombineField,
      +            testCase.orderingFields,
                   testCase.tableType,
                   s"${tmp.getCanonicalPath}/$targetTable"
                 )
      @@ -689,7 +689,7 @@ class TestTableColumnTypeMismatch extends HoodieSparkSqlTestBase with ScalaAsser
                    |tblproperties (
                    |  type = 'cow',
                    |  primaryKey = 'id',
      -             |  preCombineField = 'ts'
      +             |  orderingFields = 'ts'
                    |)
                """.stripMargin)
       
      @@ -738,7 +738,7 @@ class TestTableColumnTypeMismatch extends HoodieSparkSqlTestBase with ScalaAsser
                      |tblproperties (
                      |  type = 'cow',
                      |  primaryKey = 'id',
      -               |  preCombineField = 'ts',
      +               |  orderingFields = 'ts',
                      |  'hoodie.record.merge.mode' = '$mergeMode'
                      |)
                  """.stripMargin)
      @@ -813,7 +813,7 @@ class TestTableColumnTypeMismatch extends HoodieSparkSqlTestBase with ScalaAsser
                      |tblproperties (
                      |  type = '$tableType',
                      |  primaryKey = 'c1',
      -               |  preCombineField = 'ts'
      +               |  orderingFields = 'ts'
                      |)
                  """.stripMargin)
       
      diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/common/TestVectorizedReadWithSchemaEvolution.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/common/TestVectorizedReadWithSchemaEvolution.scala
      index ddebcd85832bb..244b579fc587f 100644
      --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/common/TestVectorizedReadWithSchemaEvolution.scala
      +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/common/TestVectorizedReadWithSchemaEvolution.scala
      @@ -47,7 +47,7 @@ class TestVectorizedReadWithSchemaEvolution extends HoodieSparkSqlTestBase {
                        | tblproperties (
                        |  type = '$tableType',
                        |  primaryKey = 'id',
      -                 |  preCombineField = 'ts'
      +                 |  orderingFields = 'ts'
                        | )
                """.stripMargin)
                   // insert data to table
      diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/ddl/TestAlterTable.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/ddl/TestAlterTable.scala
      index 99e1a93bf45b5..87d5ae5bb6662 100644
      --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/ddl/TestAlterTable.scala
      +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/ddl/TestAlterTable.scala
      @@ -53,7 +53,7 @@ class TestAlterTable extends HoodieSparkSqlTestBase {
                    | tblproperties (
                    |  type = '$tableType',
                    |  primaryKey = 'id',
      -             |  preCombineField = 'ts'
      +             |  orderingFields = 'ts'
                    | )
              """.stripMargin)
       
      @@ -164,7 +164,7 @@ class TestAlterTable extends HoodieSparkSqlTestBase {
                    | tblproperties (
                    |  type = '$tableType',
                    |  primaryKey = 'id',
      -             |  preCombineField = 'ts'
      +             |  orderingFields = 'ts'
                    | )
                    | partitioned by (dt)
              """.stripMargin)
      @@ -193,7 +193,7 @@ class TestAlterTable extends HoodieSparkSqlTestBase {
                    | tblproperties (
                    |  type = '$tableType',
                    |  primaryKey = 'id',
      -             |  preCombineField = 'ts',
      +             |  orderingFields = 'ts',
                    |  `hoodie.index.type`='BUCKET',
                    |  `hoodie.index.bucket.engine`='SIMPLE',
                    |  `hoodie.bucket.index.num.buckets`='2',
      @@ -234,7 +234,7 @@ class TestAlterTable extends HoodieSparkSqlTestBase {
                    | tblproperties (
                    |  type = '$tableType',
                    |  primaryKey = 'id',
      -             |  preCombineField = 'ts',
      +             |  orderingFields = 'ts',
                    |  hoodie.metadata.enable = 'false',
                    |  hoodie.clean.commits.retained = '100',
                    |  hoodie.clustering.inline = 'true',
      @@ -321,7 +321,7 @@ class TestAlterTable extends HoodieSparkSqlTestBase {
                  | tblproperties (
                  |  type = '$tableType',
                  |  primaryKey = 'id',
      -           |  preCombineField = 'ts'
      +           |  orderingFields = 'ts'
                  | )
            """.stripMargin)
       
      @@ -360,7 +360,7 @@ class TestAlterTable extends HoodieSparkSqlTestBase {
                    | tblproperties (
                    |  type = '$tableType',
                    |  primaryKey = 'id',
      -             |  preCombineField = 'ts'
      +             |  orderingFields = 'ts'
                    | )
              """.stripMargin)
       
      @@ -401,7 +401,7 @@ class TestAlterTable extends HoodieSparkSqlTestBase {
                    | tblproperties (
                    |  type = '$tableType',
                    |  primaryKey = 'id',
      -             |  preCombineField = 'ts',
      +             |  orderingFields = 'ts',
                    |  hoodie.write.concurrency.mode='optimistic_concurrency_control',
                    |  hoodie.clean.failed.writes.policy='LAZY',
                    |  hoodie.write.lock.provider='org.apache.hudi.client.transaction.lock.FileSystemBasedLockProvider'
      @@ -484,7 +484,7 @@ class TestAlterTable extends HoodieSparkSqlTestBase {
                    | tblproperties (
                    |  type = 'cow',
                    |  primaryKey = 'id',
      -             |  preCombineField = 'ts',
      +             |  orderingFields = 'ts',
                    |  hoodie.clean.trigger.strategy = 'NUM_COMMITS',
                    |  hoodie.clean.commits.retained = '3'
                    | )
      diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/ddl/TestAlterTableAddPartition.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/ddl/TestAlterTableAddPartition.scala
      index 3ad6f113a8278..3fcb3d4a3e12b 100644
      --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/ddl/TestAlterTableAddPartition.scala
      +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/ddl/TestAlterTableAddPartition.scala
      @@ -35,7 +35,7 @@ class TestAlterTableAddPartition extends HoodieSparkSqlTestBase {
                  | using hudi
                  | tblproperties (
                  |  primaryKey = 'id',
      -           |  preCombineField = 'ts'
      +           |  orderingFields = 'ts'
                  | )
                  |""".stripMargin)
       
      @@ -61,7 +61,7 @@ class TestAlterTableAddPartition extends HoodieSparkSqlTestBase {
                  | using hudi
                  | tblproperties (
                  |  primaryKey = 'id',
      -           |  preCombineField = 'ts'
      +           |  orderingFields = 'ts'
                  | )
                  | partitioned by (dt)
                  |""".stripMargin)
      @@ -88,7 +88,7 @@ class TestAlterTableAddPartition extends HoodieSparkSqlTestBase {
                  | using hudi
                  | tblproperties (
                  |  primaryKey = 'id',
      -           |  preCombineField = 'ts'
      +           |  orderingFields = 'ts'
                  | )
                  | partitioned by (dt)
                  |""".stripMargin)
      @@ -120,7 +120,7 @@ class TestAlterTableAddPartition extends HoodieSparkSqlTestBase {
                    | using hudi
                    | tblproperties (
                    |  primaryKey = 'id',
      -             |  preCombineField = 'ts',
      +             |  orderingFields = 'ts',
                    |  hoodie.datasource.write.hive_style_partitioning = '$hiveStyle'
                    | )
                    | partitioned by (dt)
      @@ -151,7 +151,7 @@ class TestAlterTableAddPartition extends HoodieSparkSqlTestBase {
                    | using hudi
                    | tblproperties (
                    |  primaryKey = 'id',
      -             |  preCombineField = 'ts',
      +             |  orderingFields = 'ts',
                    |  hoodie.datasource.write.hive_style_partitioning = '$hiveStyle'
                    | )
                    | partitioned by (year, month, day)
      @@ -182,7 +182,7 @@ class TestAlterTableAddPartition extends HoodieSparkSqlTestBase {
                    | using hudi
                    | tblproperties (
                    |  primaryKey = 'id',
      -             |  preCombineField = 'ts',
      +             |  orderingFields = 'ts',
                    |  hoodie.datasource.write.partitionpath.urlencode = '$urlEncode'
                    | )
                    | partitioned by (p_a)
      @@ -212,7 +212,7 @@ class TestAlterTableAddPartition extends HoodieSparkSqlTestBase {
                    | using hudi
                    | tblproperties (
                    |  primaryKey = 'id',
      -             |  preCombineField = 'ts',
      +             |  orderingFields = 'ts',
                    |  hoodie.datasource.write.partitionpath.urlencode = '$urlEncode'
                    | )
                    | partitioned by (p_a, p_b)
      diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/ddl/TestAlterTableDropPartition.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/ddl/TestAlterTableDropPartition.scala
      index 57d79f8c13825..93ba4606c17bc 100644
      --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/ddl/TestAlterTableDropPartition.scala
      +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/ddl/TestAlterTableDropPartition.scala
      @@ -20,6 +20,7 @@ package org.apache.spark.sql.hudi.ddl
       import org.apache.hudi.{DataSourceWriteOptions, HoodieCLIUtils}
       import org.apache.hudi.avro.model.{HoodieCleanMetadata, HoodieCleanPartitionMetadata}
       import org.apache.hudi.common.model.{HoodieCleaningPolicy, HoodieCommitMetadata}
      +import org.apache.hudi.common.table.HoodieTableConfig
       import org.apache.hudi.common.table.timeline.HoodieInstant
       import org.apache.hudi.common.util.{Option => HOption, PartitionPathEncodeUtils, StringUtils}
       import org.apache.hudi.config.{HoodieCleanConfig, HoodieWriteConfig}
      @@ -66,7 +67,7 @@ class TestAlterTableDropPartition extends HoodieSparkSqlTestBase {
                | using hudi
                | tblproperties (
                |  primaryKey = 'id',
      -         |  preCombineField = 'ts'
      +         |  orderingFields = 'ts'
                | )
                |""".stripMargin)
           // insert data
      @@ -93,7 +94,7 @@ class TestAlterTableDropPartition extends HoodieSparkSqlTestBase {
                | using hudi
                | tblproperties (
                |  primaryKey = 'id',
      -         |  preCombineField = 'ts',
      +         |  orderingFields = 'ts',
                |  hoodie.clean.commits.retained= '1'
                | )
                |""".stripMargin)
      @@ -121,7 +122,7 @@ class TestAlterTableDropPartition extends HoodieSparkSqlTestBase {
                 .option(HoodieWriteConfig.TBL_NAME.key, tableName)
                 .option(DataSourceWriteOptions.TABLE_TYPE.key, DataSourceWriteOptions.COW_TABLE_TYPE_OPT_VAL)
                 .option(DataSourceWriteOptions.RECORDKEY_FIELD.key, "id")
      -          .option(DataSourceWriteOptions.PRECOMBINE_FIELD.key, "ts")
      +          .option(HoodieTableConfig.ORDERING_FIELDS.key, "ts")
                 .option(DataSourceWriteOptions.PARTITIONPATH_FIELD.key, "dt")
                 .option(DataSourceWriteOptions.URL_ENCODE_PARTITIONING.key(), urlencode)
                 .option(HoodieWriteConfig.INSERT_PARALLELISM_VALUE.key, "1")
      @@ -140,7 +141,7 @@ class TestAlterTableDropPartition extends HoodieSparkSqlTestBase {
                 .option(HoodieWriteConfig.TBL_NAME.key, tableName)
                 .option(DataSourceWriteOptions.TABLE_TYPE.key, DataSourceWriteOptions.COW_TABLE_TYPE_OPT_VAL)
                 .option(DataSourceWriteOptions.RECORDKEY_FIELD.key, "id")
      -          .option(DataSourceWriteOptions.PRECOMBINE_FIELD.key, "ts")
      +          .option(HoodieTableConfig.ORDERING_FIELDS.key, "ts")
                 .option(DataSourceWriteOptions.PARTITIONPATH_FIELD.key, "dt")
                 .option(DataSourceWriteOptions.URL_ENCODE_PARTITIONING.key(), urlencode)
                 .option(DataSourceWriteOptions.KEYGENERATOR_CLASS_NAME.key, classOf[SimpleKeyGenerator].getName)
      @@ -199,7 +200,7 @@ class TestAlterTableDropPartition extends HoodieSparkSqlTestBase {
                 .option(HoodieWriteConfig.TBL_NAME.key, tableName)
                 .option(DataSourceWriteOptions.TABLE_TYPE.key, DataSourceWriteOptions.COW_TABLE_TYPE_OPT_VAL)
                 .option(DataSourceWriteOptions.RECORDKEY_FIELD.key, "id")
      -          .option(DataSourceWriteOptions.PRECOMBINE_FIELD.key, "ts")
      +          .option(HoodieTableConfig.ORDERING_FIELDS.key, "ts")
                 .option(DataSourceWriteOptions.PARTITIONPATH_FIELD.key, "dt")
                 .option(DataSourceWriteOptions.URL_ENCODE_PARTITIONING.key(), urlencode)
                 .option(HoodieWriteConfig.INSERT_PARALLELISM_VALUE.key, "1")
      @@ -214,7 +215,7 @@ class TestAlterTableDropPartition extends HoodieSparkSqlTestBase {
                    |location '$tablePath'
                    | tblproperties (
                    |  primaryKey = 'id',
      -             |  preCombineField = 'ts',
      +             |  orderingFields = 'ts',
                    |  hoodie.clean.commits.retained= '1'
                    | )
                    |""".stripMargin)
      @@ -258,7 +259,7 @@ class TestAlterTableDropPartition extends HoodieSparkSqlTestBase {
                | using hudi
                | tblproperties (
                |  primaryKey = 'id',
      -         |  preCombineField = 'ts'
      +         |  orderingFields = 'ts'
                | )
                | partitioned by (dt)
                |""".stripMargin)
      @@ -305,7 +306,7 @@ class TestAlterTableDropPartition extends HoodieSparkSqlTestBase {
                 .option(HoodieWriteConfig.TBL_NAME.key, tableName)
                 .option(DataSourceWriteOptions.TABLE_TYPE.key, DataSourceWriteOptions.COW_TABLE_TYPE_OPT_VAL)
                 .option(DataSourceWriteOptions.RECORDKEY_FIELD.key, "id")
      -          .option(DataSourceWriteOptions.PRECOMBINE_FIELD.key, "ts")
      +          .option(HoodieTableConfig.ORDERING_FIELDS.key, "ts")
                 .option(DataSourceWriteOptions.PARTITIONPATH_FIELD.key, "year,month,day")
                 .option(DataSourceWriteOptions.HIVE_STYLE_PARTITIONING.key, hiveStyle)
                 .option(HoodieWriteConfig.INSERT_PARALLELISM_VALUE.key, "1")
      @@ -329,7 +330,7 @@ class TestAlterTableDropPartition extends HoodieSparkSqlTestBase {
                 .option(HoodieWriteConfig.TBL_NAME.key, tableName)
                 .option(DataSourceWriteOptions.TABLE_TYPE.key, DataSourceWriteOptions.COW_TABLE_TYPE_OPT_VAL)
                 .option(DataSourceWriteOptions.RECORDKEY_FIELD.key, "id")
      -          .option(DataSourceWriteOptions.PRECOMBINE_FIELD.key, "ts")
      +          .option(HoodieTableConfig.ORDERING_FIELDS.key, "ts")
                 .option(DataSourceWriteOptions.PARTITIONPATH_FIELD.key, "year,month,day")
                 .option(DataSourceWriteOptions.HIVE_STYLE_PARTITIONING.key, hiveStyle)
                 .option(DataSourceWriteOptions.KEYGENERATOR_CLASS_NAME.key, classOf[ComplexKeyGenerator].getName)
      @@ -385,7 +386,7 @@ class TestAlterTableDropPartition extends HoodieSparkSqlTestBase {
                 .option(HoodieWriteConfig.TBL_NAME.key, tableName)
                 .option(DataSourceWriteOptions.TABLE_TYPE.key, DataSourceWriteOptions.COW_TABLE_TYPE_OPT_VAL)
                 .option(DataSourceWriteOptions.RECORDKEY_FIELD.key, "id")
      -          .option(DataSourceWriteOptions.PRECOMBINE_FIELD.key, "ts")
      +          .option(HoodieTableConfig.ORDERING_FIELDS.key, "ts")
                 .option(DataSourceWriteOptions.PARTITIONPATH_FIELD.key, "year,month,day")
                 .option(DataSourceWriteOptions.HIVE_STYLE_PARTITIONING.key, hiveStyle)
                 .option(HoodieWriteConfig.INSERT_PARALLELISM_VALUE.key, "1")
      @@ -400,7 +401,7 @@ class TestAlterTableDropPartition extends HoodieSparkSqlTestBase {
                    |location '$tablePath'
                    | tblproperties (
                    |  primaryKey = 'id',
      -             |  preCombineField = 'ts',
      +             |  orderingFields = 'ts',
                    |  hoodie.clean.commits.retained= '1'
                    | )
                    |""".stripMargin)
      @@ -409,7 +410,7 @@ class TestAlterTableDropPartition extends HoodieSparkSqlTestBase {
                 .option(HoodieWriteConfig.TBL_NAME.key, tableName)
                 .option(DataSourceWriteOptions.TABLE_TYPE.key, DataSourceWriteOptions.COW_TABLE_TYPE_OPT_VAL)
                 .option(DataSourceWriteOptions.RECORDKEY_FIELD.key, "id")
      -          .option(DataSourceWriteOptions.PRECOMBINE_FIELD.key, "ts")
      +          .option(HoodieTableConfig.ORDERING_FIELDS.key, "ts")
                 .option(DataSourceWriteOptions.PARTITIONPATH_FIELD.key, "year,month,day")
                 .option(DataSourceWriteOptions.HIVE_STYLE_PARTITIONING.key, hiveStyle)
                 .option(DataSourceWriteOptions.KEYGENERATOR_CLASS_NAME.key, classOf[ComplexKeyGenerator].getName)
      @@ -477,7 +478,7 @@ class TestAlterTableDropPartition extends HoodieSparkSqlTestBase {
                    | tblproperties (
                    |  type = '$tableType',
                    |  primaryKey = 'id',
      -             |  preCombineField = 'ts'
      +             |  orderingFields = 'ts'
                    | )
                    | partitioned by (dt)
              """.stripMargin)
      @@ -532,7 +533,7 @@ class TestAlterTableDropPartition extends HoodieSparkSqlTestBase {
                    | options (
                    |  primaryKey ='id',
                    |  type = '$tableType',
      -             |  preCombineField = 'ts'
      +             |  orderingFields = 'ts'
                    | )
                    | partitioned by(ts)
                    | location '$basePath'
      @@ -573,7 +574,7 @@ class TestAlterTableDropPartition extends HoodieSparkSqlTestBase {
                  | options (
                  |  primaryKey ='id',
                  |  type = 'mor',
      -           |  preCombineField = 'ts',
      +           |  orderingFields = 'ts',
                  |  hoodie.index.type = 'INMEMORY'
                  | )
                  | partitioned by(ts)
      @@ -621,7 +622,7 @@ class TestAlterTableDropPartition extends HoodieSparkSqlTestBase {
                  | options (
                  |  primaryKey ='id',
                  |  type = 'mor',
      -           |  preCombineField = 'ts',
      +           |  orderingFields = 'ts',
                  |  hoodie.index.type = 'INMEMORY'
                  | )
                  | partitioned by(ts)
      @@ -666,7 +667,7 @@ class TestAlterTableDropPartition extends HoodieSparkSqlTestBase {
                    | tblproperties (
                    |  primaryKey ='id',
                    |  type = '$tableType',
      -             |  preCombineField = 'ts'
      +             |  orderingFields = 'ts'
                    | ) partitioned by (partition_date_col)
                """.stripMargin)
               spark.sql(s"insert into $tableName values " +
      diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/ddl/TestCreateTable.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/ddl/TestCreateTable.scala
      index 6c9d57e8c175b..169e74028e84d 100644
      --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/ddl/TestCreateTable.scala
      +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/ddl/TestCreateTable.scala
      @@ -58,7 +58,7 @@ class TestCreateTable extends HoodieSparkSqlTestBase {
                |   hoodie.database.name = "databaseName",
                |   hoodie.table.name = "tableName",
                |   primaryKey = 'id',
      -         |   preCombineField = 'ts',
      +         |   orderingFields = 'ts',
                |   hoodie.datasource.write.operation = 'upsert'
                | )
              """.stripMargin)
      @@ -109,14 +109,14 @@ class TestCreateTable extends HoodieSparkSqlTestBase {
                |   hoodie.database.name = "databaseName",
                |   hoodie.table.name = "tableName",
                |   primaryKey = 'id',
      -         |   preCombineField = 'ts',
      +         |   orderingFields = 'ts',
                |   hoodie.datasource.write.operation = 'upsert'
                | )
              """.stripMargin)
           val table = spark.sessionState.catalog.getTableMetadata(TableIdentifier(tableName))
           assertResult(table.properties("type"))("cow")
           assertResult(table.properties("primaryKey"))("id")
      -    assertResult(table.properties("preCombineField"))("ts")
      +    assertResult(table.properties("orderingFields"))("ts")
           assertResult(tableName)(table.identifier.table)
           assertResult("hudi")(table.provider.get)
           assertResult(CatalogTableType.MANAGED)(table.tableType)
      @@ -139,7 +139,7 @@ class TestCreateTable extends HoodieSparkSqlTestBase {
           assertResult(true)(tableConfig.contains(HoodieTableConfig.CREATE_SCHEMA.key))
           assertResult("dt")(tableConfig(HoodieTableConfig.PARTITION_FIELDS.key))
           assertResult("id")(tableConfig(HoodieTableConfig.RECORDKEY_FIELDS.key))
      -    assertResult("ts")(tableConfig(HoodieTableConfig.PRECOMBINE_FIELDS.key))
      +    assertResult("ts")(tableConfig(HoodieTableConfig.ORDERING_FIELDS.key))
           assertResult(KeyGeneratorType.SIMPLE.name())(tableConfig(HoodieTableConfig.KEY_GENERATOR_TYPE.key))
           assertResult("default")(tableConfig(HoodieTableConfig.DATABASE_NAME.key()))
           assertResult(tableName)(tableConfig(HoodieTableConfig.NAME.key()))
      @@ -259,7 +259,7 @@ class TestCreateTable extends HoodieSparkSqlTestBase {
                    |) using hudi
                    | tblproperties (
                    |  primaryKey = 'id',
      -             |  preCombineField = 'ts1',
      +             |  orderingFields = 'ts1',
                    |  type = 'cow'
                    | )
                    | location '${tmp.getCanonicalPath}'
      @@ -277,7 +277,7 @@ class TestCreateTable extends HoodieSparkSqlTestBase {
                    |) using hudi
                    | tblproperties (
                    |  primaryKey = 'id',
      -             |  preCombineField = 'ts',
      +             |  orderingFields = 'ts',
                    |  type = 'cow1'
                    | )
                    | location '${tmp.getCanonicalPath}'
      @@ -597,7 +597,7 @@ class TestCreateTable extends HoodieSparkSqlTestBase {
                |) using hudi
                | tblproperties (
                |  primaryKey = 'id',
      -         |  preCombineField = 'ts'
      +         |  orderingFields = 'ts'
                |)
                |""".stripMargin)
           val tableLocation = spark.sessionState.catalog.getTableMetadata(TableIdentifier(tableName1)).location
      @@ -613,7 +613,7 @@ class TestCreateTable extends HoodieSparkSqlTestBase {
                |) using hudi
                | tblproperties (
                |  primaryKey = 'id',
      -         |  preCombineField = 'ts'
      +         |  orderingFields = 'ts'
                |)
                |location '$tableLocation'
            """.stripMargin)
      @@ -642,7 +642,7 @@ class TestCreateTable extends HoodieSparkSqlTestBase {
                  |) using hudi
                  | tblproperties (
                  |  primaryKey = 'id',
      -           |  preCombineField = 'ts',
      +           |  orderingFields = 'ts',
                  |  type = 'mor'
                  | )
                  | location '$parentPath/$tableName1'
      @@ -666,7 +666,7 @@ class TestCreateTable extends HoodieSparkSqlTestBase {
             val roCatalogTable = spark.sessionState.catalog.getTableMetadata(TableIdentifier(roTableName1))
             assertResult(roCatalogTable.properties("type"))("mor")
             assertResult(roCatalogTable.properties("primaryKey"))("id")
      -      assertResult(roCatalogTable.properties("preCombineField"))("ts")
      +      assertResult(roCatalogTable.properties("orderingFields"))("ts")
             assertResult(roCatalogTable.storage.properties("hoodie.query.as.ro.table"))("true")
             checkAnswer(s"select id, name, ts from $roTableName1")(
               Seq(1, "a1", 1000)
      @@ -685,7 +685,7 @@ class TestCreateTable extends HoodieSparkSqlTestBase {
             val rtCatalogTable = spark.sessionState.catalog.getTableMetadata(TableIdentifier(rtTableName1))
             assertResult(rtCatalogTable.properties("type"))("mor")
             assertResult(rtCatalogTable.properties("primaryKey"))("id")
      -      assertResult(rtCatalogTable.properties("preCombineField"))("ts")
      +      assertResult(rtCatalogTable.properties("orderingFields"))("ts")
             assertResult(rtCatalogTable.storage.properties("hoodie.query.as.ro.table"))("false")
             checkAnswer(s"select id, name, ts from $rtTableName1")(
               Seq(1, "a2", 1100)
      @@ -709,7 +709,7 @@ class TestCreateTable extends HoodieSparkSqlTestBase {
                  |) using hudi
                  | tblproperties (
                  |  primaryKey = 'id',
      -           |  preCombineField = 'ts',
      +           |  orderingFields = 'ts',
                  |  type = 'cow'
                  | )
                  | location '$parentPath/$tableName1'
      @@ -750,7 +750,7 @@ class TestCreateTable extends HoodieSparkSqlTestBase {
                  | create table $tableName3 using hudi
                  | tblproperties(
                  |    primaryKey = 'id',
      -           |    preCombineField = 'ts',
      +           |    orderingFields = 'ts',
                  |    type = 'mor',
                  |    'hoodie.query.as.ro.table' = 'true'
                  | )
      @@ -773,7 +773,7 @@ class TestCreateTable extends HoodieSparkSqlTestBase {
                  |    hoodie.database.name = "databaseName",
                  |    hoodie.table.name = "tableName",
                  |    primaryKey = 'id',
      -           |    preCombineField = 'ts',
      +           |    orderingFields = 'ts',
                  |    hoodie.datasource.write.operation = 'upsert',
                  |    type = '$tableType'
                  | )
      @@ -809,7 +809,7 @@ class TestCreateTable extends HoodieSparkSqlTestBase {
                  |    hoodie.database.name = "databaseName",
                  |    hoodie.table.name = "tableName",
                  |    primaryKey = 'id',
      -           |    preCombineField = 'ts',
      +           |    orderingFields = 'ts',
                  |    hoodie.datasource.write.operation = 'upsert',
                  |    type = '$tableType'
                  | )
      @@ -877,7 +877,7 @@ class TestCreateTable extends HoodieSparkSqlTestBase {
                 .option(HoodieWriteConfig.TBL_NAME.key, s"original_$tableName")
                 .option(TABLE_TYPE.key, COW_TABLE_TYPE_OPT_VAL)
                 .option(RECORDKEY_FIELD.key, "id")
      -          .option(PRECOMBINE_FIELD.key, "ts")
      +          .option(HoodieTableConfig.ORDERING_FIELDS.key, "ts")
                 .option(PARTITIONPATH_FIELD.key, "dt")
                 .option(HoodieWriteConfig.INSERT_PARALLELISM_VALUE.key, "1")
                 .option(HoodieWriteConfig.UPSERT_PARALLELISM_VALUE.key, "1")
      @@ -907,7 +907,7 @@ class TestCreateTable extends HoodieSparkSqlTestBase {
               val properties = metaClient.getTableConfig.getProps.asScala.toMap
               assertResult(true)(properties.contains(HoodieTableConfig.CREATE_SCHEMA.key))
               assertResult("dt")(properties(HoodieTableConfig.PARTITION_FIELDS.key))
      -        assertResult("ts")(properties(HoodieTableConfig.PRECOMBINE_FIELDS.key))
      +        assertResult("ts")(properties(HoodieTableConfig.ORDERING_FIELDS.key))
               assertResult("hudi_database")(metaClient.getTableConfig.getDatabaseName)
               assertResult(s"original_$tableName")(metaClient.getTableConfig.getTableName)
       
      @@ -956,7 +956,7 @@ class TestCreateTable extends HoodieSparkSqlTestBase {
                 .option(HoodieWriteConfig.TBL_NAME.key, tableName)
                 .option(TABLE_TYPE.key, MOR_TABLE_TYPE_OPT_VAL)
                 .option(RECORDKEY_FIELD.key, "id")
      -          .option(PRECOMBINE_FIELD.key, "ts")
      +          .option(HoodieTableConfig.ORDERING_FIELDS.key, "ts")
                 .option(PARTITIONPATH_FIELD.key, "day,hh")
                 .option(URL_ENCODE_PARTITIONING.key, "true")
                 .option(HoodieWriteConfig.INSERT_PARALLELISM_VALUE.key, "1")
      @@ -978,7 +978,7 @@ class TestCreateTable extends HoodieSparkSqlTestBase {
               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_FIELDS.key))
      +        assertResult("ts")(properties(HoodieTableConfig.ORDERING_FIELDS.key))
       
               val escapedPathPart = escapePathName(day)
       
      @@ -1025,7 +1025,7 @@ class TestCreateTable extends HoodieSparkSqlTestBase {
               .option(HoodieWriteConfig.TBL_NAME.key, tableName)
               .option(TABLE_TYPE.key, COW_TABLE_TYPE_OPT_VAL)
               .option(RECORDKEY_FIELD.key, "id")
      -        .option(PRECOMBINE_FIELD.key, "ts")
      +        .option(HoodieTableConfig.ORDERING_FIELDS.key, "ts")
               .option(PARTITIONPATH_FIELD.key, "")
               .option(HoodieWriteConfig.INSERT_PARALLELISM_VALUE.key, "1")
               .option(HoodieWriteConfig.UPSERT_PARALLELISM_VALUE.key, "1")
      @@ -1045,7 +1045,7 @@ class TestCreateTable extends HoodieSparkSqlTestBase {
             val metaClient = createMetaClient(spark, tmp.getCanonicalPath)
             val properties = metaClient.getTableConfig.getProps.asScala.toMap
             assertResult(true)(properties.contains(HoodieTableConfig.CREATE_SCHEMA.key))
      -      assertResult("ts")(properties(HoodieTableConfig.PRECOMBINE_FIELDS.key))
      +      assertResult("ts")(properties(HoodieTableConfig.ORDERING_FIELDS.key))
       
             // Test insert into
             spark.sql(s"insert into $tableName values(2, 'a2', 10, 1000)")
      @@ -1120,7 +1120,7 @@ class TestCreateTable extends HoodieSparkSqlTestBase {
                | comment "This is a simple hudi table"
                | tblproperties (
                |   primaryKey = 'id',
      -         |   preCombineField = 'ts'
      +         |   orderingFields = 'ts'
                | )
              """.stripMargin)
           val shown = spark.sql(s"show create table $tableName").head.getString(0)
      @@ -1198,7 +1198,7 @@ class TestCreateTable extends HoodieSparkSqlTestBase {
                | comment "This is a simple hudi table"
                | tblproperties (
                |   primaryKey = 'id',
      -         |   preCombineField = 'ts'
      +         |   orderingFields = 'ts'
                | )
              """.stripMargin)
           checkKeyGenerator("org.apache.hudi.keygen.NonpartitionedKeyGenerator", tableName)
      @@ -1217,7 +1217,7 @@ class TestCreateTable extends HoodieSparkSqlTestBase {
                | partitioned by (ts)
                | tblproperties (
                |   primaryKey = 'id',
      -         |   preCombineField = 'ts'
      +         |   orderingFields = 'ts'
                | )
              """.stripMargin)
           checkKeyGenerator("org.apache.hudi.keygen.SimpleKeyGenerator", tableName)
      @@ -1236,7 +1236,7 @@ class TestCreateTable extends HoodieSparkSqlTestBase {
                | partitioned by (ts)
                | tblproperties (
                |   primaryKey = 'id,name',
      -         |   preCombineField = 'ts'
      +         |   orderingFields = 'ts'
                | )
              """.stripMargin)
           checkKeyGenerator("org.apache.hudi.keygen.ComplexKeyGenerator", tableName)
      @@ -1253,7 +1253,7 @@ class TestCreateTable extends HoodieSparkSqlTestBase {
                    | LOCATION '${tmp.getCanonicalPath}/$tableName'
                    | TBLPROPERTIES (
                    |  primaryKey = 'id',
      -             |  preCombineField = 'ts'
      +             |  orderingFields = 'ts'
                    | )
                    | AS SELECT * FROM (
                    |  SELECT 1 as id, 'a1' as name, 10 as price, 1000 as ts
      @@ -1299,7 +1299,7 @@ class TestCreateTable extends HoodieSparkSqlTestBase {
                | ) using hudi
                | tblproperties (
                |   primaryKey = 'id',
      -         |   preCombineField = 'ts'
      +         |   orderingFields = 'ts'
                | )
              """.stripMargin)
           val table = spark.sessionState.catalog.getTableMetadata(TableIdentifier(tableName))
      @@ -1324,14 +1324,14 @@ class TestCreateTable extends HoodieSparkSqlTestBase {
                |   hoodie.database.name = "databaseName",
                |   hoodie.table.name = "tableName",
                |   PRIMARYKEY = 'id',
      -         |   precombineField = 'ts',
      +         |   orderingFields = 'ts',
                |   hoodie.datasource.write.operation = 'upsert'
                | )
              """.stripMargin)
           val table = spark.sessionState.catalog.getTableMetadata(TableIdentifier(tableName))
           assertResult(table.properties("type"))("cow")
           assertResult(table.properties("primaryKey"))("id")
      -    assertResult(table.properties("preCombineField"))("ts")
      +    assertResult(table.properties("orderingFields"))("ts")
         }
       
         test("Test Create Hoodie Table with existing hoodie.properties") {
      @@ -1350,7 +1350,7 @@ class TestCreateTable extends HoodieSparkSqlTestBase {
                  | tblproperties (
                  |  primaryKey ='id',
                  |  type = 'cow',
      -           |  preCombineField = 'ts'
      +           |  orderingFields = 'ts'
                  | )
              """.stripMargin)
       
      @@ -1372,7 +1372,7 @@ class TestCreateTable extends HoodieSparkSqlTestBase {
                  | tblproperties (
                  |  primaryKey ='id',
                  |  type = 'cow',
      -           |  preCombineField = 'ts'
      +           |  orderingFields = 'ts'
                  | )
              """.stripMargin)
       
      @@ -1397,7 +1397,7 @@ class TestCreateTable extends HoodieSparkSqlTestBase {
                    | tblproperties (
                    |  primaryKey ='id',
                    |  type = '$tableType',
      -             |  preCombineField = 'ts',
      +             |  orderingFields = 'ts',
                    |  hoodie.table.base.file.format = 'PARQUET'
                    | )
              """.stripMargin)
      @@ -1423,7 +1423,7 @@ class TestCreateTable extends HoodieSparkSqlTestBase {
                  | tblproperties (
                  |  primaryKey ='id',
                  |  type = 'cow',
      -           |  preCombineField = 'ts',
      +           |  orderingFields = 'ts',
                  |  hoodie.table.base.file.format = 'ORC'
                  | )
              """.stripMargin)
      @@ -1448,13 +1448,13 @@ class TestCreateTable extends HoodieSparkSqlTestBase {
                    | tblproperties (
                    |  hoodie.table.recordkey.fields ='id',
                    |  hoodie.table.type = '$tableType',
      -             |  hoodie.table.precombine.field = 'ts'
      +             |  hoodie.table.ordering.fields = 'ts'
                    | )
              """.stripMargin)
               val hoodieCatalogTable = HoodieCatalogTable(spark, TableIdentifier(tableName))
               assertResult(Array("id"))(hoodieCatalogTable.primaryKeys)
               assertResult(tableType)(hoodieCatalogTable.tableTypeName)
      -        assertResult(java.util.Collections.singletonList[String]("ts"))(hoodieCatalogTable.preCombineKeys)
      +        assertResult(java.util.Collections.singletonList[String]("ts"))(hoodieCatalogTable.orderingFields)
             }
           }
         }
      @@ -1524,7 +1524,7 @@ class TestCreateTable extends HoodieSparkSqlTestBase {
                  |) using hudi
                  | tblproperties (
                  |  primaryKey = 'id',
      -           |  preCombineField = 'ts',
      +           |  orderingFields = 'ts',
                  |  type = 'cow'
                  | )
                  | location '$parentPath/$tableName1'
      @@ -1542,7 +1542,7 @@ class TestCreateTable extends HoodieSparkSqlTestBase {
                  |) using hudi
                  | tblproperties (
                  |  primaryKey = 'id',
      -           |  preCombineField = 'ts',
      +           |  orderingFields = 'ts',
                  |  type = 'cow'
                  | )
                  | location '$parentPath/$tableName1'
      @@ -1569,7 +1569,7 @@ class TestCreateTable extends HoodieSparkSqlTestBase {
                      | options (
                      |  primaryKey ='id',
                      |  type = '$tableType',
      -               |  preCombineField = 'ts'
      +               |  orderingFields = 'ts'
                      | )
                      | partitioned by(ts)
                      | location '$basePath'
      @@ -1598,7 +1598,7 @@ class TestCreateTable extends HoodieSparkSqlTestBase {
                      | options (
                      |  primaryKey ='id',
                      |  type = '$tableType',
      -               |  preCombineField = 'ts',
      +               |  orderingFields = 'ts',
                      |  'hoodie.datasource.write.partitionpath.field' = 'segment:simple,ts:timestamp',
                      |  'hoodie.datasource.write.keygenerator.class' = 'org.apache.hudi.keygen.CustomKeyGenerator',
                      |  'hoodie.keygen.timebased.timestamp.type' = 'SCALAR',
      @@ -1632,7 +1632,7 @@ class TestCreateTable extends HoodieSparkSqlTestBase {
                      | options (
                      |  primaryKey ='id',
                      |  type = '$tableType',
      -               |  preCombineField = 'segment',
      +               |  orderingFields = 'segment',
                      |  'hoodie.datasource.write.partitionpath.field' = 'segment:simple,ts:timestamp',
                      |  'hoodie.datasource.write.keygenerator.class' = 'org.apache.hudi.keygen.CustomKeyGenerator',
                      |  'hoodie.keygen.timebased.timestamp.type' = 'SCALAR',
      diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/ddl/TestDropTable.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/ddl/TestDropTable.scala
      index 63967bfd16538..dafe532ee8c41 100644
      --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/ddl/TestDropTable.scala
      +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/ddl/TestDropTable.scala
      @@ -46,7 +46,7 @@ class TestDropTable extends HoodieSparkSqlTestBase {
                    | tblproperties (
                    |  type = '$tableType',
                    |  primaryKey = 'id',
      -             |  preCombineField = 'ts'
      +             |  orderingFields = 'ts'
                    | )
              """.stripMargin)
               spark.sql(s"DROP TABLE $tableName")
      @@ -83,7 +83,7 @@ class TestDropTable extends HoodieSparkSqlTestBase {
                    | tblproperties (
                    |  type = '$tableType',
                    |  primaryKey = 'id',
      -             |  preCombineField = 'ts'
      +             |  orderingFields = 'ts'
                    | )
              """.stripMargin)
               spark.sql(s"DROP TABLE $tableName PURGE")
      @@ -107,7 +107,7 @@ class TestDropTable extends HoodieSparkSqlTestBase {
                  | tblproperties (
                  |  type = 'mor',
                  |  primaryKey = 'id',
      -           |  preCombineField = 'ts'
      +           |  orderingFields = 'ts'
                  | )
              """.stripMargin)
       
      @@ -118,7 +118,7 @@ class TestDropTable extends HoodieSparkSqlTestBase {
                  | tblproperties (
                  |  type = 'mor',
                  |  primaryKey = 'id',
      -           |  preCombineField = 'ts'
      +           |  orderingFields = 'ts'
                  | )
              """.stripMargin)
             alterSerdeProperties(spark.sessionState.catalog, TableIdentifier(s"${tableName}_ro"),
      @@ -131,7 +131,7 @@ class TestDropTable extends HoodieSparkSqlTestBase {
                  | tblproperties (
                  |  type = 'mor',
                  |  primaryKey = 'id',
      -           |  preCombineField = 'ts'
      +           |  orderingFields = 'ts'
                  | )
              """.stripMargin)
             alterSerdeProperties(spark.sessionState.catalog, TableIdentifier(s"${tableName}_rt"),
      @@ -156,7 +156,7 @@ class TestDropTable extends HoodieSparkSqlTestBase {
                  | tblproperties (
                  |  type = 'mor',
                  |  primaryKey = 'id',
      -           |  preCombineField = 'ts'
      +           |  orderingFields = 'ts'
                  | )
              """.stripMargin)
       
      @@ -167,7 +167,7 @@ class TestDropTable extends HoodieSparkSqlTestBase {
                  | tblproperties (
                  |  type = 'mor',
                  |  primaryKey = 'id',
      -           |  preCombineField = 'ts'
      +           |  orderingFields = 'ts'
                  | )
              """.stripMargin)
             alterSerdeProperties(spark.sessionState.catalog, TableIdentifier(s"${tableName}_ro"),
      @@ -180,7 +180,7 @@ class TestDropTable extends HoodieSparkSqlTestBase {
                  | tblproperties (
                  |  type = 'mor',
                  |  primaryKey = 'id',
      -           |  preCombineField = 'ts'
      +           |  orderingFields = 'ts'
                  | )
              """.stripMargin)
             alterSerdeProperties(spark.sessionState.catalog, TableIdentifier(s"${tableName}_rt"),
      @@ -212,7 +212,7 @@ class TestDropTable extends HoodieSparkSqlTestBase {
                  | tblproperties (
                  |  type = 'mor',
                  |  primaryKey = 'id',
      -           |  preCombineField = 'ts'
      +           |  orderingFields = 'ts'
                  | )
              """.stripMargin)
       
      @@ -223,7 +223,7 @@ class TestDropTable extends HoodieSparkSqlTestBase {
                  | tblproperties (
                  |  type = 'mor',
                  |  primaryKey = 'id',
      -           |  preCombineField = 'ts'
      +           |  orderingFields = 'ts'
                  | )
              """.stripMargin)
             alterSerdeProperties(spark.sessionState.catalog, TableIdentifier(s"${tableName}_ro"),
      @@ -236,7 +236,7 @@ class TestDropTable extends HoodieSparkSqlTestBase {
                  | tblproperties (
                  |  type = 'mor',
                  |  primaryKey = 'id',
      -           |  preCombineField = 'ts'
      +           |  orderingFields = 'ts'
                  | )
              """.stripMargin)
             alterSerdeProperties(spark.sessionState.catalog, TableIdentifier(s"${tableName}_rt"),
      @@ -262,7 +262,7 @@ class TestDropTable extends HoodieSparkSqlTestBase {
                  | location '$tablePath'
                  | tblproperties (
                  |  primaryKey = 'id',
      -           |  preCombineField = 'ts'
      +           |  orderingFields = 'ts'
                  | )
                  |""".stripMargin)
       
      @@ -289,7 +289,7 @@ class TestDropTable extends HoodieSparkSqlTestBase {
                  | location '$tablePath'
                  | tblproperties (
                  |  primaryKey = 'id',
      -           |  preCombineField = 'ts',
      +           |  orderingFields = 'ts',
                  |  type = 'mor'
                  | )
                  |""".stripMargin)
      @@ -302,7 +302,7 @@ class TestDropTable extends HoodieSparkSqlTestBase {
                  | tblproperties (
                  |  type = 'mor',
                  |  primaryKey = 'id',
      -           |  preCombineField = 'ts'
      +           |  orderingFields = 'ts'
                  | )
              """.stripMargin)
             alterSerdeProperties(spark.sessionState.catalog, TableIdentifier(s"${tableName}_ro"),
      @@ -315,7 +315,7 @@ class TestDropTable extends HoodieSparkSqlTestBase {
                  | tblproperties (
                  |  type = 'mor',
                  |  primaryKey = 'id',
      -           |  preCombineField = 'ts'
      +           |  orderingFields = 'ts'
                  | )
              """.stripMargin)
             alterSerdeProperties(spark.sessionState.catalog, TableIdentifier(s"${tableName}_rt"),
      @@ -342,7 +342,7 @@ class TestDropTable extends HoodieSparkSqlTestBase {
                      |)using hudi
                      | tblproperties (
                      |  primaryKey = 'id',
      -               |  preCombineField = 'ts'
      +               |  orderingFields = 'ts'
                      | )
                      |""".stripMargin)
       
      diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/ddl/TestRepairTable.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/ddl/TestRepairTable.scala
      index ebbdc1a6a1c00..edffa6db36478 100644
      --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/ddl/TestRepairTable.scala
      +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/ddl/TestRepairTable.scala
      @@ -19,7 +19,8 @@
       
       package org.apache.spark.sql.hudi.ddl
       
      -import org.apache.hudi.DataSourceWriteOptions.{PARTITIONPATH_FIELD, PRECOMBINE_FIELD, RECORDKEY_FIELD}
      +import org.apache.hudi.DataSourceWriteOptions.{ORDERING_FIELDS, PARTITIONPATH_FIELD, RECORDKEY_FIELD}
      +import org.apache.hudi.common.table.HoodieTableConfig
       import org.apache.hudi.common.table.HoodieTableConfig.HIVE_STYLE_PARTITIONING_ENABLE
       import org.apache.hudi.config.HoodieWriteConfig.TBL_NAME
       
      @@ -45,7 +46,7 @@ class TestRepairTable extends HoodieSparkSqlTestBase {
                    | location '$basePath'
                    | tblproperties (
                    |  primaryKey = 'id',
      -             |  preCombineField = 'ts',
      +             |  orderingFields = 'ts',
                    |  hoodie.datasource.write.hive_style_partitioning = '$hiveStylePartitionEnable'
                    | )
               """.stripMargin)
      @@ -74,7 +75,7 @@ class TestRepairTable extends HoodieSparkSqlTestBase {
                    | location '$basePath'
                    | tblproperties (
                    |  primaryKey = 'id',
      -             |  preCombineField = 'ts',
      +             |  orderingFields = 'ts',
                    |  hoodie.datasource.write.hive_style_partitioning = '$hiveStylePartitionEnable'
                    | )
               """.stripMargin)
      @@ -85,7 +86,7 @@ class TestRepairTable extends HoodieSparkSqlTestBase {
                 .toDF("id", "name", "ts", "dt", "hh")
               df.write.format("hudi")
                 .option(RECORDKEY_FIELD.key, "id")
      -          .option(PRECOMBINE_FIELD.key, "ts")
      +          .option(HoodieTableConfig.ORDERING_FIELDS.key, "ts")
                 .option(PARTITIONPATH_FIELD.key, "dt,hh")
                 .option(HIVE_STYLE_PARTITIONING_ENABLE.key, hiveStylePartitionEnable)
                 .mode(SaveMode.Append)
      @@ -111,7 +112,7 @@ class TestRepairTable extends HoodieSparkSqlTestBase {
               df.write.format("hudi")
                 .option(TBL_NAME.key(), tableName)
                 .option(RECORDKEY_FIELD.key, "id")
      -          .option(PRECOMBINE_FIELD.key, "ts")
      +          .option(HoodieTableConfig.ORDERING_FIELDS.key, "ts")
                 .option(PARTITIONPATH_FIELD.key, "dt,hh")
                 .option(HIVE_STYLE_PARTITIONING_ENABLE.key, hiveStylePartitionEnable)
                 .mode(SaveMode.Append)
      @@ -150,7 +151,7 @@ class TestRepairTable extends HoodieSparkSqlTestBase {
                    | location '$basePath'
                    | tblproperties (
                    |  primaryKey = 'id',
      -             |  preCombineField = 'ts',
      +             |  orderingFields = 'ts',
                    |  hoodie.datasource.write.hive_style_partitioning = '$hiveStylePartitionEnable'
                    | )
               """.stripMargin)
      @@ -162,7 +163,7 @@ class TestRepairTable extends HoodieSparkSqlTestBase {
               df1.write.format("hudi")
                 .option(TBL_NAME.key(), tableName)
                 .option(RECORDKEY_FIELD.key, "id")
      -          .option(PRECOMBINE_FIELD.key, "ts")
      +          .option(HoodieTableConfig.ORDERING_FIELDS.key, "ts")
                 .option(PARTITIONPATH_FIELD.key, "dt")
                 .option(HIVE_STYLE_PARTITIONING_ENABLE.key, hiveStylePartitionEnable)
                 .mode(SaveMode.Append)
      @@ -177,7 +178,7 @@ class TestRepairTable extends HoodieSparkSqlTestBase {
               df2.write.format("hudi")
                 .option(TBL_NAME.key(), tableName)
                 .option(RECORDKEY_FIELD.key, "id")
      -          .option(PRECOMBINE_FIELD.key, "ts")
      +          .option(HoodieTableConfig.ORDERING_FIELDS.key, "ts")
                 .option(PARTITIONPATH_FIELD.key, "dt")
                 .option(HIVE_STYLE_PARTITIONING_ENABLE.key, hiveStylePartitionEnable)
                 .mode(SaveMode.Overwrite)
      diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/ddl/TestShowPartitions.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/ddl/TestShowPartitions.scala
      index f146afd886039..cf9ddb4f9b3e8 100644
      --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/ddl/TestShowPartitions.scala
      +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/ddl/TestShowPartitions.scala
      @@ -38,7 +38,7 @@ class TestShowPartitions extends HoodieSparkSqlTestBase {
                |) using hudi
                |tblproperties (
                |  primaryKey = 'id',
      -         |  preCombineField = 'ts'
      +         |  orderingFields = 'ts'
                )
              """.stripMargin)
           // Insert data
      @@ -65,7 +65,7 @@ class TestShowPartitions extends HoodieSparkSqlTestBase {
                | partitioned by (dt)
                | tblproperties (
                |   primaryKey = 'id',
      -         |   preCombineField = 'ts'
      +         |   orderingFields = 'ts'
                | )
              """.stripMargin)
           // Empty partitions
      @@ -116,7 +116,7 @@ class TestShowPartitions extends HoodieSparkSqlTestBase {
                | partitioned by (year, month, day)
                | tblproperties (
                |   primaryKey = 'id',
      -         |   preCombineField = 'ts'
      +         |   orderingFields = 'ts'
                | )
              """.stripMargin)
           // Empty partitions
      @@ -185,7 +185,7 @@ class TestShowPartitions extends HoodieSparkSqlTestBase {
                      | partitioned by (year, month, day)
                      | tblproperties (
                      |   primaryKey = 'id',
      -               |   preCombineField = 'ts'
      +               |   orderingFields = 'ts'
                      | )
                    """.stripMargin)
                 spark.sql(s"alter table $tableName add partition(year='2023', month='06', day='06')")
      @@ -221,7 +221,7 @@ class TestShowPartitions extends HoodieSparkSqlTestBase {
                  | partitioned by (year, month, day)
                  | tblproperties (
                  |   primaryKey = 'id',
      -           |   preCombineField = 'ts'
      +           |   orderingFields = 'ts'
                  | )
                """.stripMargin)
       
      @@ -270,7 +270,7 @@ class TestShowPartitions extends HoodieSparkSqlTestBase {
                    | partitioned by (dt)
                    | tblproperties (
                    |   primaryKey = 'id',
      -             |   preCombineField = 'ts'
      +             |   orderingFields = 'ts'
                    | )
                """.stripMargin)
       
      diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/ddl/TestSpark3DDL.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/ddl/TestSpark3DDL.scala
      index df3382b356dec..3ea5b032f2bcc 100644
      --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/ddl/TestSpark3DDL.scala
      +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/ddl/TestSpark3DDL.scala
      @@ -21,7 +21,7 @@ import org.apache.hudi.{DataSourceWriteOptions, DefaultSparkRecordMerger, Quicks
       import org.apache.hudi.common.config.{HoodieReaderConfig, HoodieStorageConfig}
       import org.apache.hudi.common.model.{HoodieRecord, HoodieTableType}
       import org.apache.hudi.common.model.HoodieRecord.HoodieRecordType
      -import org.apache.hudi.common.table.TableSchemaResolver
      +import org.apache.hudi.common.table.{HoodieTableConfig, TableSchemaResolver}
       import org.apache.hudi.common.table.timeline.HoodieInstant
       import org.apache.hudi.common.testutils.{HoodieTestDataGenerator, RawTripTestPayload}
       import org.apache.hudi.config.HoodieWriteConfig
      @@ -61,7 +61,7 @@ class TestSpark3DDL extends HoodieSparkSqlTestBase {
                | options (
                |  type = '$tableType',
                |  primaryKey = 'id',
      -         |  preCombineField = 'comb'
      +         |  orderingFields = 'comb'
                | )
                | partitioned by (par)
                    """.stripMargin)
      @@ -205,7 +205,7 @@ class TestSpark3DDL extends HoodieSparkSqlTestBase {
               val df = spark.createDataFrame(rowRdd, structType)
               df.write.format("hudi")
                 .option("hoodie.datasource.write.recordkey.field", "id")
      -          .option("hoodie.datasource.write.precombine.field", "ts")
      +          .option(HoodieTableConfig.ORDERING_FIELDS.key(), "ts")
                 .option("hoodie.datasource.write.partitionpath.field", "partition")
                 .option("hoodie.table.name", tableName)
                 .option("hoodie.datasource.write.table.type", tableType.name())
      @@ -224,7 +224,7 @@ class TestSpark3DDL extends HoodieSparkSqlTestBase {
               val df2 = spark.createDataFrame(rowRdd2, structType)
               df2.write.format("hudi")
                 .option("hoodie.datasource.write.recordkey.field", "id")
      -          .option("hoodie.datasource.write.precombine.field", "ts")
      +          .option(HoodieTableConfig.ORDERING_FIELDS.key(), "ts")
                 .option("hoodie.datasource.write.partitionpath.field", "partition")
                 .option("hoodie.table.name", tableName)
                 .option("hoodie.datasource.write.table.type", tableType.name())
      @@ -250,7 +250,7 @@ class TestSpark3DDL extends HoodieSparkSqlTestBase {
               val df3 = spark.createDataFrame(rowRdd3, structType3)
               df3.write.format("hudi")
                 .option("hoodie.datasource.write.recordkey.field", "id")
      -          .option("hoodie.datasource.write.precombine.field", "ts")
      +          .option(HoodieTableConfig.ORDERING_FIELDS.key(), "ts")
                 .option("hoodie.datasource.write.partitionpath.field", "partition")
                 .option("hoodie.table.name", tableName)
                 .option("hoodie.datasource.write.table.type", tableType.name())
      @@ -285,7 +285,7 @@ class TestSpark3DDL extends HoodieSparkSqlTestBase {
                    | tblproperties (
                    |  type = '$tableType',
                    |  primaryKey = 'id',
      -             |  preCombineField = 'ts'
      +             |  orderingFields = 'ts'
                    | )
              """.stripMargin)
       
      @@ -358,8 +358,8 @@ class TestSpark3DDL extends HoodieSparkSqlTestBase {
                 checkException(s"Alter table $tableName add columns(col_new1 int after $f)")("forbid adjust the position of ordinary columns between meta columns")
               }
               Seq("id", "comb", "par").foreach { col =>
      -          checkException(s"alter table $tableName drop column $col")("cannot support apply changes for primaryKey/CombineKey/partitionKey")
      -          checkException(s"alter table $tableName rename column $col to ${col + col}")("cannot support apply changes for primaryKey/CombineKey/partitionKey")
      +          checkException(s"alter table $tableName drop column $col")("cannot support apply changes for primaryKey/orderingFields/partitionKey")
      +          checkException(s"alter table $tableName rename column $col to ${col + col}")("cannot support apply changes for primaryKey/orderingFields/partitionKey")
               }
               // check duplicate add or rename
               // keep consistent with hive, column names insensitive
      @@ -549,7 +549,7 @@ class TestSpark3DDL extends HoodieSparkSqlTestBase {
                      | options (
                      |  type = '$tableType',
                      |  primaryKey = 'id',
      -               |  preCombineField = 'comb'
      +               |  orderingFields = 'comb'
                      | )
                      | partitioned by (par)
                    """.stripMargin)
      @@ -605,7 +605,7 @@ class TestSpark3DDL extends HoodieSparkSqlTestBase {
                      | options (
                      |  type = '$tableType',
                      |  primaryKey = 'id',
      -               |  preCombineField = 'ts'
      +               |  orderingFields = 'ts'
                      | )
                    """.stripMargin)
                 spark.sql(s"show create table ${tableName}").show(false)
      @@ -684,7 +684,7 @@ class TestSpark3DDL extends HoodieSparkSqlTestBase {
                    | options (
                    |  type = '$tableType',
                    |  primaryKey = 'id',
      -             |  preCombineField = 'ts'
      +             |  orderingFields = 'ts'
                    | )
                    """.stripMargin)
               spark.sql(s"alter table $tableName alter column name drop not null")
      @@ -714,7 +714,7 @@ class TestSpark3DDL extends HoodieSparkSqlTestBase {
                      | options (
                      |  type = '$tableType',
                      |  primaryKey = 'id',
      -               |  preCombineField = 'ts'
      +               |  orderingFields = 'ts'
                      | )
                    """.stripMargin)
                 spark.sql(s"show create table ${tableName}").show(false)
      @@ -754,7 +754,7 @@ class TestSpark3DDL extends HoodieSparkSqlTestBase {
                    | options (
                    |  type = 'mor',
                    |  primaryKey = 'id',
      -             |  preCombineField = 'ts'
      +             |  orderingFields = 'ts'
                    | )
                    """.stripMargin)
       
      @@ -1009,7 +1009,7 @@ class TestSpark3DDL extends HoodieSparkSqlTestBase {
                    | options (
                    |  type = '$tableType',
                    |  primaryKey = 'id',
      -             |  preCombineField = 'ts'
      +             |  orderingFields = 'ts'
                    | )
                    | partitioned by (ts)
                    """.stripMargin)
      @@ -1053,7 +1053,7 @@ class TestSpark3DDL extends HoodieSparkSqlTestBase {
                    | tblproperties (
                    |  primaryKey = 'id',
                    |  type = '$tableType',
      -             |  preCombineField = 'ts'
      +             |  orderingFields = 'ts'
                    |  ${if (tableType.equals("mor")) ", hoodie.index.type = 'INMEMORY'" else ""}
                    | )
                  """.stripMargin)
      @@ -1091,7 +1091,7 @@ class TestSpark3DDL extends HoodieSparkSqlTestBase {
                    | tblproperties (
                    |  primaryKey = 'id',
                    |  type = '$tableType',
      -             |  preCombineField = 'ts'
      +             |  orderingFields = 'ts'
                    |  ${if (tableType.equals("mor")) ", hoodie.index.type = 'INMEMORY'" else ""}
                    | )
                  """.stripMargin)
      @@ -1172,7 +1172,7 @@ class TestSpark3DDL extends HoodieSparkSqlTestBase {
                    |tblproperties (
                    | primaryKey = 'id',
                    | type = '$tableType',
      -             | preCombineField = 'ts'
      +             | orderingFields = 'ts'
                    |)
                    |partitioned by (region, dt)""".stripMargin)
       
      diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/ddl/TestTruncateTable.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/ddl/TestTruncateTable.scala
      index 3d290d589e249..ba824c9c68f5a 100644
      --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/ddl/TestTruncateTable.scala
      +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/ddl/TestTruncateTable.scala
      @@ -20,6 +20,7 @@
       package org.apache.spark.sql.hudi.ddl
       
       import org.apache.hudi.DataSourceWriteOptions._
      +import org.apache.hudi.common.table.HoodieTableConfig
       import org.apache.hudi.config.HoodieWriteConfig
       
       import org.apache.spark.sql.SaveMode
      @@ -42,7 +43,7 @@ class TestTruncateTable extends HoodieSparkSqlTestBase {
                   | options (
                   |  type = '$tableType',
                   |  primaryKey = 'id',
      -            |  preCombineField = 'ts'
      +            |  orderingFields = 'ts'
                   | )
              """.stripMargin)
             // Insert data
      @@ -73,7 +74,7 @@ class TestTruncateTable extends HoodieSparkSqlTestBase {
                 .option(HoodieWriteConfig.TBL_NAME.key, tableName)
                 .option(TABLE_TYPE.key, MOR_TABLE_TYPE_OPT_VAL)
                 .option(RECORDKEY_FIELD.key, "id")
      -          .option(PRECOMBINE_FIELD.key, "ts")
      +          .option(HoodieTableConfig.ORDERING_FIELDS.key, "ts")
                 .option(PARTITIONPATH_FIELD.key, "dt")
                 .option(URL_ENCODE_PARTITIONING.key(), urlencode)
                 .option(HoodieWriteConfig.INSERT_PARALLELISM_VALUE.key, "1")
      @@ -114,7 +115,7 @@ class TestTruncateTable extends HoodieSparkSqlTestBase {
                 .option(HoodieWriteConfig.TBL_NAME.key, tableName)
                 .option(TABLE_TYPE.key, COW_TABLE_TYPE_OPT_VAL)
                 .option(RECORDKEY_FIELD.key, "id")
      -          .option(PRECOMBINE_FIELD.key, "ts")
      +          .option(HoodieTableConfig.ORDERING_FIELDS.key, "ts")
                 .option(PARTITIONPATH_FIELD.key, "year,month,day")
                 .option(HIVE_STYLE_PARTITIONING.key, hiveStyle)
                 .option(HoodieWriteConfig.INSERT_PARALLELISM_VALUE.key, "1")
      diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/dml/insert/TestInsertTable.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/dml/insert/TestInsertTable.scala
      index da6a44ef4865e..72de4a0d0b5ae 100644
      --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/dml/insert/TestInsertTable.scala
      +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/dml/insert/TestInsertTable.scala
      @@ -1168,7 +1168,7 @@ class TestInsertTable extends HoodieSparkSqlTestBase {
                   .option(HoodieWriteConfig.TBL_NAME.key, tableName)
                   .option(TABLE_TYPE.key, COW_TABLE_TYPE_OPT_VAL)
                   .option(RECORDKEY_FIELD.key, "id")
      -            .option(PRECOMBINE_FIELD.key, "ts")
      +            .option(HoodieTableConfig.ORDERING_FIELDS.key, "ts")
                   .option(HoodieWriteConfig.BULKINSERT_PARALLELISM_VALUE.key, "1")
                   // test specific settings
                   .option(OPERATION.key, WriteOperationType.BULK_INSERT.value)
      @@ -1492,7 +1492,7 @@ class TestInsertTable extends HoodieSparkSqlTestBase {
               .option(HoodieWriteConfig.TBL_NAME.key, tableName)
               .option(TABLE_TYPE.key, MOR_TABLE_TYPE_OPT_VAL)
               .option(RECORDKEY_FIELD.key, "id")
      -        .option(PRECOMBINE_FIELD.key, "ts")
      +        .option(HoodieTableConfig.ORDERING_FIELDS.key, "ts")
               .option(PARTITIONPATH_FIELD.key, "day,hh")
               .option(HoodieWriteConfig.INSERT_PARALLELISM_VALUE.key, "1")
               .option(HoodieWriteConfig.UPSERT_PARALLELISM_VALUE.key, "1")
      @@ -2167,7 +2167,7 @@ class TestInsertTable extends HoodieSparkSqlTestBase {
           }
         }
       
      -  test("Test Hudi should not record empty preCombineKey in hoodie.properties") {
      +  test("Test Hudi should not record empty orderingFields in hoodie.properties") {
           withSQLConf("hoodie.datasource.write.operation" -> "insert") {
             withTempDir { tmp =>
               val tableName = generateTableName
      diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/dml/others/TestDeleteTable.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/dml/others/TestDeleteTable.scala
      index 0e7223b35fbdd..06f0a77e93f38 100644
      --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/dml/others/TestDeleteTable.scala
      +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/dml/others/TestDeleteTable.scala
      @@ -20,6 +20,7 @@
       package org.apache.spark.sql.hudi.dml.others
       
       import org.apache.hudi.DataSourceWriteOptions._
      +import org.apache.hudi.common.table.HoodieTableConfig
       import org.apache.hudi.config.HoodieWriteConfig
       
       import org.apache.spark.sql.SaveMode
      @@ -337,7 +338,7 @@ class TestDeleteTable extends HoodieSparkSqlTestBase {
                   .option(HoodieWriteConfig.TBL_NAME.key, tableName)
                   .option(TABLE_TYPE.key, MOR_TABLE_TYPE_OPT_VAL)
                   .option(RECORDKEY_FIELD.key, "id")
      -            .option(PRECOMBINE_FIELD.key, "ts")
      +            .option(HoodieTableConfig.ORDERING_FIELDS.key, "ts")
                   .option(PARTITIONPATH_FIELD.key, "dt")
                   .option(URL_ENCODE_PARTITIONING.key(), urlencode)
                   .option(HoodieWriteConfig.INSERT_PARALLELISM_VALUE.key, "1")
      diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/dml/others/TestMergeIntoTable.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/dml/others/TestMergeIntoTable.scala
      index c55367fe7ab3a..5007c95067e6d 100644
      --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/dml/others/TestMergeIntoTable.scala
      +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/dml/others/TestMergeIntoTable.scala
      @@ -760,7 +760,7 @@ class TestMergeIntoTable extends HoodieSparkSqlTestBase with ScalaAssertionSuppo
                    | tblproperties (
                    |  type = '$tableType',
                    |  primaryKey = 'id',
      -             |  preCombineField = 'v'
      +             |  orderingFields = 'v'
                    | )
                    | partitioned by(dt)
                    | location '${tmp.getCanonicalPath}/$tableName1'
      @@ -801,7 +801,7 @@ class TestMergeIntoTable extends HoodieSparkSqlTestBase with ScalaAssertionSuppo
                """.stripMargin
               )(
                 "MERGE INTO field resolution error: " +
      -            "Failed to resolve precombine field `v` w/in the source-table output")
      +            "Failed to resolve ordering fields `v` w/in the source-table output")
       
               spark.sql(
                 s"""
      @@ -889,7 +889,7 @@ class TestMergeIntoTable extends HoodieSparkSqlTestBase with ScalaAssertionSuppo
                  | when matched then delete
                  |""".stripMargin)(
               "MERGE INTO field resolution error: "
      -          + "Failed to resolve precombine field `v` w/in the source-table output")
      +          + "Failed to resolve ordering fields `v` w/in the source-table output")
       
             //
             // 2.b) set source column name to be different with target column
      diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/dml/others/TestMergeIntoTable2.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/dml/others/TestMergeIntoTable2.scala
      index 56a1fd916096b..10a90c699877c 100644
      --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/dml/others/TestMergeIntoTable2.scala
      +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/dml/others/TestMergeIntoTable2.scala
      @@ -1060,7 +1060,7 @@ class TestMergeIntoTable2 extends HoodieSparkSqlTestBase {
                        | tblproperties (
                        |  type = '$tableType',
                        |  primaryKey = 'id',
      -                 |  preCombineField = 'ts',
      +                 |  orderingFields = 'ts',
                        |  recordMergeMode = '$mergeMode'
                        | )
                        | partitioned by(dt)
      @@ -1145,7 +1145,7 @@ class TestMergeIntoTable2 extends HoodieSparkSqlTestBase {
       
                   if (mergeMode == "EVENT_TIME_ORDERING") {
                     checkException(mergeStmt)(
      -                "MERGE INTO field resolution error: No matching assignment found for target table precombine field `ts`"
      +                "MERGE INTO field resolution error: No matching assignment found for target table ordering field `ts`"
                     )
                   } else {
                     // For COMMIT_TIME_ORDERING, this should execute without error
      diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/dml/others/TestMergeModeCommitTimeOrdering.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/dml/others/TestMergeModeCommitTimeOrdering.scala
      index eac9a29e5d881..7f7e3a9f4986d 100644
      --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/dml/others/TestMergeModeCommitTimeOrdering.scala
      +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/dml/others/TestMergeModeCommitTimeOrdering.scala
      @@ -74,12 +74,12 @@ class TestMergeModeCommitTimeOrdering extends HoodieSparkSqlTestBase {
               HoodieTableConfig.RECORD_MERGE_STRATEGY_ID.key -> COMMIT_TIME_BASED_MERGE_STRATEGY_UUID)
           }
           val nonExistentConfigs = if (tableVersion.toInt == 6) {
      -      Seq(HoodieTableConfig.RECORD_MERGE_MODE.key, HoodieTableConfig.PRECOMBINE_FIELDS.key)
      +      Seq(HoodieTableConfig.RECORD_MERGE_MODE.key, HoodieTableConfig.ORDERING_FIELDS.key)
           } else {
             if (setRecordMergeConfigs) {
               Seq()
             } else {
      -        Seq(HoodieTableConfig.PRECOMBINE_FIELDS.key)
      +        Seq(HoodieTableConfig.ORDERING_FIELDS.key)
             }
           }
       
      diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/dml/others/TestMergeModeEventTimeOrdering.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/dml/others/TestMergeModeEventTimeOrdering.scala
      index 7122183e249c9..a3bc2af091066 100644
      --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/dml/others/TestMergeModeEventTimeOrdering.scala
      +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/dml/others/TestMergeModeEventTimeOrdering.scala
      @@ -71,12 +71,12 @@ class TestMergeModeEventTimeOrdering extends HoodieSparkSqlTestBase {
             Map(
               HoodieTableConfig.VERSION.key -> "6",
               HoodieTableConfig.PAYLOAD_CLASS_NAME.key -> classOf[DefaultHoodieRecordPayload].getName,
      -        HoodieTableConfig.PRECOMBINE_FIELDS.key -> "ts"
      +        HoodieTableConfig.ORDERING_FIELDS.key -> "ts"
             )
           } else {
             Map(
               HoodieTableConfig.VERSION.key -> String.valueOf(HoodieTableVersion.current().versionCode()),
      -        HoodieTableConfig.PRECOMBINE_FIELDS.key -> "ts",
      +        HoodieTableConfig.ORDERING_FIELDS.key -> "ts",
               HoodieTableConfig.RECORD_MERGE_MODE.key -> EVENT_TIME_ORDERING.name(),
               HoodieTableConfig.RECORD_MERGE_STRATEGY_ID.key -> EVENT_TIME_BASED_MERGE_STRATEGY_UUID)
           }
      diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/dml/others/TestPartialUpdateForMergeInto.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/dml/others/TestPartialUpdateForMergeInto.scala
      index 9ca649dab4cdf..c277af22595be 100644
      --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/dml/others/TestPartialUpdateForMergeInto.scala
      +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/dml/others/TestPartialUpdateForMergeInto.scala
      @@ -580,7 +580,7 @@ class TestPartialUpdateForMergeInto extends HoodieSparkSqlTestBase {
                | preCombineField = '_ts'
                |)""".stripMargin)
       
      -    val failedToResolveErrorMessage = "Failed to resolve precombine field `_ts` w/in the source-table output"
      +    val failedToResolveErrorMessage = "Failed to resolve ordering fields `_ts` w/in the source-table output"
       
           checkExceptionContain(
             s"""
      diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/feature/TestCDCForSparkSQL.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/feature/TestCDCForSparkSQL.scala
      index 7c6304dfa49f9..e316559aa5158 100644
      --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/feature/TestCDCForSparkSQL.scala
      +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/feature/TestCDCForSparkSQL.scala
      @@ -71,7 +71,7 @@ class TestCDCForSparkSQL extends HoodieSparkSqlTestBase {
                    | partitioned by (name)
                    | tblproperties (
                    |   'primaryKey' = 'id',
      -             |   'preCombineField' = 'ts',
      +             |   'orderingFields' = 'ts',
                    |   'hoodie.table.cdc.enabled' = 'true',
                    |   'hoodie.table.cdc.supplemental.logging.mode' = '$DATA_BEFORE_AFTER',
                    |   type = '$tableType'
      @@ -125,7 +125,7 @@ class TestCDCForSparkSQL extends HoodieSparkSqlTestBase {
                      | ) using hudi
                      | tblproperties (
                      |   'primaryKey' = 'id',
      -               |   'preCombineField' = 'ts',
      +               |   'orderingFields' = 'ts',
                      |   'hoodie.table.cdc.enabled' = 'true',
                      |   'hoodie.table.cdc.supplemental.logging.mode' = '${loggingMode.name()}',
                      |   $otherTableProperties
      @@ -248,7 +248,7 @@ class TestCDCForSparkSQL extends HoodieSparkSqlTestBase {
                      | partitioned by (pt)
                      | tblproperties (
                      |   'primaryKey' = 'id',
      -               |   'preCombineField' = 'ts',
      +               |   'orderingFields' = 'ts',
                      |   'hoodie.table.cdc.enabled' = 'true',
                      |   'hoodie.table.cdc.supplemental.logging.mode' = '${loggingMode.name()}',
                      |   'type' = '$tableType'
      @@ -331,7 +331,7 @@ class TestCDCForSparkSQL extends HoodieSparkSqlTestBase {
                      | partitioned by (pt)
                      | tblproperties (
                      |   'primaryKey' = 'id',
      -               |   'preCombineField' = 'ts',
      +               |   'orderingFields' = 'ts',
                      |   'hoodie.table.cdc.enabled' = 'true',
                      |   'hoodie.table.cdc.supplemental.logging.mode' = '${loggingMode.name()}',
                      |   '${ENABLE_MERGE_INTO_PARTIAL_UPDATES.key}' = 'true',
      diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/feature/TestCompactionTable.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/feature/TestCompactionTable.scala
      index 5d891e70c2b16..37fa2a4278cfa 100644
      --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/feature/TestCompactionTable.scala
      +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/feature/TestCompactionTable.scala
      @@ -38,7 +38,7 @@ class TestCompactionTable extends HoodieSparkSqlTestBase {
                  | tblproperties (
                  |  primaryKey ='id',
                  |  type = 'mor',
      -           |  preCombineField = 'ts'
      +           |  orderingFields = 'ts'
                  | )
              """.stripMargin)
             spark.sql("set hoodie.parquet.max.file.size = 10000")
      @@ -93,7 +93,7 @@ class TestCompactionTable extends HoodieSparkSqlTestBase {
                  | tblproperties (
                  |  primaryKey ='id',
                  |  type = 'mor',
      -           |  preCombineField = 'ts'
      +           |  orderingFields = 'ts'
                  | )
              """.stripMargin)
             spark.sql("set hoodie.parquet.max.file.size = 10000")
      @@ -154,7 +154,7 @@ class TestCompactionTable extends HoodieSparkSqlTestBase {
                  | tblproperties (
                  |  primaryKey ='id',
                  |  type = 'mor',
      -           |  preCombineField = 'ts'
      +           |  orderingFields = 'ts'
                  | )
              """.stripMargin)
             spark.sql("set hoodie.parquet.max.file.size = 10000")
      diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/feature/TestDataSkippingQuery.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/feature/TestDataSkippingQuery.scala
      index 6b7002d40e633..aedc972b0f083 100644
      --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/feature/TestDataSkippingQuery.scala
      +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/feature/TestDataSkippingQuery.scala
      @@ -143,7 +143,7 @@ class TestDataSkippingQuery extends HoodieSparkSqlTestBase {
                      |) using hudi
                      | tblproperties (
                      | primaryKey = 'id,name',
      -               | preCombineField = 'ts',
      +               | orderingFields = 'ts',
                      | type = '$tableType',
                      | hoodie.index.type = 'BUCKET',
                      | hoodie.bucket.index.hash.field = 'id',
      diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/feature/TestQueryMergeOnReadOptimizedTable.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/feature/TestQueryMergeOnReadOptimizedTable.scala
      index b918dc7540863..4a17974f94437 100644
      --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/feature/TestQueryMergeOnReadOptimizedTable.scala
      +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/feature/TestQueryMergeOnReadOptimizedTable.scala
      @@ -41,7 +41,7 @@ class TestQueryMergeOnReadOptimizedTable extends HoodieSparkSqlTestBase {
                  | tblproperties (
                  |  type = 'mor',
                  |  primaryKey = 'id',
      -           |  preCombineField = 'ts'
      +           |  orderingFields = 'ts'
                  | )
              """.stripMargin)
             // insert data to table
      diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/feature/index/TestExpressionIndex.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/feature/index/TestExpressionIndex.scala
      index 230895caf4fb7..2d8361a77ba67 100644
      --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/feature/index/TestExpressionIndex.scala
      +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/feature/index/TestExpressionIndex.scala
      @@ -29,7 +29,7 @@ import org.apache.hudi.client.utils.SparkMetadataWriterUtils
       import org.apache.hudi.common.config.{HoodieMetadataConfig, HoodieStorageConfig, TypedProperties}
       import org.apache.hudi.common.fs.FSUtils
       import org.apache.hudi.common.model.{FileSlice, HoodieIndexDefinition}
      -import org.apache.hudi.common.table.{HoodieTableMetaClient, HoodieTableVersion}
      +import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient, HoodieTableVersion}
       import org.apache.hudi.common.table.view.{FileSystemViewManager, HoodieTableFileSystemView}
       import org.apache.hudi.common.testutils.HoodieTestUtils
       import org.apache.hudi.common.util.Option
      @@ -1984,7 +1984,7 @@ class TestExpressionIndex extends HoodieSparkSqlTestBase {
               "hoodie.upsert.shuffle.parallelism" -> "4",
               HoodieWriteConfig.TBL_NAME.key -> tableName,
               RECORDKEY_FIELD.key -> "c1",
      -        PRECOMBINE_FIELD.key -> "c1",
      +        HoodieTableConfig.ORDERING_FIELDS.key -> "c1",
               PARTITIONPATH_FIELD.key() -> "c8",
               "hoodie.metadata.index.column.stats.enable" -> "false"
             )
      @@ -2070,7 +2070,7 @@ class TestExpressionIndex extends HoodieSparkSqlTestBase {
               HoodieWriteConfig.TBL_NAME.key -> tableName,
               TABLE_TYPE.key -> "MERGE_ON_READ",
               RECORDKEY_FIELD.key -> "c1",
      -        PRECOMBINE_FIELD.key -> "c1",
      +        HoodieTableConfig.ORDERING_FIELDS.key -> "c1",
               PARTITIONPATH_FIELD.key() -> "c8",
               // setting IndexType to be INMEMORY to simulate Global Index nature
               HoodieIndexConfig.INDEX_TYPE.key -> HoodieIndex.IndexType.INMEMORY.name(),
      @@ -2142,7 +2142,7 @@ class TestExpressionIndex extends HoodieSparkSqlTestBase {
             HoodieWriteConfig.TBL_NAME.key -> "testGetExpressionIndexRecordsUsingBloomFilter",
             TABLE_TYPE.key -> "MERGE_ON_READ",
             RECORDKEY_FIELD.key -> "c1",
      -      PRECOMBINE_FIELD.key -> "c1",
      +      HoodieTableConfig.ORDERING_FIELDS.key -> "c1",
             PARTITIONPATH_FIELD.key() -> "c8",
             // setting IndexType to be INMEMORY to simulate Global Index nature
             HoodieIndexConfig.INDEX_TYPE.key -> HoodieIndex.IndexType.INMEMORY.name()
      diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/feature/index/testHoodieBackedTableMetadataIndexLookup.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/feature/index/TestHoodieBackedTableMetadataIndexLookup.scala
      similarity index 99%
      rename from hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/feature/index/testHoodieBackedTableMetadataIndexLookup.scala
      rename to hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/feature/index/TestHoodieBackedTableMetadataIndexLookup.scala
      index 88a2c13436ed1..6371fb7925ad9 100644
      --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/feature/index/testHoodieBackedTableMetadataIndexLookup.scala
      +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/feature/index/TestHoodieBackedTableMetadataIndexLookup.scala
      @@ -24,7 +24,7 @@ import org.apache.hudi.client.common.HoodieSparkEngineContext
       import org.apache.hudi.common.data.{HoodieListData, HoodiePairData}
       import org.apache.hudi.common.engine.EngineType
       import org.apache.hudi.common.model.HoodieRecordLocation
      -import org.apache.hudi.common.table.HoodieTableMetaClient
      +import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient}
       import org.apache.hudi.common.testutils.HoodieTestDataGenerator
       import org.apache.hudi.common.util.HoodieDataUtils
       import org.apache.hudi.config.HoodieWriteConfig
      @@ -179,7 +179,7 @@ abstract class HoodieBackedTableMetadataIndexLookupTestBase extends HoodieSparkS
             "hoodie.insert.shuffle.parallelism" -> "4",
             "hoodie.upsert.shuffle.parallelism" -> "4",
             DataSourceWriteOptions.RECORDKEY_FIELD.key -> "id",
      -      DataSourceWriteOptions.PRECOMBINE_FIELD.key -> "ts",
      +      HoodieTableConfig.ORDERING_FIELDS.key -> "ts",
             HoodieWriteConfig.TBL_NAME.key -> tableName,
             DataSourceWriteOptions.OPERATION.key -> DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL
           )
      diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/feature/index/TestSecondaryIndex.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/feature/index/TestSecondaryIndex.scala
      index aedd18e0ead34..232fa431f5ee2 100644
      --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/feature/index/TestSecondaryIndex.scala
      +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/feature/index/TestSecondaryIndex.scala
      @@ -23,7 +23,7 @@ import org.apache.hudi.{DataSourceReadOptions, DataSourceWriteOptions, HoodieSpa
       import org.apache.hudi.DataSourceWriteOptions._
       import org.apache.hudi.common.config.{HoodieMetadataConfig, RecordMergeMode}
       import org.apache.hudi.common.model.WriteOperationType
      -import org.apache.hudi.common.table.{HoodieTableMetaClient, TableSchemaResolver}
      +import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient, TableSchemaResolver}
       import org.apache.hudi.common.testutils.{HoodieTestDataGenerator, HoodieTestUtils}
       import org.apache.hudi.common.testutils.RawTripTestPayload.recordsToStrings
       import org.apache.hudi.config.{HoodieClusteringConfig, HoodieCompactionConfig, HoodieWriteConfig}
      @@ -53,7 +53,7 @@ class TestSecondaryIndex extends HoodieSparkSqlTestBase {
           "hoodie.upsert.shuffle.parallelism" -> "4",
           RECORDKEY_FIELD.key -> "_row_key",
           PARTITIONPATH_FIELD.key -> "partition_path",
      -    PRECOMBINE_FIELD.key -> "timestamp",
      +    HoodieTableConfig.ORDERING_FIELDS.key -> "timestamp",
           HoodieClusteringConfig.INLINE_CLUSTERING.key -> "true",
           HoodieClusteringConfig.INLINE_CLUSTERING_MAX_COMMITS.key -> "4",
           HoodieCompactionConfig.INLINE_COMPACT.key() -> "true",
      @@ -1174,7 +1174,7 @@ class TestSecondaryIndex extends HoodieSparkSqlTestBase {
                 .option("hoodie.table.name", tableName)
                 .option("hoodie.datasource.write.table.type", "COPY_ON_WRITE")
                 .option("hoodie.datasource.write.recordkey.field", "id")
      -          .option("hoodie.datasource.write.precombine.field", "ts")
      +          .option(HoodieTableConfig.ORDERING_FIELDS.key(), "ts")
                 .option("hoodie.datasource.write.operation", "upsert")
                 .option("hoodie.schema.on.read.enable", "true")
                 .mode("append")
      diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestArchiveCommitsProcedure.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestArchiveCommitsProcedure.scala
      index e44739caf749e..c81ffcfb59d6f 100644
      --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestArchiveCommitsProcedure.scala
      +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestArchiveCommitsProcedure.scala
      @@ -36,7 +36,7 @@ class TestArchiveCommitsProcedure extends HoodieSparkProcedureTestBase {
                  | tblproperties (
                  |   primaryKey = 'id',
                  |   type = 'cow',
      -           |   preCombineField = 'ts',
      +           |   orderingFields = 'ts',
                  |   hoodie.metadata.enable = "false"
                  | )
                  |""".stripMargin)
      diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestBootstrapProcedure.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestBootstrapProcedure.scala
      index 6f81377b22aeb..bc69b434bf776 100644
      --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestBootstrapProcedure.scala
      +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestBootstrapProcedure.scala
      @@ -246,7 +246,7 @@ class TestBootstrapProcedure extends HoodieSparkProcedureTestBase {
             }
       
             spark.sql("set hoodie.bootstrap.parallelism = 20")
      -      spark.sql("set hoodie.datasource.write.precombine.field=timestamp")
      +      spark.sql("set hoodie.table.ordering.fields=timestamp")
             spark.sql("set hoodie.metadata.index.column.stats.enable = false")
       
             checkAnswer(
      diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestCallProcedure.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestCallProcedure.scala
      index 7af7a117906c4..2004f9c0c274d 100644
      --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestCallProcedure.scala
      +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestCallProcedure.scala
      @@ -37,7 +37,7 @@ class TestCallProcedure extends HoodieSparkProcedureTestBase {
                  | location '${tmp.getCanonicalPath}/$tableName'
                  | tblproperties (
                  |  primaryKey = 'id',
      -           |  preCombineField = 'ts'
      +           |  orderingFields = 'ts'
                  | )
              """.stripMargin)
             // insert data to table
      @@ -71,7 +71,7 @@ class TestCallProcedure extends HoodieSparkProcedureTestBase {
                  | location '${tmp.getCanonicalPath}/$tableName'
                  | tblproperties (
                  |  primaryKey = 'id',
      -           |  preCombineField = 'ts'
      +           |  orderingFields = 'ts'
                  | )
              """.stripMargin)
             // insert data to table
      @@ -104,7 +104,7 @@ class TestCallProcedure extends HoodieSparkProcedureTestBase {
                  | location '${tmp.getCanonicalPath}/$tableName'
                  | tblproperties (
                  |  primaryKey = 'id',
      -           |  preCombineField = 'ts'
      +           |  orderingFields = 'ts'
                  | )
              """.stripMargin)
             // insert data to table
      @@ -148,7 +148,7 @@ class TestCallProcedure extends HoodieSparkProcedureTestBase {
                  | location '${tmp.getCanonicalPath}/$tableName'
                  | tblproperties (
                  |  primaryKey = 'id',
      -           |  preCombineField = 'ts'
      +           |  orderingFields = 'ts'
                  | )
              """.stripMargin)
             // insert data to table
      @@ -187,7 +187,7 @@ class TestCallProcedure extends HoodieSparkProcedureTestBase {
                  | location '$tablePath'
                  | tblproperties (
                  |  primaryKey = 'id',
      -           |  preCombineField = 'ts'
      +           |  orderingFields = 'ts'
                  | )
              """.stripMargin)
       
      @@ -225,7 +225,7 @@ class TestCallProcedure extends HoodieSparkProcedureTestBase {
                  | location '$tablePath'
                  | tblproperties (
                  |  primaryKey = 'id',
      -           |  preCombineField = 'ts'
      +           |  orderingFields = 'ts'
                  | )
              """.stripMargin)
       
      @@ -268,7 +268,7 @@ class TestCallProcedure extends HoodieSparkProcedureTestBase {
                  | location '${tmp.getCanonicalPath}/$tableName'
                  | tblproperties (
                  |  primaryKey = 'id',
      -           |  preCombineField = 'ts'
      +           |  orderingFields = 'ts'
                  | )
              """.stripMargin)
             // insert data to table
      @@ -312,7 +312,7 @@ class TestCallProcedure extends HoodieSparkProcedureTestBase {
                  | location '${tmp.getCanonicalPath}/$tableName'
                  | tblproperties (
                  |  primaryKey = 'id',
      -           |  preCombineField = 'ts'
      +           |  orderingFields = 'ts'
                  | )
              """.stripMargin)
             // insert data to table
      diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestCleanProcedure.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestCleanProcedure.scala
      index a35696a0487a4..ae1438d7e970a 100644
      --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestCleanProcedure.scala
      +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestCleanProcedure.scala
      @@ -37,7 +37,7 @@ class TestCleanProcedure extends HoodieSparkProcedureTestBase {
                    | tblproperties (
                    |   primaryKey = 'id',
                    |   type = 'cow',
      -             |   preCombineField = 'ts'
      +             |   orderingFields = 'ts'
                    | )
                    |""".stripMargin)
       
      @@ -135,7 +135,7 @@ class TestCleanProcedure extends HoodieSparkProcedureTestBase {
                    | tblproperties (
                    |   primaryKey = 'id',
                    |   type = 'cow',
      -             |   preCombineField = 'ts',
      +             |   orderingFields = 'ts',
                    |   hoodie.clean.policy = 'KEEP_LATEST_COMMITS',
                    |   hoodie.clean.commits.retained = '2'
                    | )
      @@ -175,7 +175,7 @@ class TestCleanProcedure extends HoodieSparkProcedureTestBase {
                    | tblproperties (
                    |   primaryKey = 'id',
                    |   type = 'cow',
      -             |   preCombineField = 'ts'
      +             |   orderingFields = 'ts'
                    | )
                    |""".stripMargin)
       
      diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestClusteringProcedure.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestClusteringProcedure.scala
      index 2f89e38eeba33..0ded61f174dd7 100644
      --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestClusteringProcedure.scala
      +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestClusteringProcedure.scala
      @@ -58,7 +58,7 @@ class TestClusteringProcedure extends HoodieSparkProcedureTestBase {
                    | options (
                    |  primaryKey ='id',
                    |  type = '$tableType',
      -             |  preCombineField = 'ts'
      +             |  orderingFields = 'ts'
                    | )
                    | partitioned by(partition)
                    | location '$basePath'
      @@ -159,7 +159,7 @@ class TestClusteringProcedure extends HoodieSparkProcedureTestBase {
                    | options (
                    |  primaryKey ='id',
                    |  type = '$tableType',
      -             |  preCombineField = 'ts'
      +             |  orderingFields = 'ts'
                    | )
                    | partitioned by(partition)
                    | location '$basePath'
      @@ -243,7 +243,7 @@ class TestClusteringProcedure extends HoodieSparkProcedureTestBase {
                    | options (
                    |  primaryKey ='id',
                    |  type = '$tableType',
      -             |  preCombineField = 'ts'
      +             |  orderingFields = 'ts'
                    | )
                    | partitioned by(partition)
                    | location '$basePath'
      diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestCommitsProcedure.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestCommitsProcedure.scala
      index d15cc09bc4302..4ea4690fa4b7b 100644
      --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestCommitsProcedure.scala
      +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestCommitsProcedure.scala
      @@ -34,7 +34,7 @@ class TestCommitsProcedure extends HoodieSparkProcedureTestBase {
                  | location '${tmp.getCanonicalPath}/$tableName'
                  | tblproperties (
                  |  primaryKey = 'id',
      -           |  preCombineField = 'ts',
      +           |  orderingFields = 'ts',
                  |  hoodie.keep.max.commits = 5,
                  |  hoodie.keep.min.commits = 4,
                  |  hoodie.clean.commits.retained = 1
      @@ -84,7 +84,7 @@ class TestCommitsProcedure extends HoodieSparkProcedureTestBase {
                  | location '${tmp.getCanonicalPath}/$tableName'
                  | tblproperties (
                  |  primaryKey = 'id',
      -           |  preCombineField = 'ts',
      +           |  orderingFields = 'ts',
                  |  hoodie.keep.max.commits = 5,
                  |  hoodie.keep.min.commits = 4,
                  |  hoodie.clean.commits.retained = 1
      @@ -134,7 +134,7 @@ class TestCommitsProcedure extends HoodieSparkProcedureTestBase {
                  | location '${tmp.getCanonicalPath}/$tableName'
                  | tblproperties (
                  |  primaryKey = 'id',
      -           |  preCombineField = 'ts'
      +           |  orderingFields = 'ts'
                  | )
              """.stripMargin)
       
      @@ -172,7 +172,7 @@ class TestCommitsProcedure extends HoodieSparkProcedureTestBase {
                  | location '${tmp.getCanonicalPath}/$tableName'
                  | tblproperties (
                  |  primaryKey = 'id',
      -           |  preCombineField = 'ts'
      +           |  orderingFields = 'ts'
                  | )
              """.stripMargin)
       
      @@ -210,7 +210,7 @@ class TestCommitsProcedure extends HoodieSparkProcedureTestBase {
                  | location '${tmp.getCanonicalPath}/$tableName'
                  | tblproperties (
                  |  primaryKey = 'id',
      -           |  preCombineField = 'ts'
      +           |  orderingFields = 'ts'
                  | )
              """.stripMargin)
       
      @@ -249,7 +249,7 @@ class TestCommitsProcedure extends HoodieSparkProcedureTestBase {
                  | location '${tmp.getCanonicalPath}/$tableName1'
                  | tblproperties (
                  |  primaryKey = 'id',
      -           |  preCombineField = 'ts'
      +           |  orderingFields = 'ts'
                  | )
              """.stripMargin)
             // insert data to table1
      @@ -268,7 +268,7 @@ class TestCommitsProcedure extends HoodieSparkProcedureTestBase {
                  | location '${tmp.getCanonicalPath}/$tableName2'
                  | tblproperties (
                  |  primaryKey = 'id',
      -           |  preCombineField = 'ts'
      +           |  orderingFields = 'ts'
                  | )
              """.stripMargin)
             // insert data to table2
      @@ -308,7 +308,7 @@ class TestCommitsProcedure extends HoodieSparkProcedureTestBase {
                  | location '${tmp.getCanonicalPath}/$tableName'
                  | tblproperties (
                  |  primaryKey = 'id',
      -           |  preCombineField = 'ts'
      +           |  orderingFields = 'ts'
                  | )
            """.stripMargin)
       
      diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestCopyToTableProcedure.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestCopyToTableProcedure.scala
      index adbc4fd8210e6..f6fcab7ca9854 100644
      --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestCopyToTableProcedure.scala
      +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestCopyToTableProcedure.scala
      @@ -41,7 +41,7 @@ class TestCopyToTableProcedure extends HoodieSparkProcedureTestBase {
                  | location '${tmp.getCanonicalPath}/$tableName'
                  | tblproperties (
                  |  primaryKey = 'id',
      -           |  preCombineField = 'ts'
      +           |  orderingFields = 'ts'
                  | )
              """.stripMargin)
       
      @@ -77,7 +77,7 @@ class TestCopyToTableProcedure extends HoodieSparkProcedureTestBase {
                  | location '${tmp.getCanonicalPath}/$tableName'
                  | tblproperties (
                  |  primaryKey = 'id',
      -           |  preCombineField = 'ts'
      +           |  orderingFields = 'ts'
                  | )
              """.stripMargin)
       
      @@ -123,7 +123,7 @@ class TestCopyToTableProcedure extends HoodieSparkProcedureTestBase {
                  | location '${tmp.getCanonicalPath}/$tableName'
                  | tblproperties (
                  |  primaryKey = 'id',
      -           |  preCombineField = 'ts'
      +           |  orderingFields = 'ts'
                  | )
              """.stripMargin)
       
      @@ -183,7 +183,7 @@ class TestCopyToTableProcedure extends HoodieSparkProcedureTestBase {
                  | location '$tablePath'
                  | tblproperties (
                  |  primaryKey = 'id',
      -           |  preCombineField = 'ts'
      +           |  orderingFields = 'ts'
                  | )
              """.stripMargin)
       
      @@ -233,7 +233,7 @@ class TestCopyToTableProcedure extends HoodieSparkProcedureTestBase {
                  | options (
                  |  type='mor',
                  |  primaryKey = 'id',
      -           |  preCombineField = 'ts',
      +           |  orderingFields = 'ts',
                  |  hoodie.compact.inline.max.delta.commits='5',
                  |  hoodie.compact.inline='true'
                  |
      @@ -283,7 +283,7 @@ class TestCopyToTableProcedure extends HoodieSparkProcedureTestBase {
                  | location '${tmp.getCanonicalPath}/$tableName'
                  | tblproperties (
                  |  primaryKey = 'id',
      -           |  preCombineField = 'ts'
      +           |  orderingFields = 'ts'
                  | )
              """.stripMargin)
       
      @@ -333,7 +333,7 @@ class TestCopyToTableProcedure extends HoodieSparkProcedureTestBase {
                  | location '${tmp.getCanonicalPath}/$tableName'
                  | tblproperties (
                  |  primaryKey = 'id',
      -           |  preCombineField = 'ts'
      +           |  orderingFields = 'ts'
                  | )
              """.stripMargin)
       
      diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestCopyToTempViewProcedure.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestCopyToTempViewProcedure.scala
      index 86f6c80cb8b56..0466681d611de 100644
      --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestCopyToTempViewProcedure.scala
      +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestCopyToTempViewProcedure.scala
      @@ -39,7 +39,7 @@ class TestCopyToTempViewProcedure extends HoodieSparkSqlTestBase {
                  | location '${tmp.getCanonicalPath}/$tableName'
                  | tblproperties (
                  |  primaryKey = 'id',
      -           |  preCombineField = 'ts'
      +           |  orderingFields = 'ts'
                  | )
              """.stripMargin)
       
      @@ -76,7 +76,7 @@ class TestCopyToTempViewProcedure extends HoodieSparkSqlTestBase {
                  | location '${tmp.getCanonicalPath}/$tableName'
                  | tblproperties (
                  |  primaryKey = 'id',
      -           |  preCombineField = 'ts'
      +           |  orderingFields = 'ts'
                  | )
              """.stripMargin)
       
      diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestExportInstantsProcedure.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestExportInstantsProcedure.scala
      index b6a83e64fa064..372c6cc5e7c55 100644
      --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestExportInstantsProcedure.scala
      +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestExportInstantsProcedure.scala
      @@ -34,7 +34,7 @@ class TestExportInstantsProcedure extends HoodieSparkProcedureTestBase {
                  | location '${tmp.getCanonicalPath}/$tableName'
                  | tblproperties (
                  |  primaryKey = 'id',
      -           |  preCombineField = 'ts'
      +           |  orderingFields = 'ts'
                  | )
              """.stripMargin)
       
      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 69b07f2c9cd7a..8d61a64199aa2 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
      @@ -34,7 +34,7 @@ class TestFsViewProcedure extends HoodieSparkProcedureTestBase {
                  | location '${tmp.getCanonicalPath}/$tableName'
                  | tblproperties (
                  |  primaryKey = 'id',
      -           |  preCombineField = 'ts'
      +           |  orderingFields = 'ts'
                  | )
              """.stripMargin)
             // insert data to table
      @@ -76,7 +76,7 @@ class TestFsViewProcedure extends HoodieSparkProcedureTestBase {
                  | location '${tmp.getCanonicalPath}/$tableName'
                  | tblproperties (
                  |  primaryKey = 'id',
      -           |  preCombineField = 'ts'
      +           |  orderingFields = 'ts'
                  | )
              """.stripMargin)
             // insert data to table
      @@ -114,7 +114,7 @@ class TestFsViewProcedure extends HoodieSparkProcedureTestBase {
                  | location '${tmp.getCanonicalPath}/$tableName'
                  | tblproperties (
                  |  primaryKey = 'id',
      -           |  preCombineField = 'ts'
      +           |  orderingFields = 'ts'
                  | )
              """.stripMargin)
             // insert data to table
      @@ -163,7 +163,7 @@ class TestFsViewProcedure extends HoodieSparkProcedureTestBase {
                  | tblproperties (
                  |  type = 'mor',
                  |  primaryKey = 'id',
      -           |  preCombineField = 'ts'
      +           |  orderingFields = 'ts'
                  | )
              """.stripMargin)
             // insert data to table
      @@ -198,7 +198,7 @@ class TestFsViewProcedure extends HoodieSparkProcedureTestBase {
                  | location '${tmp.getCanonicalPath}/$tableName'
                  | tblproperties (
                  |  primaryKey = 'id',
      -           |  preCombineField = 'ts',
      +           |  orderingFields = 'ts',
                  |  'hoodie.parquet.small.file.limit' = '0'
                  | )
              """.stripMargin)
      @@ -230,7 +230,7 @@ class TestFsViewProcedure extends HoodieSparkProcedureTestBase {
                  | location '${tmp.getCanonicalPath}/$tableName'
                  | tblproperties (
                  |  primaryKey = 'id',
      -           |  preCombineField = 'ts',
      +           |  orderingFields = 'ts',
                  |  'hoodie.parquet.small.file.limit' = '0'
                  | )
              """.stripMargin)
      diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestHoodieLogFileProcedure.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestHoodieLogFileProcedure.scala
      index c9a8c8ec8471c..4f8b56bcce157 100644
      --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestHoodieLogFileProcedure.scala
      +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestHoodieLogFileProcedure.scala
      @@ -39,7 +39,7 @@ class TestHoodieLogFileProcedure extends HoodieSparkProcedureTestBase {
                  | tblproperties (
                  |  type = 'mor',
                  |  primaryKey = 'id',
      -           |  preCombineField = 'ts'
      +           |  orderingFields = 'ts'
                  | )
              """.stripMargin)
             // insert data to table
      @@ -79,7 +79,7 @@ class TestHoodieLogFileProcedure extends HoodieSparkProcedureTestBase {
                  | tblproperties (
                  |  type = 'mor',
                  |  primaryKey = 'id',
      -           |  preCombineField = 'ts'
      +           |  orderingFields = 'ts'
                  | )
              """.stripMargin)
             // insert data to table
      diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestMetadataProcedure.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestMetadataProcedure.scala
      index 0b440435e2cd4..e33c83d7a778d 100644
      --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestMetadataProcedure.scala
      +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestMetadataProcedure.scala
      @@ -34,7 +34,7 @@ class TestMetadataProcedure extends HoodieSparkProcedureTestBase {
                  | location '${tmp.getCanonicalPath}/$tableName'
                  | tblproperties (
                  |  primaryKey = 'id',
      -           |  preCombineField = 'ts'
      +           |  orderingFields = 'ts'
                  | )
              """.stripMargin)
             // insert data to table
      @@ -70,7 +70,7 @@ class TestMetadataProcedure extends HoodieSparkProcedureTestBase {
                  | location '${tmp.getCanonicalPath}/$tableName'
                  | tblproperties (
                  |  primaryKey = 'id',
      -           |  preCombineField = 'ts'
      +           |  orderingFields = 'ts'
                  | )
              """.stripMargin)
             // insert data to table
      @@ -90,7 +90,7 @@ class TestMetadataProcedure extends HoodieSparkProcedureTestBase {
                  | location '${tmp.getCanonicalPath}/$tableName_1'
                  | tblproperties (
                  |  primaryKey = 'id',
      -           |  preCombineField = 'ts'
      +           |  orderingFields = 'ts'
                  | )
              """.stripMargin)
             // insert data to table
      @@ -128,7 +128,7 @@ class TestMetadataProcedure extends HoodieSparkProcedureTestBase {
                  | location '${tmp.getCanonicalPath}/$tableName'
                  | tblproperties (
                  |  primaryKey = 'id',
      -           |  preCombineField = 'ts'
      +           |  orderingFields = 'ts'
                  | )
              """.stripMargin)
             // insert data to table
      @@ -170,7 +170,7 @@ class TestMetadataProcedure extends HoodieSparkProcedureTestBase {
                  | location '${tmp.getCanonicalPath}/$tableName'
                  | tblproperties (
                  |  primaryKey = 'c1',
      -           |  preCombineField = 'c8',
      +           |  orderingFields = 'c8',
                  |  hoodie.metadata.enable="true",
                  |  hoodie.metadata.index.column.stats.enable="true"
                  | )
      @@ -287,7 +287,7 @@ class TestMetadataProcedure extends HoodieSparkProcedureTestBase {
                  | location '${tmp.getCanonicalPath}/$tableName'
                  | tblproperties (
                  |  primaryKey = 'id',
      -           |  preCombineField = 'ts',
      +           |  orderingFields = 'ts',
                  |  hoodie.metadata.metrics.enable = 'true'
                  | )
              """.stripMargin)
      @@ -319,7 +319,7 @@ class TestMetadataProcedure extends HoodieSparkProcedureTestBase {
                  | partitioned by (ts)
                  | tblproperties (
                  |  primaryKey = 'id',
      -           |  preCombineField = 'ts'
      +           |  orderingFields = 'ts'
                  | )
              """.stripMargin)
             // insert data to table
      @@ -350,7 +350,7 @@ class TestMetadataProcedure extends HoodieSparkProcedureTestBase {
                  | partitioned by (ts)
                  | tblproperties (
                  |  primaryKey = 'id',
      -           |  preCombineField = 'ts'
      +           |  orderingFields = 'ts'
                  | )
              """.stripMargin)
             // insert data to table
      @@ -393,7 +393,7 @@ class TestMetadataProcedure extends HoodieSparkProcedureTestBase {
                  | partitioned by (ts)
                  | tblproperties (
                  |  primaryKey = 'id',
      -           |  preCombineField = 'ts'
      +           |  orderingFields = 'ts'
                  | )
              """.stripMargin)
             // insert data to table
      diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestRepairsProcedure.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestRepairsProcedure.scala
      index 0fc23f5af60b3..726bbd5070b8e 100644
      --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestRepairsProcedure.scala
      +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestRepairsProcedure.scala
      @@ -58,7 +58,7 @@ class TestRepairsProcedure extends HoodieSparkProcedureTestBase {
                  | location '$tablePath'
                  | tblproperties (
                  |  primaryKey = 'id',
      -           |  preCombineField = 'ts'
      +           |  orderingFields = 'ts'
                  | )
              """.stripMargin)
             // create commit instant
      @@ -110,7 +110,7 @@ class TestRepairsProcedure extends HoodieSparkProcedureTestBase {
                  | location '$tablePath'
                  | tblproperties (
                  |  primaryKey = 'id',
      -           |  preCombineField = 'ts'
      +           |  orderingFields = 'ts'
                  | )
              """.stripMargin)
       
      @@ -148,7 +148,7 @@ class TestRepairsProcedure extends HoodieSparkProcedureTestBase {
                 |[hoodie.table.initial.version,$tableVersion,$tableVersion]
                 |[hoodie.table.keygenerator.type,NON_PARTITION,null]
                 |[hoodie.table.name,,]
      -          |[hoodie.table.precombine.field,ts,null]
      +          |[hoodie.table.ordering.fields,ts,null]
                 |[hoodie.table.recordkey.fields,id,null]
                 |[hoodie.table.type,COPY_ON_WRITE,COPY_ON_WRITE]
                 |[hoodie.table.version,,]
      @@ -198,7 +198,7 @@ class TestRepairsProcedure extends HoodieSparkProcedureTestBase {
                  | location '$tablePath'
                  | tblproperties (
                  |  primaryKey = 'id',
      -           |  preCombineField = 'ts'
      +           |  orderingFields = 'ts'
                  | )
              """.stripMargin)
             var metaClient = createMetaClient(spark, tablePath)
      diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestRunRollbackInflightTableServiceProcedure.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestRunRollbackInflightTableServiceProcedure.scala
      index 3b5008f7d0539..cff62f0d511ae 100644
      --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestRunRollbackInflightTableServiceProcedure.scala
      +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestRunRollbackInflightTableServiceProcedure.scala
      @@ -40,7 +40,7 @@ class TestRunRollbackInflightTableServiceProcedure extends HoodieSparkProcedureT
                  | options (
                  |  primaryKey ='id',
                  |  type = 'cow',
      -           |  preCombineField = 'ts'
      +           |  orderingFields = 'ts'
                  | )
                  | partitioned by(ts)
                  | location '$basePath'
      @@ -84,7 +84,7 @@ class TestRunRollbackInflightTableServiceProcedure extends HoodieSparkProcedureT
                    | options (
                    |  primaryKey ='id',
                    |  type = 'mor',
      -             |  preCombineField = 'ts'
      +             |  orderingFields = 'ts'
                    | )
                    | partitioned by(ts)
                    | location '$basePath'
      diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestShowInvalidParquetProcedure.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestShowInvalidParquetProcedure.scala
      index d074ab56fccab..94c2a57503faa 100644
      --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestShowInvalidParquetProcedure.scala
      +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestShowInvalidParquetProcedure.scala
      @@ -45,7 +45,7 @@ class TestShowInvalidParquetProcedure extends HoodieSparkProcedureTestBase {
                  | location '$basePath'
                  | tblproperties (
                  |  primaryKey = 'id',
      -           |  preCombineField = 'ts'
      +           |  orderingFields = 'ts'
                  | )
              """.stripMargin)
             // insert data to table
      @@ -100,7 +100,7 @@ class TestShowInvalidParquetProcedure extends HoodieSparkProcedureTestBase {
                  | location '$basePath'
                  | tblproperties (
                  |  primaryKey = 'id',
      -           |  preCombineField = 'ts'
      +           |  orderingFields = 'ts'
                  | )
              """.stripMargin)
             // insert data to table
      @@ -148,7 +148,7 @@ class TestShowInvalidParquetProcedure extends HoodieSparkProcedureTestBase {
                  | location '$basePath'
                  | tblproperties (
                  |  primaryKey = 'id',
      -           |  preCombineField = 'ts'
      +           |  orderingFields = 'ts'
                  | )
              """.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..4fa8851d1db14 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
      @@ -37,7 +37,7 @@ class TestStatsProcedure extends HoodieSparkProcedureTestBase {
                  | location '$tablePath'
                  | tblproperties (
                  |  primaryKey = 'id',
      -           |  preCombineField = 'ts'
      +           |  orderingFields = 'ts'
                  | )
              """.stripMargin)
             // insert data to table
      @@ -75,7 +75,7 @@ class TestStatsProcedure extends HoodieSparkProcedureTestBase {
                  | location '$tablePath'
                  | tblproperties (
                  |  primaryKey = 'id',
      -           |  preCombineField = 'ts'
      +           |  orderingFields = 'ts'
                  | )
              """.stripMargin)
             // insert data to table
      @@ -131,7 +131,7 @@ class TestStatsProcedure extends HoodieSparkProcedureTestBase {
                  | location '$tablePath'
                  | tblproperties (
                  |  primaryKey = 'id',
      -           |  preCombineField = 'ts'
      +           |  orderingFields = 'ts'
                  | )
              """.stripMargin)
             // insert data to table
      diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestTTLProcedure.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestTTLProcedure.scala
      index 0c4fb7a29b884..43b3836fa0446 100644
      --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestTTLProcedure.scala
      +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestTTLProcedure.scala
      @@ -31,7 +31,7 @@ import org.apache.hudi.config.HoodieWriteConfig
       
       import org.apache.spark.api.java.JavaSparkContext
       
      -import java.util.Properties
      +import java.util.{Collections, Properties}
       
       import scala.collection.JavaConverters._
       
      @@ -66,7 +66,7 @@ class TestTTLProcedure extends HoodieSparkProcedureTestBase with SparkDatasetMix
                    | location '$basePath'
                    | tblproperties (
                    |   primaryKey = '_row_key',
      -             |   preCombineField = '_row_key',
      +             |   orderingFields = '_row_key',
                    |   type = 'cow'
                    | )
                    |""".stripMargin)
      @@ -114,7 +114,6 @@ class TestTTLProcedure extends HoodieSparkProcedureTestBase with SparkDatasetMix
             .newBuilder
             .withPath(basePath)
             .withSchema(TRIP_EXAMPLE_SCHEMA)
      -      .withPreCombineField("_row_key")
             .forTable(tableName)
      -
      +      .withProps(Collections.singletonMap(HoodieTableConfig.ORDERING_FIELDS.key(), "_row_key"))
       }
      diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestTruncateTableProcedure.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestTruncateTableProcedure.scala
      index 22de9a39a5ab7..d59bf9936b366 100644
      --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestTruncateTableProcedure.scala
      +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestTruncateTableProcedure.scala
      @@ -43,7 +43,7 @@ class TestTruncateTableProcedure extends HoodieSparkProcedureTestBase {
                  | location '$tablePath'
                  | tblproperties (
                  |  primaryKey = 'id',
      -           |  preCombineField = 'ts'
      +           |  orderingFields = 'ts'
                  | )
            """.stripMargin)
       
      diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/config/JsonKafkaPostProcessorConfig.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/config/JsonKafkaPostProcessorConfig.java
      index 28dd11cc1037c..393a60c186c12 100644
      --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/config/JsonKafkaPostProcessorConfig.java
      +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/config/JsonKafkaPostProcessorConfig.java
      @@ -28,7 +28,7 @@
       
       import static org.apache.hudi.common.util.ConfigUtils.DELTA_STREAMER_CONFIG_PREFIX;
       import static org.apache.hudi.common.util.ConfigUtils.STREAMER_CONFIG_PREFIX;
      -import static org.apache.hudi.utilities.sources.processor.maxwell.PreCombineFieldType.DATE_STRING;
      +import static org.apache.hudi.utilities.sources.processor.maxwell.OrderingFieldType.DATE_STRING;
       
       /**
        * Json Kafka Post Processor Configs
      @@ -62,18 +62,20 @@ public class JsonKafkaPostProcessorConfig extends HoodieConfig {
             .markAdvanced()
             .withDocumentation("Table name regex");
       
      -  public static final ConfigProperty PRECOMBINE_FIELD_TYPE = ConfigProperty
      -      .key(STREAMER_CONFIG_PREFIX + "source.json.kafka.post.processor.maxwell.precombine.field.type")
      +  public static final ConfigProperty ORDERING_FIELDS_TYPE = ConfigProperty
      +      .key(STREAMER_CONFIG_PREFIX + "source.json.kafka.post.processor.maxwell.ordering.fields.type")
             .defaultValue(DATE_STRING.toString())
      -      .withAlternatives(DELTA_STREAMER_CONFIG_PREFIX + "source.json.kafka.post.processor.maxwell.precombine.field.type")
      +      .withAlternatives(DELTA_STREAMER_CONFIG_PREFIX + "source.json.kafka.post.processor.maxwell.precombine.field.type",
      +          STREAMER_CONFIG_PREFIX + "source.json.kafka.post.processor.maxwell.precombine.field.type")
             .markAdvanced()
      -      .withDocumentation("Data type of the preCombine field. could be NON_TIMESTAMP, DATE_STRING,"
      +      .withDocumentation("Data type of the ordering field. could be NON_TIMESTAMP, DATE_STRING,"
                 + "UNIX_TIMESTAMP or EPOCHMILLISECONDS. DATE_STRING by default");
       
      -  public static final ConfigProperty PRECOMBINE_FIELD_FORMAT = ConfigProperty
      -      .key(STREAMER_CONFIG_PREFIX + "source.json.kafka.post.processor.maxwell.precombine.field.format")
      +  public static final ConfigProperty ORDERING_FIELDS_FORMAT = ConfigProperty
      +      .key(STREAMER_CONFIG_PREFIX + "source.json.kafka.post.processor.maxwell.ordering.fields.format")
             .defaultValue("yyyy-MM-dd HH:mm:ss")
      -      .withAlternatives(DELTA_STREAMER_CONFIG_PREFIX + "source.json.kafka.post.processor.maxwell.precombine.field.format")
      +      .withAlternatives(DELTA_STREAMER_CONFIG_PREFIX + "source.json.kafka.post.processor.maxwell.precombine.field.format",
      +          STREAMER_CONFIG_PREFIX + "source.json.kafka.post.processor.maxwell.precombine.field.format")
             .markAdvanced()
             .withDocumentation("When the preCombine filed is in DATE_STRING format, use should tell hoodie"
                 + "what format it is. 'yyyy-MM-dd HH:mm:ss' by default");
      diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/processor/maxwell/MaxwellJsonKafkaSourcePostProcessor.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/processor/maxwell/MaxwellJsonKafkaSourcePostProcessor.java
      index 50fd71c949f85..da5c049a986dd 100644
      --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/processor/maxwell/MaxwellJsonKafkaSourcePostProcessor.java
      +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/processor/maxwell/MaxwellJsonKafkaSourcePostProcessor.java
      @@ -20,9 +20,9 @@
       
       import org.apache.hudi.common.config.TypedProperties;
       import org.apache.hudi.common.model.HoodieRecord;
      +import org.apache.hudi.common.util.ConfigUtils;
       import org.apache.hudi.common.util.DateTimeUtils;
       import org.apache.hudi.common.util.Option;
      -import org.apache.hudi.config.HoodieWriteConfig;
       import org.apache.hudi.utilities.config.JsonKafkaPostProcessorConfig;
       import org.apache.hudi.utilities.exception.HoodieSourcePostProcessException;
       import org.apache.hudi.utilities.sources.processor.JsonKafkaSourcePostProcessor;
      @@ -37,11 +37,11 @@
       import java.util.regex.Pattern;
       
       import static org.apache.hudi.common.util.ConfigUtils.getStringWithAltKeys;
      -import static org.apache.hudi.utilities.sources.processor.maxwell.PreCombineFieldType.DATE_STRING;
      -import static org.apache.hudi.utilities.sources.processor.maxwell.PreCombineFieldType.EPOCHMILLISECONDS;
      -import static org.apache.hudi.utilities.sources.processor.maxwell.PreCombineFieldType.NON_TIMESTAMP;
      -import static org.apache.hudi.utilities.sources.processor.maxwell.PreCombineFieldType.UNIX_TIMESTAMP;
      -import static org.apache.hudi.utilities.sources.processor.maxwell.PreCombineFieldType.valueOf;
      +import static org.apache.hudi.utilities.sources.processor.maxwell.OrderingFieldType.DATE_STRING;
      +import static org.apache.hudi.utilities.sources.processor.maxwell.OrderingFieldType.EPOCHMILLISECONDS;
      +import static org.apache.hudi.utilities.sources.processor.maxwell.OrderingFieldType.NON_TIMESTAMP;
      +import static org.apache.hudi.utilities.sources.processor.maxwell.OrderingFieldType.UNIX_TIMESTAMP;
      +import static org.apache.hudi.utilities.sources.processor.maxwell.OrderingFieldType.valueOf;
       
       /**
        * A {@link JsonKafkaSourcePostProcessor} help to extract fresh data from maxwell json string and tag the record as
      @@ -114,34 +114,34 @@ private String processDelete(JsonNode inputJson, ObjectNode result) {
           // tag this record as delete.
           result.put(HoodieRecord.HOODIE_IS_DELETED_FIELD, true);
       
      -    PreCombineFieldType preCombineFieldType =
      +    OrderingFieldType orderingFieldsType =
               valueOf(getStringWithAltKeys(
      -            this.props, JsonKafkaPostProcessorConfig.PRECOMBINE_FIELD_TYPE, true).toUpperCase(Locale.ROOT));
      +            this.props, JsonKafkaPostProcessorConfig.ORDERING_FIELDS_TYPE, true).toUpperCase(Locale.ROOT));
       
           // maxwell won't update the `update_time`(delete time) field of the record which is tagged as delete. so if we
           // want to delete this record correctly, we should update its `update_time` to a time closer to where the
           // delete operation actually occurred. here we use `ts` from maxwell json string as this 'delete' time.
       
           // we can update the `update_time`(delete time) only when it is in timestamp format.
      -    if (!preCombineFieldType.equals(NON_TIMESTAMP)) {
      -      String preCombineField = this.props.getString(HoodieWriteConfig.PRECOMBINE_FIELD_NAME.key(), null);
      +    if (!orderingFieldsType.equals(NON_TIMESTAMP)) {
      +      String orderingFields = ConfigUtils.getOrderingFieldsStrDuringWrite(props);
       
             // ts from maxwell
             long ts = inputJson.get(TS).longValue();
       
             // convert the `update_time`(delete time) to the proper format.
      -      if (preCombineFieldType.equals(DATE_STRING)) {
      +      if (orderingFieldsType.equals(DATE_STRING)) {
               // DATE_STRING format
      -        String timeFormat = getStringWithAltKeys(this.props, JsonKafkaPostProcessorConfig.PRECOMBINE_FIELD_FORMAT, true);
      -        result.put(preCombineField, DateTimeUtils.formatUnixTimestamp(ts, timeFormat));
      -      } else if (preCombineFieldType.equals(EPOCHMILLISECONDS)) {
      +        String timeFormat = getStringWithAltKeys(this.props, JsonKafkaPostProcessorConfig.ORDERING_FIELDS_FORMAT, true);
      +        result.put(orderingFields, DateTimeUtils.formatUnixTimestamp(ts, timeFormat));
      +      } else if (orderingFieldsType.equals(EPOCHMILLISECONDS)) {
               // EPOCHMILLISECONDS format
      -        result.put(preCombineField, ts * 1000L);
      -      } else if (preCombineFieldType.equals(UNIX_TIMESTAMP)) {
      +        result.put(orderingFields, ts * 1000L);
      +      } else if (orderingFieldsType.equals(UNIX_TIMESTAMP)) {
               // UNIX_TIMESTAMP format
      -        result.put(preCombineField, ts);
      +        result.put(orderingFields, ts);
             } else {
      -        throw new HoodieSourcePostProcessException("Unsupported preCombine time format " + preCombineFieldType);
      +        throw new HoodieSourcePostProcessException("Unsupported preCombine time format " + orderingFieldsType);
             }
           }
           return result.toString();
      diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/processor/maxwell/PreCombineFieldType.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/processor/maxwell/OrderingFieldType.java
      similarity index 94%
      rename from hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/processor/maxwell/PreCombineFieldType.java
      rename to hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/processor/maxwell/OrderingFieldType.java
      index d3969a02bc4f3..ec76a4fa2233e 100644
      --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/processor/maxwell/PreCombineFieldType.java
      +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/processor/maxwell/OrderingFieldType.java
      @@ -19,9 +19,9 @@
       package org.apache.hudi.utilities.sources.processor.maxwell;
       
       /**
      - * Enum of preCombine field time type.
      + * Enum of ordering field time type.
        */
      -public enum PreCombineFieldType {
      +public enum OrderingFieldType {
         /**
          * Not a timestamp type field
          */
      diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/streamer/BootstrapExecutor.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/streamer/BootstrapExecutor.java
      index a5ea67823f5b0..cd0948ed4703d 100644
      --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/streamer/BootstrapExecutor.java
      +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/streamer/BootstrapExecutor.java
      @@ -59,7 +59,6 @@
       import static org.apache.hudi.common.table.HoodieTableConfig.POPULATE_META_FIELDS;
       import static org.apache.hudi.common.table.HoodieTableConfig.TIMELINE_HISTORY_PATH;
       import static org.apache.hudi.common.table.HoodieTableConfig.TIMELINE_TIMEZONE;
      -import static org.apache.hudi.config.HoodieWriteConfig.PRECOMBINE_FIELD_NAME;
       import static org.apache.hudi.config.HoodieWriteConfig.WRITE_TABLE_VERSION;
       import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_SYNC_BUCKET_SYNC;
       import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_SYNC_BUCKET_SYNC_SPEC;
      @@ -207,7 +206,7 @@ private void initializeTable() throws IOException {
               .setTableType(cfg.tableType)
               .setTableName(cfg.targetTableName)
               .setRecordKeyFields(props.getString(RECORDKEY_FIELD_NAME.key()))
      -        .setPreCombineFields(props.getString(PRECOMBINE_FIELD_NAME.key(), null))
      +        .setOrderingFields(ConfigUtils.getOrderingFieldsStrDuringWrite(props))
               .setTableVersion(ConfigUtils.getIntWithAltKeys(props, WRITE_TABLE_VERSION))
               .setTableFormat(props.getString(HoodieTableConfig.TABLE_FORMAT.key(), HoodieTableConfig.TABLE_FORMAT.defaultValue()))
               .setPopulateMetaFields(props.getBoolean(
      diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/streamer/HoodieStreamer.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/streamer/HoodieStreamer.java
      index 191308ad4ddcf..829eea731c550 100644
      --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/streamer/HoodieStreamer.java
      +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/streamer/HoodieStreamer.java
      @@ -272,8 +272,8 @@ public static class Config implements Serializable {
                   + "JsonKafkaSource, AvroKafkaSource, HiveIncrPullSource}")
           public String sourceClassName = JsonDFSSource.class.getName();
       
      -    @Parameter(names = {"--source-ordering-field"}, description = "Comma separated list of fields within source record to decide how"
      -        + " to break ties between records with same key in input data.")
      +    @Parameter(names = {"--source-ordering-fields", "--source-ordering-field"}, description = "Comma separated list of fields within source record to decide how"
      +        + " to break ties between records with same key in input data. --source-ordering-field is deprecated, please use --source-ordering-fields instead")
           public String sourceOrderingFields = null;
       
           @Parameter(names = {"--payload-class"}, description = "Deprecated. "
      diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/streamer/StreamSync.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/streamer/StreamSync.java
      index 88552c003a1f0..bee324723590b 100644
      --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/streamer/StreamSync.java
      +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/streamer/StreamSync.java
      @@ -442,7 +442,7 @@ HoodieTableMetaClient initializeEmptyTable(HoodieTableMetaClient.TableBuilder ta
               .setPopulateMetaFields(props.getBoolean(HoodieTableConfig.POPULATE_META_FIELDS.key(),
                   HoodieTableConfig.POPULATE_META_FIELDS.defaultValue()))
               .setKeyGeneratorClassProp(keyGenClassName)
      -        .setPreCombineFields(cfg.sourceOrderingFields)
      +        .setOrderingFields(cfg.sourceOrderingFields)
               .setPartitionMetafileUseBaseFormat(props.getBoolean(HoodieTableConfig.PARTITION_METAFILE_USE_BASE_FORMAT.key(),
                   HoodieTableConfig.PARTITION_METAFILE_USE_BASE_FORMAT.defaultValue()))
               .setCDCEnabled(props.getBoolean(HoodieTableConfig.CDC_ENABLED.key(),
      diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/TestHoodieMetadataTableValidator.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/TestHoodieMetadataTableValidator.java
      index 4d88ef28bd070..5ea5a575dfddc 100644
      --- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/TestHoodieMetadataTableValidator.java
      +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/TestHoodieMetadataTableValidator.java
      @@ -34,6 +34,7 @@
       import org.apache.hudi.common.model.HoodieLogFile;
       import org.apache.hudi.common.model.HoodieRecord;
       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.log.HoodieLogFormat;
       import org.apache.hudi.common.table.log.block.HoodieAvroDataBlock;
      @@ -191,7 +192,7 @@ public void testMetadataTableValidation(String viewStorageTypeForFSListing, Stri
           writeOptions.put(DataSourceWriteOptions.TABLE_NAME().key(), "test_table");
           writeOptions.put(DataSourceWriteOptions.TABLE_TYPE().key(), "MERGE_ON_READ");
           writeOptions.put(DataSourceWriteOptions.RECORDKEY_FIELD().key(), "_row_key");
      -    writeOptions.put(DataSourceWriteOptions.PRECOMBINE_FIELD().key(), "timestamp");
      +    writeOptions.put(HoodieTableConfig.ORDERING_FIELDS.key(), "timestamp");
           writeOptions.put(DataSourceWriteOptions.PARTITIONPATH_FIELD().key(), "partition_path");
       
           Dataset inserts = makeInsertDf("000", 5).cache();
      @@ -250,7 +251,7 @@ void missingLogFileFailsValidation() throws Exception {
           writeOptions.put("hoodie.table.name", "test_table");
           writeOptions.put(DataSourceWriteOptions.TABLE_TYPE().key(), "MERGE_ON_READ");
           writeOptions.put(DataSourceWriteOptions.RECORDKEY_FIELD().key(), "_row_key");
      -    writeOptions.put(DataSourceWriteOptions.PRECOMBINE_FIELD().key(), "timestamp");
      +    writeOptions.put(HoodieTableConfig.ORDERING_FIELDS.key(), "timestamp");
           writeOptions.put(DataSourceWriteOptions.PARTITIONPATH_FIELD().key(), "partition_path");
       
           Dataset inserts = makeInsertDf("000", 5).cache();
      @@ -314,7 +315,7 @@ public void testSecondaryIndexValidation() throws Exception {
                     + "hoodie.metadata.record.index.enable = 'true', "
                     + "hoodie.datasource.write.recordkey.field = 'record_key_col', "
                     + "hoodie.enable.data.skipping = 'true', "
      -              + "hoodie.datasource.write.precombine.field = 'ts', "
      +              + "hoodie.table.ordering.fields = 'ts', "
                     + "hoodie.datasource.write.payload.class = 'org.apache.hudi.common.model.OverwriteWithLatestAvroPayload'"
                     + ") "
                     + "partitioned by(partition_key_col) "
      @@ -365,7 +366,7 @@ public void testGetFSSecondaryKeyToRecordKeys() throws Exception {
                     + "hoodie.metadata.record.index.enable = 'true', "
                     + "hoodie.datasource.write.recordkey.field = 'record_key_col', "
                     + "hoodie.enable.data.skipping = 'true', "
      -              + "hoodie.datasource.write.precombine.field = 'ts'"
      +              + "hoodie.table.ordering.fields = 'ts'"
                     + ") "
                     + "partitioned by(partition_key_col) "
                     + "location '" + basePath + "'");
      @@ -417,7 +418,7 @@ public void testColumnStatsValidation(String tableType) {
           writeOptions.put("hoodie.table.name", "test_table");
           writeOptions.put(DataSourceWriteOptions.TABLE_TYPE().key(), tableType);
           writeOptions.put(DataSourceWriteOptions.RECORDKEY_FIELD().key(), "_row_key");
      -    writeOptions.put(DataSourceWriteOptions.PRECOMBINE_FIELD().key(), "timestamp");
      +    writeOptions.put(HoodieTableConfig.ORDERING_FIELDS.key(), "timestamp");
           writeOptions.put(DataSourceWriteOptions.PARTITIONPATH_FIELD().key(), "partition_path");
           writeOptions.put(HoodieMetadataConfig.ENABLE_METADATA_INDEX_COLUMN_STATS.key(), "true");
       
      @@ -449,7 +450,7 @@ public void testPartitionStatsValidation(String tableType) throws Exception {
             writeOptions.put("hoodie.table.name", "test_table");
             writeOptions.put(DataSourceWriteOptions.TABLE_TYPE().key(), tableType);
             writeOptions.put(DataSourceWriteOptions.RECORDKEY_FIELD().key(), "_row_key");
      -      writeOptions.put(DataSourceWriteOptions.PRECOMBINE_FIELD().key(), "timestamp");
      +      writeOptions.put(HoodieTableConfig.ORDERING_FIELDS.key(), "timestamp");
             writeOptions.put(DataSourceWriteOptions.PARTITIONPATH_FIELD().key(), "partition_path");
             writeOptions.put(HoodieMetadataConfig.ENABLE_METADATA_INDEX_COLUMN_STATS.key(), "true");
       
      @@ -517,7 +518,7 @@ public void testAdditionalPartitionsinMDT(boolean testFailureCase) throws IOExce
           writeOptions.put(DataSourceWriteOptions.TABLE_NAME().key(), "test_table");
           writeOptions.put(DataSourceWriteOptions.TABLE_TYPE().key(), "MERGE_ON_READ");
           writeOptions.put(DataSourceWriteOptions.RECORDKEY_FIELD().key(), "_row_key");
      -    writeOptions.put(DataSourceWriteOptions.PRECOMBINE_FIELD().key(), "timestamp");
      +    writeOptions.put(HoodieTableConfig.ORDERING_FIELDS.key(), "timestamp");
           writeOptions.put(DataSourceWriteOptions.PARTITIONPATH_FIELD().key(), "partition_path");
       
           // constructor of HoodieMetadataValidator instantiates HoodieTableMetaClient. hence creating an actual table. but rest of tests is mocked.
      @@ -598,7 +599,7 @@ public void testAdditionalFilesInMetadata(Integer lastNFileSlices, boolean ignor
             writeOptions.put(DataSourceWriteOptions.TABLE_NAME().key(), "test_table");
             writeOptions.put(DataSourceWriteOptions.TABLE_TYPE().key(), "MERGE_ON_READ");
             writeOptions.put(DataSourceWriteOptions.RECORDKEY_FIELD().key(), "_row_key");
      -      writeOptions.put(DataSourceWriteOptions.PRECOMBINE_FIELD().key(), "timestamp");
      +      writeOptions.put(HoodieTableConfig.ORDERING_FIELDS.key(), "timestamp");
             writeOptions.put(HoodieCompactionConfig.INLINE_COMPACT_NUM_DELTA_COMMITS.key(), "2");
       
             Dataset inserts = makeInsertDf("000", 10).cache();
      @@ -721,7 +722,7 @@ public void testAdditionalPartitionsinMdtEndToEnd(boolean ignoreFailed) throws E
             writeOptions.put("hoodie.table.name", "test_table");
             writeOptions.put(DataSourceWriteOptions.TABLE_TYPE().key(), "MERGE_ON_READ");
             writeOptions.put(DataSourceWriteOptions.RECORDKEY_FIELD().key(), "_row_key");
      -      writeOptions.put(DataSourceWriteOptions.PRECOMBINE_FIELD().key(), "timestamp");
      +      writeOptions.put(HoodieTableConfig.ORDERING_FIELDS.key(), "timestamp");
             writeOptions.put(DataSourceWriteOptions.PARTITIONPATH_FIELD().key(),"partition_path");
             writeOptions.put(HoodieCompactionConfig.INLINE_COMPACT_NUM_DELTA_COMMITS.key(), "2");
       
      @@ -1178,7 +1179,7 @@ public void testRecordIndexMismatch(boolean ignoreFailed) throws IOException {
           writeOptions.put("hoodie.table.name", "test_table");
           writeOptions.put(DataSourceWriteOptions.TABLE_TYPE().key(), "COPY_ON_WRITE");
           writeOptions.put(DataSourceWriteOptions.RECORDKEY_FIELD().key(), "_row_key");
      -    writeOptions.put(DataSourceWriteOptions.PRECOMBINE_FIELD().key(), "timestamp");
      +    writeOptions.put(HoodieTableConfig.ORDERING_FIELDS.key(), "timestamp");
           writeOptions.put(DataSourceWriteOptions.OPERATION().key(),"bulk_insert");
           writeOptions.put(HoodieMetadataConfig.RECORD_INDEX_ENABLE_PROP.key(), "true");
       
      @@ -1275,7 +1276,7 @@ public void testRliValidationFalsePositiveCase() throws Exception {
             writeOptions.put(DataSourceWriteOptions.TABLE_NAME().key(), "test_table");
             writeOptions.put(DataSourceWriteOptions.TABLE_TYPE().key(), "MERGE_ON_READ");
             writeOptions.put(DataSourceWriteOptions.RECORDKEY_FIELD().key(), "_row_key");
      -      writeOptions.put(DataSourceWriteOptions.PRECOMBINE_FIELD().key(), "timestamp");
      +      writeOptions.put(HoodieTableConfig.ORDERING_FIELDS.key(), "timestamp");
             writeOptions.put(DataSourceWriteOptions.PARTITIONPATH_FIELD().key(), "partition_path");
       
             Dataset inserts = makeInsertDf("000", 5).cache();
      @@ -1401,7 +1402,7 @@ void testLogDetailMaxLength() {
           writeOptions.put("hoodie.table.name", "test_table");
           writeOptions.put(DataSourceWriteOptions.TABLE_TYPE().key(), "MERGE_ON_READ");
           writeOptions.put(DataSourceWriteOptions.RECORDKEY_FIELD().key(), "_row_key");
      -    writeOptions.put(DataSourceWriteOptions.PRECOMBINE_FIELD().key(), "timestamp");
      +    writeOptions.put(HoodieTableConfig.ORDERING_FIELDS.key(), "timestamp");
           writeOptions.put(DataSourceWriteOptions.PARTITIONPATH_FIELD().key(), "partition_path");
       
           // Create a large dataset to generate long validation messages
      diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/TestJsonKafkaSourcePostProcessor.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/TestJsonKafkaSourcePostProcessor.java
      index 1f1a4e2b5c1f8..afc441c58d638 100644
      --- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/TestJsonKafkaSourcePostProcessor.java
      +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/TestJsonKafkaSourcePostProcessor.java
      @@ -240,8 +240,8 @@ public void testMaxwellJsonKafkaSourcePostProcessor() throws IOException {
           });
       
           // test delete
      -    props.setProperty(JsonKafkaPostProcessorConfig.PRECOMBINE_FIELD_TYPE.key(), "DATE_STRING");
      -    props.setProperty(JsonKafkaPostProcessorConfig.PRECOMBINE_FIELD_FORMAT.key(), "yyyy-MM-dd HH:mm:ss");
      +    props.setProperty(JsonKafkaPostProcessorConfig.ORDERING_FIELDS_TYPE.key(), "DATE_STRING");
      +    props.setProperty(JsonKafkaPostProcessorConfig.ORDERING_FIELDS_FORMAT.key(), "yyyy-MM-dd HH:mm:ss");
           props.setProperty(HoodieWriteConfig.PRECOMBINE_FIELD_NAME.key(), "update_time");
       
           JavaRDD inputDelete = jsc().parallelize(Collections.singletonList(hudiMaxwell01Delete));
      @@ -262,7 +262,7 @@ public void testMaxwellJsonKafkaSourcePostProcessor() throws IOException {
               });
       
           // test preCombine field is not time
      -    props.setProperty(JsonKafkaPostProcessorConfig.PRECOMBINE_FIELD_TYPE.key(), "NON_TIMESTAMP");
      +    props.setProperty(JsonKafkaPostProcessorConfig.ORDERING_FIELDS_TYPE.key(), "NON_TIMESTAMP");
           props.setProperty(HoodieWriteConfig.PRECOMBINE_FIELD_NAME.key(), "id");
       
           JavaRDD inputDelete2 = jsc().parallelize(Collections.singletonList(hudiMaxwell01Delete));
      diff --git a/packaging/bundle-validation/utilities/hoodieapp.properties b/packaging/bundle-validation/utilities/hoodieapp.properties
      index 6d2382fc89045..835b09dcee41f 100644
      --- a/packaging/bundle-validation/utilities/hoodieapp.properties
      +++ b/packaging/bundle-validation/utilities/hoodieapp.properties
      @@ -16,7 +16,7 @@
       
       hoodie.datasource.write.recordkey.field=key
       hoodie.datasource.write.partitionpath.field=date
      -hoodie.datasource.write.precombine.field=ts
      +hoodie.table.ordering.fields=ts
       hoodie.metadata.enable=true
       hoodie.deltastreamer.source.dfs.root=file:///opt/bundle-validation/data/stocks/data
       hoodie.deltastreamer.schemaprovider.target.schema.file=file:///opt/bundle-validation/data/stocks/schema.avsc