diff --git a/conf/hudi-defaults.conf.template b/conf/hudi-defaults.conf.template index 175dbaf23d739..fbcedb3f18b9c 100644 --- a/conf/hudi-defaults.conf.template +++ b/conf/hudi-defaults.conf.template @@ -20,7 +20,7 @@ # Example: # hoodie.datasource.hive_sync.jdbcurl jdbc:hive2://localhost:10000 -# hoodie.datasource.hive_sync.use_jdbc true +# hoodie.datasource.hive_sync.mode jdbc # hoodie.datasource.hive_sync.support_timestamp false # hoodie.index.type BLOOM # hoodie.metadata.enable false diff --git a/docker/demo/config/hoodie-incr.properties b/docker/demo/config/hoodie-incr.properties index 80f474b1e7716..47bfc95b53cbe 100644 --- a/docker/demo/config/hoodie-incr.properties +++ b/docker/demo/config/hoodie-incr.properties @@ -28,5 +28,6 @@ hoodie.deltastreamer.source.hoodieincr.path=/docker_hoodie_sync_valid_test hoodie.deltastreamer.source.hoodieincr.read_latest_on_missing_ckpt=true # hive sync hoodie.datasource.hive_sync.table=docker_hoodie_sync_valid_test_2 +hoodie.datasource.hive_sync.mode=jdbc hoodie.datasource.hive_sync.jdbcurl=jdbc:hive2://hiveserver:10000 -hoodie.datasource.hive_sync.partition_fields=partition \ No newline at end of file +hoodie.datasource.hive_sync.partition_fields=partition diff --git a/docker/demo/sparksql-incremental.commands b/docker/demo/sparksql-incremental.commands index 9ec586e49d854..3d7da63703c2c 100644 --- a/docker/demo/sparksql-incremental.commands +++ b/docker/demo/sparksql-incremental.commands @@ -47,6 +47,7 @@ spark.sql("select key, `_hoodie_partition_path` as datestr, symbol, ts, open, cl option(HoodieWriteConfig.TBL_NAME.key(), "stock_ticks_derived_mor"). option(HoodieSyncConfig.META_SYNC_TABLE_NAME.key(), "stock_ticks_derived_mor"). option(HoodieSyncConfig.META_SYNC_DATABASE_NAME.key(), "default"). + option(HiveSyncConfigHolder.HIVE_SYNC_MODE.key(), "jdbc"). option(HiveSyncConfigHolder.HIVE_URL.key(), "jdbc:hive2://hiveserver:10000"). option(HiveSyncConfigHolder.HIVE_USER.key(), "hive"). option(HiveSyncConfigHolder.HIVE_PASS.key(), "hive"). @@ -79,6 +80,7 @@ spark.sql("select key, `_hoodie_partition_path` as datestr, symbol, ts, open, cl option(HoodieWriteConfig.TBL_NAME.key(), "stock_ticks_derived_mor_bs"). option(HoodieSyncConfig.META_SYNC_TABLE_NAME.key(), "stock_ticks_derived_mor_bs"). option(HoodieSyncConfig.META_SYNC_DATABASE_NAME.key(), "default"). + option(HiveSyncConfigHolder.HIVE_SYNC_MODE.key(), "jdbc"). option(HiveSyncConfigHolder.HIVE_URL.key(), "jdbc:hive2://hiveserver:10000"). option(HiveSyncConfigHolder.HIVE_USER.key(), "hive"). option(HiveSyncConfigHolder.HIVE_PASS.key(), "hive"). diff --git a/hudi-aws/src/main/java/org/apache/hudi/aws/sync/AWSGlueCatalogSyncClient.java b/hudi-aws/src/main/java/org/apache/hudi/aws/sync/AWSGlueCatalogSyncClient.java index f322ef79cafe0..79651154ed63b 100644 --- a/hudi-aws/src/main/java/org/apache/hudi/aws/sync/AWSGlueCatalogSyncClient.java +++ b/hudi-aws/src/main/java/org/apache/hudi/aws/sync/AWSGlueCatalogSyncClient.java @@ -64,9 +64,9 @@ import java.util.stream.Collectors; import static org.apache.hudi.aws.utils.S3Utils.s3aToS3; -import static org.apache.hudi.common.util.MapUtils.nonEmpty; import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_CREATE_MANAGED_TABLE; import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_SUPPORT_TIMESTAMP_TYPE; +import static org.apache.hudi.common.util.MapUtils.isNullOrEmpty; import static org.apache.hudi.hive.util.HiveSchemaUtil.getPartitionKeyType; import static org.apache.hudi.hive.util.HiveSchemaUtil.parquetSchemaToMapSchema; import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_DATABASE_NAME; @@ -193,11 +193,11 @@ public void dropPartitions(String tableName, List partitionsToDrop) { */ @Override public void updateTableProperties(String tableName, Map tableProperties) { - if (nonEmpty(tableProperties)) { + if (isNullOrEmpty(tableProperties)) { return; } try { - updateTableParameters(awsGlue, databaseName, tableName, tableProperties, true); + updateTableParameters(awsGlue, databaseName, tableName, tableProperties, false); } catch (Exception e) { throw new HoodieGlueSyncException("Fail to update properties for table " + tableId(databaseName, tableName), e); } @@ -210,10 +210,7 @@ public void updateTableSchema(String tableName, MessageType newSchema) { try { Table table = getTable(awsGlue, databaseName, tableName); Map newSchemaMap = parquetSchemaToMapSchema(newSchema, config.getBoolean(HIVE_SUPPORT_TIMESTAMP_TYPE), false); - List newColumns = newSchemaMap.keySet().stream().map(key -> { - String keyType = getPartitionKeyType(newSchemaMap, key); - return new Column().withName(key).withType(keyType.toLowerCase()).withComment(""); - }).collect(Collectors.toList()); + List newColumns = getColumnsFromSchema(newSchemaMap); StorageDescriptor sd = table.getStorageDescriptor(); sd.setColumns(newColumns); @@ -258,15 +255,7 @@ public void createTable(String tableName, try { Map mapSchema = parquetSchemaToMapSchema(storageSchema, config.getBoolean(HIVE_SUPPORT_TIMESTAMP_TYPE), false); - List schemaWithoutPartitionKeys = new ArrayList<>(); - for (String key : mapSchema.keySet()) { - String keyType = getPartitionKeyType(mapSchema, key); - Column column = new Column().withName(key).withType(keyType.toLowerCase()).withComment(""); - // In Glue, the full schema should exclude the partition keys - if (!config.getSplitStrings(META_SYNC_PARTITION_FIELDS).contains(key)) { - schemaWithoutPartitionKeys.add(column); - } - } + List schemaWithoutPartitionKeys = getColumnsFromSchema(mapSchema); // now create the schema partition List schemaPartitionKeys = config.getSplitStrings(META_SYNC_PARTITION_FIELDS).stream().map(partitionKey -> { @@ -419,6 +408,19 @@ public void deleteLastReplicatedTimeStamp(String tableName) { throw new UnsupportedOperationException("Not supported: `deleteLastReplicatedTimeStamp`"); } + private List getColumnsFromSchema(Map mapSchema) { + List cols = new ArrayList<>(); + for (String key : mapSchema.keySet()) { + // In Glue, the full schema should exclude the partition keys + if (!config.getSplitStrings(META_SYNC_PARTITION_FIELDS).contains(key)) { + String keyType = getPartitionKeyType(mapSchema, key); + Column column = new Column().withName(key).withType(keyType.toLowerCase()).withComment(""); + cols.add(column); + } + } + return cols; + } + private enum TableType { MANAGED_TABLE, EXTERNAL_TABLE, diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/SparkMain.java b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/SparkMain.java index 41357453510da..e94c38bd16af8 100644 --- a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/SparkMain.java +++ b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/SparkMain.java @@ -35,7 +35,7 @@ import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.common.util.ValidationUtils; import org.apache.hudi.config.HoodieBootstrapConfig; -import org.apache.hudi.config.HoodieCompactionConfig; +import org.apache.hudi.config.HoodieCleanConfig; import org.apache.hudi.config.HoodieIndexConfig; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieSavepointException; @@ -538,7 +538,7 @@ private static SparkRDDWriteClient createHoodieClient(JavaSparkContext jsc, Stri private static HoodieWriteConfig getWriteConfig(String basePath, Boolean rollbackUsingMarkers, boolean lazyCleanPolicy) { return HoodieWriteConfig.newBuilder().withPath(basePath) .withRollbackUsingMarkers(rollbackUsingMarkers) - .withCompactionConfig(HoodieCompactionConfig.newBuilder().withFailedWritesCleaningPolicy(lazyCleanPolicy ? HoodieFailedWritesCleaningPolicy.LAZY : + .withCleanConfig(HoodieCleanConfig.newBuilder().withFailedWritesCleaningPolicy(lazyCleanPolicy ? HoodieFailedWritesCleaningPolicy.LAZY : HoodieFailedWritesCleaningPolicy.EAGER).build()) .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build()).build(); } diff --git a/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestArchivedCommitsCommand.java b/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestArchivedCommitsCommand.java index d822ad658920b..b936202bd0d80 100644 --- a/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestArchivedCommitsCommand.java +++ b/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestArchivedCommitsCommand.java @@ -30,7 +30,8 @@ import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.table.view.FileSystemViewStorageConfig; import org.apache.hudi.common.testutils.HoodieTestUtils; -import org.apache.hudi.config.HoodieCompactionConfig; +import org.apache.hudi.config.HoodieArchivalConfig; +import org.apache.hudi.config.HoodieCleanConfig; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.table.HoodieSparkTable; import org.apache.hudi.client.HoodieTimelineArchiver; @@ -72,7 +73,8 @@ public void init() throws Exception { // Generate archive HoodieWriteConfig cfg = HoodieWriteConfig.newBuilder().withPath(tablePath) .withSchema(HoodieTestCommitMetadataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2) - .withCompactionConfig(HoodieCompactionConfig.newBuilder().retainCommits(1).archiveCommitsWith(2, 3).build()) + .withArchivalConfig(HoodieArchivalConfig.newBuilder().archiveCommitsWith(2, 3).build()) + .withCleanConfig(HoodieCleanConfig.newBuilder().retainCommits(1).build()) .withFileSystemViewConfig(FileSystemViewStorageConfig.newBuilder() .withRemoteServerPort(timelineServicePort).build()) .forTable("test-trip-table").build(); diff --git a/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestCommitsCommand.java b/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestCommitsCommand.java index b23c6fd150dc0..e03699f66e2cb 100644 --- a/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestCommitsCommand.java +++ b/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestCommitsCommand.java @@ -37,7 +37,8 @@ import org.apache.hudi.common.testutils.HoodieTestUtils; import org.apache.hudi.common.util.NumericUtils; import org.apache.hudi.common.util.Option; -import org.apache.hudi.config.HoodieCompactionConfig; +import org.apache.hudi.config.HoodieArchivalConfig; +import org.apache.hudi.config.HoodieCleanConfig; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.table.HoodieSparkTable; import org.apache.hudi.client.HoodieTimelineArchiver; @@ -212,7 +213,8 @@ public void testShowArchivedCommits(boolean enableMetadataTable) throws Exceptio // Generate archive HoodieWriteConfig cfg = HoodieWriteConfig.newBuilder().withPath(tablePath1) .withSchema(HoodieTestCommitMetadataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2) - .withCompactionConfig(HoodieCompactionConfig.newBuilder().retainCommits(1).archiveCommitsWith(2, 3).build()) + .withCleanConfig(HoodieCleanConfig.newBuilder().retainCommits(1).build()) + .withArchivalConfig(HoodieArchivalConfig.newBuilder().archiveCommitsWith(2, 3).build()) .withFileSystemViewConfig(FileSystemViewStorageConfig.newBuilder() .withRemoteServerPort(timelineServicePort).build()) .withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(enableMetadataTable).build()) @@ -266,7 +268,8 @@ public void testShowArchivedCommitsWithMultiCommitsFile(boolean enableMetadataTa // Generate archive HoodieWriteConfig cfg = HoodieWriteConfig.newBuilder().withPath(tablePath1) .withSchema(HoodieTestCommitMetadataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2) - .withCompactionConfig(HoodieCompactionConfig.newBuilder().retainCommits(1).archiveCommitsWith(2, 3).build()) + .withCleanConfig(HoodieCleanConfig.newBuilder().retainCommits(1).build()) + .withArchivalConfig(HoodieArchivalConfig.newBuilder().archiveCommitsWith(2, 3).build()) .withFileSystemViewConfig(FileSystemViewStorageConfig.newBuilder() .withRemoteServerPort(timelineServicePort).build()) .withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(enableMetadataTable).build()) diff --git a/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestCompactionCommand.java b/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestCompactionCommand.java index 17c1002f6b0dd..bc5ba168e3ed8 100644 --- a/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestCompactionCommand.java +++ b/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestCompactionCommand.java @@ -39,7 +39,8 @@ import org.apache.hudi.common.testutils.CompactionTestUtils; import org.apache.hudi.common.testutils.HoodieTestUtils; import org.apache.hudi.common.util.Option; -import org.apache.hudi.config.HoodieCompactionConfig; +import org.apache.hudi.config.HoodieArchivalConfig; +import org.apache.hudi.config.HoodieCleanConfig; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.table.HoodieSparkTable; @@ -166,7 +167,8 @@ private void generateArchive() throws IOException { // Generate archive HoodieWriteConfig cfg = HoodieWriteConfig.newBuilder().withPath(tablePath) .withSchema(HoodieTestCommitMetadataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2) - .withCompactionConfig(HoodieCompactionConfig.newBuilder().retainCommits(1).archiveCommitsWith(2, 3).build()) + .withCleanConfig(HoodieCleanConfig.newBuilder().retainCommits(1).build()) + .withArchivalConfig(HoodieArchivalConfig.newBuilder().archiveCommitsWith(2, 3).build()) .withFileSystemViewConfig(FileSystemViewStorageConfig.newBuilder() .withRemoteServerPort(timelineServicePort).build()) .forTable("test-trip-table").build(); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieWriteClient.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieWriteClient.java index 961965353b7ff..24400c5eda90a 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieWriteClient.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieWriteClient.java @@ -39,6 +39,7 @@ import org.apache.hudi.client.transaction.TransactionManager; import org.apache.hudi.client.utils.TransactionUtils; import org.apache.hudi.common.HoodiePendingRollbackInfo; +import org.apache.hudi.common.config.HoodieCommonConfig; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy; @@ -276,15 +277,21 @@ private void saveInternalSchema(HoodieTable table, String instantTime, HoodieCom TableSchemaResolver schemaUtil = new TableSchemaResolver(table.getMetaClient()); String historySchemaStr = schemaUtil.getTableHistorySchemaStrFromCommitMetadata().orElse(""); FileBasedInternalSchemaStorageManager schemasManager = new FileBasedInternalSchemaStorageManager(table.getMetaClient()); - if (!historySchemaStr.isEmpty()) { - InternalSchema internalSchema = InternalSchemaUtils.searchSchema(Long.parseLong(instantTime), - SerDeHelper.parseSchemas(historySchemaStr)); + if (!historySchemaStr.isEmpty() || Boolean.parseBoolean(config.getString(HoodieCommonConfig.RECONCILE_SCHEMA.key()))) { + InternalSchema internalSchema; Schema avroSchema = HoodieAvroUtils.createHoodieWriteSchema(new Schema.Parser().parse(config.getSchema())); - InternalSchema evolvedSchema = AvroSchemaEvolutionUtils.evolveSchemaFromNewAvroSchema(avroSchema, internalSchema); + if (historySchemaStr.isEmpty()) { + internalSchema = AvroInternalSchemaConverter.convert(avroSchema); + internalSchema.setSchemaId(Long.parseLong(instantTime)); + } else { + internalSchema = InternalSchemaUtils.searchSchema(Long.parseLong(instantTime), + SerDeHelper.parseSchemas(historySchemaStr)); + } + InternalSchema evolvedSchema = AvroSchemaEvolutionUtils.reconcileSchema(avroSchema, internalSchema); if (evolvedSchema.equals(internalSchema)) { metadata.addMetadata(SerDeHelper.LATEST_SCHEMA, SerDeHelper.toJson(evolvedSchema)); //TODO save history schema by metaTable - schemasManager.persistHistorySchemaStr(instantTime, historySchemaStr); + schemasManager.persistHistorySchemaStr(instantTime, historySchemaStr.isEmpty() ? SerDeHelper.inheritSchemas(evolvedSchema, "") : historySchemaStr); } else { evolvedSchema.setSchemaId(Long.parseLong(instantTime)); String newSchemaStr = SerDeHelper.toJson(evolvedSchema); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieArchivalConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieArchivalConfig.java new file mode 100644 index 0000000000000..32bccc3a3d18f --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieArchivalConfig.java @@ -0,0 +1,194 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.config; + +import org.apache.hudi.common.config.ConfigClassProperty; +import org.apache.hudi.common.config.ConfigGroups; +import org.apache.hudi.common.config.ConfigProperty; +import org.apache.hudi.common.config.HoodieConfig; + +import javax.annotation.concurrent.Immutable; +import java.io.File; +import java.io.FileReader; +import java.io.IOException; +import java.util.Properties; + +/** + * Archival related config. + */ +@Immutable +@ConfigClassProperty(name = "Archival Configs", + groupName = ConfigGroups.Names.WRITE_CLIENT, + description = "Configurations that control archival.") +public class HoodieArchivalConfig extends HoodieConfig { + + public static final ConfigProperty AUTO_ARCHIVE = ConfigProperty + .key("hoodie.archive.automatic") + .defaultValue("true") + .withDocumentation("When enabled, the archival table service is invoked immediately after each commit," + + " to archive commits if we cross a maximum value of commits." + + " It's recommended to enable this, to ensure number of active commits is bounded."); + + public static final ConfigProperty ASYNC_ARCHIVE = ConfigProperty + .key("hoodie.archive.async") + .defaultValue("false") + .sinceVersion("0.11.0") + .withDocumentation("Only applies when " + AUTO_ARCHIVE.key() + " is turned on. " + + "When turned on runs archiver async with writing, which can speed up overall write performance."); + + public static final ConfigProperty MAX_COMMITS_TO_KEEP = ConfigProperty + .key("hoodie.keep.max.commits") + .defaultValue("30") + .withDocumentation("Archiving service moves older entries from timeline into an archived log after each write, to " + + " keep the metadata overhead constant, even as the table size grows." + + "This config controls the maximum number of instants to retain in the active timeline. "); + + public static final ConfigProperty DELETE_ARCHIVED_INSTANT_PARALLELISM_VALUE = ConfigProperty + .key("hoodie.archive.delete.parallelism") + .defaultValue(100) + .withDocumentation("Parallelism for deleting archived hoodie commits."); + + public static final ConfigProperty MIN_COMMITS_TO_KEEP = ConfigProperty + .key("hoodie.keep.min.commits") + .defaultValue("20") + .withDocumentation("Similar to " + MAX_COMMITS_TO_KEEP.key() + ", but controls the minimum number of" + + "instants to retain in the active timeline."); + + public static final ConfigProperty COMMITS_ARCHIVAL_BATCH_SIZE = ConfigProperty + .key("hoodie.commits.archival.batch") + .defaultValue(String.valueOf(10)) + .withDocumentation("Archiving of instants is batched in best-effort manner, to pack more instants into a single" + + " archive log. This config controls such archival batch size."); + + public static final ConfigProperty ARCHIVE_MERGE_FILES_BATCH_SIZE = ConfigProperty + .key("hoodie.archive.merge.files.batch.size") + .defaultValue(10) + .withDocumentation("The number of small archive files to be merged at once."); + + public static final ConfigProperty ARCHIVE_MERGE_SMALL_FILE_LIMIT_BYTES = ConfigProperty + .key("hoodie.archive.merge.small.file.limit.bytes") + .defaultValue(20L * 1024 * 1024) + .withDocumentation("This config sets the archive file size limit below which an archive file becomes a candidate to be selected as such a small file."); + + public static final ConfigProperty ARCHIVE_MERGE_ENABLE = ConfigProperty + .key("hoodie.archive.merge.enable") + .defaultValue(false) + .withDocumentation("When enable, hoodie will auto merge several small archive files into larger one. It's" + + " useful when storage scheme doesn't support append operation."); + + /** + * @deprecated Use {@link #MAX_COMMITS_TO_KEEP} and its methods instead + */ + @Deprecated + public static final String MAX_COMMITS_TO_KEEP_PROP = MAX_COMMITS_TO_KEEP.key(); + /** + * @deprecated Use {@link #MIN_COMMITS_TO_KEEP} and its methods instead + */ + @Deprecated + public static final String MIN_COMMITS_TO_KEEP_PROP = MIN_COMMITS_TO_KEEP.key(); + /** + * @deprecated Use {@link #COMMITS_ARCHIVAL_BATCH_SIZE} and its methods instead + */ + @Deprecated + public static final String COMMITS_ARCHIVAL_BATCH_SIZE_PROP = COMMITS_ARCHIVAL_BATCH_SIZE.key(); + /** @deprecated Use {@link #MAX_COMMITS_TO_KEEP} and its methods instead */ + @Deprecated + private static final String DEFAULT_MAX_COMMITS_TO_KEEP = MAX_COMMITS_TO_KEEP.defaultValue(); + /** + * @deprecated Use {@link #MIN_COMMITS_TO_KEEP} and its methods instead + */ + @Deprecated + private static final String DEFAULT_MIN_COMMITS_TO_KEEP = MIN_COMMITS_TO_KEEP.defaultValue(); + /** + * @deprecated Use {@link #COMMITS_ARCHIVAL_BATCH_SIZE} and its methods instead + */ + @Deprecated + private static final String DEFAULT_COMMITS_ARCHIVAL_BATCH_SIZE = COMMITS_ARCHIVAL_BATCH_SIZE.defaultValue(); + + private HoodieArchivalConfig() { + super(); + } + + public static HoodieArchivalConfig.Builder newBuilder() { + return new HoodieArchivalConfig.Builder(); + } + + public static class Builder { + + private final HoodieArchivalConfig archivalConfig = new HoodieArchivalConfig(); + + public HoodieArchivalConfig.Builder fromFile(File propertiesFile) throws IOException { + try (FileReader reader = new FileReader(propertiesFile)) { + this.archivalConfig.getProps().load(reader); + return this; + } + } + + public HoodieArchivalConfig.Builder fromProperties(Properties props) { + this.archivalConfig.getProps().putAll(props); + return this; + } + + public HoodieArchivalConfig.Builder withAutoArchive(Boolean autoArchive) { + archivalConfig.setValue(AUTO_ARCHIVE, String.valueOf(autoArchive)); + return this; + } + + public HoodieArchivalConfig.Builder withAsyncArchive(Boolean asyncArchive) { + archivalConfig.setValue(ASYNC_ARCHIVE, String.valueOf(asyncArchive)); + return this; + } + + public HoodieArchivalConfig.Builder archiveCommitsWith(int minToKeep, int maxToKeep) { + archivalConfig.setValue(MIN_COMMITS_TO_KEEP, String.valueOf(minToKeep)); + archivalConfig.setValue(MAX_COMMITS_TO_KEEP, String.valueOf(maxToKeep)); + return this; + } + + public HoodieArchivalConfig.Builder withArchiveMergeFilesBatchSize(int number) { + archivalConfig.setValue(ARCHIVE_MERGE_FILES_BATCH_SIZE, String.valueOf(number)); + return this; + } + + public HoodieArchivalConfig.Builder withArchiveMergeSmallFileLimit(long size) { + archivalConfig.setValue(ARCHIVE_MERGE_SMALL_FILE_LIMIT_BYTES, String.valueOf(size)); + return this; + } + + public HoodieArchivalConfig.Builder withArchiveMergeEnable(boolean enable) { + archivalConfig.setValue(ARCHIVE_MERGE_ENABLE, String.valueOf(enable)); + return this; + } + + public HoodieArchivalConfig.Builder withArchiveDeleteParallelism(int archiveDeleteParallelism) { + archivalConfig.setValue(DELETE_ARCHIVED_INSTANT_PARALLELISM_VALUE, String.valueOf(archiveDeleteParallelism)); + return this; + } + + public HoodieArchivalConfig.Builder withCommitsArchivalBatchSize(int batchSize) { + archivalConfig.setValue(COMMITS_ARCHIVAL_BATCH_SIZE, String.valueOf(batchSize)); + return this; + } + + public HoodieArchivalConfig build() { + archivalConfig.setDefaults(HoodieArchivalConfig.class.getName()); + return archivalConfig; + } + } +} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieCleanConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieCleanConfig.java new file mode 100644 index 0000000000000..7b665f9b2b876 --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieCleanConfig.java @@ -0,0 +1,297 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.config; + +import org.apache.hudi.common.config.ConfigClassProperty; +import org.apache.hudi.common.config.ConfigGroups; +import org.apache.hudi.common.config.ConfigProperty; +import org.apache.hudi.common.config.HoodieConfig; +import org.apache.hudi.common.model.HoodieCleaningPolicy; +import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy; +import org.apache.hudi.table.action.clean.CleaningTriggerStrategy; + +import javax.annotation.concurrent.Immutable; +import java.io.File; +import java.io.FileReader; +import java.io.IOException; +import java.util.Arrays; +import java.util.Properties; +import java.util.stream.Collectors; + +/** + * Clean related config. + */ +@Immutable +@ConfigClassProperty(name = "Clean Configs", + groupName = ConfigGroups.Names.WRITE_CLIENT, + description = "Cleaning (reclamation of older/unused file groups/slices).") +public class HoodieCleanConfig extends HoodieConfig { + + public static final ConfigProperty AUTO_CLEAN = ConfigProperty + .key("hoodie.clean.automatic") + .defaultValue("true") + .withDocumentation("When enabled, the cleaner table service is invoked immediately after each commit," + + " to delete older file slices. It's recommended to enable this, to ensure metadata and data storage" + + " growth is bounded."); + + public static final ConfigProperty ASYNC_CLEAN = ConfigProperty + .key("hoodie.clean.async") + .defaultValue("false") + .withDocumentation("Only applies when " + AUTO_CLEAN.key() + " is turned on. " + + "When turned on runs cleaner async with writing, which can speed up overall write performance."); + + public static final ConfigProperty CLEANER_COMMITS_RETAINED = ConfigProperty + .key("hoodie.cleaner.commits.retained") + .defaultValue("10") + .withDocumentation("Number of commits to retain, without cleaning. This will be retained for num_of_commits * time_between_commits " + + "(scheduled). This also directly translates into how much data retention the table supports for incremental queries."); + + public static final ConfigProperty CLEANER_HOURS_RETAINED = ConfigProperty.key("hoodie.cleaner.hours.retained") + .defaultValue("24") + .withDocumentation("Number of hours for which commits need to be retained. This config provides a more flexible option as" + + "compared to number of commits retained for cleaning service. Setting this property ensures all the files, but the latest in a file group," + + " corresponding to commits with commit times older than the configured number of hours to be retained are cleaned."); + + public static final ConfigProperty CLEANER_POLICY = ConfigProperty + .key("hoodie.cleaner.policy") + .defaultValue(HoodieCleaningPolicy.KEEP_LATEST_COMMITS.name()) + .withDocumentation("Cleaning policy to be used. The cleaner service deletes older file slices files to re-claim space." + + " By default, cleaner spares the file slices written by the last N commits, determined by " + CLEANER_COMMITS_RETAINED.key() + + " Long running query plans may often refer to older file slices and will break if those are cleaned, before the query has had" + + " a chance to run. So, it is good to make sure that the data is retained for more than the maximum query execution time"); + + public static final ConfigProperty CLEAN_TRIGGER_STRATEGY = ConfigProperty + .key("hoodie.clean.trigger.strategy") + .defaultValue(CleaningTriggerStrategy.NUM_COMMITS.name()) + .withDocumentation("Controls how cleaning is scheduled. Valid options: " + + Arrays.stream(CleaningTriggerStrategy.values()).map(Enum::name).collect(Collectors.joining(","))); + + public static final ConfigProperty CLEAN_MAX_COMMITS = ConfigProperty + .key("hoodie.clean.max.commits") + .defaultValue("1") + .withDocumentation("Number of commits after the last clean operation, before scheduling of a new clean is attempted."); + + public static final ConfigProperty CLEANER_FILE_VERSIONS_RETAINED = ConfigProperty + .key("hoodie.cleaner.fileversions.retained") + .defaultValue("3") + .withDocumentation("When " + HoodieCleaningPolicy.KEEP_LATEST_FILE_VERSIONS.name() + " cleaning policy is used, " + + " the minimum number of file slices to retain in each file group, during cleaning."); + + public static final ConfigProperty CLEANER_INCREMENTAL_MODE_ENABLE = ConfigProperty + .key("hoodie.cleaner.incremental.mode") + .defaultValue("true") + .withDocumentation("When enabled, the plans for each cleaner service run is computed incrementally off the events " + + " in the timeline, since the last cleaner run. This is much more efficient than obtaining listings for the full" + + " table for each planning (even with a metadata table)."); + + public static final ConfigProperty FAILED_WRITES_CLEANER_POLICY = ConfigProperty + .key("hoodie.cleaner.policy.failed.writes") + .defaultValue(HoodieFailedWritesCleaningPolicy.EAGER.name()) + .withDocumentation("Cleaning policy for failed writes to be used. Hudi will delete any files written by " + + "failed writes to re-claim space. Choose to perform this rollback of failed writes eagerly before " + + "every writer starts (only supported for single writer) or lazily by the cleaner (required for multi-writers)"); + + public static final ConfigProperty CLEANER_PARALLELISM_VALUE = ConfigProperty + .key("hoodie.cleaner.parallelism") + .defaultValue("200") + .withDocumentation("Parallelism for the cleaning operation. Increase this if cleaning becomes slow."); + + public static final ConfigProperty ALLOW_MULTIPLE_CLEANS = ConfigProperty + .key("hoodie.clean.allow.multiple") + .defaultValue(true) + .sinceVersion("0.11.0") + .withDocumentation("Allows scheduling/executing multiple cleans by enabling this config. If users prefer to strictly ensure clean requests should be mutually exclusive, " + + ".i.e. a 2nd clean will not be scheduled if another clean is not yet completed to avoid repeat cleaning of same files, they might want to disable this config."); + + public static final ConfigProperty CLEANER_BOOTSTRAP_BASE_FILE_ENABLE = ConfigProperty + .key("hoodie.cleaner.delete.bootstrap.base.file") + .defaultValue("false") + .withDocumentation("When set to true, cleaner also deletes the bootstrap base file when it's skeleton base file is " + + " cleaned. Turn this to true, if you want to ensure the bootstrap dataset storage is reclaimed over time, as the" + + " table receives updates/deletes. Another reason to turn this on, would be to ensure data residing in bootstrap " + + " base files are also physically deleted, to comply with data privacy enforcement processes."); + + + /** @deprecated Use {@link #CLEANER_POLICY} and its methods instead */ + @Deprecated + public static final String CLEANER_POLICY_PROP = CLEANER_POLICY.key(); + /** @deprecated Use {@link #AUTO_CLEAN} and its methods instead */ + @Deprecated + public static final String AUTO_CLEAN_PROP = AUTO_CLEAN.key(); + /** @deprecated Use {@link #ASYNC_CLEAN} and its methods instead */ + @Deprecated + public static final String ASYNC_CLEAN_PROP = ASYNC_CLEAN.key(); + /** @deprecated Use {@link #CLEANER_FILE_VERSIONS_RETAINED} and its methods instead */ + @Deprecated + public static final String CLEANER_FILE_VERSIONS_RETAINED_PROP = CLEANER_FILE_VERSIONS_RETAINED.key(); + /** + * @deprecated Use {@link #CLEANER_COMMITS_RETAINED} and its methods instead + */ + @Deprecated + public static final String CLEANER_COMMITS_RETAINED_PROP = CLEANER_COMMITS_RETAINED.key(); + /** + * @deprecated Use {@link #CLEANER_INCREMENTAL_MODE_ENABLE} and its methods instead + */ + @Deprecated + public static final String CLEANER_INCREMENTAL_MODE = CLEANER_INCREMENTAL_MODE_ENABLE.key(); + /** + * @deprecated Use {@link #CLEANER_BOOTSTRAP_BASE_FILE_ENABLE} and its methods instead + */ + @Deprecated + public static final String CLEANER_BOOTSTRAP_BASE_FILE_ENABLED = CLEANER_BOOTSTRAP_BASE_FILE_ENABLE.key(); + /** + * @deprecated Use {@link #CLEANER_PARALLELISM_VALUE} and its methods instead + */ + @Deprecated + public static final String CLEANER_PARALLELISM = CLEANER_PARALLELISM_VALUE.key(); + /** + * @deprecated Use {@link #CLEANER_PARALLELISM_VALUE} and its methods instead + */ + @Deprecated + public static final String DEFAULT_CLEANER_PARALLELISM = CLEANER_PARALLELISM_VALUE.defaultValue(); + /** @deprecated Use {@link #CLEANER_POLICY} and its methods instead */ + @Deprecated + private static final String DEFAULT_CLEANER_POLICY = CLEANER_POLICY.defaultValue(); + /** @deprecated Use {@link #FAILED_WRITES_CLEANER_POLICY} and its methods instead */ + @Deprecated + public static final String FAILED_WRITES_CLEANER_POLICY_PROP = FAILED_WRITES_CLEANER_POLICY.key(); + /** @deprecated Use {@link #FAILED_WRITES_CLEANER_POLICY} and its methods instead */ + @Deprecated + private static final String DEFAULT_FAILED_WRITES_CLEANER_POLICY = FAILED_WRITES_CLEANER_POLICY.defaultValue(); + /** @deprecated Use {@link #AUTO_CLEAN} and its methods instead */ + @Deprecated + private static final String DEFAULT_AUTO_CLEAN = AUTO_CLEAN.defaultValue(); + /** + * @deprecated Use {@link #ASYNC_CLEAN} and its methods instead + */ + @Deprecated + private static final String DEFAULT_ASYNC_CLEAN = ASYNC_CLEAN.defaultValue(); + /** + * @deprecated Use {@link #CLEANER_INCREMENTAL_MODE_ENABLE} and its methods instead + */ + @Deprecated + private static final String DEFAULT_INCREMENTAL_CLEANER = CLEANER_INCREMENTAL_MODE_ENABLE.defaultValue(); + /** @deprecated Use {@link #CLEANER_FILE_VERSIONS_RETAINED} and its methods instead */ + @Deprecated + private static final String DEFAULT_CLEANER_FILE_VERSIONS_RETAINED = CLEANER_FILE_VERSIONS_RETAINED.defaultValue(); + /** @deprecated Use {@link #CLEANER_COMMITS_RETAINED} and its methods instead */ + @Deprecated + private static final String DEFAULT_CLEANER_COMMITS_RETAINED = CLEANER_COMMITS_RETAINED.defaultValue(); + /** + * @deprecated Use {@link #CLEANER_BOOTSTRAP_BASE_FILE_ENABLE} and its methods instead + */ + @Deprecated + private static final String DEFAULT_CLEANER_BOOTSTRAP_BASE_FILE_ENABLED = CLEANER_BOOTSTRAP_BASE_FILE_ENABLE.defaultValue(); + + private HoodieCleanConfig() { + super(); + } + + public static HoodieCleanConfig.Builder newBuilder() { + return new HoodieCleanConfig.Builder(); + } + + public static class Builder { + + private final HoodieCleanConfig cleanConfig = new HoodieCleanConfig(); + + public HoodieCleanConfig.Builder fromFile(File propertiesFile) throws IOException { + try (FileReader reader = new FileReader(propertiesFile)) { + this.cleanConfig.getProps().load(reader); + return this; + } + } + + public HoodieCleanConfig.Builder fromProperties(Properties props) { + this.cleanConfig.getProps().putAll(props); + return this; + } + + public HoodieCleanConfig.Builder withAutoClean(Boolean autoClean) { + cleanConfig.setValue(AUTO_CLEAN, String.valueOf(autoClean)); + return this; + } + + public HoodieCleanConfig.Builder withAsyncClean(Boolean asyncClean) { + cleanConfig.setValue(ASYNC_CLEAN, String.valueOf(asyncClean)); + return this; + } + + public HoodieCleanConfig.Builder withIncrementalCleaningMode(Boolean incrementalCleaningMode) { + cleanConfig.setValue(CLEANER_INCREMENTAL_MODE_ENABLE, String.valueOf(incrementalCleaningMode)); + return this; + } + + public HoodieCleanConfig.Builder withCleaningTriggerStrategy(String cleaningTriggerStrategy) { + cleanConfig.setValue(CLEAN_TRIGGER_STRATEGY, cleaningTriggerStrategy); + return this; + } + + public HoodieCleanConfig.Builder withMaxCommitsBeforeCleaning(int maxCommitsBeforeCleaning) { + cleanConfig.setValue(CLEAN_MAX_COMMITS, String.valueOf(maxCommitsBeforeCleaning)); + return this; + } + + public HoodieCleanConfig.Builder withCleanerPolicy(HoodieCleaningPolicy policy) { + cleanConfig.setValue(CLEANER_POLICY, policy.name()); + return this; + } + + public HoodieCleanConfig.Builder retainFileVersions(int fileVersionsRetained) { + cleanConfig.setValue(CLEANER_FILE_VERSIONS_RETAINED, String.valueOf(fileVersionsRetained)); + return this; + } + + public HoodieCleanConfig.Builder retainCommits(int commitsRetained) { + cleanConfig.setValue(CLEANER_COMMITS_RETAINED, String.valueOf(commitsRetained)); + return this; + } + + public HoodieCleanConfig.Builder cleanerNumHoursRetained(int cleanerHoursRetained) { + cleanConfig.setValue(CLEANER_HOURS_RETAINED, String.valueOf(cleanerHoursRetained)); + return this; + } + + public HoodieCleanConfig.Builder allowMultipleCleans(boolean allowMultipleCleanSchedules) { + cleanConfig.setValue(ALLOW_MULTIPLE_CLEANS, String.valueOf(allowMultipleCleanSchedules)); + return this; + } + + public HoodieCleanConfig.Builder withCleanerParallelism(int cleanerParallelism) { + cleanConfig.setValue(CLEANER_PARALLELISM_VALUE, String.valueOf(cleanerParallelism)); + return this; + } + + public HoodieCleanConfig.Builder withCleanBootstrapBaseFileEnabled(Boolean cleanBootstrapSourceFileEnabled) { + cleanConfig.setValue(CLEANER_BOOTSTRAP_BASE_FILE_ENABLE, String.valueOf(cleanBootstrapSourceFileEnabled)); + return this; + } + + public HoodieCleanConfig.Builder withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy failedWritesPolicy) { + cleanConfig.setValue(FAILED_WRITES_CLEANER_POLICY, failedWritesPolicy.name()); + return this; + } + + public HoodieCleanConfig build() { + cleanConfig.setDefaults(HoodieCleanConfig.class.getName()); + HoodieCleaningPolicy.valueOf(cleanConfig.getString(CLEANER_POLICY)); + return cleanConfig; + } + } +} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieCompactionConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieCompactionConfig.java index 4003a07de7f0b..d1d0e6726173b 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieCompactionConfig.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieCompactionConfig.java @@ -22,11 +22,6 @@ import org.apache.hudi.common.config.ConfigGroups; import org.apache.hudi.common.config.ConfigProperty; import org.apache.hudi.common.config.HoodieConfig; -import org.apache.hudi.common.model.HoodieCleaningPolicy; -import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy; -import org.apache.hudi.common.model.OverwriteWithLatestAvroPayload; -import org.apache.hudi.common.util.ValidationUtils; -import org.apache.hudi.table.action.clean.CleaningTriggerStrategy; import org.apache.hudi.table.action.compact.CompactionTriggerStrategy; import org.apache.hudi.table.action.compact.strategy.CompactionStrategy; import org.apache.hudi.table.action.compact.strategy.LogFileSizeBasedCompactionStrategy; @@ -47,57 +42,9 @@ @ConfigClassProperty(name = "Compaction Configs", groupName = ConfigGroups.Names.WRITE_CLIENT, description = "Configurations that control compaction " - + "(merging of log files onto a new base files) as well as " - + "cleaning (reclamation of older/unused file groups/slices).") + + "(merging of log files onto a new base files).") public class HoodieCompactionConfig extends HoodieConfig { - public static final ConfigProperty AUTO_ARCHIVE = ConfigProperty - .key("hoodie.archive.automatic") - .defaultValue("true") - .withDocumentation("When enabled, the archival table service is invoked immediately after each commit," - + " to archive commits if we cross a maximum value of commits." - + " It's recommended to enable this, to ensure number of active commits is bounded."); - - public static final ConfigProperty ASYNC_ARCHIVE = ConfigProperty - .key("hoodie.archive.async") - .defaultValue("false") - .sinceVersion("0.11.0") - .withDocumentation("Only applies when " + AUTO_ARCHIVE.key() + " is turned on. " - + "When turned on runs archiver async with writing, which can speed up overall write performance."); - - public static final ConfigProperty AUTO_CLEAN = ConfigProperty - .key("hoodie.clean.automatic") - .defaultValue("true") - .withDocumentation("When enabled, the cleaner table service is invoked immediately after each commit," - + " to delete older file slices. It's recommended to enable this, to ensure metadata and data storage" - + " growth is bounded."); - - public static final ConfigProperty ASYNC_CLEAN = ConfigProperty - .key("hoodie.clean.async") - .defaultValue("false") - .withDocumentation("Only applies when " + AUTO_CLEAN.key() + " is turned on. " - + "When turned on runs cleaner async with writing, which can speed up overall write performance."); - - public static final ConfigProperty CLEANER_COMMITS_RETAINED = ConfigProperty - .key("hoodie.cleaner.commits.retained") - .defaultValue("10") - .withDocumentation("Number of commits to retain, without cleaning. This will be retained for num_of_commits * time_between_commits " - + "(scheduled). This also directly translates into how much data retention the table supports for incremental queries."); - - public static final ConfigProperty CLEANER_HOURS_RETAINED = ConfigProperty.key("hoodie.cleaner.hours.retained") - .defaultValue("24") - .withDocumentation("Number of hours for which commits need to be retained. This config provides a more flexible option as" - + "compared to number of commits retained for cleaning service. Setting this property ensures all the files, but the latest in a file group," - + " corresponding to commits with commit times older than the configured number of hours to be retained are cleaned."); - - public static final ConfigProperty CLEANER_POLICY = ConfigProperty - .key("hoodie.cleaner.policy") - .defaultValue(HoodieCleaningPolicy.KEEP_LATEST_COMMITS.name()) - .withDocumentation("Cleaning policy to be used. The cleaner service deletes older file slices files to re-claim space." - + " By default, cleaner spares the file slices written by the last N commits, determined by " + CLEANER_COMMITS_RETAINED.key() - + " Long running query plans may often refer to older file slices and will break if those are cleaned, before the query has had" - + " a chance to run. So, it is good to make sure that the data is retained for more than the maximum query execution time"); - public static final ConfigProperty INLINE_COMPACT = ConfigProperty .key("hoodie.compact.inline") .defaultValue("false") @@ -130,62 +77,6 @@ public class HoodieCompactionConfig extends HoodieConfig { .withDocumentation("Controls how compaction scheduling is triggered, by time or num delta commits or combination of both. " + "Valid options: " + Arrays.stream(CompactionTriggerStrategy.values()).map(Enum::name).collect(Collectors.joining(","))); - public static final ConfigProperty CLEAN_TRIGGER_STRATEGY = ConfigProperty - .key("hoodie.clean.trigger.strategy") - .defaultValue(CleaningTriggerStrategy.NUM_COMMITS.name()) - .withDocumentation("Controls how cleaning is scheduled. Valid options: " - + Arrays.stream(CleaningTriggerStrategy.values()).map(Enum::name).collect(Collectors.joining(","))); - - public static final ConfigProperty CLEAN_MAX_COMMITS = ConfigProperty - .key("hoodie.clean.max.commits") - .defaultValue("1") - .withDocumentation("Number of commits after the last clean operation, before scheduling of a new clean is attempted."); - - public static final ConfigProperty CLEANER_FILE_VERSIONS_RETAINED = ConfigProperty - .key("hoodie.cleaner.fileversions.retained") - .defaultValue("3") - .withDocumentation("When " + HoodieCleaningPolicy.KEEP_LATEST_FILE_VERSIONS.name() + " cleaning policy is used, " - + " the minimum number of file slices to retain in each file group, during cleaning."); - - public static final ConfigProperty CLEANER_INCREMENTAL_MODE_ENABLE = ConfigProperty - .key("hoodie.cleaner.incremental.mode") - .defaultValue("true") - .withDocumentation("When enabled, the plans for each cleaner service run is computed incrementally off the events " - + " in the timeline, since the last cleaner run. This is much more efficient than obtaining listings for the full" - + " table for each planning (even with a metadata table)."); - - public static final ConfigProperty MAX_COMMITS_TO_KEEP = ConfigProperty - .key("hoodie.keep.max.commits") - .defaultValue("30") - .withDocumentation("Archiving service moves older entries from timeline into an archived log after each write, to " - + " keep the metadata overhead constant, even as the table size grows." - + "This config controls the maximum number of instants to retain in the active timeline. "); - - public static final ConfigProperty DELETE_ARCHIVED_INSTANT_PARALLELISM_VALUE = ConfigProperty - .key("hoodie.archive.delete.parallelism") - .defaultValue(100) - .withDocumentation("Parallelism for deleting archived hoodie commits."); - - public static final ConfigProperty MIN_COMMITS_TO_KEEP = ConfigProperty - .key("hoodie.keep.min.commits") - .defaultValue("20") - .withDocumentation("Similar to " + MAX_COMMITS_TO_KEEP.key() + ", but controls the minimum number of" - + "instants to retain in the active timeline."); - - public static final ConfigProperty COMMITS_ARCHIVAL_BATCH_SIZE = ConfigProperty - .key("hoodie.commits.archival.batch") - .defaultValue(String.valueOf(10)) - .withDocumentation("Archiving of instants is batched in best-effort manner, to pack more instants into a single" - + " archive log. This config controls such archival batch size."); - - public static final ConfigProperty CLEANER_BOOTSTRAP_BASE_FILE_ENABLE = ConfigProperty - .key("hoodie.cleaner.delete.bootstrap.base.file") - .defaultValue("false") - .withDocumentation("When set to true, cleaner also deletes the bootstrap base file when it's skeleton base file is " - + " cleaned. Turn this to true, if you want to ensure the bootstrap dataset storage is reclaimed over time, as the" - + " table receives updates/deletes. Another reason to turn this on, would be to ensure data residing in bootstrap " - + " base files are also physically deleted, to comply with data privacy enforcement processes."); - public static final ConfigProperty PARQUET_SMALL_FILE_LIMIT = ConfigProperty .key("hoodie.parquet.small.file.limit") .defaultValue(String.valueOf(104857600)) @@ -202,11 +93,6 @@ public class HoodieCompactionConfig extends HoodieConfig { + " Hudi will search commits in the reverse order, until we find a commit that has totalBytesWritten " + " larger than (PARQUET_SMALL_FILE_LIMIT_BYTES * this_threshold)"); - public static final ConfigProperty CLEANER_PARALLELISM_VALUE = ConfigProperty - .key("hoodie.cleaner.parallelism") - .defaultValue("200") - .withDocumentation("Parallelism for the cleaning operation. Increase this if cleaning becomes slow."); - // 500GB of target IO per compaction (both read and write public static final ConfigProperty TARGET_IO_PER_COMPACTION_IN_MB = ConfigProperty .key("hoodie.compaction.target.io") @@ -227,13 +113,6 @@ public class HoodieCompactionConfig extends HoodieConfig { + "compaction during each compaction run. By default. Hudi picks the log file " + "with most accumulated unmerged data"); - public static final ConfigProperty PAYLOAD_CLASS_NAME = ConfigProperty - .key("hoodie.compaction.payload.class") - .defaultValue(OverwriteWithLatestAvroPayload.class.getName()) - .withDocumentation("This needs to be same as class used during insert/upserts. Just like writing, compaction also uses " - + "the record payload class to merge records in the log against each other, merge again with the base file and " - + "produce the final record to be written after compaction."); - public static final ConfigProperty COMPACTION_LAZY_BLOCK_READ_ENABLE = ConfigProperty .key("hoodie.compaction.lazy.block.read") .defaultValue("true") @@ -247,13 +126,6 @@ public class HoodieCompactionConfig extends HoodieConfig { .withDocumentation("HoodieLogFormatReader reads a logfile in the forward direction starting from pos=0 to pos=file_length. " + "If this config is set to true, the reader reads the logfile in reverse direction, from pos=file_length to pos=0"); - public static final ConfigProperty FAILED_WRITES_CLEANER_POLICY = ConfigProperty - .key("hoodie.cleaner.policy.failed.writes") - .defaultValue(HoodieFailedWritesCleaningPolicy.EAGER.name()) - .withDocumentation("Cleaning policy for failed writes to be used. Hudi will delete any files written by " - + "failed writes to re-claim space. Choose to perform this rollback of failed writes eagerly before " - + "every writer starts (only supported for single writer) or lazily by the cleaner (required for multi-writers)"); - public static final ConfigProperty TARGET_PARTITIONS_PER_DAYBASED_COMPACTION = ConfigProperty .key("hoodie.compaction.daybased.target.partitions") .defaultValue("10") @@ -290,39 +162,8 @@ public class HoodieCompactionConfig extends HoodieConfig { .withDocumentation("The average record size. If not explicitly specified, hudi will compute the " + "record size estimate compute dynamically based on commit metadata. " + " This is critical in computing the insert parallelism and bin-packing inserts into small files."); - - public static final ConfigProperty ALLOW_MULTIPLE_CLEANS = ConfigProperty - .key("hoodie.clean.allow.multiple") - .defaultValue(true) - .sinceVersion("0.11.0") - .withDocumentation("Allows scheduling/executing multiple cleans by enabling this config. If users prefer to strictly ensure clean requests should be mutually exclusive, " - + ".i.e. a 2nd clean will not be scheduled if another clean is not yet completed to avoid repeat cleaning of same files, they might want to disable this config."); - - public static final ConfigProperty ARCHIVE_MERGE_FILES_BATCH_SIZE = ConfigProperty - .key("hoodie.archive.merge.files.batch.size") - .defaultValue(10) - .withDocumentation("The number of small archive files to be merged at once."); - - public static final ConfigProperty ARCHIVE_MERGE_SMALL_FILE_LIMIT_BYTES = ConfigProperty - .key("hoodie.archive.merge.small.file.limit.bytes") - .defaultValue(20L * 1024 * 1024) - .withDocumentation("This config sets the archive file size limit below which an archive file becomes a candidate to be selected as such a small file."); - - public static final ConfigProperty ARCHIVE_MERGE_ENABLE = ConfigProperty - .key("hoodie.archive.merge.enable") - .defaultValue(false) - .withDocumentation("When enable, hoodie will auto merge several small archive files into larger one. It's" - + " useful when storage scheme doesn't support append operation."); - - /** @deprecated Use {@link #CLEANER_POLICY} and its methods instead */ - @Deprecated - public static final String CLEANER_POLICY_PROP = CLEANER_POLICY.key(); - /** @deprecated Use {@link #AUTO_CLEAN} and its methods instead */ - @Deprecated - public static final String AUTO_CLEAN_PROP = AUTO_CLEAN.key(); - /** @deprecated Use {@link #ASYNC_CLEAN} and its methods instead */ - @Deprecated - public static final String ASYNC_CLEAN_PROP = ASYNC_CLEAN.key(); + + /** @deprecated Use {@link #INLINE_COMPACT} and its methods instead */ @Deprecated public static final String INLINE_COMPACT_PROP = INLINE_COMPACT.key(); @@ -335,39 +176,6 @@ public class HoodieCompactionConfig extends HoodieConfig { /** @deprecated Use {@link #INLINE_COMPACT_TRIGGER_STRATEGY} and its methods instead */ @Deprecated public static final String INLINE_COMPACT_TRIGGER_STRATEGY_PROP = INLINE_COMPACT_TRIGGER_STRATEGY.key(); - /** @deprecated Use {@link #CLEANER_FILE_VERSIONS_RETAINED} and its methods instead */ - @Deprecated - public static final String CLEANER_FILE_VERSIONS_RETAINED_PROP = CLEANER_FILE_VERSIONS_RETAINED.key(); - /** - * @deprecated Use {@link #CLEANER_COMMITS_RETAINED} and its methods instead - */ - @Deprecated - public static final String CLEANER_COMMITS_RETAINED_PROP = CLEANER_COMMITS_RETAINED.key(); - /** - * @deprecated Use {@link #CLEANER_INCREMENTAL_MODE_ENABLE} and its methods instead - */ - @Deprecated - public static final String CLEANER_INCREMENTAL_MODE = CLEANER_INCREMENTAL_MODE_ENABLE.key(); - /** - * @deprecated Use {@link #MAX_COMMITS_TO_KEEP} and its methods instead - */ - @Deprecated - public static final String MAX_COMMITS_TO_KEEP_PROP = MAX_COMMITS_TO_KEEP.key(); - /** - * @deprecated Use {@link #MIN_COMMITS_TO_KEEP} and its methods instead - */ - @Deprecated - public static final String MIN_COMMITS_TO_KEEP_PROP = MIN_COMMITS_TO_KEEP.key(); - /** - * @deprecated Use {@link #COMMITS_ARCHIVAL_BATCH_SIZE} and its methods instead - */ - @Deprecated - public static final String COMMITS_ARCHIVAL_BATCH_SIZE_PROP = COMMITS_ARCHIVAL_BATCH_SIZE.key(); - /** - * @deprecated Use {@link #CLEANER_BOOTSTRAP_BASE_FILE_ENABLE} and its methods instead - */ - @Deprecated - public static final String CLEANER_BOOTSTRAP_BASE_FILE_ENABLED = CLEANER_BOOTSTRAP_BASE_FILE_ENABLE.key(); /** * @deprecated Use {@link #PARQUET_SMALL_FILE_LIMIT} and its methods instead */ @@ -418,16 +226,6 @@ public class HoodieCompactionConfig extends HoodieConfig { */ @Deprecated public static final String DEFAULT_COPY_ON_WRITE_TABLE_RECORD_SIZE_ESTIMATE = COPY_ON_WRITE_RECORD_SIZE_ESTIMATE.defaultValue(); - /** - * @deprecated Use {@link #CLEANER_PARALLELISM_VALUE} and its methods instead - */ - @Deprecated - public static final String CLEANER_PARALLELISM = CLEANER_PARALLELISM_VALUE.key(); - /** - * @deprecated Use {@link #CLEANER_PARALLELISM_VALUE} and its methods instead - */ - @Deprecated - public static final String DEFAULT_CLEANER_PARALLELISM = CLEANER_PARALLELISM_VALUE.defaultValue(); /** * @deprecated Use {@link #TARGET_IO_PER_COMPACTION_IN_MB} and its methods instead */ @@ -446,12 +244,6 @@ public class HoodieCompactionConfig extends HoodieConfig { /** @deprecated Use {@link #COMPACTION_STRATEGY} and its methods instead */ @Deprecated public static final String DEFAULT_COMPACTION_STRATEGY = COMPACTION_STRATEGY.defaultValue(); - /** @deprecated Use {@link #PAYLOAD_CLASS_NAME} and its methods instead */ - @Deprecated - public static final String DEFAULT_PAYLOAD_CLASS = PAYLOAD_CLASS_NAME.defaultValue(); - /** @deprecated Use {@link #PAYLOAD_CLASS_NAME} and its methods instead */ - @Deprecated - public static final String PAYLOAD_CLASS_PROP = PAYLOAD_CLASS_NAME.key(); /** @deprecated Use {@link #COMPACTION_LAZY_BLOCK_READ_ENABLE} and its methods instead */ @Deprecated public static final String COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP = COMPACTION_LAZY_BLOCK_READ_ENABLE.key(); @@ -464,33 +256,11 @@ public class HoodieCompactionConfig extends HoodieConfig { /** @deprecated Use {@link #COMPACTION_REVERSE_LOG_READ_ENABLE} and its methods instead */ @Deprecated public static final String DEFAULT_COMPACTION_REVERSE_LOG_READ_ENABLED = COMPACTION_REVERSE_LOG_READ_ENABLE.defaultValue(); - /** @deprecated Use {@link #CLEANER_POLICY} and its methods instead */ - @Deprecated - private static final String DEFAULT_CLEANER_POLICY = CLEANER_POLICY.defaultValue(); - /** @deprecated Use {@link #FAILED_WRITES_CLEANER_POLICY} and its methods instead */ - @Deprecated - public static final String FAILED_WRITES_CLEANER_POLICY_PROP = FAILED_WRITES_CLEANER_POLICY.key(); - /** @deprecated Use {@link #FAILED_WRITES_CLEANER_POLICY} and its methods instead */ - @Deprecated - private static final String DEFAULT_FAILED_WRITES_CLEANER_POLICY = FAILED_WRITES_CLEANER_POLICY.defaultValue(); - /** @deprecated Use {@link #AUTO_CLEAN} and its methods instead */ - @Deprecated - private static final String DEFAULT_AUTO_CLEAN = AUTO_CLEAN.defaultValue(); - /** - * @deprecated Use {@link #ASYNC_CLEAN} and its methods instead - */ - @Deprecated - private static final String DEFAULT_ASYNC_CLEAN = ASYNC_CLEAN.defaultValue(); /** * @deprecated Use {@link #INLINE_COMPACT} and its methods instead */ @Deprecated private static final String DEFAULT_INLINE_COMPACT = INLINE_COMPACT.defaultValue(); - /** - * @deprecated Use {@link #CLEANER_INCREMENTAL_MODE_ENABLE} and its methods instead - */ - @Deprecated - private static final String DEFAULT_INCREMENTAL_CLEANER = CLEANER_INCREMENTAL_MODE_ENABLE.defaultValue(); /** @deprecated Use {@link #INLINE_COMPACT_NUM_DELTA_COMMITS} and its methods instead */ @Deprecated private static final String DEFAULT_INLINE_COMPACT_NUM_DELTA_COMMITS = INLINE_COMPACT_NUM_DELTA_COMMITS.defaultValue(); @@ -500,30 +270,6 @@ public class HoodieCompactionConfig extends HoodieConfig { /** @deprecated Use {@link #INLINE_COMPACT_TRIGGER_STRATEGY} and its methods instead */ @Deprecated private static final String DEFAULT_INLINE_COMPACT_TRIGGER_STRATEGY = INLINE_COMPACT_TRIGGER_STRATEGY.defaultValue(); - /** @deprecated Use {@link #CLEANER_FILE_VERSIONS_RETAINED} and its methods instead */ - @Deprecated - private static final String DEFAULT_CLEANER_FILE_VERSIONS_RETAINED = CLEANER_FILE_VERSIONS_RETAINED.defaultValue(); - /** @deprecated Use {@link #CLEANER_COMMITS_RETAINED} and its methods instead */ - @Deprecated - private static final String DEFAULT_CLEANER_COMMITS_RETAINED = CLEANER_COMMITS_RETAINED.defaultValue(); - /** @deprecated Use {@link #MAX_COMMITS_TO_KEEP} and its methods instead */ - @Deprecated - private static final String DEFAULT_MAX_COMMITS_TO_KEEP = MAX_COMMITS_TO_KEEP.defaultValue(); - /** - * @deprecated Use {@link #MIN_COMMITS_TO_KEEP} and its methods instead - */ - @Deprecated - private static final String DEFAULT_MIN_COMMITS_TO_KEEP = MIN_COMMITS_TO_KEEP.defaultValue(); - /** - * @deprecated Use {@link #COMMITS_ARCHIVAL_BATCH_SIZE} and its methods instead - */ - @Deprecated - private static final String DEFAULT_COMMITS_ARCHIVAL_BATCH_SIZE = COMMITS_ARCHIVAL_BATCH_SIZE.defaultValue(); - /** - * @deprecated Use {@link #CLEANER_BOOTSTRAP_BASE_FILE_ENABLE} and its methods instead - */ - @Deprecated - private static final String DEFAULT_CLEANER_BOOTSTRAP_BASE_FILE_ENABLED = CLEANER_BOOTSTRAP_BASE_FILE_ENABLE.defaultValue(); /** @deprecated Use {@link #TARGET_PARTITIONS_PER_DAYBASED_COMPACTION} and its methods instead */ @Deprecated public static final String TARGET_PARTITIONS_PER_DAYBASED_COMPACTION_PROP = TARGET_PARTITIONS_PER_DAYBASED_COMPACTION.key(); @@ -555,31 +301,6 @@ public Builder fromProperties(Properties props) { return this; } - public Builder withAutoArchive(Boolean autoArchive) { - compactionConfig.setValue(AUTO_ARCHIVE, String.valueOf(autoArchive)); - return this; - } - - public Builder withAsyncArchive(Boolean asyncArchive) { - compactionConfig.setValue(ASYNC_ARCHIVE, String.valueOf(asyncArchive)); - return this; - } - - public Builder withAutoClean(Boolean autoClean) { - compactionConfig.setValue(AUTO_CLEAN, String.valueOf(autoClean)); - return this; - } - - public Builder withAsyncClean(Boolean asyncClean) { - compactionConfig.setValue(ASYNC_CLEAN, String.valueOf(asyncClean)); - return this; - } - - public Builder withIncrementalCleaningMode(Boolean incrementalCleaningMode) { - compactionConfig.setValue(CLEANER_INCREMENTAL_MODE_ENABLE, String.valueOf(incrementalCleaningMode)); - return this; - } - public Builder withInlineCompaction(Boolean inlineCompaction) { compactionConfig.setValue(INLINE_COMPACT, String.valueOf(inlineCompaction)); return this; @@ -595,57 +316,6 @@ public Builder withInlineCompactionTriggerStrategy(CompactionTriggerStrategy com return this; } - public Builder withCleaningTriggerStrategy(String cleaningTriggerStrategy) { - compactionConfig.setValue(CLEAN_TRIGGER_STRATEGY, cleaningTriggerStrategy); - return this; - } - - public Builder withMaxCommitsBeforeCleaning(int maxCommitsBeforeCleaning) { - compactionConfig.setValue(CLEAN_MAX_COMMITS, String.valueOf(maxCommitsBeforeCleaning)); - return this; - } - - public Builder withCleanerPolicy(HoodieCleaningPolicy policy) { - compactionConfig.setValue(CLEANER_POLICY, policy.name()); - return this; - } - - public Builder retainFileVersions(int fileVersionsRetained) { - compactionConfig.setValue(CLEANER_FILE_VERSIONS_RETAINED, String.valueOf(fileVersionsRetained)); - return this; - } - - public Builder retainCommits(int commitsRetained) { - compactionConfig.setValue(CLEANER_COMMITS_RETAINED, String.valueOf(commitsRetained)); - return this; - } - - public Builder cleanerNumHoursRetained(int cleanerHoursRetained) { - compactionConfig.setValue(CLEANER_HOURS_RETAINED, String.valueOf(cleanerHoursRetained)); - return this; - } - - public Builder archiveCommitsWith(int minToKeep, int maxToKeep) { - compactionConfig.setValue(MIN_COMMITS_TO_KEEP, String.valueOf(minToKeep)); - compactionConfig.setValue(MAX_COMMITS_TO_KEEP, String.valueOf(maxToKeep)); - return this; - } - - public Builder withArchiveMergeFilesBatchSize(int number) { - compactionConfig.setValue(ARCHIVE_MERGE_FILES_BATCH_SIZE, String.valueOf(number)); - return this; - } - - public Builder withArchiveMergeSmallFileLimit(long size) { - compactionConfig.setValue(ARCHIVE_MERGE_SMALL_FILE_LIMIT_BYTES, String.valueOf(size)); - return this; - } - - public Builder withArchiveMergeEnable(boolean enable) { - compactionConfig.setValue(ARCHIVE_MERGE_ENABLE, String.valueOf(enable)); - return this; - } - public Builder compactionSmallFileSize(long smallFileLimitBytes) { compactionConfig.setValue(PARQUET_SMALL_FILE_LIMIT, String.valueOf(smallFileLimitBytes)); return this; @@ -671,26 +341,11 @@ public Builder approxRecordSize(int recordSizeEstimate) { return this; } - public Builder allowMultipleCleans(boolean allowMultipleCleanSchedules) { - compactionConfig.setValue(ALLOW_MULTIPLE_CLEANS, String.valueOf(allowMultipleCleanSchedules)); - return this; - } - - public Builder withCleanerParallelism(int cleanerParallelism) { - compactionConfig.setValue(CLEANER_PARALLELISM_VALUE, String.valueOf(cleanerParallelism)); - return this; - } - public Builder withCompactionStrategy(CompactionStrategy compactionStrategy) { compactionConfig.setValue(COMPACTION_STRATEGY, compactionStrategy.getClass().getName()); return this; } - public Builder withPayloadClass(String payloadClassName) { - compactionConfig.setValue(PAYLOAD_CLASS_NAME, payloadClassName); - return this; - } - public Builder withTargetIOPerCompactionInMB(long targetIOPerCompactionInMB) { compactionConfig.setValue(TARGET_IO_PER_COMPACTION_IN_MB, String.valueOf(targetIOPerCompactionInMB)); return this; @@ -701,11 +356,6 @@ public Builder withMaxNumDeltaCommitsBeforeCompaction(int maxNumDeltaCommitsBefo return this; } - public Builder withArchiveDeleteParallelism(int archiveDeleteParallelism) { - compactionConfig.setValue(DELETE_ARCHIVED_INSTANT_PARALLELISM_VALUE, String.valueOf(archiveDeleteParallelism)); - return this; - } - public Builder withMaxDeltaSecondsBeforeCompaction(int maxDeltaSecondsBeforeCompaction) { compactionConfig.setValue(INLINE_COMPACT_TIME_DELTA_SECONDS, String.valueOf(maxDeltaSecondsBeforeCompaction)); return this; @@ -736,49 +386,8 @@ public Builder withPreserveCommitMetadata(boolean preserveCommitMetadata) { return this; } - public Builder withCommitsArchivalBatchSize(int batchSize) { - compactionConfig.setValue(COMMITS_ARCHIVAL_BATCH_SIZE, String.valueOf(batchSize)); - return this; - } - - public Builder withCleanBootstrapBaseFileEnabled(Boolean cleanBootstrapSourceFileEnabled) { - compactionConfig.setValue(CLEANER_BOOTSTRAP_BASE_FILE_ENABLE, String.valueOf(cleanBootstrapSourceFileEnabled)); - return this; - } - - public Builder withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy failedWritesPolicy) { - compactionConfig.setValue(FAILED_WRITES_CLEANER_POLICY, failedWritesPolicy.name()); - return this; - } - public HoodieCompactionConfig build() { compactionConfig.setDefaults(HoodieCompactionConfig.class.getName()); - // validation - HoodieCleaningPolicy.valueOf(compactionConfig.getString(CLEANER_POLICY)); - - // Ensure minInstantsToKeep > cleanerCommitsRetained, otherwise we will archive some - // commit instant on timeline, that still has not been cleaned. Could miss some data via incr pull - int minInstantsToKeep = Integer.parseInt(compactionConfig.getStringOrDefault(HoodieCompactionConfig.MIN_COMMITS_TO_KEEP)); - int maxInstantsToKeep = Integer.parseInt(compactionConfig.getStringOrDefault(HoodieCompactionConfig.MAX_COMMITS_TO_KEEP)); - int cleanerCommitsRetained = - Integer.parseInt(compactionConfig.getStringOrDefault(HoodieCompactionConfig.CLEANER_COMMITS_RETAINED)); - ValidationUtils.checkArgument(maxInstantsToKeep > minInstantsToKeep, - String.format( - "Increase %s=%d to be greater than %s=%d.", - HoodieCompactionConfig.MAX_COMMITS_TO_KEEP.key(), maxInstantsToKeep, - HoodieCompactionConfig.MIN_COMMITS_TO_KEEP.key(), minInstantsToKeep)); - ValidationUtils.checkArgument(minInstantsToKeep > cleanerCommitsRetained, - String.format( - "Increase %s=%d to be greater than %s=%d. Otherwise, there is risk of incremental pull " - + "missing data from few instants.", - HoodieCompactionConfig.MIN_COMMITS_TO_KEEP.key(), minInstantsToKeep, - HoodieCompactionConfig.CLEANER_COMMITS_RETAINED.key(), cleanerCommitsRetained)); - - boolean inlineCompact = compactionConfig.getBoolean(HoodieCompactionConfig.INLINE_COMPACT); - boolean inlineCompactSchedule = compactionConfig.getBoolean(HoodieCompactionConfig.SCHEDULE_INLINE_COMPACT); - ValidationUtils.checkArgument(!(inlineCompact && inlineCompactSchedule), String.format("Either of inline compaction (%s) or " - + "schedule inline compaction (%s) can be enabled. Both can't be set to true at the same time. %s, %s", HoodieCompactionConfig.INLINE_COMPACT.key(), - HoodieCompactionConfig.SCHEDULE_INLINE_COMPACT.key(), inlineCompact, inlineCompactSchedule)); return compactionConfig; } } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodiePayloadConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodiePayloadConfig.java index 2989d8c2b36d9..2a05752aa6e1e 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodiePayloadConfig.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodiePayloadConfig.java @@ -22,6 +22,7 @@ import org.apache.hudi.common.config.ConfigGroups; import org.apache.hudi.common.config.ConfigProperty; import org.apache.hudi.common.config.HoodieConfig; +import org.apache.hudi.common.model.OverwriteWithLatestAvroPayload; import java.io.File; import java.io.FileReader; @@ -52,6 +53,20 @@ public class HoodiePayloadConfig extends HoodieConfig { .withDocumentation("Table column/field name to derive timestamp associated with the records. This can" + "be useful for e.g, determining the freshness of the table."); + public static final ConfigProperty PAYLOAD_CLASS_NAME = ConfigProperty + .key("hoodie.compaction.payload.class") + .defaultValue(OverwriteWithLatestAvroPayload.class.getName()) + .withDocumentation("This needs to be same as class used during insert/upserts. Just like writing, compaction also uses " + + "the record payload class to merge records in the log against each other, merge again with the base file and " + + "produce the final record to be written after compaction."); + + /** @deprecated Use {@link #PAYLOAD_CLASS_NAME} and its methods instead */ + @Deprecated + public static final String DEFAULT_PAYLOAD_CLASS = PAYLOAD_CLASS_NAME.defaultValue(); + /** @deprecated Use {@link #PAYLOAD_CLASS_NAME} and its methods instead */ + @Deprecated + public static final String PAYLOAD_CLASS_PROP = PAYLOAD_CLASS_NAME.key(); + private HoodiePayloadConfig() { super(); } @@ -86,6 +101,11 @@ public Builder withPayloadEventTimeField(String payloadEventTimeField) { return this; } + public HoodiePayloadConfig.Builder withPayloadClass(String payloadClassName) { + payloadConfig.setValue(PAYLOAD_CLASS_NAME, payloadClassName); + return this; + } + public HoodiePayloadConfig build() { payloadConfig.setDefaults(HoodiePayloadConfig.class.getName()); return payloadConfig; diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieStorageConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieStorageConfig.java index ba3888863d557..fc1798f206fbc 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieStorageConfig.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieStorageConfig.java @@ -119,16 +119,16 @@ public class HoodieStorageConfig extends HoodieConfig { .withDocumentation("Whether to use dictionary encoding"); public static final ConfigProperty PARQUET_WRITE_LEGACY_FORMAT_ENABLED = ConfigProperty - .key("hoodie.parquet.writelegacyformat.enabled") - .defaultValue("false") - .withDocumentation("Sets spark.sql.parquet.writeLegacyFormat. If true, data will be written in a way of Spark 1.4 and earlier. " - + "For example, decimal values will be written in Parquet's fixed-length byte array format which other systems such as Apache Hive and Apache Impala use. " - + "If false, the newer format in Parquet will be used. For example, decimals will be written in int-based format."); + .key("hoodie.parquet.writelegacyformat.enabled") + .defaultValue("false") + .withDocumentation("Sets spark.sql.parquet.writeLegacyFormat. If true, data will be written in a way of Spark 1.4 and earlier. " + + "For example, decimal values will be written in Parquet's fixed-length byte array format which other systems such as Apache Hive and Apache Impala use. " + + "If false, the newer format in Parquet will be used. For example, decimals will be written in int-based format."); public static final ConfigProperty PARQUET_OUTPUT_TIMESTAMP_TYPE = ConfigProperty - .key("hoodie.parquet.outputtimestamptype") - .defaultValue("TIMESTAMP_MICROS") - .withDocumentation("Sets spark.sql.parquet.outputTimestampType. Parquet timestamp type to use when Spark writes data to Parquet files."); + .key("hoodie.parquet.outputtimestamptype") + .defaultValue("TIMESTAMP_MICROS") + .withDocumentation("Sets spark.sql.parquet.outputTimestampType. Parquet timestamp type to use when Spark writes data to Parquet files."); public static final ConfigProperty HFILE_COMPRESSION_ALGORITHM_NAME = ConfigProperty .key("hoodie.hfile.compression.algorithm") 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 d18238fa4b6b8..4d07097c07c88 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 @@ -89,6 +89,8 @@ import java.util.function.Supplier; import java.util.stream.Collectors; +import static org.apache.hudi.config.HoodieCleanConfig.CLEANER_POLICY; + /** * Class storing configs for the HoodieWriteClient. */ @@ -1148,31 +1150,31 @@ public boolean populateMetaFields() { * compaction properties. */ public HoodieCleaningPolicy getCleanerPolicy() { - return HoodieCleaningPolicy.valueOf(getString(HoodieCompactionConfig.CLEANER_POLICY)); + return HoodieCleaningPolicy.valueOf(getString(CLEANER_POLICY)); } public int getCleanerFileVersionsRetained() { - return getInt(HoodieCompactionConfig.CLEANER_FILE_VERSIONS_RETAINED); + return getInt(HoodieCleanConfig.CLEANER_FILE_VERSIONS_RETAINED); } public int getCleanerCommitsRetained() { - return getInt(HoodieCompactionConfig.CLEANER_COMMITS_RETAINED); + return getInt(HoodieCleanConfig.CLEANER_COMMITS_RETAINED); } public int getCleanerHoursRetained() { - return getInt(HoodieCompactionConfig.CLEANER_HOURS_RETAINED); + return getInt(HoodieCleanConfig.CLEANER_HOURS_RETAINED); } public int getMaxCommitsToKeep() { - return getInt(HoodieCompactionConfig.MAX_COMMITS_TO_KEEP); + return getInt(HoodieArchivalConfig.MAX_COMMITS_TO_KEEP); } public int getMinCommitsToKeep() { - return getInt(HoodieCompactionConfig.MIN_COMMITS_TO_KEEP); + return getInt(HoodieArchivalConfig.MIN_COMMITS_TO_KEEP); } public int getArchiveMergeFilesBatchSize() { - return getInt(HoodieCompactionConfig.ARCHIVE_MERGE_FILES_BATCH_SIZE); + return getInt(HoodieArchivalConfig.ARCHIVE_MERGE_FILES_BATCH_SIZE); } public int getParquetSmallFileLimit() { @@ -1192,7 +1194,7 @@ public int getCopyOnWriteRecordSizeEstimate() { } public boolean allowMultipleCleans() { - return getBoolean(HoodieCompactionConfig.ALLOW_MULTIPLE_CLEANS); + return getBoolean(HoodieCleanConfig.ALLOW_MULTIPLE_CLEANS); } public boolean shouldAutoTuneInsertSplits() { @@ -1200,43 +1202,43 @@ public boolean shouldAutoTuneInsertSplits() { } public int getCleanerParallelism() { - return getInt(HoodieCompactionConfig.CLEANER_PARALLELISM_VALUE); + return getInt(HoodieCleanConfig.CLEANER_PARALLELISM_VALUE); } public int getCleaningMaxCommits() { - return getInt(HoodieCompactionConfig.CLEAN_MAX_COMMITS); + return getInt(HoodieCleanConfig.CLEAN_MAX_COMMITS); } public CleaningTriggerStrategy getCleaningTriggerStrategy() { - return CleaningTriggerStrategy.valueOf(getString(HoodieCompactionConfig.CLEAN_TRIGGER_STRATEGY)); + return CleaningTriggerStrategy.valueOf(getString(HoodieCleanConfig.CLEAN_TRIGGER_STRATEGY)); } public boolean isAutoClean() { - return getBoolean(HoodieCompactionConfig.AUTO_CLEAN); + return getBoolean(HoodieCleanConfig.AUTO_CLEAN); } public boolean getArchiveMergeEnable() { - return getBoolean(HoodieCompactionConfig.ARCHIVE_MERGE_ENABLE); + return getBoolean(HoodieArchivalConfig.ARCHIVE_MERGE_ENABLE); } public long getArchiveMergeSmallFileLimitBytes() { - return getLong(HoodieCompactionConfig.ARCHIVE_MERGE_SMALL_FILE_LIMIT_BYTES); + return getLong(HoodieArchivalConfig.ARCHIVE_MERGE_SMALL_FILE_LIMIT_BYTES); } public boolean isAutoArchive() { - return getBoolean(HoodieCompactionConfig.AUTO_ARCHIVE); + return getBoolean(HoodieArchivalConfig.AUTO_ARCHIVE); } public boolean isAsyncArchive() { - return getBoolean(HoodieCompactionConfig.ASYNC_ARCHIVE); + return getBoolean(HoodieArchivalConfig.ASYNC_ARCHIVE); } public boolean isAsyncClean() { - return getBoolean(HoodieCompactionConfig.ASYNC_CLEAN); + return getBoolean(HoodieCleanConfig.ASYNC_CLEAN); } public boolean incrementalCleanerModeEnabled() { - return getBoolean(HoodieCompactionConfig.CLEANER_INCREMENTAL_MODE_ENABLE); + return getBoolean(HoodieCleanConfig.CLEANER_INCREMENTAL_MODE_ENABLE); } public boolean inlineCompactionEnabled() { @@ -1280,7 +1282,7 @@ public Boolean getCompactionReverseLogReadEnabled() { } public int getArchiveDeleteParallelism() { - return getInt(HoodieCompactionConfig.DELETE_ARCHIVED_INSTANT_PARALLELISM_VALUE); + return getInt(HoodieArchivalConfig.DELETE_ARCHIVED_INSTANT_PARALLELISM_VALUE); } public boolean inlineClusteringEnabled() { @@ -1321,7 +1323,7 @@ public int getAsyncClusterMaxCommits() { } public String getPayloadClass() { - return getString(HoodieCompactionConfig.PAYLOAD_CLASS_NAME); + return getString(HoodiePayloadConfig.PAYLOAD_CLASS_NAME); } public int getTargetPartitionsPerDayBasedCompaction() { @@ -1329,11 +1331,11 @@ public int getTargetPartitionsPerDayBasedCompaction() { } public int getCommitArchivalBatchSize() { - return getInt(HoodieCompactionConfig.COMMITS_ARCHIVAL_BATCH_SIZE); + return getInt(HoodieArchivalConfig.COMMITS_ARCHIVAL_BATCH_SIZE); } public Boolean shouldCleanBootstrapBaseFile() { - return getBoolean(HoodieCompactionConfig.CLEANER_BOOTSTRAP_BASE_FILE_ENABLE); + return getBoolean(HoodieCleanConfig.CLEANER_BOOTSTRAP_BASE_FILE_ENABLE); } public String getClusteringUpdatesStrategyClass() { @@ -1342,7 +1344,7 @@ public String getClusteringUpdatesStrategyClass() { public HoodieFailedWritesCleaningPolicy getFailedWritesCleanPolicy() { return HoodieFailedWritesCleaningPolicy - .valueOf(getString(HoodieCompactionConfig.FAILED_WRITES_CLEANER_POLICY)); + .valueOf(getString(HoodieCleanConfig.FAILED_WRITES_CLEANER_POLICY)); } /** @@ -2117,6 +2119,8 @@ public static class Builder { private boolean isIndexConfigSet = false; private boolean isStorageConfigSet = false; private boolean isCompactionConfigSet = false; + private boolean isCleanConfigSet = false; + private boolean isArchivalConfigSet = false; private boolean isClusteringConfigSet = false; private boolean isOptimizeConfigSet = false; private boolean isMetricsConfigSet = false; @@ -2284,6 +2288,18 @@ public Builder withCompactionConfig(HoodieCompactionConfig compactionConfig) { return this; } + public Builder withCleanConfig(HoodieCleanConfig cleanConfig) { + writeConfig.getProps().putAll(cleanConfig.getProps()); + isCleanConfigSet = true; + return this; + } + + public Builder withArchivalConfig(HoodieArchivalConfig cleanConfig) { + writeConfig.getProps().putAll(cleanConfig.getProps()); + isArchivalConfigSet = true; + return this; + } + public Builder withClusteringConfig(HoodieClusteringConfig clusteringConfig) { writeConfig.getProps().putAll(clusteringConfig.getProps()); isClusteringConfigSet = true; @@ -2517,6 +2533,10 @@ protected void setDefaults() { writeConfig.getProps()).build()); writeConfig.setDefaultOnCondition(!isCompactionConfigSet, HoodieCompactionConfig.newBuilder().fromProperties(writeConfig.getProps()).build()); + writeConfig.setDefaultOnCondition(!isCleanConfigSet, + HoodieCleanConfig.newBuilder().fromProperties(writeConfig.getProps()).build()); + writeConfig.setDefaultOnCondition(!isArchivalConfigSet, + HoodieArchivalConfig.newBuilder().fromProperties(writeConfig.getProps()).build()); writeConfig.setDefaultOnCondition(!isClusteringConfigSet, HoodieClusteringConfig.newBuilder().withEngineType(engineType) .fromProperties(writeConfig.getProps()).build()); @@ -2587,10 +2607,10 @@ private void autoAdjustConfigsForConcurrencyMode(boolean isLockProviderPropertyS if (WriteConcurrencyMode.OPTIMISTIC_CONCURRENCY_CONTROL.value() .equalsIgnoreCase(writeConcurrencyMode)) { // In this case, we assume that the user takes care of setting the lock provider used - writeConfig.setValue(HoodieCompactionConfig.FAILED_WRITES_CLEANER_POLICY.key(), + writeConfig.setValue(HoodieCleanConfig.FAILED_WRITES_CLEANER_POLICY.key(), HoodieFailedWritesCleaningPolicy.LAZY.name()); LOG.info(String.format("Automatically set %s=%s since optimistic concurrency control is used", - HoodieCompactionConfig.FAILED_WRITES_CLEANER_POLICY.key(), + HoodieCleanConfig.FAILED_WRITES_CLEANER_POLICY.key(), HoodieFailedWritesCleaningPolicy.LAZY.name())); } } @@ -2602,9 +2622,34 @@ private void validate() { Objects.requireNonNull(writeConfig.getString(BASE_PATH)); if (writeConfig.getString(WRITE_CONCURRENCY_MODE) .equalsIgnoreCase(WriteConcurrencyMode.OPTIMISTIC_CONCURRENCY_CONTROL.value())) { - ValidationUtils.checkArgument(!writeConfig.getString(HoodieCompactionConfig.FAILED_WRITES_CLEANER_POLICY) + ValidationUtils.checkArgument(!writeConfig.getString(HoodieCleanConfig.FAILED_WRITES_CLEANER_POLICY) .equals(HoodieFailedWritesCleaningPolicy.EAGER.name()), "To enable optimistic concurrency control, set hoodie.cleaner.policy.failed.writes=LAZY"); } + + HoodieCleaningPolicy.valueOf(writeConfig.getString(CLEANER_POLICY)); + // Ensure minInstantsToKeep > cleanerCommitsRetained, otherwise we will archive some + // commit instant on timeline, that still has not been cleaned. Could miss some data via incr pull + int minInstantsToKeep = Integer.parseInt(writeConfig.getStringOrDefault(HoodieArchivalConfig.MIN_COMMITS_TO_KEEP)); + int maxInstantsToKeep = Integer.parseInt(writeConfig.getStringOrDefault(HoodieArchivalConfig.MAX_COMMITS_TO_KEEP)); + int cleanerCommitsRetained = + Integer.parseInt(writeConfig.getStringOrDefault(HoodieCleanConfig.CLEANER_COMMITS_RETAINED)); + ValidationUtils.checkArgument(maxInstantsToKeep > minInstantsToKeep, + String.format( + "Increase %s=%d to be greater than %s=%d.", + HoodieArchivalConfig.MAX_COMMITS_TO_KEEP.key(), maxInstantsToKeep, + HoodieArchivalConfig.MIN_COMMITS_TO_KEEP.key(), minInstantsToKeep)); + ValidationUtils.checkArgument(minInstantsToKeep > cleanerCommitsRetained, + String.format( + "Increase %s=%d to be greater than %s=%d. Otherwise, there is risk of incremental pull " + + "missing data from few instants.", + HoodieArchivalConfig.MIN_COMMITS_TO_KEEP.key(), minInstantsToKeep, + HoodieCleanConfig.CLEANER_COMMITS_RETAINED.key(), cleanerCommitsRetained)); + + boolean inlineCompact = writeConfig.getBoolean(HoodieCompactionConfig.INLINE_COMPACT); + boolean inlineCompactSchedule = writeConfig.getBoolean(HoodieCompactionConfig.SCHEDULE_INLINE_COMPACT); + ValidationUtils.checkArgument(!(inlineCompact && inlineCompactSchedule), String.format("Either of inline compaction (%s) or " + + "schedule inline compaction (%s) can be enabled. Both can't be set to true at the same time. %s, %s", HoodieCompactionConfig.INLINE_COMPACT.key(), + HoodieCompactionConfig.SCHEDULE_INLINE_COMPACT.key(), inlineCompact, inlineCompactSchedule)); } public HoodieWriteConfig build() { diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java index e36adf6be53d3..f4ee3fc9f2424 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java @@ -58,6 +58,8 @@ import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.ValidationUtils; import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.config.HoodieArchivalConfig; +import org.apache.hudi.config.HoodieCleanConfig; import org.apache.hudi.config.HoodieCompactionConfig; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.config.metrics.HoodieMetricsConfig; @@ -255,20 +257,24 @@ private HoodieWriteConfig createMetadataWriteConfig(HoodieWriteConfig writeConfi .withPath(HoodieTableMetadata.getMetadataTableBasePath(writeConfig.getBasePath())) .withSchema(HoodieMetadataRecord.getClassSchema().toString()) .forTable(tableName) - .withCompactionConfig(HoodieCompactionConfig.newBuilder() + // we will trigger cleaning manually, to control the instant times + .withCleanConfig(HoodieCleanConfig.newBuilder() .withAsyncClean(writeConfig.isMetadataAsyncClean()) - // we will trigger cleaning manually, to control the instant times .withAutoClean(false) .withCleanerParallelism(parallelism) .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS) .withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.LAZY) .retainCommits(writeConfig.getMetadataCleanerCommitsRetained()) + .build()) + // we will trigger archive manually, to ensure only regular writer invokes it + .withArchivalConfig(HoodieArchivalConfig.newBuilder() .archiveCommitsWith(minCommitsToKeep, maxCommitsToKeep) - // we will trigger compaction manually, to control the instant times + .withAutoArchive(false) + .build()) + // we will trigger compaction manually, to control the instant times + .withCompactionConfig(HoodieCompactionConfig.newBuilder() .withInlineCompaction(false) .withMaxNumDeltaCommitsBeforeCompaction(writeConfig.getMetadataCompactDeltaCommitMax()) - // we will trigger archive manually, to ensure only regular writer invokes it - .withAutoArchive(false) // by default, the HFile does not keep the metadata fields, set up as false // to always use the metadata of the new record. .withPreserveCommitMetadata(false) diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/HoodieMergeHelper.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/HoodieMergeHelper.java index 3e2d8abdd7466..5d1a55453d162 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/HoodieMergeHelper.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/HoodieMergeHelper.java @@ -100,7 +100,7 @@ public void runMerge(HoodieTable>, HoodieData params = new HashMap<>(3); - params.put(HoodieCompactionConfig.CLEANER_COMMITS_RETAINED.key(), "1"); - params.put(HoodieCompactionConfig.MAX_COMMITS_TO_KEEP.key(), "5"); - params.put(HoodieCompactionConfig.MIN_COMMITS_TO_KEEP.key(), "2"); + params.put(HoodieCleanConfig.CLEANER_COMMITS_RETAINED.key(), "1"); + params.put(HoodieArchivalConfig.MAX_COMMITS_TO_KEEP.key(), "5"); + params.put(HoodieArchivalConfig.MIN_COMMITS_TO_KEEP.key(), "2"); if (withAlternative) { params.put("hoodie.avro.schema.externalTransformation", "true"); } else { diff --git a/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/HoodieSparkUtils.scala b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/HoodieSparkUtils.scala index 7a8f8a1580d97..49827166258ba 100644 --- a/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/HoodieSparkUtils.scala +++ b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/HoodieSparkUtils.scala @@ -21,7 +21,6 @@ package org.apache.hudi import org.apache.avro.Schema import org.apache.avro.generic.GenericRecord import org.apache.hadoop.fs.{FileSystem, Path} -import org.apache.hudi.avro.HoodieAvroUtils.rewriteRecord import org.apache.hudi.client.utils.SparkRowSerDe import org.apache.hudi.common.config.TypedProperties import org.apache.hudi.common.model.HoodieRecord @@ -39,8 +38,10 @@ import org.apache.spark.sql.catalyst.encoders.RowEncoder import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression, Literal} import org.apache.spark.sql.sources._ import org.apache.spark.sql.types.{StringType, StructField, StructType} - import java.util.Properties + +import org.apache.hudi.avro.HoodieAvroUtils + import scala.collection.JavaConverters._ object HoodieSparkUtils extends SparkAdapterSupport { @@ -162,11 +163,11 @@ object HoodieSparkUtils extends SparkAdapterSupport { if (rows.isEmpty) { Iterator.empty } else { + val readerAvroSchema = new Schema.Parser().parse(readerAvroSchemaStr) val transform: GenericRecord => GenericRecord = if (sameSchema) identity else { - val readerAvroSchema = new Schema.Parser().parse(readerAvroSchemaStr) - rewriteRecord(_, readerAvroSchema) + HoodieAvroUtils.rewriteRecordDeep(_, readerAvroSchema) } // Since caller might request to get records in a different ("evolved") schema, we will be rewriting from diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestClientRollback.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestClientRollback.java index f6315eec7d211..629b16cdb88f0 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestClientRollback.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestClientRollback.java @@ -38,7 +38,7 @@ import org.apache.hudi.common.testutils.HoodieTestDataGenerator; import org.apache.hudi.common.testutils.HoodieTestTable; import org.apache.hudi.common.util.collection.Pair; -import org.apache.hudi.config.HoodieCompactionConfig; +import org.apache.hudi.config.HoodieCleanConfig; import org.apache.hudi.config.HoodieIndexConfig; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieRollbackException; @@ -80,7 +80,7 @@ public class TestClientRollback extends HoodieClientTestBase { */ @Test public void testSavepointAndRollback() throws Exception { - HoodieWriteConfig cfg = getConfigBuilder().withCompactionConfig(HoodieCompactionConfig.newBuilder() + HoodieWriteConfig cfg = getConfigBuilder().withCleanConfig(HoodieCleanConfig.newBuilder() .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS).retainCommits(1).build()).build(); try (SparkRDDWriteClient client = getHoodieWriteClient(cfg)) { HoodieTestDataGenerator.writePartitionMetadataDeprecated(fs, HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS, basePath); @@ -214,7 +214,7 @@ public void testRollbackCommit() throws Exception { HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath) .withRollbackUsingMarkers(false) - .withCompactionConfig(HoodieCompactionConfig.newBuilder() + .withCleanConfig(HoodieCleanConfig.newBuilder() .withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.LAZY).build()) .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.INMEMORY).build()).build(); @@ -329,7 +329,7 @@ public void testFailedRollbackCommit( .enable(enableMetadataTable) .build() ) - .withCompactionConfig(HoodieCompactionConfig.newBuilder() + .withCleanConfig(HoodieCleanConfig.newBuilder() .withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.LAZY).build()) .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.INMEMORY).build()).build(); @@ -436,7 +436,7 @@ public void testAutoRollbackInflightCommit() throws Exception { // Set Failed Writes rollback to LAZY HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath) .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.INMEMORY).build()) - .withCompactionConfig(HoodieCompactionConfig.newBuilder() + .withCleanConfig(HoodieCleanConfig.newBuilder() .withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.LAZY).build()).build(); HoodieTableMetadataWriter metadataWriter = SparkHoodieBackedTableMetadataWriter.create(hadoopConf, config, context); @@ -530,7 +530,7 @@ public void testRollbackWithRequestedRollbackPlan(boolean enableMetadataTable, b .enable(enableMetadataTable) .build() ) - .withCompactionConfig(HoodieCompactionConfig.newBuilder() + .withCleanConfig(HoodieCleanConfig.newBuilder() .withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.LAZY).build()) .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.INMEMORY).build()).build(); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestHoodieClientMultiWriter.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestHoodieClientMultiWriter.java index 3aeca0f275891..268674e78d87a 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestHoodieClientMultiWriter.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestHoodieClientMultiWriter.java @@ -33,6 +33,8 @@ import org.apache.hudi.common.testutils.HoodieTestUtils; import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieClusteringConfig; +import org.apache.hudi.config.HoodieCleanConfig; +import org.apache.hudi.config.HoodieArchivalConfig; import org.apache.hudi.config.HoodieCompactionConfig; import org.apache.hudi.config.HoodieLockConfig; import org.apache.hudi.config.HoodieWriteConfig; @@ -100,9 +102,11 @@ public void testHoodieClientBasicMultiWriter(HoodieTableType tableType) throws E properties.setProperty(FILESYSTEM_LOCK_PATH_PROP_KEY, basePath + "/.hoodie/.locks"); properties.setProperty(LockConfiguration.LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP_KEY, "3000"); HoodieWriteConfig writeConfig = getConfigBuilder() - .withCompactionConfig(HoodieCompactionConfig.newBuilder() + .withCleanConfig(HoodieCleanConfig.newBuilder() .withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.LAZY) - .withAutoArchive(false).withAutoClean(false).build()) + .withAutoClean(false).build()) + .withArchivalConfig(HoodieArchivalConfig.newBuilder() + .withAutoArchive(false).build()) .withWriteConcurrencyMode(WriteConcurrencyMode.OPTIMISTIC_CONCURRENCY_CONTROL) // Timeline-server-based markers are not used for multi-writer tests .withMarkersType(MarkerType.DIRECT.name()) @@ -192,9 +196,11 @@ public void testMultiWriterWithInsertsToDistinctPartitions(HoodieTableType table properties.setProperty(LockConfiguration.LOCK_ACQUIRE_CLIENT_NUM_RETRIES_PROP_KEY, "20"); HoodieWriteConfig cfg = getConfigBuilder() + .withCleanConfig(HoodieCleanConfig.newBuilder() + .withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.LAZY) + .build()) .withCompactionConfig(HoodieCompactionConfig.newBuilder() .withInlineCompaction(false) - .withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.LAZY) .withMaxNumDeltaCommitsBeforeCompaction(2) .build()) .withWriteConcurrencyMode(WriteConcurrencyMode.OPTIMISTIC_CONCURRENCY_CONTROL) @@ -265,9 +271,12 @@ private void testMultiWriterWithAsyncTableServicesWithConflict(HoodieTableType t properties.setProperty(LockConfiguration.LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP_KEY, "3000"); // Disabling embedded timeline server, it doesn't work with multiwriter HoodieWriteConfig.Builder writeConfigBuilder = getConfigBuilder() - .withCompactionConfig(HoodieCompactionConfig.newBuilder().withAutoClean(false) - .withInlineCompaction(false).withAsyncClean(true) - .withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.LAZY) + .withCleanConfig(HoodieCleanConfig.newBuilder() + .withAutoClean(false) + .withAsyncClean(true) + .withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.LAZY).build()) + .withCompactionConfig(HoodieCompactionConfig.newBuilder() + .withInlineCompaction(false) .withMaxNumDeltaCommitsBeforeCompaction(2).build()) .withEmbeddedTimelineServerEnabled(false) // Timeline-server-based markers are not used for multi-writer tests @@ -402,7 +411,8 @@ public void testHoodieClientMultiWriterWithClustering(HoodieTableType tableType) properties.setProperty(FILESYSTEM_LOCK_PATH_PROP_KEY, basePath + "/.hoodie/.locks"); properties.setProperty(LockConfiguration.LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP_KEY, "3000"); HoodieWriteConfig.Builder writeConfigBuilder = getConfigBuilder() - .withCompactionConfig(HoodieCompactionConfig.newBuilder().withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.LAZY) + .withCleanConfig(HoodieCleanConfig.newBuilder() + .withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.LAZY) .withAutoClean(false).build()) .withWriteConcurrencyMode(WriteConcurrencyMode.OPTIMISTIC_CONCURRENCY_CONTROL) // Timeline-server-based markers are not used for multi-writer tests @@ -453,7 +463,8 @@ public void testHoodieClientMultiWriterAutoCommitForConflict() throws Exception properties.setProperty(LockConfiguration.LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP_KEY, "3000"); properties.setProperty(LockConfiguration.LOCK_ACQUIRE_CLIENT_NUM_RETRIES_PROP_KEY, "100"); HoodieWriteConfig.Builder writeConfigBuilder = getConfigBuilder() - .withCompactionConfig(HoodieCompactionConfig.newBuilder().withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.LAZY) + .withCleanConfig(HoodieCleanConfig.newBuilder() + .withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.LAZY) .withAutoClean(false).build()) .withWriteConcurrencyMode(WriteConcurrencyMode.OPTIMISTIC_CONCURRENCY_CONTROL) // Timeline-server-based markers are not used for multi-writer tests @@ -536,7 +547,8 @@ public void testHoodieClientMultiWriterAutoCommitNonConflict() throws Exception properties.setProperty(LockConfiguration.LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP_KEY, "3000"); properties.setProperty(LockConfiguration.LOCK_ACQUIRE_CLIENT_NUM_RETRIES_PROP_KEY, "100"); HoodieWriteConfig.Builder writeConfigBuilder = getConfigBuilder() - .withCompactionConfig(HoodieCompactionConfig.newBuilder().withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.LAZY) + .withCleanConfig(HoodieCleanConfig.newBuilder() + .withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.LAZY) .withAutoClean(false).build()) .withWriteConcurrencyMode(WriteConcurrencyMode.OPTIMISTIC_CONCURRENCY_CONTROL) // Timeline-server-based markers are not used for multi-writer tests diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieBackedMetadata.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieBackedMetadata.java index 1a618a01dfe24..6d410ded1ccca 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieBackedMetadata.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieBackedMetadata.java @@ -82,11 +82,13 @@ import org.apache.hudi.common.util.hash.ColumnIndexID; import org.apache.hudi.common.util.hash.PartitionIndexID; import org.apache.hudi.config.HoodieClusteringConfig; +import org.apache.hudi.config.HoodieCleanConfig; +import org.apache.hudi.config.HoodieArchivalConfig; import org.apache.hudi.config.HoodieCompactionConfig; -import org.apache.hudi.config.HoodieIndexConfig; import org.apache.hudi.config.HoodieLockConfig; -import org.apache.hudi.config.HoodieStorageConfig; +import org.apache.hudi.config.HoodieIndexConfig; import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.config.HoodieStorageConfig; import org.apache.hudi.exception.HoodieMetadataException; import org.apache.hudi.index.HoodieIndex; import org.apache.hudi.io.storage.HoodieHFileReader; @@ -476,7 +478,13 @@ public void testMetadataTableArchival() throws Exception { .archiveCommitsWith(3, 4) .retainCommits(1) .build()) - .withCompactionConfig(HoodieCompactionConfig.newBuilder().archiveCommitsWith(2, 3).retainCommits(1).build()).build(); + .withCleanConfig(HoodieCleanConfig.newBuilder() + .retainCommits(1) + .build()) + .withArchivalConfig(HoodieArchivalConfig.newBuilder() + .archiveCommitsWith(2, 3) + .build()) + .build(); initWriteConfigAndMetatableWriter(writeConfig, true); AtomicInteger commitTime = new AtomicInteger(1); @@ -637,8 +645,9 @@ public void testMetadataTableDeletePartition(HoodieTableType tableType) throws I initPath(); int maxCommits = 1; HoodieWriteConfig cfg = getConfigBuilder(TRIP_EXAMPLE_SCHEMA, HoodieIndex.IndexType.BLOOM, HoodieFailedWritesCleaningPolicy.EAGER) - .withCompactionConfig(HoodieCompactionConfig.newBuilder() - .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS).retainCommits(maxCommits).build()) + .withCleanConfig(HoodieCleanConfig.newBuilder() + .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS).retainCommits(maxCommits) + .build()) .withParallelism(1, 1).withBulkInsertParallelism(1).withFinalizeWriteParallelism(1).withDeleteParallelism(1) .withConsistencyGuardConfig(ConsistencyGuardConfig.newBuilder().withConsistencyCheckEnabled(true).build()) .withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(true).build()) @@ -1172,8 +1181,15 @@ public void testManualRollbacks(final boolean populateMateFields) throws Excepti .withMaxNumDeltaCommitsBeforeCompaction(maxDeltaCommitsBeforeCompaction) .withPopulateMetaFields(populateMateFields) .build()) - .withCompactionConfig(HoodieCompactionConfig.newBuilder().archiveCommitsWith(minArchiveCommitsDataset, minArchiveCommitsDataset + 1) - .retainCommits(1).retainFileVersions(1).withAutoClean(false).withAsyncClean(true).build()) + .withCleanConfig(HoodieCleanConfig.newBuilder() + .retainCommits(1) + .retainFileVersions(1) + .withAutoClean(false) + .withAsyncClean(true) + .build()) + .withArchivalConfig(HoodieArchivalConfig.newBuilder() + .archiveCommitsWith(minArchiveCommitsDataset, minArchiveCommitsDataset + 1) + .build()) .build(); initWriteConfigAndMetatableWriter(writeConfig, true); @@ -1399,10 +1415,13 @@ public void testColStatsPrefixLookup() throws IOException { // disable small file handling so that every insert goes to a new file group. HoodieWriteConfig writeConfig = getWriteConfigBuilder(true, true, false) .withRollbackUsingMarkers(false) + .withCleanConfig(HoodieCleanConfig.newBuilder() + .withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.EAGER) + .withAutoClean(false).retainCommits(1).retainFileVersions(1) + .build()) .withCompactionConfig(HoodieCompactionConfig.newBuilder().compactionSmallFileSize(0) .withInlineCompaction(false).withMaxNumDeltaCommitsBeforeCompaction(1) - .withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.EAGER) - .withAutoClean(false).retainCommits(1).retainFileVersions(1).build()) + .build()) .withMetadataConfig(HoodieMetadataConfig.newBuilder() .enable(true) .withMetadataIndexColumnStats(true) @@ -1612,7 +1631,7 @@ public void testMetadataMultiWriter() throws Exception { properties.setProperty(LockConfiguration.LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP_KEY, "1000"); properties.setProperty(LockConfiguration.LOCK_ACQUIRE_CLIENT_NUM_RETRIES_PROP_KEY, "20"); HoodieWriteConfig writeConfig = getWriteConfigBuilder(true, true, false) - .withCompactionConfig(HoodieCompactionConfig.newBuilder() + .withCleanConfig(HoodieCleanConfig.newBuilder() .withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.LAZY).withAutoClean(false).build()) .withWriteConcurrencyMode(WriteConcurrencyMode.OPTIMISTIC_CONCURRENCY_CONTROL) .withLockConfig(HoodieLockConfig.newBuilder().withLockProvider(InProcessLockProvider.class).build()) @@ -1676,8 +1695,9 @@ public void testMultiWriterForDoubleLocking() throws Exception { properties.setProperty(LockConfiguration.LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP_KEY, "3000"); HoodieWriteConfig writeConfig = getWriteConfigBuilder(true, true, false) - .withCompactionConfig(HoodieCompactionConfig.newBuilder() - .withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.LAZY).withAutoClean(true).retainCommits(4).build()) + .withCleanConfig(HoodieCleanConfig.newBuilder() + .withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.LAZY).withAutoClean(true).retainCommits(4) + .build()) .withAutoCommit(false) .withWriteConcurrencyMode(WriteConcurrencyMode.OPTIMISTIC_CONCURRENCY_CONTROL) .withLockConfig(HoodieLockConfig.newBuilder().withLockProvider(InProcessLockProvider.class).build()) @@ -1853,9 +1873,12 @@ public void testCleaningArchivingAndCompaction() throws Exception { .withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(true) .archiveCommitsWith(40, 60).retainCommits(1) .withMaxNumDeltaCommitsBeforeCompaction(maxDeltaCommitsBeforeCompaction).build()) - .withCompactionConfig(HoodieCompactionConfig.newBuilder().archiveCommitsWith(2, 4) + .withCleanConfig(HoodieCleanConfig.newBuilder() .withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.NEVER) - .retainCommits(1).retainFileVersions(1).withAutoClean(true).withAsyncClean(false).build()) + .retainCommits(1).retainFileVersions(1).withAutoClean(true).withAsyncClean(false) + .build()) + .withArchivalConfig(HoodieArchivalConfig.newBuilder() + .archiveCommitsWith(2, 4).build()) .build(); List records; @@ -2006,8 +2029,8 @@ public void testRollbackDuringUpgradeForDoubleLocking() throws IOException, Inte properties.setProperty(LockConfiguration.LOCK_ACQUIRE_CLIENT_NUM_RETRIES_PROP_KEY, "3"); properties.setProperty(LockConfiguration.LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP_KEY, "3000"); HoodieWriteConfig writeConfig = getWriteConfigBuilder(false, true, false) - .withCompactionConfig(HoodieCompactionConfig.newBuilder() - .withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.LAZY).withAutoClean(false).build()) + .withCleanConfig(HoodieCleanConfig.newBuilder() + .withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.LAZY).withAutoClean(false).build()) .withWriteConcurrencyMode(WriteConcurrencyMode.OPTIMISTIC_CONCURRENCY_CONTROL) .withLockConfig(HoodieLockConfig.newBuilder().withLockProvider(InProcessLockProvider.class).build()) .withProperties(properties) @@ -2034,10 +2057,8 @@ public void testRollbackDuringUpgradeForDoubleLocking() throws IOException, Inte // set hoodie.table.version to 2 in hoodie.properties file changeTableVersion(HoodieTableVersion.TWO); - writeConfig = getWriteConfigBuilder(true, true, false) - .withRollbackUsingMarkers(false) - .withCompactionConfig(HoodieCompactionConfig.newBuilder() - .withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.LAZY).withAutoClean(false).build()) + writeConfig = getWriteConfigBuilder(true, true, false).withRollbackUsingMarkers(false).withCleanConfig(HoodieCleanConfig.newBuilder() + .withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.LAZY).withAutoClean(false).build()) .withWriteConcurrencyMode(WriteConcurrencyMode.OPTIMISTIC_CONCURRENCY_CONTROL) .withLockConfig(HoodieLockConfig.newBuilder().withLockProvider(InProcessLockProvider.class).build()) .withProperties(properties) @@ -2119,7 +2140,7 @@ public void testDeletePartitions() throws Exception { int maxCommits = 1; HoodieWriteConfig cfg = getConfigBuilder(TRIP_EXAMPLE_SCHEMA, HoodieIndex.IndexType.BLOOM, HoodieFailedWritesCleaningPolicy.EAGER) - .withCompactionConfig(HoodieCompactionConfig.newBuilder() + .withCleanConfig(HoodieCleanConfig.newBuilder() .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS).retainCommits(maxCommits).build()) .withParallelism(1, 1).withBulkInsertParallelism(1).withFinalizeWriteParallelism(1).withDeleteParallelism(1) .withConsistencyGuardConfig(ConsistencyGuardConfig.newBuilder().withConsistencyCheckEnabled(true).build()) @@ -2285,13 +2306,13 @@ private void doWriteInsertAndUpsert(HoodieTestTable testTable) throws Exception private HoodieWriteConfig getSmallInsertWriteConfig(int insertSplitSize, String schemaStr, long smallFileSize, boolean mergeAllowDuplicateInserts) { HoodieWriteConfig.Builder builder = getConfigBuilder(schemaStr, HoodieIndex.IndexType.BLOOM, HoodieFailedWritesCleaningPolicy.EAGER); - return builder - .withCompactionConfig( + return builder.withCompactionConfig( HoodieCompactionConfig.newBuilder() .compactionSmallFileSize(smallFileSize) // Set rollback to LAZY so no inflights are deleted - .withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.LAZY) .insertSplitSize(insertSplitSize).build()) + .withCleanConfig(HoodieCleanConfig.newBuilder() + .withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.LAZY).build()) .withStorageConfig( HoodieStorageConfig.newBuilder() .hfileMaxFileSize(dataGen.getEstimatedFileSizeInBytes(200)) @@ -2307,8 +2328,8 @@ public HoodieWriteConfig.Builder getConfigBuilder(String schemaStr, HoodieIndex. .withTimelineLayoutVersion(TimelineLayoutVersion.CURR_VERSION) .withWriteStatusClass(MetadataMergeWriteStatus.class) .withConsistencyGuardConfig(ConsistencyGuardConfig.newBuilder().withConsistencyCheckEnabled(true).build()) - .withCompactionConfig(HoodieCompactionConfig.newBuilder().withFailedWritesCleaningPolicy(cleaningPolicy) - .compactionSmallFileSize(1024 * 1024).build()) + .withCleanConfig(HoodieCleanConfig.newBuilder().withFailedWritesCleaningPolicy(cleaningPolicy).build()) + .withCompactionConfig(HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024 * 1024).build()) .withStorageConfig(HoodieStorageConfig.newBuilder().hfileMaxFileSize(1024 * 1024).parquetMaxFileSize(1024 * 1024).orcMaxFileSize(1024 * 1024).build()) .forTable("test-trip-table") .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(indexType).build()) diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieClientOnCopyOnWriteStorage.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieClientOnCopyOnWriteStorage.java index cebf3145bfd28..8ba459b77226d 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieClientOnCopyOnWriteStorage.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieClientOnCopyOnWriteStorage.java @@ -71,12 +71,13 @@ import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.common.util.collection.Pair; -import org.apache.hudi.config.HoodieClusteringConfig; import org.apache.hudi.config.HoodieCompactionConfig; import org.apache.hudi.config.HoodieIndexConfig; -import org.apache.hudi.config.HoodiePreCommitValidatorConfig; +import org.apache.hudi.config.HoodieClusteringConfig; import org.apache.hudi.config.HoodieStorageConfig; import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.config.HoodiePreCommitValidatorConfig; +import org.apache.hudi.config.HoodieCleanConfig; import org.apache.hudi.data.HoodieJavaRDD; import org.apache.hudi.exception.HoodieCommitException; import org.apache.hudi.exception.HoodieCorruptedDataException; @@ -360,7 +361,7 @@ public void testPreCommitValidationWithMultipleInflights() throws Exception { .withPrecommitValidatorSingleResultSqlQueries(COUNT_SQL_QUERY_FOR_VALIDATION + "#" + 500) .build(); HoodieWriteConfig config = getConfigBuilder() - .withCompactionConfig(HoodieCompactionConfig.newBuilder().withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.NEVER).build()) + .withCleanConfig(HoodieCleanConfig.newBuilder().withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.NEVER).build()) .withPreCommitValidatorConfig(validatorConfig) .build(); @@ -386,7 +387,7 @@ public void testPreCommitValidationWithMultipleInflights() throws Exception { .withPrecommitValidatorSingleResultSqlQueries(COUNT_SQL_QUERY_FOR_VALIDATION + "#" + numRecords) .build(); config = getConfigBuilder() - .withCompactionConfig(HoodieCompactionConfig.newBuilder().withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.NEVER).build()) + .withCleanConfig(HoodieCleanConfig.newBuilder().withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.NEVER).build()) .withPreCommitValidatorConfig(validatorConfig) .build(); String instant2 = HoodieActiveTimeline.createNewInstantTime(); @@ -921,7 +922,7 @@ private void testUpsertsUpdatePartitionPath(IndexType indexType, HoodieWriteConf .setTimelineLayoutVersion(VERSION_0) .initTable(metaClient.getHadoopConf(), metaClient.getBasePath()); // Set rollback to LAZY so no inflights are deleted - hoodieWriteConfig.getProps().put(HoodieCompactionConfig.FAILED_WRITES_CLEANER_POLICY.key(), + hoodieWriteConfig.getProps().put(HoodieCleanConfig.FAILED_WRITES_CLEANER_POLICY.key(), HoodieFailedWritesCleaningPolicy.LAZY.name()); SparkRDDWriteClient client = getHoodieWriteClient(hoodieWriteConfig); @@ -2606,17 +2607,16 @@ private HoodieWriteConfig getSmallInsertWriteConfig(int insertSplitSize, String if (!populateMetaFields) { builder.withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(IndexType.SIMPLE).build()); } - return builder - .withCompactionConfig( - HoodieCompactionConfig.newBuilder() - .compactionSmallFileSize(smallFileSize) - // Set rollback to LAZY so no inflights are deleted - .withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.LAZY) - .insertSplitSize(insertSplitSize).build()) - .withStorageConfig( - HoodieStorageConfig.newBuilder() - .hfileMaxFileSize(dataGen.getEstimatedFileSizeInBytes(200)) - .parquetMaxFileSize(dataGen.getEstimatedFileSizeInBytes(200)).build()) + return builder.withCompactionConfig(HoodieCompactionConfig.newBuilder() + .compactionSmallFileSize(smallFileSize) + // Set rollback to LAZY so no inflights are deleted + .insertSplitSize(insertSplitSize).build()) + .withCleanConfig(HoodieCleanConfig.newBuilder() + .withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.LAZY) + .build()) + .withStorageConfig(HoodieStorageConfig.newBuilder() + .hfileMaxFileSize(dataGen.getEstimatedFileSizeInBytes(200)) + .parquetMaxFileSize(dataGen.getEstimatedFileSizeInBytes(200)).build()) .withMergeAllowDuplicateOnInserts(mergeAllowDuplicateInserts) .withProps(props) .build(); @@ -2636,7 +2636,7 @@ protected HoodieInstant createRequestedReplaceInstant(HoodieTableMetaClient meta private HoodieWriteConfig getParallelWritingWriteConfig(HoodieFailedWritesCleaningPolicy cleaningPolicy, boolean populateMetaFields) { return getConfigBuilder() .withEmbeddedTimelineServerEnabled(false) - .withCompactionConfig(HoodieCompactionConfig.newBuilder() + .withCleanConfig(HoodieCleanConfig.newBuilder() .withFailedWritesCleaningPolicy(cleaningPolicy) .withAutoClean(false).build()) .withTimelineLayoutVersion(1) diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieMetadataBase.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieMetadataBase.java index 2e387be54452a..29c653daee61a 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieMetadataBase.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieMetadataBase.java @@ -36,9 +36,11 @@ import org.apache.hudi.common.testutils.HoodieMetadataTestTable; import org.apache.hudi.common.testutils.HoodieTestTable; import org.apache.hudi.common.util.Option; -import org.apache.hudi.config.HoodieCompactionConfig; +import org.apache.hudi.config.HoodieCleanConfig; +import org.apache.hudi.config.HoodieArchivalConfig; import org.apache.hudi.config.HoodieIndexConfig; import org.apache.hudi.config.HoodieStorageConfig; +import org.apache.hudi.config.HoodieCompactionConfig; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.config.metrics.HoodieMetricsConfig; import org.apache.hudi.config.metrics.HoodieMetricsGraphiteConfig; @@ -338,9 +340,11 @@ protected HoodieWriteConfig.Builder getWriteConfigBuilder(HoodieFailedWritesClea .withParallelism(2, 2).withDeleteParallelism(2).withRollbackParallelism(2).withFinalizeWriteParallelism(2) .withAutoCommit(autoCommit) .withCompactionConfig(HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024 * 1024 * 1024) - .withInlineCompaction(false).withMaxNumDeltaCommitsBeforeCompaction(1) + .withInlineCompaction(false).withMaxNumDeltaCommitsBeforeCompaction(1).build()) + .withCleanConfig(HoodieCleanConfig.newBuilder() .withFailedWritesCleaningPolicy(policy) - .withAutoClean(false).retainCommits(1).retainFileVersions(1).build()) + .withAutoClean(false).retainCommits(1).retainFileVersions(1) + .build()) .withStorageConfig(HoodieStorageConfig.newBuilder().hfileMaxFileSize(1024 * 1024 * 1024).build()) .withEmbeddedTimelineServerEnabled(true).forTable("test-trip-table") .withFileSystemViewConfig(new FileSystemViewStorageConfig.Builder() @@ -390,16 +394,20 @@ protected HoodieWriteConfig getMetadataWriteConfig(HoodieWriteConfig writeConfig .withPath(HoodieTableMetadata.getMetadataTableBasePath(writeConfig.getBasePath())) .withSchema(HoodieMetadataRecord.getClassSchema().toString()) .forTable(writeConfig.getTableName() + METADATA_TABLE_NAME_SUFFIX) - .withCompactionConfig(HoodieCompactionConfig.newBuilder() + // we will trigger cleaning manually, to control the instant times + .withCleanConfig(HoodieCleanConfig.newBuilder() .withAsyncClean(writeConfig.isMetadataAsyncClean()) - // we will trigger cleaning manually, to control the instant times .withAutoClean(false) .withCleanerParallelism(parallelism) .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS) .withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.LAZY) .retainCommits(writeConfig.getMetadataCleanerCommitsRetained()) - .archiveCommitsWith(minCommitsToKeep, maxCommitsToKeep) - // we will trigger compaction manually, to control the instant times + .build()) + // we will trigger archival manually, to control the instant times + .withArchivalConfig(HoodieArchivalConfig.newBuilder() + .archiveCommitsWith(minCommitsToKeep, maxCommitsToKeep).build()) + // we will trigger compaction manually, to control the instant times + .withCompactionConfig(HoodieCompactionConfig.newBuilder() .withInlineCompaction(false) .withMaxNumDeltaCommitsBeforeCompaction(writeConfig.getMetadataCompactDeltaCommitMax()).build()) .withParallelism(parallelism, parallelism) diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieMetadataBootstrap.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieMetadataBootstrap.java index bdbc9e72d3f4a..8531030a5cc24 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieMetadataBootstrap.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieMetadataBootstrap.java @@ -26,7 +26,8 @@ import org.apache.hudi.common.testutils.HoodieMetadataTestTable; import org.apache.hudi.common.testutils.HoodieTestDataGenerator; import org.apache.hudi.common.testutils.HoodieTestTable; -import org.apache.hudi.config.HoodieCompactionConfig; +import org.apache.hudi.config.HoodieArchivalConfig; +import org.apache.hudi.config.HoodieCleanConfig; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.log4j.LogManager; @@ -275,7 +276,10 @@ private void doWriteInsertAndUpsert(HoodieTestTable testTable) throws Exception private HoodieWriteConfig getWriteConfig(int minArchivalCommits, int maxArchivalCommits) throws Exception { return HoodieWriteConfig.newBuilder().withPath(basePath) .withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2) - .withCompactionConfig(HoodieCompactionConfig.newBuilder().retainCommits(1).archiveCommitsWith(minArchivalCommits, maxArchivalCommits).build()) + .withCleanConfig(HoodieCleanConfig.newBuilder() + .retainCommits(1).build()) + .withArchivalConfig(HoodieArchivalConfig.newBuilder() + .archiveCommitsWith(minArchivalCommits, maxArchivalCommits).build()) .withFileSystemViewConfig(FileSystemViewStorageConfig.newBuilder() .withRemoteServerPort(timelineServicePort).build()) .withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(true).build()) diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/hbase/TestSparkHoodieHBaseIndex.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/hbase/TestSparkHoodieHBaseIndex.java index 87bcad04bc85e..407fb8de0e812 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/hbase/TestSparkHoodieHBaseIndex.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/hbase/TestSparkHoodieHBaseIndex.java @@ -32,6 +32,8 @@ import org.apache.hudi.common.testutils.HoodieTestDataGenerator; import org.apache.hudi.common.testutils.HoodieTestUtils; import org.apache.hudi.common.util.Option; +import org.apache.hudi.config.HoodieCleanConfig; +import org.apache.hudi.config.HoodieArchivalConfig; import org.apache.hudi.config.HoodieCompactionConfig; import org.apache.hudi.config.HoodieHBaseIndexConfig; import org.apache.hudi.config.HoodieIndexConfig; @@ -477,9 +479,9 @@ private JavaRDD generateAndCommitRecords(SparkRDDWriteClient write public void testHbaseTagLocationForArchivedCommits() throws Exception { // Load to memory Map params = new HashMap(); - params.put(HoodieCompactionConfig.CLEANER_COMMITS_RETAINED.key(), "1"); - params.put(HoodieCompactionConfig.MAX_COMMITS_TO_KEEP.key(), "3"); - params.put(HoodieCompactionConfig.MIN_COMMITS_TO_KEEP.key(), "2"); + params.put(HoodieCleanConfig.CLEANER_COMMITS_RETAINED.key(), "1"); + params.put(HoodieArchivalConfig.MAX_COMMITS_TO_KEEP.key(), "3"); + params.put(HoodieArchivalConfig.MIN_COMMITS_TO_KEEP.key(), "2"); HoodieWriteConfig config = getConfigBuilder(100, false, false).withProps(params).build(); SparkHoodieHBaseIndex index = new SparkHoodieHBaseIndex(config); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/TestHoodieTimelineArchiver.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/TestHoodieTimelineArchiver.java index d412052c2dbbf..4f41c4a44d9fb 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/TestHoodieTimelineArchiver.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/TestHoodieTimelineArchiver.java @@ -45,6 +45,8 @@ import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.config.HoodieCompactionConfig; +import org.apache.hudi.config.HoodieArchivalConfig; +import org.apache.hudi.config.HoodieCleanConfig; import org.apache.hudi.config.HoodieLockConfig; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieException; @@ -181,13 +183,14 @@ private HoodieWriteConfig initTestTableAndGetWriteConfig(boolean enableMetadata, init(tableType); HoodieWriteConfig writeConfig = HoodieWriteConfig.newBuilder().withPath(basePath) .withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2) - .withCompactionConfig(HoodieCompactionConfig.newBuilder().retainCommits(1).archiveCommitsWith(minArchivalCommits, maxArchivalCommits) - .withMaxNumDeltaCommitsBeforeCompaction(maxDeltaCommits) + .withCleanConfig(HoodieCleanConfig.newBuilder().retainCommits(1).withFailedWritesCleaningPolicy(failedWritesCleaningPolicy).build()) + .withArchivalConfig(HoodieArchivalConfig.newBuilder() .withArchiveMergeEnable(enableArchiveMerge) .withArchiveMergeFilesBatchSize(archiveFilesBatch) .withArchiveMergeSmallFileLimit(size) - .withFailedWritesCleaningPolicy(failedWritesCleaningPolicy) - .build()) + .archiveCommitsWith(minArchivalCommits, maxArchivalCommits).build()) + .withCompactionConfig(HoodieCompactionConfig.newBuilder() + .withMaxNumDeltaCommitsBeforeCompaction(maxDeltaCommits).build()) .withFileSystemViewConfig(FileSystemViewStorageConfig.newBuilder() .withRemoteServerPort(timelineServicePort).build()) .withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(enableMetadata) @@ -566,7 +569,8 @@ public void testArchiveCommitSavepointNoHole(boolean enableMetadataTable) throws init(); HoodieWriteConfig cfg = HoodieWriteConfig.newBuilder().withPath(basePath) .withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2).forTable("test-trip-table") - .withCompactionConfig(HoodieCompactionConfig.newBuilder().retainCommits(1).archiveCommitsWith(2, 5).build()) + .withArchivalConfig(HoodieArchivalConfig.newBuilder().archiveCommitsWith(2, 5).build()) + .withCleanConfig(HoodieCleanConfig.newBuilder().retainCommits(1).build()) .withFileSystemViewConfig(FileSystemViewStorageConfig.newBuilder() .withRemoteServerPort(timelineServicePort).build()) .withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(enableMetadataTable).build()) @@ -716,7 +720,8 @@ public void testArchiveCommitTimeline(boolean enableMetadataTable) throws Except HoodieWriteConfig cfg = HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA) .withParallelism(2, 2).forTable("test-trip-table") - .withCompactionConfig(HoodieCompactionConfig.newBuilder().retainCommits(1).archiveCommitsWith(2, 3).build()) + .withCleanConfig(HoodieCleanConfig.newBuilder().retainCommits(1).build()) + .withArchivalConfig(HoodieArchivalConfig.newBuilder().archiveCommitsWith(2, 3).build()) .withFileSystemViewConfig(FileSystemViewStorageConfig.newBuilder() .withRemoteServerPort(timelineServicePort).build()) .withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(enableMetadataTable).build()) @@ -881,9 +886,9 @@ public void testArchiveCompletedRollbackAndClean(boolean isEmpty, boolean enable HoodieWriteConfig cfg = HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA) .withParallelism(2, 2).forTable("test-trip-table") - .withCompactionConfig(HoodieCompactionConfig.newBuilder().retainCommits(1).archiveCommitsWith(minInstantsToKeep, maxInstantsToKeep).build()) - .withFileSystemViewConfig(FileSystemViewStorageConfig.newBuilder() - .withRemoteServerPort(timelineServicePort).build()) + .withCleanConfig(HoodieCleanConfig.newBuilder().retainCommits(1).build()) + .withArchivalConfig(HoodieArchivalConfig.newBuilder().archiveCommitsWith(minInstantsToKeep, maxInstantsToKeep).build()) + .withFileSystemViewConfig(FileSystemViewStorageConfig.newBuilder().withRemoteServerPort(timelineServicePort).build()) .withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(enableMetadataTable).build()) .build(); metaClient = HoodieTableMetaClient.reload(metaClient); @@ -940,7 +945,8 @@ public void testArchiveInflightClean(boolean enableMetadataTable) throws Excepti HoodieWriteConfig cfg = HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA) .withParallelism(2, 2).forTable("test-trip-table") - .withCompactionConfig(HoodieCompactionConfig.newBuilder().retainCommits(1).archiveCommitsWith(2, 3).build()) + .withCleanConfig(HoodieCleanConfig.newBuilder().retainCommits(1).build()) + .withArchivalConfig(HoodieArchivalConfig.newBuilder().archiveCommitsWith(2, 3).build()) .withFileSystemViewConfig(FileSystemViewStorageConfig.newBuilder() .withRemoteServerPort(timelineServicePort).build()) .withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(enableMetadataTable).build()) @@ -1146,8 +1152,8 @@ public void testArchivalAndCompactionInMetadataTable() throws Exception { // Test configs where metadata table has more aggressive archival configs than the compaction config HoodieWriteConfig writeConfig = HoodieWriteConfig.newBuilder().withPath(basePath) .withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2) - .withCompactionConfig(HoodieCompactionConfig.newBuilder() - .retainCommits(1).archiveCommitsWith(2, 4).build()) + .withArchivalConfig(HoodieArchivalConfig.newBuilder().archiveCommitsWith(2, 4).build()) + .withCleanConfig(HoodieCleanConfig.newBuilder().retainCommits(1).build()) .withFileSystemViewConfig(FileSystemViewStorageConfig.newBuilder() .withRemoteServerPort(timelineServicePort).build()) .withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(true) diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/TestCleaner.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/TestCleaner.java index b8545b0f63809..8da877940b36b 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/TestCleaner.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/TestCleaner.java @@ -70,6 +70,7 @@ import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.config.HoodieCleanConfig; import org.apache.hudi.config.HoodieCompactionConfig; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieIOException; @@ -261,11 +262,15 @@ public void testMultiClean() { HoodieWriteConfig writeConfig = getConfigBuilder() .withFileSystemViewConfig(new FileSystemViewStorageConfig.Builder() .withEnableBackupForRemoteFileSystemView(false).build()) - .withCompactionConfig(HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024 * 1024 * 1024) - .withInlineCompaction(false).withMaxNumDeltaCommitsBeforeCompaction(1) + + .withCleanConfig(HoodieCleanConfig.newBuilder() .withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.EAGER) .allowMultipleCleans(false) - .withAutoClean(false).retainCommits(1).retainFileVersions(1).build()) + .withAutoClean(false).retainCommits(1).retainFileVersions(1) + .build()) + .withCompactionConfig(HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024 * 1024 * 1024) + .withInlineCompaction(false).withMaxNumDeltaCommitsBeforeCompaction(1) + .build()) .withEmbeddedTimelineServerEnabled(false).build(); int index = 0; @@ -334,8 +339,9 @@ private void testInsertAndCleanByVersions( throws Exception { int maxVersions = 2; // keep upto 2 versions for each file HoodieWriteConfig cfg = getConfigBuilder() - .withCompactionConfig(HoodieCompactionConfig.newBuilder() - .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_FILE_VERSIONS).retainFileVersions(maxVersions).build()) + .withCleanConfig(HoodieCleanConfig.newBuilder() + .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_FILE_VERSIONS) + .retainFileVersions(maxVersions).build()) .withParallelism(1, 1).withBulkInsertParallelism(1).withFinalizeWriteParallelism(1).withDeleteParallelism(1) .withConsistencyGuardConfig(ConsistencyGuardConfig.newBuilder().withConsistencyCheckEnabled(true).build()) .build(); @@ -503,7 +509,7 @@ private void testInsertAndCleanByCommits( throws Exception { int maxCommits = 3; // keep upto 3 commits from the past HoodieWriteConfig cfg = getConfigBuilder() - .withCompactionConfig(HoodieCompactionConfig.newBuilder() + .withCleanConfig(HoodieCleanConfig.newBuilder() .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS).retainCommits(maxCommits).build()) .withParallelism(1, 1).withBulkInsertParallelism(1).withFinalizeWriteParallelism(1).withDeleteParallelism(1) .withConsistencyGuardConfig(ConsistencyGuardConfig.newBuilder().withConsistencyCheckEnabled(true).build()) @@ -579,7 +585,7 @@ private void testFailedInsertAndCleanByCommits( HoodieWriteConfig cfg = getConfigBuilder() .withAutoCommit(false) .withHeartbeatIntervalInMs(3000) - .withCompactionConfig(HoodieCompactionConfig.newBuilder() + .withCleanConfig(HoodieCleanConfig.newBuilder() .withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.LAZY) .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS).retainCommits(maxCommits).build()) .withParallelism(1, 1).withBulkInsertParallelism(1).withFinalizeWriteParallelism(1).withDeleteParallelism(1) @@ -732,8 +738,8 @@ public void testCleanEmptyInstants() throws Exception { HoodieWriteConfig.newBuilder() .withPath(basePath) .withMetadataConfig(HoodieMetadataConfig.newBuilder().withAssumeDatePartitioning(true).build()) - .withCompactionConfig(HoodieCompactionConfig.newBuilder() - .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS).build()) + .withCleanConfig(HoodieCleanConfig.newBuilder() + .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS).build()) .build(); metaClient = HoodieTableMetaClient.reload(metaClient); @@ -787,8 +793,9 @@ public void testCleanWithReplaceCommits() throws Exception { .withMetadataConfig(HoodieMetadataConfig.newBuilder() .withMaxNumDeltaCommitsBeforeCompaction(1) .withAssumeDatePartitioning(true).build()) - .withCompactionConfig(HoodieCompactionConfig.newBuilder() - .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS).retainCommits(2).build()) + .withCleanConfig(HoodieCleanConfig.newBuilder() + .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS) + .retainCommits(2).build()) .build(); HoodieTableMetadataWriter metadataWriter = SparkHoodieBackedTableMetadataWriter.create(hadoopConf, config, context); @@ -1140,7 +1147,7 @@ public void testCleanMarkerDataFilesOnRollback() throws Exception { public void testCleaningWithZeroPartitionPaths() throws Exception { HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath) .withMetadataConfig(HoodieMetadataConfig.newBuilder().withAssumeDatePartitioning(true).build()) - .withCompactionConfig(HoodieCompactionConfig.newBuilder() + .withCleanConfig(HoodieCleanConfig.newBuilder() .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS).retainCommits(2).build()) .build(); @@ -1164,7 +1171,7 @@ public void testCleaningWithZeroPartitionPaths() throws Exception { public void testKeepLatestCommitsWithPendingCompactions() throws Exception { HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath) .withMetadataConfig(HoodieMetadataConfig.newBuilder().withAssumeDatePartitioning(true).build()) - .withCompactionConfig(HoodieCompactionConfig.newBuilder() + .withCleanConfig(HoodieCleanConfig.newBuilder() .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS).retainCommits(2).build()) .build(); // Deletions: @@ -1188,7 +1195,7 @@ public void testKeepLatestVersionsWithPendingCompactions(boolean retryFailure) t HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath) .withMetadataConfig(HoodieMetadataConfig.newBuilder().withAssumeDatePartitioning(true).build()) - .withCompactionConfig(HoodieCompactionConfig.newBuilder() + .withCleanConfig(HoodieCleanConfig.newBuilder() .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_FILE_VERSIONS).retainFileVersions(2).build()) .build(); // Deletions: @@ -1212,8 +1219,8 @@ public void testCleanPreviousCorruptedCleanFiles() throws IOException { HoodieWriteConfig.newBuilder() .withPath(basePath) .withMetadataConfig(HoodieMetadataConfig.newBuilder().withAssumeDatePartitioning(true).build()) - .withCompactionConfig(HoodieCompactionConfig.newBuilder() - .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_FILE_VERSIONS).retainFileVersions(1).build()) + .withCleanConfig(HoodieCleanConfig.newBuilder() + .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_FILE_VERSIONS).retainFileVersions(1).build()) .build(); String commitTime = makeNewCommitTime(1, "%09d"); @@ -1241,7 +1248,7 @@ public void testRerunFailedClean(boolean simulateMetadataFailure) throws Excepti .withMetadataConfig(HoodieMetadataConfig.newBuilder() .withMaxNumDeltaCommitsBeforeCompaction(1) .withAssumeDatePartitioning(true).build()) - .withCompactionConfig(HoodieCompactionConfig.newBuilder() + .withCleanConfig(HoodieCleanConfig.newBuilder() .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS).retainCommits(2).build()) .build(); @@ -1317,7 +1324,7 @@ private void testInsertAndCleanFailedWritesByVersions( HoodieWriteConfig cfg = getConfigBuilder() .withAutoCommit(false) .withHeartbeatIntervalInMs(3000) - .withCompactionConfig(HoodieCompactionConfig.newBuilder() + .withCleanConfig(HoodieCleanConfig.newBuilder() .withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.LAZY) .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_FILE_VERSIONS).retainFileVersions(maxVersions).build()) .withParallelism(1, 1).withBulkInsertParallelism(1).withFinalizeWriteParallelism(1).withDeleteParallelism(1) diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestCleanPlanExecutor.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestCleanPlanExecutor.java index bd015baec9dbb..9fcac64c002f1 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestCleanPlanExecutor.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestCleanPlanExecutor.java @@ -37,7 +37,7 @@ import org.apache.hudi.common.util.CollectionUtils; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.collection.Pair; -import org.apache.hudi.config.HoodieCompactionConfig; +import org.apache.hudi.config.HoodieCleanConfig; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.metadata.HoodieTableMetadataWriter; import org.apache.hudi.metadata.SparkHoodieBackedTableMetadataWriter; @@ -77,14 +77,14 @@ public class TestCleanPlanExecutor extends TestCleaner { @Test public void testInvalidCleaningTriggerStrategy() { HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath) - .withMetadataConfig(HoodieMetadataConfig.newBuilder().withAssumeDatePartitioning(true).enable(false).build()) - .withCompactionConfig(HoodieCompactionConfig.newBuilder() - .withIncrementalCleaningMode(true) - .withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.EAGER) - .withCleanBootstrapBaseFileEnabled(true) - .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS).retainCommits(2) - .withCleaningTriggerStrategy("invalid_strategy").build()) - .build(); + .withMetadataConfig(HoodieMetadataConfig.newBuilder().withAssumeDatePartitioning(true).enable(false).build()) + .withCleanConfig(HoodieCleanConfig.newBuilder() + .withIncrementalCleaningMode(true) + .withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.EAGER) + .withCleanBootstrapBaseFileEnabled(true) + .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS).retainCommits(2) + .withCleaningTriggerStrategy("invalid_strategy").build()) + .withMetadataConfig(HoodieMetadataConfig.newBuilder().withAssumeDatePartitioning(true).enable(false).build()).build(); Exception e = assertThrows(IllegalArgumentException.class, () -> runCleaner(config, true), "should fail when invalid trigger strategy is provided!"); assertTrue(e.getMessage().contains("No enum constant org.apache.hudi.table.action.clean.CleaningTriggerStrategy.invalid_strategy")); } @@ -108,18 +108,15 @@ public void testKeepLatestCommits( boolean simulateFailureRetry, boolean simulateMetadataFailure, boolean enableIncrementalClean, boolean enableBootstrapSourceClean) throws Exception { HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath) - .withMetadataConfig( - HoodieMetadataConfig.newBuilder() - .withAssumeDatePartitioning(true) - .build()) - .withCompactionConfig(HoodieCompactionConfig.newBuilder() + .withMetadataConfig(HoodieMetadataConfig.newBuilder().withAssumeDatePartitioning(true).build()) + .withCleanConfig(HoodieCleanConfig.newBuilder() .withIncrementalCleaningMode(enableIncrementalClean) .withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.EAGER) .withCleanBootstrapBaseFileEnabled(enableBootstrapSourceClean) .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS) .retainCommits(2) - .withMaxCommitsBeforeCleaning(2).build()) - .build(); + .withMaxCommitsBeforeCleaning(2) + .build()).build(); HoodieTestTable testTable = HoodieTestTable.of(metaClient); String p0 = "2020/01/01"; @@ -274,7 +271,7 @@ public void testKeepLatestFileVersions() throws Exception { HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath) .withMetadataConfig(HoodieMetadataConfig.newBuilder().withAssumeDatePartitioning(true).build()) - .withCompactionConfig(HoodieCompactionConfig.newBuilder() + .withCleanConfig(HoodieCleanConfig.newBuilder() .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_FILE_VERSIONS).retainFileVersions(1).build()) .build(); @@ -353,7 +350,7 @@ public void testKeepLatestFileVersionsWithBootstrapFileClean() throws Exception HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath) .withMetadataConfig(HoodieMetadataConfig.newBuilder().withAssumeDatePartitioning(true).build()) - .withCompactionConfig(HoodieCompactionConfig.newBuilder() + .withCleanConfig(HoodieCleanConfig.newBuilder() .withCleanBootstrapBaseFileEnabled(true) .withCleanerParallelism(1) .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_FILE_VERSIONS).retainFileVersions(1).build()) @@ -453,18 +450,15 @@ public void testKeepLatestFileVersionsWithBootstrapFileClean() throws Exception @Test public void testKeepLatestFileVersionsMOR() throws Exception { - HoodieWriteConfig config = - HoodieWriteConfig.newBuilder().withPath(basePath) - .withMetadataConfig( - HoodieMetadataConfig.newBuilder() - .withAssumeDatePartitioning(true) - // Column Stats Index is disabled, since these tests construct tables which are - // not valid (empty commit metadata, invalid parquet files) - .withMetadataIndexColumnStats(false) - .build()) - .withCompactionConfig(HoodieCompactionConfig.newBuilder() - .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_FILE_VERSIONS).retainFileVersions(1).build()) - .build(); + HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath) + .withMetadataConfig(HoodieMetadataConfig.newBuilder().withAssumeDatePartitioning(true) + // Column Stats Index is disabled, since these tests construct tables which are + // not valid (empty commit metadata, invalid parquet files) + .withMetadataIndexColumnStats(false) + .build()) + .withCleanConfig(HoodieCleanConfig.newBuilder() + .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_FILE_VERSIONS).retainFileVersions(1) + .build()).build(); HoodieTableMetaClient metaClient = HoodieTestUtils.init(hadoopConf, basePath, HoodieTableType.MERGE_ON_READ); HoodieTestTable testTable = HoodieTestTable.of(metaClient); @@ -497,18 +491,14 @@ public void testKeepLatestFileVersionsMOR() throws Exception { @Test public void testKeepLatestCommitsMOR() throws Exception { - HoodieWriteConfig config = - HoodieWriteConfig.newBuilder().withPath(basePath) - .withMetadataConfig( - HoodieMetadataConfig.newBuilder() - .withAssumeDatePartitioning(true) + HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath) + .withMetadataConfig(HoodieMetadataConfig.newBuilder().withAssumeDatePartitioning(true) // Column Stats Index is disabled, since these tests construct tables which are // not valid (empty commit metadata, invalid parquet files) - .withMetadataIndexColumnStats(false) - .build()) - .withCompactionConfig(HoodieCompactionConfig.newBuilder() - .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS).retainCommits(1).build()) - .build(); + .withMetadataIndexColumnStats(false).build()) + .withCleanConfig(HoodieCleanConfig.newBuilder() + .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS).retainCommits(1).build()) + .build(); HoodieTableMetaClient metaClient = HoodieTestUtils.init(hadoopConf, basePath, HoodieTableType.MERGE_ON_READ); HoodieTestTable testTable = HoodieTestTable.of(metaClient); @@ -552,11 +542,12 @@ public void testKeepXHoursWithCleaning( boolean enableIncrementalClean, boolean enableBootstrapSourceClean) throws Exception { HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath) .withMetadataConfig(HoodieMetadataConfig.newBuilder().withAssumeDatePartitioning(true).build()) - .withCompactionConfig(HoodieCompactionConfig.newBuilder() + .withCleanConfig(HoodieCleanConfig.newBuilder() .withIncrementalCleaningMode(enableIncrementalClean) .withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.EAGER) .withCleanBootstrapBaseFileEnabled(enableBootstrapSourceClean) - .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_BY_HOURS).cleanerNumHoursRetained(2).build()) + .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_BY_HOURS).cleanerNumHoursRetained(2) + .build()) .build(); HoodieTestTable testTable = HoodieTestTable.of(metaClient); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestHoodieSparkCopyOnWriteTableArchiveWithReplace.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestHoodieSparkCopyOnWriteTableArchiveWithReplace.java index b4d6aefa71fe6..baff4ebac8752 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestHoodieSparkCopyOnWriteTableArchiveWithReplace.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestHoodieSparkCopyOnWriteTableArchiveWithReplace.java @@ -27,7 +27,8 @@ import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.testutils.HoodieTestDataGenerator; import org.apache.hudi.common.util.Option; -import org.apache.hudi.config.HoodieCompactionConfig; +import org.apache.hudi.config.HoodieArchivalConfig; +import org.apache.hudi.config.HoodieCleanConfig; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.testutils.SparkClientFunctionalTestHarness; @@ -54,8 +55,9 @@ public class TestHoodieSparkCopyOnWriteTableArchiveWithReplace extends SparkClie public void testDeletePartitionAndArchive(boolean metadataEnabled) throws IOException { HoodieTableMetaClient metaClient = getHoodieMetaClient(HoodieTableType.COPY_ON_WRITE); HoodieWriteConfig writeConfig = getConfigBuilder(true) - .withCompactionConfig(HoodieCompactionConfig.newBuilder().archiveCommitsWith(2, 3).retainCommits(1).build()) - .withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(metadataEnabled).build()) + .withCleanConfig(HoodieCleanConfig.newBuilder().retainCommits(1).build()) + .withArchivalConfig(HoodieArchivalConfig.newBuilder().archiveCommitsWith(2, 3).build()) + .withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(metadataEnabled).build()) .build(); try (SparkRDDWriteClient client = getHoodieWriteClient(writeConfig); HoodieTestDataGenerator dataGen = new HoodieTestDataGenerator(DEFAULT_PARTITION_PATHS)) { diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestHoodieSparkMergeOnReadTableRollback.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestHoodieSparkMergeOnReadTableRollback.java index c33f0bcc2c5aa..2f0e585ec90a0 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestHoodieSparkMergeOnReadTableRollback.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestHoodieSparkMergeOnReadTableRollback.java @@ -43,6 +43,7 @@ import org.apache.hudi.common.testutils.HoodieTestTable; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.config.HoodieCleanConfig; import org.apache.hudi.config.HoodieCompactionConfig; import org.apache.hudi.config.HoodieStorageConfig; import org.apache.hudi.config.HoodieWriteConfig; @@ -556,7 +557,7 @@ void testRestoreWithCleanedUpCommits() throws Exception { // trigger clean. creating a new client with aggresive cleaner configs so that clean will kick in immediately. cfgBuilder = getConfigBuilder(false) - .withCompactionConfig(HoodieCompactionConfig.newBuilder().retainCommits(1).build()) + .withCleanConfig(HoodieCleanConfig.newBuilder().retainCommits(1).build()) // Timeline-server-based markers are not used for multi-rollback tests .withMarkersType(MarkerType.DIRECT.name()); addConfigsForPopulateMetaFields(cfgBuilder, populateMetaFields); @@ -977,10 +978,13 @@ private long getNumLogFilesInLatestFileSlice(HoodieTableMetaClient metaClient, H private HoodieWriteConfig getWriteConfig(boolean autoCommit, boolean rollbackUsingMarkers) { HoodieWriteConfig.Builder cfgBuilder = getConfigBuilder(autoCommit).withRollbackUsingMarkers(rollbackUsingMarkers) + .withCleanConfig(HoodieCleanConfig.newBuilder() + .withAutoClean(false) + .withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.LAZY) + .build()) .withCompactionConfig(HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024 * 1024 * 1024L) .withInlineCompaction(false).withMaxNumDeltaCommitsBeforeCompaction(3) - .withAutoClean(false) - .withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.LAZY).build()); + .build()); return cfgBuilder.build(); } diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestBase.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestBase.java index 16fd48af6c014..900674a677588 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestBase.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestBase.java @@ -38,10 +38,11 @@ import org.apache.hudi.common.testutils.HoodieTestDataGenerator; import org.apache.hudi.common.testutils.RawTripTestPayload; import org.apache.hudi.common.util.Option; +import org.apache.hudi.config.HoodieCleanConfig; +import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.config.HoodieCompactionConfig; -import org.apache.hudi.config.HoodieIndexConfig; import org.apache.hudi.config.HoodieStorageConfig; -import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.config.HoodieIndexConfig; import org.apache.hudi.index.HoodieIndex; import org.apache.hudi.index.HoodieIndex.IndexType; import org.apache.hudi.index.SparkHoodieIndexFactory; @@ -146,8 +147,8 @@ public HoodieWriteConfig.Builder getConfigBuilder(String schemaStr, IndexType in .withTimelineLayoutVersion(TimelineLayoutVersion.CURR_VERSION) .withWriteStatusClass(MetadataMergeWriteStatus.class) .withConsistencyGuardConfig(ConsistencyGuardConfig.newBuilder().withConsistencyCheckEnabled(true).build()) - .withCompactionConfig(HoodieCompactionConfig.newBuilder().withFailedWritesCleaningPolicy(cleaningPolicy) - .compactionSmallFileSize(1024 * 1024).build()) + .withCleanConfig(HoodieCleanConfig.newBuilder().withFailedWritesCleaningPolicy(cleaningPolicy).build()) + .withCompactionConfig(HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024 * 1024).build()) .withStorageConfig(HoodieStorageConfig.newBuilder().hfileMaxFileSize(1024 * 1024).parquetMaxFileSize(1024 * 1024).orcMaxFileSize(1024 * 1024).build()) .forTable("test-trip-table") .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(indexType).build()) diff --git a/hudi-common/src/main/java/org/apache/hudi/avro/HoodieAvroUtils.java b/hudi-common/src/main/java/org/apache/hudi/avro/HoodieAvroUtils.java index 66066040275bf..fa65461bfdb0e 100644 --- a/hudi-common/src/main/java/org/apache/hudi/avro/HoodieAvroUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/avro/HoodieAvroUtils.java @@ -745,15 +745,18 @@ public static GenericRecord rewriteRecordWithNewSchema(IndexedRecord oldRecord, * b) For GenericRecord, copy over the data from the old schema to the new schema or set default values for all fields of this transformed schema * * @param oldRecord oldRecord to be rewritten + * @param oldAvroSchema old avro schema. * @param newSchema newSchema used to rewrite oldRecord * @param renameCols a map store all rename cols, (k, v)-> (colNameFromNewSchema, colNameFromOldSchema) * @param fieldNames track the full name of visited field when we travel new schema. * @return newRecord for new Schema */ - private static Object rewriteRecordWithNewSchema(Object oldRecord, Schema oldSchema, Schema newSchema, Map renameCols, Deque fieldNames) { + private static Object rewriteRecordWithNewSchema(Object oldRecord, Schema oldAvroSchema, Schema newSchema, Map renameCols, Deque fieldNames) { if (oldRecord == null) { return null; } + // try to get real schema for union type + Schema oldSchema = getActualSchemaFromUnion(oldAvroSchema, oldRecord); switch (newSchema.getType()) { case RECORD: if (!(oldRecord instanceof IndexedRecord)) { @@ -761,39 +764,32 @@ private static Object rewriteRecordWithNewSchema(Object oldRecord, Schema oldSch } IndexedRecord indexedRecord = (IndexedRecord) oldRecord; List fields = newSchema.getFields(); - Map helper = new HashMap<>(); - + GenericData.Record newRecord = new GenericData.Record(newSchema); for (int i = 0; i < fields.size(); i++) { Schema.Field field = fields.get(i); String fieldName = field.name(); fieldNames.push(fieldName); if (oldSchema.getField(field.name()) != null) { Schema.Field oldField = oldSchema.getField(field.name()); - helper.put(i, rewriteRecordWithNewSchema(indexedRecord.get(oldField.pos()), oldField.schema(), fields.get(i).schema(), renameCols, fieldNames)); + newRecord.put(i, rewriteRecordWithNewSchema(indexedRecord.get(oldField.pos()), oldField.schema(), fields.get(i).schema(), renameCols, fieldNames)); } else { String fieldFullName = createFullName(fieldNames); - String[] colNamePartsFromOldSchema = renameCols.getOrDefault(fieldFullName, "").split("\\."); - String lastColNameFromOldSchema = colNamePartsFromOldSchema[colNamePartsFromOldSchema.length - 1]; + String fieldNameFromOldSchema = renameCols.getOrDefault(fieldFullName, ""); // deal with rename - if (oldSchema.getField(field.name()) == null && oldSchema.getField(lastColNameFromOldSchema) != null) { + if (oldSchema.getField(field.name()) == null && oldSchema.getField(fieldNameFromOldSchema) != null) { // find rename - Schema.Field oldField = oldSchema.getField(lastColNameFromOldSchema); - helper.put(i, rewriteRecordWithNewSchema(indexedRecord.get(oldField.pos()), oldField.schema(), fields.get(i).schema(), renameCols, fieldNames)); - } - } - fieldNames.pop(); - } - GenericData.Record newRecord = new GenericData.Record(newSchema); - for (int i = 0; i < fields.size(); i++) { - if (helper.containsKey(i)) { - newRecord.put(i, helper.get(i)); - } else { - if (fields.get(i).defaultVal() instanceof JsonProperties.Null) { - newRecord.put(i, null); + Schema.Field oldField = oldSchema.getField(fieldNameFromOldSchema); + newRecord.put(i, rewriteRecordWithNewSchema(indexedRecord.get(oldField.pos()), oldField.schema(), fields.get(i).schema(), renameCols, fieldNames)); } else { - newRecord.put(i, fields.get(i).defaultVal()); + // deal with default value + if (fields.get(i).defaultVal() instanceof JsonProperties.Null) { + newRecord.put(i, null); + } else { + newRecord.put(i, fields.get(i).defaultVal()); + } } } + fieldNames.pop(); } return newRecord; case ARRAY: @@ -1028,4 +1024,8 @@ public GenericRecord next() { } }; } + + public static GenericRecord rewriteRecordDeep(GenericRecord oldRecord, Schema newSchema) { + return rewriteRecordWithNewSchema(oldRecord, newSchema, Collections.EMPTY_MAP); + } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieCommonConfig.java b/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieCommonConfig.java index cc62bcc32824f..917cfe621f11e 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieCommonConfig.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieCommonConfig.java @@ -36,6 +36,13 @@ public class HoodieCommonConfig extends HoodieConfig { .defaultValue(false) .withDocumentation("Enables support for Schema Evolution feature"); + public static final ConfigProperty RECONCILE_SCHEMA = ConfigProperty + .key("hoodie.datasource.write.reconcile.schema") + .defaultValue(false) + .withDocumentation("When a new batch of write has records with old schema, but latest table schema got " + + "evolved, this config will upgrade the records to leverage latest table schema(default values will be " + + "injected to missing fields). If not, the write batch would fail."); + public static final ConfigProperty SPILLABLE_DISK_MAP_TYPE = ConfigProperty .key("hoodie.common.spillable.diskmap.type") .defaultValue(ExternalSpillableMap.DiskMapType.BITCASK) diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/TableSchemaResolver.java b/hudi-common/src/main/java/org/apache/hudi/common/table/TableSchemaResolver.java index 4f99926887692..5fc989e2e5185 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/TableSchemaResolver.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/TableSchemaResolver.java @@ -23,6 +23,7 @@ import org.apache.avro.Schema.Field; import org.apache.avro.SchemaCompatibility; import org.apache.avro.generic.IndexedRecord; +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.io.hfile.CacheConfig; @@ -261,6 +262,11 @@ private MessageType getTableParquetSchemaFromDataFile() { } } + public static MessageType convertAvroSchemaToParquet(Schema schema, Configuration hadoopConf) { + AvroSchemaConverter avroSchemaConverter = new AvroSchemaConverter(hadoopConf); + return avroSchemaConverter.convert(schema); + } + private Schema convertParquetSchemaToAvro(MessageType parquetSchema) { AvroSchemaConverter avroSchemaConverter = new AvroSchemaConverter(metaClient.getHadoopConf()); return avroSchemaConverter.convert(parquetSchema); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/log/AbstractHoodieLogRecordReader.java b/hudi-common/src/main/java/org/apache/hudi/common/table/log/AbstractHoodieLogRecordReader.java index 9687136444eeb..16a264e06ddcd 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/log/AbstractHoodieLogRecordReader.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/log/AbstractHoodieLogRecordReader.java @@ -57,8 +57,8 @@ import java.io.IOException; import java.util.ArrayDeque; import java.util.Arrays; +import java.util.Collections; import java.util.Deque; -import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Set; @@ -380,7 +380,7 @@ private void processDataBlock(HoodieDataBlock dataBlock, Option keySpec Option schemaOption = getMergedSchema(dataBlock); while (recordIterator.hasNext()) { IndexedRecord currentRecord = recordIterator.next(); - IndexedRecord record = schemaOption.isPresent() ? HoodieAvroUtils.rewriteRecordWithNewSchema(currentRecord, schemaOption.get(), new HashMap<>()) : currentRecord; + IndexedRecord record = schemaOption.isPresent() ? HoodieAvroUtils.rewriteRecordWithNewSchema(currentRecord, schemaOption.get(), Collections.emptyMap()) : currentRecord; processNextRecord(createHoodieRecord(record, this.hoodieTableMetaClient.getTableConfig(), this.payloadClassFQN, this.preCombineField, this.withOperationField, this.simpleKeyGenFields, this.partitionName)); totalLogRecords.incrementAndGet(); diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/strategy/AllPendingCompactionPlanSelectStrategy.java b/hudi-common/src/main/java/org/apache/hudi/exception/HoodieCatalogException.java similarity index 53% rename from hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/strategy/AllPendingCompactionPlanSelectStrategy.java rename to hudi-common/src/main/java/org/apache/hudi/exception/HoodieCatalogException.java index 23b6708ff304c..0ac8c8d7b6554 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/strategy/AllPendingCompactionPlanSelectStrategy.java +++ b/hudi-common/src/main/java/org/apache/hudi/exception/HoodieCatalogException.java @@ -7,7 +7,7 @@ * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, @@ -16,20 +16,29 @@ * limitations under the License. */ -package org.apache.hudi.sink.compact.strategy; - -import java.util.List; -import java.util.stream.Collectors; -import org.apache.hudi.common.table.timeline.HoodieInstant; -import org.apache.hudi.common.table.timeline.HoodieTimeline; -import org.apache.hudi.sink.compact.FlinkCompactionConfig; +package org.apache.hudi.exception; /** - * Select all pending compaction plan to compact + *

+ * Exception thrown for Hoodie Catalog errors. + *

*/ -public class AllPendingCompactionPlanSelectStrategy implements CompactionPlanSelectStrategy { - @Override - public List select(HoodieTimeline pendingCompactionTimeline, FlinkCompactionConfig config) { - return pendingCompactionTimeline.getInstants().collect(Collectors.toList()); +public class HoodieCatalogException extends RuntimeException { + + public HoodieCatalogException() { + super(); } + + public HoodieCatalogException(String message) { + super(message); + } + + public HoodieCatalogException(String message, Throwable t) { + super(message, t); + } + + public HoodieCatalogException(Throwable t) { + super(t); + } + } diff --git a/hudi-common/src/main/java/org/apache/hudi/internal/schema/action/InternalSchemaMerger.java b/hudi-common/src/main/java/org/apache/hudi/internal/schema/action/InternalSchemaMerger.java index bcea9b957b3ea..cd9bae0541cdc 100644 --- a/hudi-common/src/main/java/org/apache/hudi/internal/schema/action/InternalSchemaMerger.java +++ b/hudi-common/src/main/java/org/apache/hudi/internal/schema/action/InternalSchemaMerger.java @@ -68,10 +68,7 @@ public InternalSchemaMerger(InternalSchema fileSchema, InternalSchema querySchem } public InternalSchemaMerger(InternalSchema fileSchema, InternalSchema querySchema, boolean ignoreRequiredAttribute, boolean useColumnTypeFromFileSchema) { - this.fileSchema = fileSchema; - this.querySchema = querySchema; - this.ignoreRequiredAttribute = ignoreRequiredAttribute; - this.useColumnTypeFromFileSchema = useColumnTypeFromFileSchema; + this(fileSchema, querySchema, ignoreRequiredAttribute, useColumnTypeFromFileSchema, true); } /** @@ -151,14 +148,15 @@ private Types.Field dealWithRename(int fieldId, Type newType, Types.Field oldFie Types.Field fieldFromFileSchema = fileSchema.findField(fieldId); String nameFromFileSchema = fieldFromFileSchema.name(); String nameFromQuerySchema = querySchema.findField(fieldId).name(); + String finalFieldName = useColNameFromFileSchema ? nameFromFileSchema : nameFromQuerySchema; Type typeFromFileSchema = fieldFromFileSchema.type(); // Current design mechanism guarantees nestedType change is not allowed, so no need to consider. if (newType.isNestedType()) { return Types.Field.get(oldField.fieldId(), oldField.isOptional(), - useColNameFromFileSchema ? nameFromFileSchema : nameFromQuerySchema, newType, oldField.doc()); + finalFieldName, newType, oldField.doc()); } else { return Types.Field.get(oldField.fieldId(), oldField.isOptional(), - useColNameFromFileSchema ? nameFromFileSchema : nameFromQuerySchema, useColumnTypeFromFileSchema ? typeFromFileSchema : newType, oldField.doc()); + finalFieldName, useColumnTypeFromFileSchema ? typeFromFileSchema : newType, oldField.doc()); } } diff --git a/hudi-common/src/main/java/org/apache/hudi/internal/schema/utils/AvroSchemaEvolutionUtils.java b/hudi-common/src/main/java/org/apache/hudi/internal/schema/utils/AvroSchemaEvolutionUtils.java index e57fce4357b25..520a6b9ec75b2 100644 --- a/hudi-common/src/main/java/org/apache/hudi/internal/schema/utils/AvroSchemaEvolutionUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/internal/schema/utils/AvroSchemaEvolutionUtils.java @@ -33,37 +33,33 @@ * Utility methods to support evolve old avro schema based on a given schema. */ public class AvroSchemaEvolutionUtils { + /** - * Support evolution from a new avroSchema. - * Now hoodie support implicitly add columns when hoodie write operation, - * This ability needs to be preserved, so implicitly evolution for internalSchema should supported. - * - * @param evolvedSchema implicitly evolution of avro when hoodie write operation - * @param oldSchema old internalSchema - * @param supportPositionReorder support position reorder - * @return evolution Schema + * Support reconcile from a new avroSchema. + * 1) incoming data has missing columns that were already defined in the table –> null values will be injected into missing columns + * 2) incoming data contains new columns not defined yet in the table -> columns will be added to the table schema (incoming dataframe?) + * 3) incoming data has missing columns that are already defined in the table and new columns not yet defined in the table -> + * new columns will be added to the table schema, missing columns will be injected with null values + * 4) support nested schema change. + * Notice: + * the incoming schema should not have delete/rename semantics. + * for example: incoming schema: int a, int b, int d; oldTableSchema int a, int b, int c, int d + * we must guarantee the column c is missing semantic, instead of delete semantic. + * @param incomingSchema implicitly evolution of avro when hoodie write operation + * @param oldTableSchema old internalSchema + * @return reconcile Schema */ - public static InternalSchema evolveSchemaFromNewAvroSchema(Schema evolvedSchema, InternalSchema oldSchema, Boolean supportPositionReorder) { - InternalSchema evolvedInternalSchema = AvroInternalSchemaConverter.convert(evolvedSchema); + public static InternalSchema reconcileSchema(Schema incomingSchema, InternalSchema oldTableSchema) { + InternalSchema inComingInternalSchema = AvroInternalSchemaConverter.convert(incomingSchema); // do check, only support add column evolution - List colNamesFromEvolved = evolvedInternalSchema.getAllColsFullName(); - List colNamesFromOldSchema = oldSchema.getAllColsFullName(); - List diffFromOldSchema = colNamesFromOldSchema.stream().filter(f -> !colNamesFromEvolved.contains(f)).collect(Collectors.toList()); + List colNamesFromIncoming = inComingInternalSchema.getAllColsFullName(); + List colNamesFromOldSchema = oldTableSchema.getAllColsFullName(); + List diffFromOldSchema = colNamesFromOldSchema.stream().filter(f -> !colNamesFromIncoming.contains(f)).collect(Collectors.toList()); List newFields = new ArrayList<>(); - if (colNamesFromEvolved.size() == colNamesFromOldSchema.size() && diffFromOldSchema.size() == 0) { - // no changes happen - if (supportPositionReorder) { - evolvedInternalSchema.getRecord().fields().forEach(f -> newFields.add(oldSchema.getRecord().field(f.name()))); - return new InternalSchema(newFields); - } - return oldSchema; - } - // try to find all added columns - if (diffFromOldSchema.size() != 0) { - throw new UnsupportedOperationException("Cannot evolve schema implicitly, find delete/rename operation"); + if (colNamesFromIncoming.size() == colNamesFromOldSchema.size() && diffFromOldSchema.size() == 0) { + return oldTableSchema; } - - List diffFromEvolutionSchema = colNamesFromEvolved.stream().filter(f -> !colNamesFromOldSchema.contains(f)).collect(Collectors.toList()); + List diffFromEvolutionSchema = colNamesFromIncoming.stream().filter(f -> !colNamesFromOldSchema.contains(f)).collect(Collectors.toList()); // Remove redundancy from diffFromEvolutionSchema. // for example, now we add a struct col in evolvedSchema, the struct col is " user struct " // when we do diff operation: user, user.name, user.age will appeared in the resultSet which is redundancy, user.name and user.age should be excluded. @@ -77,29 +73,27 @@ public static InternalSchema evolveSchemaFromNewAvroSchema(Schema evolvedSchema, // find redundancy, skip it continue; } - finalAddAction.put(evolvedInternalSchema.findIdByName(name), name); + finalAddAction.put(inComingInternalSchema.findIdByName(name), name); } - TableChanges.ColumnAddChange addChange = TableChanges.ColumnAddChange.get(oldSchema); + TableChanges.ColumnAddChange addChange = TableChanges.ColumnAddChange.get(oldTableSchema); finalAddAction.entrySet().stream().forEach(f -> { String name = f.getValue(); int splitPoint = name.lastIndexOf("."); String parentName = splitPoint > 0 ? name.substring(0, splitPoint) : ""; String rawName = splitPoint > 0 ? name.substring(splitPoint + 1) : name; - addChange.addColumns(parentName, rawName, evolvedInternalSchema.findType(name), null); + // try to infer add position. + java.util.Optional inferPosition = + colNamesFromIncoming.stream().filter(c -> + c.lastIndexOf(".") == splitPoint + && c.startsWith(parentName) + && inComingInternalSchema.findIdByName(c) > inComingInternalSchema.findIdByName(name) + && oldTableSchema.findIdByName(c) > 0).sorted((s1, s2) -> oldTableSchema.findIdByName(s1) - oldTableSchema.findIdByName(s2)).findFirst(); + addChange.addColumns(parentName, rawName, inComingInternalSchema.findType(name), null); + inferPosition.map(i -> addChange.addPositionChange(name, i, "before")); }); - InternalSchema res = SchemaChangeUtils.applyTableChanges2Schema(oldSchema, addChange); - if (supportPositionReorder) { - evolvedInternalSchema.getRecord().fields().forEach(f -> newFields.add(oldSchema.getRecord().field(f.name()))); - return new InternalSchema(newFields); - } else { - return res; - } - } - - public static InternalSchema evolveSchemaFromNewAvroSchema(Schema evolvedSchema, InternalSchema oldSchema) { - return evolveSchemaFromNewAvroSchema(evolvedSchema, oldSchema, false); + return SchemaChangeUtils.applyTableChanges2Schema(oldTableSchema, addChange); } /** diff --git a/hudi-common/src/main/java/org/apache/hudi/internal/schema/utils/InternalSchemaUtils.java b/hudi-common/src/main/java/org/apache/hudi/internal/schema/utils/InternalSchemaUtils.java index a784b409b8f2f..c799c236d0db0 100644 --- a/hudi-common/src/main/java/org/apache/hudi/internal/schema/utils/InternalSchemaUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/internal/schema/utils/InternalSchemaUtils.java @@ -273,7 +273,7 @@ public static String createFullName(String name, Deque fieldNames) { * * @param oldSchema oldSchema * @param newSchema newSchema which modified from oldSchema - * @return renameCols Map. (k, v) -> (colNameFromNewSchema, colNameFromOldSchema) + * @return renameCols Map. (k, v) -> (colNameFromNewSchema, colNameLastPartFromOldSchema) */ public static Map collectRenameCols(InternalSchema oldSchema, InternalSchema newSchema) { List colNamesFromWriteSchema = oldSchema.getAllColsFullName(); @@ -281,6 +281,9 @@ public static Map collectRenameCols(InternalSchema oldSchema, In int filedIdFromWriteSchema = oldSchema.findIdByName(f); // try to find the cols which has the same id, but have different colName; return newSchema.getAllIds().contains(filedIdFromWriteSchema) && !newSchema.findfullName(filedIdFromWriteSchema).equalsIgnoreCase(f); - }).collect(Collectors.toMap(e -> newSchema.findfullName(oldSchema.findIdByName(e)), e -> e)); + }).collect(Collectors.toMap(e -> newSchema.findfullName(oldSchema.findIdByName(e)), e -> { + int lastDotIndex = e.lastIndexOf("."); + return e.substring(lastDotIndex == -1 ? 0 : lastDotIndex + 1); + })); } } diff --git a/hudi-common/src/test/java/org/apache/hudi/avro/TestHoodieAvroUtils.java b/hudi-common/src/test/java/org/apache/hudi/avro/TestHoodieAvroUtils.java index bd0254da3dc6e..f2c02d627f131 100644 --- a/hudi-common/src/test/java/org/apache/hudi/avro/TestHoodieAvroUtils.java +++ b/hudi-common/src/test/java/org/apache/hudi/avro/TestHoodieAvroUtils.java @@ -27,12 +27,14 @@ import org.apache.avro.generic.GenericData; import org.apache.avro.generic.GenericRecord; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Assertions; import java.io.IOException; import java.math.BigDecimal; import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Arrays; +import java.util.HashMap; import java.util.List; import java.util.Map; @@ -98,6 +100,12 @@ public class TestHoodieAvroUtils { + "{\"name\":\"student\",\"type\":{\"name\":\"student\",\"type\":\"record\",\"fields\":[" + "{\"name\":\"firstname\",\"type\":[\"null\" ,\"string\"],\"default\": null},{\"name\":\"lastname\",\"type\":[\"null\" ,\"string\"],\"default\": null}]}}]}"; + private static String SCHEMA_WITH_NESTED_FIELD_RENAMED = "{\"name\":\"MyClass\",\"type\":\"record\",\"namespace\":\"com.acme.avro\",\"fields\":[" + + "{\"name\":\"fn\",\"type\":\"string\"}," + + "{\"name\":\"ln\",\"type\":\"string\"}," + + "{\"name\":\"ss\",\"type\":{\"name\":\"ss\",\"type\":\"record\",\"fields\":[" + + "{\"name\":\"fn\",\"type\":[\"null\" ,\"string\"],\"default\": null},{\"name\":\"ln\",\"type\":[\"null\" ,\"string\"],\"default\": null}]}}]}"; + @Test public void testPropsPresent() { Schema schema = HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(EXAMPLE_SCHEMA)); @@ -342,4 +350,26 @@ public void testGetNestedFieldSchema() throws IOException { assertEquals(Schema.create(Schema.Type.STRING), getNestedFieldSchemaFromWriteSchema(rec3.getSchema(), "student.firstname")); assertEquals(Schema.create(Schema.Type.STRING), getNestedFieldSchemaFromWriteSchema(nestedSchema, "student.firstname")); } + + @Test + public void testReWriteAvroRecordWithNewSchema() { + Schema nestedSchema = new Schema.Parser().parse(SCHEMA_WITH_NESTED_FIELD); + GenericRecord rec3 = new GenericData.Record(nestedSchema); + rec3.put("firstname", "person1"); + rec3.put("lastname", "person2"); + GenericRecord studentRecord = new GenericData.Record(rec3.getSchema().getField("student").schema()); + studentRecord.put("firstname", "person1"); + studentRecord.put("lastname", "person2"); + rec3.put("student", studentRecord); + + Schema nestedSchemaRename = new Schema.Parser().parse(SCHEMA_WITH_NESTED_FIELD_RENAMED); + Map colRenames = new HashMap<>(); + colRenames.put("fn", "firstname"); + colRenames.put("ln", "lastname"); + colRenames.put("ss", "student"); + colRenames.put("ss.fn", "firstname"); + colRenames.put("ss.ln", "lastname"); + GenericRecord studentRecordRename = HoodieAvroUtils.rewriteRecordWithNewSchema(rec3, nestedSchemaRename, colRenames); + Assertions.assertEquals(GenericData.get().validate(nestedSchemaRename, studentRecordRename), true); + } } diff --git a/hudi-common/src/test/java/org/apache/hudi/common/util/TestDFSPropertiesConfiguration.java b/hudi-common/src/test/java/org/apache/hudi/common/util/TestDFSPropertiesConfiguration.java index 465739340dc86..a122f414f93d9 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/util/TestDFSPropertiesConfiguration.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/util/TestDFSPropertiesConfiguration.java @@ -185,7 +185,7 @@ public void testLoadGlobalConfFile() { DFSPropertiesConfiguration.refreshGlobalProps(); assertEquals(5, DFSPropertiesConfiguration.getGlobalProps().size()); assertEquals("jdbc:hive2://localhost:10000", DFSPropertiesConfiguration.getGlobalProps().get("hoodie.datasource.hive_sync.jdbcurl")); - assertEquals("true", DFSPropertiesConfiguration.getGlobalProps().get("hoodie.datasource.hive_sync.use_jdbc")); + assertEquals("jdbc", DFSPropertiesConfiguration.getGlobalProps().get("hoodie.datasource.hive_sync.mode")); assertEquals("false", DFSPropertiesConfiguration.getGlobalProps().get("hoodie.datasource.hive_sync.support_timestamp")); assertEquals("BLOOM", DFSPropertiesConfiguration.getGlobalProps().get("hoodie.index.type")); assertEquals("true", DFSPropertiesConfiguration.getGlobalProps().get("hoodie.metadata.enable")); diff --git a/hudi-common/src/test/java/org/apache/hudi/internal/schema/utils/TestAvroSchemaEvolutionUtils.java b/hudi-common/src/test/java/org/apache/hudi/internal/schema/utils/TestAvroSchemaEvolutionUtils.java index 3850ef07b90a3..6126c479c6154 100644 --- a/hudi-common/src/test/java/org/apache/hudi/internal/schema/utils/TestAvroSchemaEvolutionUtils.java +++ b/hudi-common/src/test/java/org/apache/hudi/internal/schema/utils/TestAvroSchemaEvolutionUtils.java @@ -38,6 +38,7 @@ import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -45,6 +46,17 @@ public class TestAvroSchemaEvolutionUtils { + String schemaStr = "{\"type\":\"record\",\"name\":\"newTableName\",\"fields\":[{\"name\":\"id\",\"type\":\"int\"},{\"name\":\"data\"," + + "\"type\":[\"null\",\"string\"],\"default\":null},{\"name\":\"preferences\",\"type\":[\"null\"," + + "{\"type\":\"record\",\"name\":\"newTableName_preferences\",\"fields\":[{\"name\":\"feature1\"," + + "\"type\":\"boolean\"},{\"name\":\"feature2\",\"type\":[\"null\",\"boolean\"],\"default\":null}]}]," + + "\"default\":null},{\"name\":\"locations\",\"type\":{\"type\":\"map\",\"values\":{\"type\":\"record\"," + + "\"name\":\"newTableName_locations\",\"fields\":[{\"name\":\"lat\",\"type\":\"float\"},{\"name\":\"long\"," + + "\"type\":\"float\"}]}}},{\"name\":\"points\",\"type\":[\"null\",{\"type\":\"array\",\"items\":[\"null\"," + + "{\"type\":\"record\",\"name\":\"newTableName_points\",\"fields\":[{\"name\":\"x\",\"type\":\"long\"}," + + "{\"name\":\"y\",\"type\":\"long\"}]}]}],\"default\":null},{\"name\":\"doubles\",\"type\":{\"type\":\"array\",\"items\":\"double\"}}," + + "{\"name\":\"properties\",\"type\":[\"null\",{\"type\":\"map\",\"values\":[\"null\",\"string\"]}],\"default\":null}]}"; + @Test public void testPrimitiveTypes() { Schema[] avroPrimitives = new Schema[] { @@ -146,16 +158,6 @@ public void testArrayType() { @Test public void testComplexConvert() { - String schemaStr = "{\"type\":\"record\",\"name\":\"newTableName\",\"fields\":[{\"name\":\"id\",\"type\":\"int\"},{\"name\":\"data\"," - + "\"type\":[\"null\",\"string\"],\"default\":null},{\"name\":\"preferences\",\"type\":[\"null\"," - + "{\"type\":\"record\",\"name\":\"newTableName_preferences\",\"fields\":[{\"name\":\"feature1\"," - + "\"type\":\"boolean\"},{\"name\":\"feature2\",\"type\":[\"null\",\"boolean\"],\"default\":null}]}]," - + "\"default\":null},{\"name\":\"locations\",\"type\":{\"type\":\"map\",\"values\":{\"type\":\"record\"," - + "\"name\":\"newTableName_locations\",\"fields\":[{\"name\":\"lat\",\"type\":\"float\"},{\"name\":\"long\"," - + "\"type\":\"float\"}]}}},{\"name\":\"points\",\"type\":[\"null\",{\"type\":\"array\",\"items\":[\"null\"," - + "{\"type\":\"record\",\"name\":\"newTableName_points\",\"fields\":[{\"name\":\"x\",\"type\":\"long\"}," - + "{\"name\":\"y\",\"type\":\"long\"}]}]}],\"default\":null},{\"name\":\"doubles\",\"type\":{\"type\":\"array\",\"items\":\"double\"}}," - + "{\"name\":\"properties\",\"type\":[\"null\",{\"type\":\"map\",\"values\":[\"null\",\"string\"]}],\"default\":null}]}"; Schema schema = new Schema.Parser().parse(schemaStr); InternalSchema internalSchema = new InternalSchema(Types.Field.get(0, false, "id", Types.IntType.get()), @@ -284,7 +286,7 @@ public void testReWriteRecordWithTypeChanged() { .updateColumnType("col6", Types.StringType.get()); InternalSchema newSchema = SchemaChangeUtils.applyTableChanges2Schema(internalSchema, updateChange); Schema newAvroSchema = AvroInternalSchemaConverter.convert(newSchema, avroSchema.getName()); - GenericRecord newRecord = HoodieAvroUtils.rewriteRecordWithNewSchema(avroRecord, newAvroSchema, new HashMap<>()); + GenericRecord newRecord = HoodieAvroUtils.rewriteRecordWithNewSchema(avroRecord, newAvroSchema, Collections.emptyMap()); Assertions.assertEquals(GenericData.get().validate(newAvroSchema, newRecord), true); } @@ -349,9 +351,26 @@ public void testReWriteNestRecord() { ); Schema newAvroSchema = AvroInternalSchemaConverter.convert(newRecord, schema.getName()); - GenericRecord newAvroRecord = HoodieAvroUtils.rewriteRecordWithNewSchema(avroRecord, newAvroSchema, new HashMap<>()); + GenericRecord newAvroRecord = HoodieAvroUtils.rewriteRecordWithNewSchema(avroRecord, newAvroSchema, Collections.emptyMap()); // test the correctly of rewrite Assertions.assertEquals(GenericData.get().validate(newAvroSchema, newAvroRecord), true); + + // test rewrite with rename + InternalSchema internalSchema = AvroInternalSchemaConverter.convert(schema); + // do change rename operation + TableChanges.ColumnUpdateChange updateChange = TableChanges.ColumnUpdateChange.get(internalSchema); + updateChange + .renameColumn("id", "idx") + .renameColumn("data", "datax") + .renameColumn("preferences.feature1", "f1") + .renameColumn("preferences.feature2", "f2") + .renameColumn("locations.value.lat", "lt"); + InternalSchema internalSchemaRename = SchemaChangeUtils.applyTableChanges2Schema(internalSchema, updateChange); + Schema avroSchemaRename = AvroInternalSchemaConverter.convert(internalSchemaRename, schema.getName()); + Map renameCols = InternalSchemaUtils.collectRenameCols(internalSchema, internalSchemaRename); + GenericRecord avroRecordRename = HoodieAvroUtils.rewriteRecordWithNewSchema(avroRecord, avroSchemaRename, renameCols); + // test the correctly of rewrite + Assertions.assertEquals(GenericData.get().validate(avroSchemaRename, avroRecordRename), true); } @Test @@ -395,7 +414,7 @@ public void testEvolutionSchemaFromNewAvroSchema() { ); evolvedRecord = (Types.RecordType)InternalSchemaBuilder.getBuilder().refreshNewId(evolvedRecord, new AtomicInteger(0)); Schema evolvedAvroSchema = AvroInternalSchemaConverter.convert(evolvedRecord, "test1"); - InternalSchema result = AvroSchemaEvolutionUtils.evolveSchemaFromNewAvroSchema(evolvedAvroSchema, oldSchema); + InternalSchema result = AvroSchemaEvolutionUtils.reconcileSchema(evolvedAvroSchema, oldSchema); Types.RecordType checkedRecord = Types.RecordType.get( Types.Field.get(0, false, "id", Types.IntType.get()), Types.Field.get(1, true, "data", Types.StringType.get()), @@ -419,4 +438,37 @@ public void testEvolutionSchemaFromNewAvroSchema() { ); Assertions.assertEquals(result.getRecord(), checkedRecord); } + + @Test + public void testReconcileSchema() { + // simple schema test + // a: boolean, b: int, c: long, d: date + Schema schema = create("simple", + new Schema.Field("a", AvroInternalSchemaConverter.nullableSchema(Schema.create(Schema.Type.BOOLEAN)), null, JsonProperties.NULL_VALUE), + new Schema.Field("b", AvroInternalSchemaConverter.nullableSchema(Schema.create(Schema.Type.INT)), null, JsonProperties.NULL_VALUE), + new Schema.Field("c", AvroInternalSchemaConverter.nullableSchema(Schema.create(Schema.Type.LONG)), null, JsonProperties.NULL_VALUE), + new Schema.Field("d", AvroInternalSchemaConverter.nullableSchema(LogicalTypes.date().addToSchema(Schema.create(Schema.Type.INT))), null, JsonProperties.NULL_VALUE)); + // a: boolean, c: long, c_1: long, d: date + Schema incomingSchema = create("simpleIncoming", + new Schema.Field("a", AvroInternalSchemaConverter.nullableSchema(Schema.create(Schema.Type.BOOLEAN)), null, JsonProperties.NULL_VALUE), + new Schema.Field("a1", AvroInternalSchemaConverter.nullableSchema(Schema.create(Schema.Type.LONG)), null, JsonProperties.NULL_VALUE), + new Schema.Field("c", AvroInternalSchemaConverter.nullableSchema(Schema.create(Schema.Type.LONG)), null, JsonProperties.NULL_VALUE), + new Schema.Field("c1", AvroInternalSchemaConverter.nullableSchema(Schema.create(Schema.Type.LONG)), null, JsonProperties.NULL_VALUE), + new Schema.Field("c2", AvroInternalSchemaConverter.nullableSchema(Schema.create(Schema.Type.LONG)), null, JsonProperties.NULL_VALUE), + new Schema.Field("d", AvroInternalSchemaConverter.nullableSchema(LogicalTypes.date().addToSchema(Schema.create(Schema.Type.INT))), null, JsonProperties.NULL_VALUE), + new Schema.Field("d1", AvroInternalSchemaConverter.nullableSchema(LogicalTypes.date().addToSchema(Schema.create(Schema.Type.INT))), null, JsonProperties.NULL_VALUE), + new Schema.Field("d2", AvroInternalSchemaConverter.nullableSchema(LogicalTypes.date().addToSchema(Schema.create(Schema.Type.INT))), null, JsonProperties.NULL_VALUE)); + + Schema simpleCheckSchema = new Schema.Parser().parse("{\"type\":\"record\",\"name\":\"simpleReconcileSchema\",\"fields\":[{\"name\":\"a\",\"type\":[\"null\",\"boolean\"],\"default\":null}," + + "{\"name\":\"b\",\"type\":[\"null\",\"int\"],\"default\":null},{\"name\":\"a1\",\"type\":[\"null\",\"long\"],\"default\":null}," + + "{\"name\":\"c\",\"type\":[\"null\",\"long\"],\"default\":null}," + + "{\"name\":\"c1\",\"type\":[\"null\",\"long\"],\"default\":null},{\"name\":\"c2\",\"type\":[\"null\",\"long\"],\"default\":null}," + + "{\"name\":\"d\",\"type\":[\"null\",{\"type\":\"int\",\"logicalType\":\"date\"}],\"default\":null}," + + "{\"name\":\"d1\",\"type\":[\"null\",{\"type\":\"int\",\"logicalType\":\"date\"}],\"default\":null}," + + "{\"name\":\"d2\",\"type\":[\"null\",{\"type\":\"int\",\"logicalType\":\"date\"}],\"default\":null}]}"); + + Schema simpleReconcileSchema = AvroInternalSchemaConverter.convert(AvroSchemaEvolutionUtils + .reconcileSchema(incomingSchema, AvroInternalSchemaConverter.convert(schema)), "simpleReconcileSchema"); + Assertions.assertEquals(simpleReconcileSchema, simpleCheckSchema); + } } diff --git a/hudi-common/src/test/resources/external-config/hudi-defaults.conf b/hudi-common/src/test/resources/external-config/hudi-defaults.conf index 1133adb4d7735..2e4c3a5d75429 100644 --- a/hudi-common/src/test/resources/external-config/hudi-defaults.conf +++ b/hudi-common/src/test/resources/external-config/hudi-defaults.conf @@ -20,7 +20,7 @@ # Example: hoodie.datasource.hive_sync.jdbcurl jdbc:hive2://localhost:10000 -hoodie.datasource.hive_sync.use_jdbc true +hoodie.datasource.hive_sync.mode jdbc hoodie.datasource.hive_sync.support_timestamp false hoodie.index.type BLOOM hoodie.metadata.enable true diff --git a/hudi-examples/hudi-examples-java/src/main/java/org/apache/hudi/examples/java/HoodieJavaWriteClientExample.java b/hudi-examples/hudi-examples-java/src/main/java/org/apache/hudi/examples/java/HoodieJavaWriteClientExample.java index 4890a6529a52c..6e20ee1190661 100644 --- a/hudi-examples/hudi-examples-java/src/main/java/org/apache/hudi/examples/java/HoodieJavaWriteClientExample.java +++ b/hudi-examples/hudi-examples-java/src/main/java/org/apache/hudi/examples/java/HoodieJavaWriteClientExample.java @@ -27,7 +27,7 @@ import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.table.HoodieTableMetaClient; -import org.apache.hudi.config.HoodieCompactionConfig; +import org.apache.hudi.config.HoodieArchivalConfig; import org.apache.hudi.config.HoodieIndexConfig; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.examples.common.HoodieExampleDataGenerator; @@ -85,7 +85,7 @@ public static void main(String[] args) throws Exception { .withSchema(HoodieExampleDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2) .withDeleteParallelism(2).forTable(tableName) .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.INMEMORY).build()) - .withCompactionConfig(HoodieCompactionConfig.newBuilder().archiveCommitsWith(20, 30).build()).build(); + .withArchivalConfig(HoodieArchivalConfig.newBuilder().archiveCommitsWith(20, 30).build()).build(); HoodieJavaWriteClient client = new HoodieJavaWriteClient<>(new HoodieJavaEngineContext(hadoopConf), cfg); diff --git a/hudi-examples/hudi-examples-spark/src/main/java/org/apache/hudi/examples/spark/HoodieWriteClientExample.java b/hudi-examples/hudi-examples-spark/src/main/java/org/apache/hudi/examples/spark/HoodieWriteClientExample.java index 1afc180531a16..299fe992fa00a 100644 --- a/hudi-examples/hudi-examples-spark/src/main/java/org/apache/hudi/examples/spark/HoodieWriteClientExample.java +++ b/hudi-examples/hudi-examples-spark/src/main/java/org/apache/hudi/examples/spark/HoodieWriteClientExample.java @@ -29,7 +29,7 @@ import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.util.Option; -import org.apache.hudi.config.HoodieCompactionConfig; +import org.apache.hudi.config.HoodieArchivalConfig; import org.apache.hudi.config.HoodieIndexConfig; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.examples.common.HoodieExampleDataGenerator; @@ -99,7 +99,7 @@ public static void main(String[] args) throws Exception { .withSchema(HoodieExampleDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2) .withDeleteParallelism(2).forTable(tableName) .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build()) - .withCompactionConfig(HoodieCompactionConfig.newBuilder().archiveCommitsWith(20, 30).build()).build(); + .withArchivalConfig(HoodieArchivalConfig.newBuilder().archiveCommitsWith(20, 30).build()).build(); SparkRDDWriteClient client = new SparkRDDWriteClient<>(new HoodieSparkEngineContext(jsc), cfg); // inserts diff --git a/hudi-flink-datasource/hudi-flink/pom.xml b/hudi-flink-datasource/hudi-flink/pom.xml index 5ad323f93442d..4a22206e06c32 100644 --- a/hudi-flink-datasource/hudi-flink/pom.xml +++ b/hudi-flink-datasource/hudi-flink/pom.xml @@ -269,7 +269,30 @@ - + + ${hive.groupid} + hive-metastore + ${hive.version} + provided + + + javax.transaction + jta + + + javax.transaction + transaction-api + + + javax.mail + mail + + + org.eclipse.jetty.aggregate + * + + + 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 7dc5deb791d4d..b1a3372e08930 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 @@ -755,12 +755,6 @@ private FlinkOptions() { .defaultValue(false) .withDescription("Assume partitioning is yyyy/mm/dd, default false"); - public static final ConfigOption HIVE_SYNC_USE_JDBC = ConfigOptions - .key("hive_sync.use_jdbc") - .booleanType() - .defaultValue(true) - .withDescription("Use JDBC when hive synchronization is enabled, default true"); - public static final ConfigOption HIVE_SYNC_AUTO_CREATE_DB = ConfigOptions .key("hive_sync.auto_create_db") .booleanType() diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/configuration/HadoopConfigurations.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/configuration/HadoopConfigurations.java index 8666151fe492d..db696a322ed71 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/configuration/HadoopConfigurations.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/configuration/HadoopConfigurations.java @@ -23,6 +23,9 @@ import org.apache.flink.configuration.Configuration; import org.apache.hadoop.fs.Path; +import java.io.File; +import java.util.ArrayList; +import java.util.List; import java.util.Map; /** @@ -54,6 +57,45 @@ public static org.apache.hadoop.conf.Configuration getHadoopConf(Configuration c return hadoopConf; } + /** + * Returns a new hadoop configuration that is initialized with the given hadoopConfDir. + * + * @param hadoopConfDir Hadoop conf directory path. + * @return A Hadoop configuration instance. + */ + public static org.apache.hadoop.conf.Configuration getHadoopConfiguration(String hadoopConfDir) { + if (new File(hadoopConfDir).exists()) { + List possiableConfFiles = new ArrayList(); + File coreSite = new File(hadoopConfDir, "core-site.xml"); + if (coreSite.exists()) { + possiableConfFiles.add(coreSite); + } + File hdfsSite = new File(hadoopConfDir, "hdfs-site.xml"); + if (hdfsSite.exists()) { + possiableConfFiles.add(hdfsSite); + } + File yarnSite = new File(hadoopConfDir, "yarn-site.xml"); + if (yarnSite.exists()) { + possiableConfFiles.add(yarnSite); + } + // Add mapred-site.xml. We need to read configurations like compression codec. + File mapredSite = new File(hadoopConfDir, "mapred-site.xml"); + if (mapredSite.exists()) { + possiableConfFiles.add(mapredSite); + } + if (possiableConfFiles.isEmpty()) { + return null; + } else { + org.apache.hadoop.conf.Configuration hadoopConfiguration = new org.apache.hadoop.conf.Configuration(); + for (File confFile : possiableConfFiles) { + hadoopConfiguration.addResource(new Path(confFile.getAbsolutePath())); + } + return hadoopConfiguration; + } + } + return null; + } + /** * Creates a Hive configuration with configured dir path or empty if no Hive conf dir is set. */ diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/CompactionPlanSourceFunction.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/CompactionPlanSourceFunction.java index fe55089988e53..883ba8bd114cd 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/CompactionPlanSourceFunction.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/CompactionPlanSourceFunction.java @@ -20,6 +20,7 @@ import org.apache.hudi.avro.model.HoodieCompactionPlan; import org.apache.hudi.common.model.CompactionOperation; +import org.apache.hudi.common.util.collection.Pair; import org.apache.flink.api.common.functions.AbstractRichFunction; import org.apache.flink.configuration.Configuration; @@ -28,8 +29,7 @@ import org.slf4j.LoggerFactory; import java.util.List; - -import static java.util.stream.Collectors.toList; +import java.util.stream.Collectors; /** * Flink hudi compaction source function. @@ -53,18 +53,12 @@ public class CompactionPlanSourceFunction extends AbstractRichFunction implement protected static final Logger LOG = LoggerFactory.getLogger(CompactionPlanSourceFunction.class); /** - * Compaction instant time. - */ - private final String compactionInstantTime; - - /** - * The compaction plan. + * compaction plan instant -> compaction plan */ - private final HoodieCompactionPlan compactionPlan; + private final List> compactionPlans; - public CompactionPlanSourceFunction(HoodieCompactionPlan compactionPlan, String compactionInstantTime) { - this.compactionPlan = compactionPlan; - this.compactionInstantTime = compactionInstantTime; + public CompactionPlanSourceFunction(List> compactionPlans) { + this.compactionPlans = compactionPlans; } @Override @@ -74,11 +68,14 @@ public void open(Configuration parameters) throws Exception { @Override public void run(SourceContext sourceContext) throws Exception { - List operations = this.compactionPlan.getOperations().stream() - .map(CompactionOperation::convertFromAvroRecordInstance).collect(toList()); - LOG.info("CompactionPlanFunction compacting " + operations + " files"); - for (CompactionOperation operation : operations) { - sourceContext.collect(new CompactionPlanEvent(compactionInstantTime, operation)); + for (Pair pair : compactionPlans) { + HoodieCompactionPlan compactionPlan = pair.getRight(); + List operations = compactionPlan.getOperations().stream() + .map(CompactionOperation::convertFromAvroRecordInstance).collect(Collectors.toList()); + LOG.info("CompactionPlanFunction compacting " + operations + " files"); + for (CompactionOperation operation : operations) { + sourceContext.collect(new CompactionPlanEvent(pair.getLeft(), operation)); + } } } diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/FlinkCompactionConfig.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/FlinkCompactionConfig.java index 02041690f1dec..449b06846156c 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/FlinkCompactionConfig.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/FlinkCompactionConfig.java @@ -20,10 +20,10 @@ import org.apache.hudi.config.HoodieMemoryConfig; import org.apache.hudi.configuration.FlinkOptions; +import org.apache.hudi.sink.compact.strategy.CompactionPlanStrategy; import com.beust.jcommander.Parameter; import org.apache.flink.configuration.Configuration; -import org.apache.hudi.sink.compact.strategy.SingleCompactionPlanSelectStrategy; /** * Configurations for Hoodie Flink compaction. @@ -102,7 +102,7 @@ public class FlinkCompactionConfig extends Configuration { @Parameter(names = {"--seq"}, description = "Compaction plan execution sequence, two options are supported:\n" + "1). FIFO: execute the oldest plan first;\n" + "2). LIFO: execute the latest plan first, by default LIFO", required = false) - public String compactionSeq = SEQ_LIFO; + public String compactionSeq = SEQ_FIFO; @Parameter(names = {"--service"}, description = "Flink Compaction runs in service mode, disable by default") public Boolean serviceMode = false; @@ -111,21 +111,21 @@ public class FlinkCompactionConfig extends Configuration { description = "Min compaction interval of async compaction service, default 10 minutes") public Integer minCompactionIntervalSeconds = 600; - @Parameter(names = {"--select-strategy"}, description = "The strategy define how to select compaction plan to compact.\n" - + "1). SingleCompactionPlanSelectStrategy: Select first or last compaction plan." - + "2). MultiCompactionPlanSelectStrategy: Select first or last n compaction plan (n is defined by compactionPlanMaxSelect)." - + "3). AllPendingCompactionPlanSelectStrategy: Select all pending compaction plan" - + "4). InstantCompactionPlanSelectStrategy: Select the compaction plan that instant is specified by compactionPlanInstant") - public String compactionPlanSelectStrategy = SingleCompactionPlanSelectStrategy.class.getName(); + @Parameter(names = {"--plan-select-strategy"}, description = "The strategy define how to select compaction plan to compact.\n" + + "1). num_instants: select plans by specific number of instants, it's the default strategy with 1 instant at a time;\n" + + "3). all: Select all pending compaction plan;\n" + + "4). instants: Select the compaction plan by specific instants") + public String compactionPlanSelectStrategy = CompactionPlanStrategy.NUM_INSTANTS; - @Parameter(names = {"--select-max-number"}, description = "Max number of compaction plan would be selected in compaction." + @Parameter(names = {"--max-num-plans"}, description = "Max number of compaction plan would be selected in compaction." + "It's only effective for MultiCompactionPlanSelectStrategy.") - public Integer compactionPlanMaxSelect = 10; + public Integer maxNumCompactionPlans = 1; - @Parameter(names = {"--select-instant"}, description = "Specify the compaction plan instant to compact and you can specify more than" - + "one instant in a time by using comma." - + "It's only effective for InstantCompactionPlanSelectStrategy.") + @Parameter(names = {"--target-instants"}, description = "Specify the compaction plan instants to compact,\n" + + "Multiple instants are supported by comma separated instant time.\n" + + "It's only effective for 'instants' plan selection strategy.") public String compactionPlanInstant; + @Parameter(names = {"--spillable_map_path"}, description = "Default file path prefix for spillable map.", required = false) public String spillableMapPath = HoodieMemoryConfig.SPILLABLE_MAP_BASE_PATH.defaultValue(); 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 f56b5a2f0fb1d..e2d2972a0de43 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 @@ -18,11 +18,6 @@ package org.apache.hudi.sink.compact; -import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator; -import org.apache.flink.streaming.api.functions.ProcessFunction; -import org.apache.flink.util.Collector; -import org.apache.flink.util.OutputTag; import org.apache.hudi.async.HoodieAsyncTableService; import org.apache.hudi.avro.model.HoodieCompactionPlan; import org.apache.hudi.client.HoodieFlinkWriteClient; @@ -31,11 +26,10 @@ import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.util.CompactionUtils; import org.apache.hudi.common.util.Option; -import org.apache.hudi.common.util.ReflectionUtils; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.configuration.FlinkOptions; import org.apache.hudi.exception.HoodieException; -import org.apache.hudi.sink.compact.strategy.CompactionPlanSelectStrategy; +import org.apache.hudi.sink.compact.strategy.CompactionPlanStrategies; import org.apache.hudi.table.HoodieFlinkTable; import org.apache.hudi.util.CompactionUtil; import org.apache.hudi.util.StreamerUtil; @@ -228,9 +222,8 @@ private void compact() throws Exception { } // fetch the instant based on the configured execution sequence - HoodieTimeline timeline = table.getActiveTimeline(); - List requested = ((CompactionPlanSelectStrategy) ReflectionUtils.loadClass(cfg.compactionPlanSelectStrategy)) - .select(timeline.filterPendingCompactionTimeline(), cfg); + HoodieTimeline pendingCompactionTimeline = table.getActiveTimeline().filterPendingCompactionTimeline(); + List requested = CompactionPlanStrategies.getStrategy(cfg).select(pendingCompactionTimeline); if (requested.isEmpty()) { // do nothing. LOG.info("No compaction plan scheduled, turns on the compaction plan schedule with --schedule option"); @@ -240,7 +233,7 @@ private void compact() throws Exception { List compactionInstantTimes = requested.stream().map(HoodieInstant::getTimestamp).collect(Collectors.toList()); compactionInstantTimes.forEach(timestamp -> { HoodieInstant inflightInstant = HoodieTimeline.getCompactionInflightInstant(timestamp); - if (timeline.containsInstant(inflightInstant)) { + if (pendingCompactionTimeline.containsInstant(inflightInstant)) { LOG.info("Rollback inflight compaction instant: [" + timestamp + "]"); table.rollbackInflightCompaction(inflightInstant); table.getMetaClient().reloadActiveTimeline(); @@ -254,13 +247,11 @@ private void compact() throws Exception { try { return Pair.of(timestamp, CompactionUtils.getCompactionPlan(table.getMetaClient(), timestamp)); } catch (IOException e) { - throw new HoodieException(e); + throw new HoodieException("Get compaction plan at instant " + timestamp + " error", e); } }) // reject empty compaction plan - .filter(pair -> !(pair.getRight() == null - || pair.getRight().getOperations() == null - || pair.getRight().getOperations().isEmpty())) + .filter(pair -> validCompactionPlan(pair.getRight())) .collect(Collectors.toList()); if (compactionPlans.isEmpty()) { @@ -270,7 +261,6 @@ private void compact() throws Exception { } List instants = compactionInstantTimes.stream().map(HoodieTimeline::getCompactionRequestedInstant).collect(Collectors.toList()); - HoodieTimeline pendingCompactionTimeline = table.getActiveTimeline().filterPendingCompactionTimeline(); for (HoodieInstant instant : instants) { if (!pendingCompactionTimeline.containsInstant(instant)) { // this means that the compaction plan was written to auxiliary path(.tmp) @@ -297,34 +287,19 @@ private void compact() throws Exception { } table.getMetaClient().reloadActiveTimeline(); - // use side-output to make operations that is in the same plan to be placed in the same stream - // keyby() cannot sure that different operations are in the different stream - DataStream source = env.addSource(new MultiCompactionPlanSourceFunction(compactionPlans)) + env.addSource(new CompactionPlanSourceFunction(compactionPlans)) .name("compaction_source") - .uid("uid_compaction_source"); - - SingleOutputStreamOperator operator = source.rebalance() + .uid("uid_compaction_source") + .rebalance() .transform("compact_task", TypeInformation.of(CompactionCommitEvent.class), new ProcessOperator<>(new CompactFunction(conf))) .setParallelism(compactionParallelism) - .process(new ProcessFunction() { - @Override - public void processElement(CompactionCommitEvent event, ProcessFunction.Context context, Collector out) { - context.output(new OutputTag<>(event.getInstant(), TypeInformation.of(CompactionCommitEvent.class)), event); - } - }) - .name("group_by_compaction_plan") - .uid("uid_group_by_compaction_plan") + .addSink(new CompactionCommitSink(conf)) + .name("compaction_commit") + .uid("uid_compaction_commit") .setParallelism(1); - compactionPlans.forEach(pair -> - operator.getSideOutput(new OutputTag<>(pair.getLeft(), TypeInformation.of(CompactionCommitEvent.class))) - .addSink(new CompactionCommitSink(conf)) - .name("clean_commits " + pair.getLeft()) - .uid("uid_clean_commits_" + pair.getLeft()) - .setParallelism(1)); - env.execute("flink_hudi_compaction_" + String.join(",", compactionInstantTimes)); } @@ -342,4 +317,8 @@ public void shutDown() { shutdownAsyncService(false); } } + + private static boolean validCompactionPlan(HoodieCompactionPlan plan) { + return plan != null && plan.getOperations() != null && plan.getOperations().size() > 0; + } } diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/MultiCompactionPlanSourceFunction.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/MultiCompactionPlanSourceFunction.java deleted file mode 100644 index 8a8c3f6b4eeb3..0000000000000 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/MultiCompactionPlanSourceFunction.java +++ /dev/null @@ -1,90 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hudi.sink.compact; - -import static java.util.stream.Collectors.toList; - -import java.util.List; -import org.apache.flink.api.common.functions.AbstractRichFunction; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.streaming.api.functions.source.SourceFunction; -import org.apache.hudi.avro.model.HoodieCompactionPlan; -import org.apache.hudi.common.model.CompactionOperation; -import org.apache.hudi.common.util.collection.Pair; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * Flink hudi compaction source function. - * - *

This function read the compaction plan as {@link CompactionOperation}s then assign the compaction task - * event {@link CompactionPlanEvent} to downstream operators. - * - *

The compaction instant time is specified explicitly with strategies: - * - *

    - *
  • If the timeline has no inflight instants, - * use {@link org.apache.hudi.common.table.timeline.HoodieActiveTimeline#createNewInstantTime()} - * as the instant time;
  • - *
  • If the timeline has inflight instants, - * use the median instant time between [last complete instant time, earliest inflight instant time] - * as the instant time.
  • - *
- */ -public class MultiCompactionPlanSourceFunction extends AbstractRichFunction implements SourceFunction { - - protected static final Logger LOG = LoggerFactory.getLogger(MultiCompactionPlanSourceFunction.class); - - /** - * compaction plan instant -> compaction plan - */ - private final List> compactionPlans; - - public MultiCompactionPlanSourceFunction(List> compactionPlans) { - this.compactionPlans = compactionPlans; - } - - @Override - public void open(Configuration parameters) throws Exception { - // no operation - } - - @Override - public void run(SourceContext sourceContext) throws Exception { - for (Pair pair : compactionPlans) { - HoodieCompactionPlan compactionPlan = pair.getRight(); - List operations = compactionPlan.getOperations().stream() - .map(CompactionOperation::convertFromAvroRecordInstance).collect(toList()); - LOG.info("CompactionPlanFunction compacting " + operations + " files"); - for (CompactionOperation operation : operations) { - sourceContext.collect(new CompactionPlanEvent(pair.getLeft(), operation)); - } - } - } - - @Override - public void close() throws Exception { - // no operation - } - - @Override - public void cancel() { - // no operation - } -} diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/strategy/CompactionPlanStrategies.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/strategy/CompactionPlanStrategies.java new file mode 100644 index 0000000000000..662dcabda3220 --- /dev/null +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/strategy/CompactionPlanStrategies.java @@ -0,0 +1,74 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.sink.compact.strategy; + +import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.util.StringUtils; +import org.apache.hudi.sink.compact.FlinkCompactionConfig; +import org.apache.hudi.util.CompactionUtil; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Locale; +import java.util.stream.Collectors; + +/** + * Factory clazz for CompactionPlanStrategy. + */ +public class CompactionPlanStrategies { + private static final Logger LOG = LoggerFactory.getLogger(CompactionPlanStrategies.class); + + private CompactionPlanStrategies() { + } + + public static CompactionPlanStrategy getStrategy(FlinkCompactionConfig config) { + switch (config.compactionPlanSelectStrategy.toLowerCase(Locale.ROOT)) { + case CompactionPlanStrategy.ALL: + return pendingCompactionTimeline -> pendingCompactionTimeline.getInstants().collect(Collectors.toList()); + case CompactionPlanStrategy.INSTANTS: + return pendingCompactionTimeline -> { + if (StringUtils.isNullOrEmpty(config.compactionPlanInstant)) { + LOG.warn("None instant is selected"); + return Collections.emptyList(); + } + List instants = Arrays.asList(config.compactionPlanInstant.split(",")); + return pendingCompactionTimeline.getInstants() + .filter(instant -> instants.contains(instant.getTimestamp())) + .collect(Collectors.toList()); + }; + case CompactionPlanStrategy.NUM_INSTANTS: + return pendingCompactionTimeline -> { + List pendingCompactionPlanInstants = pendingCompactionTimeline.getInstants().collect(Collectors.toList()); + if (CompactionUtil.isLIFO(config.compactionSeq)) { + Collections.reverse(pendingCompactionPlanInstants); + } + int range = Math.min(config.maxNumCompactionPlans, pendingCompactionPlanInstants.size()); + return pendingCompactionPlanInstants.subList(0, range); + }; + default: + throw new UnsupportedOperationException("Unknown compaction plan strategy: " + + config.compactionPlanSelectStrategy + + ", supported strategies:[num_instants,instants,all]"); + } + } +} diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/strategy/CompactionPlanSelectStrategy.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/strategy/CompactionPlanStrategy.java similarity index 81% rename from hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/strategy/CompactionPlanSelectStrategy.java rename to hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/strategy/CompactionPlanStrategy.java index a41fcef198139..e209ff53391fc 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/strategy/CompactionPlanSelectStrategy.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/strategy/CompactionPlanStrategy.java @@ -18,17 +18,21 @@ package org.apache.hudi.sink.compact.strategy; -import java.util.List; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; -import org.apache.hudi.sink.compact.FlinkCompactionConfig; + +import java.util.List; /** - * CompactionRangeStrategy + * Compaction plan selection strategy. */ -public interface CompactionPlanSelectStrategy { +public interface CompactionPlanStrategy { + String ALL = "all"; + String INSTANTS = "instants"; + String NUM_INSTANTS = "num_instants"; + /** - * Define how to select compaction plan to compact + * Define how to select compaction plan to compact. */ - List select(HoodieTimeline pendingCompactionTimeline, FlinkCompactionConfig config); + List select(HoodieTimeline pendingCompactionTimeline); } diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/strategy/InstantCompactionPlanSelectStrategy.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/strategy/InstantCompactionPlanSelectStrategy.java deleted file mode 100644 index 45382b70c4def..0000000000000 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/strategy/InstantCompactionPlanSelectStrategy.java +++ /dev/null @@ -1,50 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hudi.sink.compact.strategy; - -import java.util.Arrays; -import java.util.Collections; -import java.util.List; -import java.util.stream.Collectors; -import org.apache.hudi.common.table.timeline.HoodieInstant; -import org.apache.hudi.common.table.timeline.HoodieTimeline; -import org.apache.hudi.common.util.StringUtils; -import org.apache.hudi.sink.compact.FlinkCompactionConfig; -import org.apache.hudi.sink.compact.HoodieFlinkCompactor; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * Specify the compaction plan instant to compact - */ -public class InstantCompactionPlanSelectStrategy implements CompactionPlanSelectStrategy { - protected static final Logger LOG = LoggerFactory.getLogger(HoodieFlinkCompactor.class); - - @Override - public List select(HoodieTimeline pendingCompactionTimeline, FlinkCompactionConfig config) { - if (StringUtils.isNullOrEmpty(config.compactionPlanInstant)) { - LOG.warn("None instant is selected"); - return Collections.emptyList(); - } - List instants = Arrays.asList(config.compactionPlanInstant.split(",")); - return pendingCompactionTimeline.getInstants() - .filter(instant -> instants.contains(instant.getTimestamp())) - .collect(Collectors.toList()); - } -} diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/strategy/MultiCompactionPlanSelectStrategy.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/strategy/MultiCompactionPlanSelectStrategy.java deleted file mode 100644 index ee0e93653f87d..0000000000000 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/strategy/MultiCompactionPlanSelectStrategy.java +++ /dev/null @@ -1,42 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hudi.sink.compact.strategy; - -import java.util.Collections; -import java.util.List; -import java.util.stream.Collectors; -import org.apache.hudi.common.table.timeline.HoodieInstant; -import org.apache.hudi.common.table.timeline.HoodieTimeline; -import org.apache.hudi.sink.compact.FlinkCompactionConfig; -import org.apache.hudi.util.CompactionUtil; - -/** - * Select multi compaction plan to compact - */ -public class MultiCompactionPlanSelectStrategy implements CompactionPlanSelectStrategy { - @Override - public List select(HoodieTimeline pendingCompactionTimeline, FlinkCompactionConfig config) { - List pendingCompactionPlanInstants = pendingCompactionTimeline.getInstants().collect(Collectors.toList()); - if (CompactionUtil.isLIFO(config.compactionSeq)) { - Collections.reverse(pendingCompactionPlanInstants); - } - int range = Math.min(config.compactionPlanMaxSelect, pendingCompactionPlanInstants.size()); - return pendingCompactionPlanInstants.subList(0, range); - } -} diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/strategy/SingleCompactionPlanSelectStrategy.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/strategy/SingleCompactionPlanSelectStrategy.java deleted file mode 100644 index 7ca939866ceec..0000000000000 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/strategy/SingleCompactionPlanSelectStrategy.java +++ /dev/null @@ -1,43 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hudi.sink.compact.strategy; - -import java.util.Collections; -import java.util.List; -import org.apache.hudi.common.table.timeline.HoodieInstant; -import org.apache.hudi.common.table.timeline.HoodieTimeline; -import org.apache.hudi.common.util.Option; -import org.apache.hudi.sink.compact.FlinkCompactionConfig; -import org.apache.hudi.util.CompactionUtil; - -/** - * Select one compaction plan to compact - */ -public class SingleCompactionPlanSelectStrategy implements CompactionPlanSelectStrategy { - @Override - public List select(HoodieTimeline pendingCompactionTimeline, FlinkCompactionConfig config) { - Option compactionPlanInstant = CompactionUtil.isLIFO(config.compactionSeq) - ? pendingCompactionTimeline.lastInstant() - : pendingCompactionTimeline.firstInstant(); - if (compactionPlanInstant.isPresent()) { - return Collections.singletonList(compactionPlanInstant.get()); - } - return Collections.emptyList(); - } -} diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/utils/HiveSyncContext.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/utils/HiveSyncContext.java index e34adac580f70..cceab5a6157c9 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/utils/HiveSyncContext.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/utils/HiveSyncContext.java @@ -43,7 +43,6 @@ import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_TABLE_SERDE_PROPERTIES; import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_URL; import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_USER; -import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_USE_JDBC; import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_USE_PRE_APACHE_INPUT_FORMAT; import static org.apache.hudi.hive.HiveSyncConfigHolder.METASTORE_URIS; import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_ASSUME_DATE_PARTITION; @@ -105,7 +104,6 @@ public static Properties buildSyncConfig(Configuration conf) { props.setPropertyIfNonNull(HIVE_TABLE_SERDE_PROPERTIES.key(), conf.getString(FlinkOptions.HIVE_SYNC_TABLE_SERDE_PROPERTIES)); props.setPropertyIfNonNull(META_SYNC_PARTITION_FIELDS.key(), String.join(",", FilePathUtils.extractHivePartitionFields(conf))); props.setPropertyIfNonNull(META_SYNC_PARTITION_EXTRACTOR_CLASS.key(), conf.getString(FlinkOptions.HIVE_SYNC_PARTITION_EXTRACTOR_CLASS_NAME)); - props.setPropertyIfNonNull(HIVE_USE_JDBC.key(), String.valueOf(conf.getBoolean(FlinkOptions.HIVE_SYNC_USE_JDBC))); props.setPropertyIfNonNull(META_SYNC_USE_FILE_LISTING_FROM_METADATA.key(), String.valueOf(conf.getBoolean(FlinkOptions.METADATA_ENABLED))); props.setPropertyIfNonNull(HIVE_IGNORE_EXCEPTIONS.key(), String.valueOf(conf.getBoolean(FlinkOptions.HIVE_SYNC_IGNORE_EXCEPTIONS))); props.setPropertyIfNonNull(HIVE_SUPPORT_TIMESTAMP_TYPE.key(), String.valueOf(conf.getBoolean(FlinkOptions.HIVE_SYNC_SUPPORT_TIMESTAMP))); 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 e9574dd52bedd..1083754ca204b 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 @@ -320,9 +320,6 @@ public class FlinkStreamerConfig extends Configuration { @Parameter(names = {"--hive-sync-assume-date-partitioning"}, description = "Assume partitioning is yyyy/mm/dd, default false") public Boolean hiveSyncAssumeDatePartition = false; - @Parameter(names = {"--hive-sync-use-jdbc"}, description = "Use JDBC when hive synchronization is enabled, default true") - public Boolean hiveSyncUseJdbc = true; - @Parameter(names = {"--hive-sync-auto-create-db"}, description = "Auto create hive database if it does not exists, default true") public Boolean hiveSyncAutoCreateDb = true; @@ -419,7 +416,6 @@ public static org.apache.flink.configuration.Configuration toFlinkConfig(FlinkSt conf.setString(FlinkOptions.HIVE_SYNC_PARTITION_FIELDS, config.hiveSyncPartitionFields); conf.setString(FlinkOptions.HIVE_SYNC_PARTITION_EXTRACTOR_CLASS_NAME, config.hiveSyncPartitionExtractorClass); conf.setBoolean(FlinkOptions.HIVE_SYNC_ASSUME_DATE_PARTITION, config.hiveSyncAssumeDatePartition); - conf.setBoolean(FlinkOptions.HIVE_SYNC_USE_JDBC, config.hiveSyncUseJdbc); conf.setBoolean(FlinkOptions.HIVE_SYNC_AUTO_CREATE_DB, config.hiveSyncAutoCreateDb); conf.setBoolean(FlinkOptions.HIVE_SYNC_IGNORE_EXCEPTIONS, config.hiveSyncIgnoreExceptions); conf.setBoolean(FlinkOptions.HIVE_SYNC_SKIP_RO_SUFFIX, config.hiveSyncSkipRoSuffix); diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/catalog/HoodieCatalogFactory.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/catalog/HoodieCatalogFactory.java index 8ab632ba5a5aa..77e9df8fdd56f 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/catalog/HoodieCatalogFactory.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/catalog/HoodieCatalogFactory.java @@ -18,6 +18,8 @@ package org.apache.hudi.table.catalog; +import org.apache.hudi.exception.HoodieCatalogException; + import org.apache.flink.configuration.ConfigOption; import org.apache.flink.configuration.Configuration; import org.apache.flink.table.catalog.Catalog; @@ -30,8 +32,8 @@ import java.util.HashSet; import java.util.Set; +import static org.apache.flink.table.factories.FactoryUtil.PROPERTY_VERSION; import static org.apache.hudi.table.catalog.CatalogOptions.CATALOG_PATH; -import static org.apache.hudi.table.catalog.CatalogOptions.DEFAULT_DATABASE; /** * A catalog factory impl that creates {@link HoodieCatalog}. @@ -52,21 +54,35 @@ public Catalog createCatalog(Context context) { FactoryUtil.createCatalogFactoryHelper(this, context); helper.validate(); - return new HoodieCatalog( - context.getName(), - (Configuration) helper.getOptions()); + if (helper.getOptions().get(HoodieCatalogFactoryOptions.MODE).equalsIgnoreCase("hms")) { + return new HoodieHiveCatalog( + context.getName(), + helper.getOptions().get(HoodieCatalogFactoryOptions.DEFAULT_DATABASE), + helper.getOptions().get(HoodieCatalogFactoryOptions.HIVE_CONF_DIR), + helper.getOptions().get(HoodieCatalogFactoryOptions.INIT_FS_TABLE)); + } else if (helper.getOptions().get(HoodieCatalogFactoryOptions.MODE).equalsIgnoreCase("dfs")) { + return new HoodieCatalog( + context.getName(), + (Configuration) helper.getOptions()); + } else { + throw new HoodieCatalogException("hoodie catalog supports only the hms and dfs modes."); + } } @Override public Set> requiredOptions() { - Set> options = new HashSet<>(); - options.add(CATALOG_PATH); - options.add(DEFAULT_DATABASE); - return options; + return Collections.emptySet(); } @Override public Set> optionalOptions() { - return Collections.emptySet(); + final Set> options = new HashSet<>(); + options.add(HoodieCatalogFactoryOptions.DEFAULT_DATABASE); + options.add(PROPERTY_VERSION); + options.add(HoodieCatalogFactoryOptions.HIVE_CONF_DIR); + options.add(HoodieCatalogFactoryOptions.MODE); + options.add(CATALOG_PATH); + options.add(HoodieCatalogFactoryOptions.INIT_FS_TABLE); + return options; } } diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/catalog/HoodieCatalogFactoryOptions.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/catalog/HoodieCatalogFactoryOptions.java new file mode 100644 index 0000000000000..9ee4f1eb4b2c2 --- /dev/null +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/catalog/HoodieCatalogFactoryOptions.java @@ -0,0 +1,47 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.table.catalog; + +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ConfigOptions; +import org.apache.flink.table.catalog.CommonCatalogOptions; + +/** {@link ConfigOption}s for {@link HoodieHiveCatalog}. */ +public class HoodieCatalogFactoryOptions { + public static final String DEFAULT_DB = "default"; + public static final String HIVE_SITE_FILE = "hive-site.xml"; + + public static final ConfigOption DEFAULT_DATABASE = + ConfigOptions.key(CommonCatalogOptions.DEFAULT_DATABASE_KEY) + .stringType() + .defaultValue(DEFAULT_DB); + + public static final ConfigOption HIVE_CONF_DIR = + ConfigOptions.key("hive-conf-dir").stringType().noDefaultValue(); + + public static final ConfigOption MODE = + ConfigOptions.key("mode").stringType().defaultValue("dfs"); + + public static final ConfigOption INIT_FS_TABLE = + ConfigOptions.key("init.fs.table").booleanType().defaultValue(true); + + private HoodieCatalogFactoryOptions() { + + } +} diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/catalog/HoodieCatalogUtil.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/catalog/HoodieCatalogUtil.java new file mode 100644 index 0000000000000..b90bffe17b5c3 --- /dev/null +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/catalog/HoodieCatalogUtil.java @@ -0,0 +1,99 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.table.catalog; + +import org.apache.hudi.configuration.HadoopConfigurations; + +import org.apache.flink.table.catalog.exceptions.CatalogException; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.conf.HiveConf; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; + +import java.io.File; +import java.io.IOException; +import java.io.InputStream; +import java.net.URL; + +import static org.apache.flink.util.StringUtils.isNullOrWhitespaceOnly; +import static org.apache.hudi.table.catalog.HoodieCatalogFactoryOptions.HIVE_SITE_FILE; + +/** + * Utilities for Hoodie Catalog. + */ +public class HoodieCatalogUtil { + private static final Logger LOG = LoggerFactory.getLogger(HoodieCatalogUtil.class); + + /** + * Returns a new {@code HiveConf}. + * + * @param hiveConfDir Hive conf directory path. + * @return A HiveConf instance. + */ + public static HiveConf createHiveConf(@Nullable String hiveConfDir) { + // create HiveConf from hadoop configuration with hadoop conf directory configured. + Configuration hadoopConf = HadoopConfigurations.getHadoopConfiguration(hiveConfDir); + if (isNullOrWhitespaceOnly(hiveConfDir) || hadoopConf == null) { + hadoopConf = HadoopConfigurations.getHadoopConf(new org.apache.flink.configuration.Configuration()); + } + + // ignore all the static conf file URLs that HiveConf may have set + HiveConf.setHiveSiteLocation(null); + HiveConf.setLoadMetastoreConfig(false); + HiveConf.setLoadHiveServer2Config(false); + HiveConf hiveConf = new HiveConf(hadoopConf, HiveConf.class); + + LOG.info("Setting hive conf dir as {}", hiveConfDir); + + if (hiveConfDir != null) { + Path hiveSite = new Path(hiveConfDir, HIVE_SITE_FILE); + if (!hiveSite.toUri().isAbsolute()) { + // treat relative URI as local file to be compatible with previous behavior + hiveSite = new Path(new File(hiveSite.toString()).toURI()); + } + try (InputStream inputStream = hiveSite.getFileSystem(hadoopConf).open(hiveSite)) { + hiveConf.addResource(inputStream, hiveSite.toString()); + // trigger a read from the conf so that the input stream is read + isEmbeddedMetastore(hiveConf); + } catch (IOException e) { + throw new CatalogException( + "Failed to load hive-site.xml from specified path:" + hiveSite, e); + } + } else { + // user doesn't provide hive conf dir, we try to find it in classpath + URL hiveSite = + Thread.currentThread().getContextClassLoader().getResource(HIVE_SITE_FILE); + if (hiveSite != null) { + LOG.info("Found {} in classpath: {}", HIVE_SITE_FILE, hiveSite); + hiveConf.addResource(hiveSite); + } + } + return hiveConf; + } + + /** + * Check whether the hive.metastore.uris is empty + */ + public static boolean isEmbeddedMetastore(HiveConf hiveConf) { + return isNullOrWhitespaceOnly(hiveConf.getVar(HiveConf.ConfVars.METASTOREURIS)); + } +} 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 new file mode 100644 index 0000000000000..33e3d4460a66f --- /dev/null +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/catalog/HoodieHiveCatalog.java @@ -0,0 +1,908 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.table.catalog; + +import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.model.HoodieFileFormat; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.TableSchemaResolver; +import org.apache.hudi.common.util.StringUtils; +import org.apache.hudi.configuration.FlinkOptions; +import org.apache.hudi.exception.HoodieCatalogException; +import org.apache.hudi.hadoop.utils.HoodieInputFormatUtils; +import org.apache.hudi.sync.common.util.ConfigUtils; +import org.apache.hudi.table.format.FilePathUtils; +import org.apache.hudi.util.AvroSchemaConverter; +import org.apache.hudi.util.StreamerUtil; + +import org.apache.avro.Schema; +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.sql.parser.hive.ddl.SqlAlterHiveDatabase; +import org.apache.flink.sql.parser.hive.ddl.SqlAlterHiveDatabaseOwner; +import org.apache.flink.sql.parser.hive.ddl.SqlCreateHiveDatabase; +import org.apache.flink.table.catalog.AbstractCatalog; +import org.apache.flink.table.catalog.CatalogBaseTable; +import org.apache.flink.table.catalog.CatalogDatabase; +import org.apache.flink.table.catalog.CatalogDatabaseImpl; +import org.apache.flink.table.catalog.CatalogFunction; +import org.apache.flink.table.catalog.CatalogPartition; +import org.apache.flink.table.catalog.CatalogPartitionSpec; +import org.apache.flink.table.catalog.CatalogPropertiesUtil; +import org.apache.flink.table.catalog.CatalogTable; +import org.apache.flink.table.catalog.CatalogView; +import org.apache.flink.table.catalog.ObjectPath; +import org.apache.flink.table.catalog.exceptions.CatalogException; +import org.apache.flink.table.catalog.exceptions.DatabaseAlreadyExistException; +import org.apache.flink.table.catalog.exceptions.DatabaseNotEmptyException; +import org.apache.flink.table.catalog.exceptions.DatabaseNotExistException; +import org.apache.flink.table.catalog.exceptions.FunctionAlreadyExistException; +import org.apache.flink.table.catalog.exceptions.FunctionNotExistException; +import org.apache.flink.table.catalog.exceptions.PartitionAlreadyExistsException; +import org.apache.flink.table.catalog.exceptions.PartitionNotExistException; +import org.apache.flink.table.catalog.exceptions.PartitionSpecInvalidException; +import org.apache.flink.table.catalog.exceptions.TableAlreadyExistException; +import org.apache.flink.table.catalog.exceptions.TableNotExistException; +import org.apache.flink.table.catalog.exceptions.TableNotPartitionedException; +import org.apache.flink.table.catalog.exceptions.TablePartitionedException; +import org.apache.flink.table.catalog.stats.CatalogColumnStatistics; +import org.apache.flink.table.catalog.stats.CatalogTableStatistics; +import org.apache.flink.table.expressions.Expression; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.metastore.IMetaStoreClient; +import org.apache.hadoop.hive.metastore.api.AlreadyExistsException; +import org.apache.hadoop.hive.metastore.api.Database; +import org.apache.hadoop.hive.metastore.api.FieldSchema; +import org.apache.hadoop.hive.metastore.api.InvalidOperationException; +import org.apache.hadoop.hive.metastore.api.NoSuchObjectException; +import org.apache.hadoop.hive.metastore.api.PrincipalType; +import org.apache.hadoop.hive.metastore.api.SerDeInfo; +import org.apache.hadoop.hive.metastore.api.StorageDescriptor; +import org.apache.hadoop.hive.metastore.api.Table; +import org.apache.hadoop.hive.metastore.api.UnknownDBException; +import org.apache.hadoop.hive.ql.metadata.Hive; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.thrift.TException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; + +import static org.apache.flink.sql.parser.hive.ddl.SqlAlterHiveDatabase.ALTER_DATABASE_OP; +import static org.apache.flink.sql.parser.hive.ddl.SqlAlterHiveDatabaseOwner.DATABASE_OWNER_NAME; +import static org.apache.flink.sql.parser.hive.ddl.SqlAlterHiveDatabaseOwner.DATABASE_OWNER_TYPE; +import static org.apache.flink.table.factories.FactoryUtil.CONNECTOR; +import static org.apache.flink.util.Preconditions.checkArgument; +import static org.apache.flink.util.Preconditions.checkNotNull; +import static org.apache.flink.util.StringUtils.isNullOrWhitespaceOnly; +import static org.apache.hudi.configuration.FlinkOptions.PATH; +import static org.apache.hudi.table.catalog.HoodieCatalogFactoryOptions.DEFAULT_DB; +import static org.apache.hudi.table.catalog.TableOptionProperties.COMMENT; +import static org.apache.hudi.table.catalog.TableOptionProperties.PK_COLUMNS; +import static org.apache.hudi.table.catalog.TableOptionProperties.PK_CONSTRAINT_NAME; +import static org.apache.hudi.table.catalog.TableOptionProperties.SPARK_SOURCE_PROVIDER; + +/** + * A catalog implementation for Hoodie based on MetaStore. + */ +public class HoodieHiveCatalog extends AbstractCatalog { + private static final Logger LOG = LoggerFactory.getLogger(HoodieHiveCatalog.class); + + private final HiveConf hiveConf; + private final boolean iniTable; + private IMetaStoreClient client; + + public HoodieHiveCatalog(String catalogName, String defaultDatabase, String hiveConf, boolean iniTable) { + this(catalogName, defaultDatabase, HoodieCatalogUtil.createHiveConf(hiveConf), iniTable, false); + } + + public HoodieHiveCatalog(String catalogName, String defaultDatabase, HiveConf hiveConf, boolean iniTable, boolean allowEmbedded) { + super(catalogName, defaultDatabase == null ? DEFAULT_DB : defaultDatabase); + this.hiveConf = hiveConf; + this.iniTable = iniTable; + if (!allowEmbedded) { + checkArgument( + !HoodieCatalogUtil.isEmbeddedMetastore(this.hiveConf), + "Embedded metastore is not allowed. Make sure you have set a valid value for " + + HiveConf.ConfVars.METASTOREURIS.toString()); + } + LOG.info("Created Hoodie Catalog '{}' in hms mode", catalogName); + } + + @Override + public void open() throws CatalogException { + if (this.client == null) { + try { + this.client = Hive.get(hiveConf).getMSC(); + } catch (Exception e) { + throw new HoodieCatalogException("Failed to create HiveMetaStoreClient", e); + } + LOG.info("Connected to Hive metastore"); + } + if (!databaseExists(getDefaultDatabase())) { + throw new HoodieCatalogException( + String.format( + "Configured default database %s doesn't exist in catalog %s.", + getDefaultDatabase(), getName())); + } + } + + @Override + public void close() throws CatalogException { + if (client != null) { + client.close(); + client = null; + LOG.info("Close connection to Hive metastore"); + } + } + + public HiveConf getHiveConf() { + return hiveConf; + } + + // ------ databases ------ + + @Override + public List listDatabases() throws CatalogException { + try { + return client.getAllDatabases(); + } catch (TException e) { + throw new HoodieCatalogException( + String.format("Failed to list all databases in %s", getName()), e); + } + } + + public Database getHiveDatabase(String databaseName) throws DatabaseNotExistException { + try { + return client.getDatabase(databaseName); + } catch (NoSuchObjectException e) { + throw new DatabaseNotExistException(getName(), databaseName); + } catch (TException e) { + throw new HoodieCatalogException( + String.format("Failed to get database %s from %s", databaseName, getName()), e); + } + } + + @Override + public CatalogDatabase getDatabase(String databaseName) + throws DatabaseNotExistException, CatalogException { + Database hiveDatabase = getHiveDatabase(databaseName); + + Map properties = new HashMap<>(hiveDatabase.getParameters()); + + properties.put(SqlCreateHiveDatabase.DATABASE_LOCATION_URI, hiveDatabase.getLocationUri()); + + return new CatalogDatabaseImpl(properties, hiveDatabase.getDescription()); + } + + @Override + public boolean databaseExists(String databaseName) throws CatalogException { + try { + return client.getDatabase(databaseName) != null; + } catch (NoSuchObjectException e) { + return false; + } catch (TException e) { + throw new HoodieCatalogException( + String.format( + "Failed to determine whether database %s exists or not", databaseName), + e); + } + } + + @Override + public void createDatabase( + String databaseName, CatalogDatabase database, boolean ignoreIfExists) + throws DatabaseAlreadyExistException, CatalogException { + checkArgument( + !isNullOrWhitespaceOnly(databaseName), "databaseName cannot be null or empty"); + checkNotNull(database, "database cannot be null"); + + Map properties = database.getProperties(); + + String dbLocationUri = properties.remove(SqlCreateHiveDatabase.DATABASE_LOCATION_URI); + + Database hiveDatabase = + new Database(databaseName, database.getComment(), dbLocationUri, properties); + + try { + client.createDatabase(hiveDatabase); + } catch (AlreadyExistsException e) { + if (!ignoreIfExists) { + throw new DatabaseAlreadyExistException(getName(), hiveDatabase.getName()); + } + } catch (TException e) { + throw new HoodieCatalogException( + String.format("Failed to create database %s", hiveDatabase.getName()), e); + } + } + + @Override + public void dropDatabase(String name, boolean ignoreIfNotExists, boolean cascade) + throws DatabaseNotExistException, DatabaseNotEmptyException, CatalogException { + try { + client.dropDatabase(name, true, ignoreIfNotExists, cascade); + } catch (NoSuchObjectException e) { + if (!ignoreIfNotExists) { + throw new DatabaseNotExistException(getName(), name); + } + } catch (InvalidOperationException e) { + throw new DatabaseNotEmptyException(getName(), name); + } catch (TException e) { + throw new HoodieCatalogException(String.format("Failed to drop database %s", name), e); + } + } + + @Override + public void alterDatabase( + String databaseName, CatalogDatabase newDatabase, boolean ignoreIfNotExists) + throws DatabaseNotExistException, CatalogException { + checkArgument( + !isNullOrWhitespaceOnly(databaseName), "databaseName cannot be null or empty"); + checkNotNull(newDatabase, "newDatabase cannot be null"); + + // client.alterDatabase doesn't throw any exception if there is no existing database + Database hiveDB; + try { + hiveDB = getHiveDatabase(databaseName); + } catch (DatabaseNotExistException e) { + if (!ignoreIfNotExists) { + throw new DatabaseNotExistException(getName(), databaseName); + } + + return; + } + + try { + client.alterDatabase(databaseName, alterDatabase(hiveDB, newDatabase)); + } catch (TException e) { + throw new HoodieCatalogException( + String.format("Failed to alter database %s", databaseName), e); + } + } + + private static Database alterDatabase(Database hiveDB, CatalogDatabase newDatabase) { + Map newParams = newDatabase.getProperties(); + String opStr = newParams.remove(ALTER_DATABASE_OP); + if (opStr == null) { + // by default is to alter db properties + opStr = SqlAlterHiveDatabase.AlterHiveDatabaseOp.CHANGE_PROPS.name(); + } + String newLocation = newParams.remove(SqlCreateHiveDatabase.DATABASE_LOCATION_URI); + SqlAlterHiveDatabase.AlterHiveDatabaseOp op = + SqlAlterHiveDatabase.AlterHiveDatabaseOp.valueOf(opStr); + switch (op) { + case CHANGE_PROPS: + hiveDB.setParameters(newParams); + break; + case CHANGE_LOCATION: + hiveDB.setLocationUri(newLocation); + break; + case CHANGE_OWNER: + String ownerName = newParams.remove(DATABASE_OWNER_NAME); + String ownerType = newParams.remove(DATABASE_OWNER_TYPE); + hiveDB.setOwnerName(ownerName); + switch (ownerType) { + case SqlAlterHiveDatabaseOwner.ROLE_OWNER: + hiveDB.setOwnerType(PrincipalType.ROLE); + break; + case SqlAlterHiveDatabaseOwner.USER_OWNER: + hiveDB.setOwnerType(PrincipalType.USER); + break; + default: + throw new CatalogException("Unsupported database owner type: " + ownerType); + } + break; + default: + throw new CatalogException("Unsupported alter database op:" + opStr); + } + // is_generic is deprecated, remove it + if (hiveDB.getParameters() != null) { + hiveDB.getParameters().remove(CatalogPropertiesUtil.IS_GENERIC); + } + return hiveDB; + } + + // ------ tables ------ + + private Table checkHoodieTable(Table hiveTable) { + if (!hiveTable.getParameters().getOrDefault(CONNECTOR.key(), "").equalsIgnoreCase("hudi") + && !hiveTable.getParameters().getOrDefault(SPARK_SOURCE_PROVIDER, "").equalsIgnoreCase("hudi")) { + throw new HoodieCatalogException(String.format("the %s is not hoodie table", hiveTable.getTableName())); + } + return hiveTable; + } + + private boolean isFlinkTable(Table hiveTable) { + return hiveTable.getParameters().getOrDefault(CONNECTOR.key(), "").equalsIgnoreCase("hudi"); + } + + private org.apache.avro.Schema getLatestTableSchema(String path) { + if (StringUtils.isNullOrEmpty(path) || !StreamerUtil.tableExists(path, hiveConf)) { + return null; + } + + try { + HoodieTableMetaClient metaClient = StreamerUtil.createMetaClient(path, hiveConf); + return new TableSchemaResolver(metaClient).getTableAvroSchema(false); + } catch (Exception e) { + LOG.warn("Error while resolving the latest table schema", e); + } + return null; + } + + @VisibleForTesting + public Table getHiveTable(ObjectPath tablePath) throws TableNotExistException { + try { + Table hiveTable = client.getTable(tablePath.getDatabaseName(), tablePath.getObjectName()); + return checkHoodieTable(hiveTable); + } catch (NoSuchObjectException e) { + throw new TableNotExistException(getName(), tablePath); + } catch (TException e) { + throw new HoodieCatalogException(String.format("Failed to get table %s from Hive metastore", tablePath.getObjectName())); + } + } + + private Table translateSparkTable2Flink(ObjectPath tablePath, Table hiveTable) { + if (!isFlinkTable(hiveTable)) { + try { + Map parameters = hiveTable.getParameters(); + parameters.putAll(TableOptionProperties.translateSparkTableProperties2Flink(hiveTable)); + String path = hiveTable.getSd().getLocation(); + parameters.put(PATH.key(), path); + if (!parameters.containsKey(FlinkOptions.HIVE_STYLE_PARTITIONING.key())) { + Path hoodieTablePath = new Path(path); + boolean hiveStyle = Arrays.stream(FSUtils.getFs(hoodieTablePath, hiveConf).listStatus(hoodieTablePath)) + .map(fileStatus -> fileStatus.getPath().getName()) + .filter(f -> !f.equals(".hoodie") && !f.equals("default")) + .anyMatch(FilePathUtils::isHiveStylePartitioning); + parameters.put(FlinkOptions.HIVE_STYLE_PARTITIONING.key(), String.valueOf(hiveStyle)); + } + client.alter_table(tablePath.getDatabaseName(), tablePath.getObjectName(), hiveTable); + } catch (Exception e) { + throw new HoodieCatalogException("Failed to update table schema", e); + } + } + return hiveTable; + } + + @Override + public CatalogBaseTable getTable(ObjectPath tablePath) throws TableNotExistException, CatalogException { + checkNotNull(tablePath, "tablePath cannot be null"); + Table hiveTable = getHiveTable(tablePath); + hiveTable = translateSparkTable2Flink(tablePath, hiveTable); + String path = hiveTable.getSd().getLocation(); + Map parameters = hiveTable.getParameters(); + Schema latestTableSchema = getLatestTableSchema(path); + org.apache.flink.table.api.Schema schema; + if (latestTableSchema != null) { + org.apache.flink.table.api.Schema.Builder builder = org.apache.flink.table.api.Schema.newBuilder() + .fromRowDataType(AvroSchemaConverter.convertToDataType(latestTableSchema)); + String pkConstraintName = parameters.get(PK_CONSTRAINT_NAME); + if (!StringUtils.isNullOrEmpty(pkConstraintName)) { + builder.primaryKeyNamed(pkConstraintName, StringUtils.split(parameters.get(PK_COLUMNS), ",")); + } + schema = builder.build(); + } else { + LOG.warn("{} does not have any hoodie schema, and use hive table schema to infer the table schema", tablePath); + schema = TableOptionProperties.convertTableSchema(hiveTable); + } + return CatalogTable.of(schema, parameters.get(COMMENT), + TableOptionProperties.getFieldNames(hiveTable.getPartitionKeys()), parameters); + } + + @Override + public void createTable(ObjectPath tablePath, CatalogBaseTable table, boolean ignoreIfExists) + throws TableAlreadyExistException, DatabaseNotExistException, CatalogException { + checkNotNull(tablePath, "tablePath cannot be null"); + checkNotNull(table, "table cannot be null"); + + if (!databaseExists(tablePath.getDatabaseName())) { + throw new DatabaseNotExistException(getName(), tablePath.getDatabaseName()); + } + + if (!table.getOptions().getOrDefault(CONNECTOR.key(), "").equalsIgnoreCase("hudi")) { + throw new HoodieCatalogException(String.format("The %s is not hoodie table", tablePath.getObjectName())); + } + + if (table instanceof CatalogView) { + throw new HoodieCatalogException("Hoodie catalog does not support to CREATE VIEW."); + } + + try { + boolean isMorTable = table.getOptions().getOrDefault(FlinkOptions.TABLE_TYPE.key(), + FlinkOptions.TABLE_TYPE.defaultValue()).equalsIgnoreCase(FlinkOptions.TABLE_TYPE_MERGE_ON_READ); + Table hiveTable = instantiateHiveTable(tablePath, table, inferTablePath(tablePath, table), isMorTable); + //create hive table + client.createTable(hiveTable); + //init hoodie metaClient + initTableIfNotExists(tablePath, (CatalogTable)table); + } catch (AlreadyExistsException e) { + if (!ignoreIfExists) { + throw new TableAlreadyExistException(getName(), tablePath, e); + } + } catch (Exception e) { + throw new HoodieCatalogException( + String.format("Failed to create table %s", tablePath.getFullName()), e); + } + } + + private void initTableIfNotExists(ObjectPath tablePath, CatalogTable catalogTable) { + if (!iniTable) { + LOG.info("Skip init table."); + return; + } + Configuration flinkConf = Configuration.fromMap(applyOptionsHook(catalogTable.getOptions())); + flinkConf.addAllToProperties(hiveConf.getAllProperties()); + final String avroSchema = AvroSchemaConverter.convertToSchema(catalogTable.getSchema().toPhysicalRowDataType().getLogicalType()).toString(); + flinkConf.setString(FlinkOptions.SOURCE_AVRO_SCHEMA, avroSchema); + + // stores two copies of options: + // - partition keys + // - primary keys + // because the HoodieTableMetaClient is a heavy impl, we try to avoid initializing it + // when calling #getTable. + + if (catalogTable.getUnresolvedSchema().getPrimaryKey().isPresent()) { + final String pkColumns = String.join(",", catalogTable.getUnresolvedSchema().getPrimaryKey().get().getColumnNames()); + String recordKey = flinkConf.get(FlinkOptions.RECORD_KEY_FIELD); + if (!Objects.equals(pkColumns, recordKey)) { + throw new HoodieCatalogException(String.format("%s and %s are the different", pkColumns, recordKey)); + } + } + + if (catalogTable.isPartitioned()) { + final String partitions = String.join(",", catalogTable.getPartitionKeys()); + flinkConf.setString(FlinkOptions.PARTITION_PATH_FIELD, partitions); + } + + if (!flinkConf.getOptional(PATH).isPresent()) { + flinkConf.setString(PATH, inferTablePath(tablePath, catalogTable)); + } + + flinkConf.setString(FlinkOptions.TABLE_NAME, tablePath.getObjectName()); + try { + StreamerUtil.initTableIfNotExists(flinkConf); + } catch (IOException e) { + throw new HoodieCatalogException("Initialize table exception.", e); + } + } + + private String inferTablePath(ObjectPath tablePath, CatalogBaseTable table) { + String location = table.getOptions().getOrDefault(PATH.key(), ""); + if (StringUtils.isNullOrEmpty(location)) { + try { + Path dbLocation = new Path(client.getDatabase(tablePath.getDatabaseName()).getLocationUri()); + location = new Path(dbLocation, tablePath.getObjectName()).toString(); + } catch (TException e) { + throw new HoodieCatalogException(String.format("Failed to infer hoodie table path for table %s", tablePath), e); + } + } + return location; + } + + private Map applyOptionsHook(Map options) { + Map properties = new HashMap<>(options); + if (!options.containsKey(FlinkOptions.RECORD_KEY_FIELD.key())) { + properties.put(FlinkOptions.RECORD_KEY_FIELD.key(), FlinkOptions.RECORD_KEY_FIELD.defaultValue()); + } + if (!options.containsKey(FlinkOptions.PRECOMBINE_FIELD.key())) { + properties.put(FlinkOptions.PRECOMBINE_FIELD.key(), FlinkOptions.PRECOMBINE_FIELD.defaultValue()); + } + if (!options.containsKey(FlinkOptions.TABLE_TYPE.key())) { + properties.put(FlinkOptions.TABLE_TYPE.key(), FlinkOptions.TABLE_TYPE.defaultValue()); + } + return properties; + } + + private Table instantiateHiveTable(ObjectPath tablePath, CatalogBaseTable table, String location, boolean useRealTimeInputFormat) throws IOException { + // let Hive set default parameters for us, e.g. serialization.format + Table hiveTable = + org.apache.hadoop.hive.ql.metadata.Table.getEmptyTable( + tablePath.getDatabaseName(), tablePath.getObjectName()); + hiveTable.setOwner(UserGroupInformation.getCurrentUser().getUserName()); + hiveTable.setCreateTime((int) (System.currentTimeMillis() / 1000)); + + Map properties = applyOptionsHook(table.getOptions()); + properties.put("EXTERNAL", "TRUE"); + // Table comment + if (table.getComment() != null) { + properties.put(COMMENT, table.getComment()); + } + + //set sd + StorageDescriptor sd = new StorageDescriptor(); + List allColumns = TableOptionProperties.createHiveColumns(table.getSchema()); + + // Table columns and partition keys + if (table instanceof CatalogTable) { + CatalogTable catalogTable = (CatalogTable) table; + + if (catalogTable.isPartitioned()) { + int partitionKeySize = catalogTable.getPartitionKeys().size(); + List regularColumns = + allColumns.subList(0, allColumns.size() - partitionKeySize); + List partitionColumns = + allColumns.subList( + allColumns.size() - partitionKeySize, allColumns.size()); + + sd.setCols(regularColumns); + hiveTable.setPartitionKeys(partitionColumns); + } else { + sd.setCols(allColumns); + hiveTable.setPartitionKeys(new ArrayList<>()); + } + } else { + sd.setCols(allColumns); + } + + HoodieFileFormat baseFileFormat = HoodieFileFormat.PARQUET; + //ignore uber input Format + String inputFormatClassName = HoodieInputFormatUtils.getInputFormatClassName(baseFileFormat, useRealTimeInputFormat); + String outputFormatClassName = HoodieInputFormatUtils.getOutputFormatClassName(baseFileFormat); + String serDeClassName = HoodieInputFormatUtils.getSerDeClassName(baseFileFormat); + sd.setInputFormat(inputFormatClassName); + sd.setOutputFormat(outputFormatClassName); + Map serdeProperties = new HashMap<>(); + serdeProperties.put("path", location); + serdeProperties.put(ConfigUtils.IS_QUERY_AS_RO_TABLE, String.valueOf(!useRealTimeInputFormat)); + serdeProperties.put("serialization.format", "1"); + sd.setSerdeInfo(new SerDeInfo(null, serDeClassName, serdeProperties)); + + sd.setLocation(location); + + hiveTable.setSd(sd); + + properties.putAll(TableOptionProperties.translateFlinkTableProperties2Spark((CatalogTable)table, hiveConf)); + + //set pk + if (table.getUnresolvedSchema().getPrimaryKey().isPresent()) { + String pkColumns = String.join(",", table.getUnresolvedSchema().getPrimaryKey().get().getColumnNames()); + String recordKey = properties.getOrDefault(FlinkOptions.RECORD_KEY_FIELD.key(), FlinkOptions.RECORD_KEY_FIELD.defaultValue()); + if (!Objects.equals(pkColumns, recordKey)) { + throw new HoodieCatalogException( + String.format("If the table has primaryKey, the primaryKey should be the the same as the recordKey, but pk %s and recordKey %s are the different", + pkColumns, + recordKey)); + } + properties.put(PK_CONSTRAINT_NAME, table.getUnresolvedSchema().getPrimaryKey().get().getConstraintName()); + properties.put(PK_COLUMNS, pkColumns); + } + + if (!properties.containsKey(FlinkOptions.PATH.key())) { + properties.put(FlinkOptions.PATH.key(), location); + } + + hiveTable.setParameters(properties); + return hiveTable; + } + + @Override + public List listTables(String databaseName) + throws DatabaseNotExistException, CatalogException { + checkArgument( + !isNullOrWhitespaceOnly(databaseName), "databaseName cannot be null or empty"); + + try { + return client.getAllTables(databaseName); + } catch (UnknownDBException e) { + throw new DatabaseNotExistException(getName(), databaseName); + } catch (TException e) { + throw new HoodieCatalogException( + String.format("Failed to list tables in database %s", databaseName), e); + } + } + + @Override + public List listViews(String databaseName) + throws DatabaseNotExistException, CatalogException { + throw new HoodieCatalogException("Hoodie catalog does not support to listViews"); + } + + @Override + public boolean tableExists(ObjectPath tablePath) throws CatalogException { + checkNotNull(tablePath, "tablePath cannot be null"); + + try { + return client.tableExists(tablePath.getDatabaseName(), tablePath.getObjectName()); + } catch (UnknownDBException e) { + return false; + } catch (TException e) { + throw new CatalogException( + String.format( + "Failed to check whether table %s exists or not.", + tablePath.getFullName()), + e); + } + } + + @Override + public void dropTable(ObjectPath tablePath, boolean ignoreIfNotExists) + throws TableNotExistException, CatalogException { + checkNotNull(tablePath, "tablePath cannot be null"); + + try { + client.dropTable( + tablePath.getDatabaseName(), + tablePath.getObjectName(), + // Indicate whether associated data should be deleted. + // Set to 'true' for now because Flink tables shouldn't have data in Hive. Can + // be changed later if necessary + true, + ignoreIfNotExists); + } catch (NoSuchObjectException e) { + if (!ignoreIfNotExists) { + throw new TableNotExistException(getName(), tablePath); + } + } catch (TException e) { + throw new HoodieCatalogException( + String.format("Failed to drop table %s", tablePath.getFullName()), e); + } + } + + @Override + public void renameTable(ObjectPath tablePath, String newTableName, boolean ignoreIfNotExists) + throws TableNotExistException, TableAlreadyExistException, CatalogException { + checkNotNull(tablePath, "tablePath cannot be null"); + checkArgument( + !isNullOrWhitespaceOnly(newTableName), "newTableName cannot be null or empty"); + + try { + // alter_table() doesn't throw a clear exception when target table doesn't exist. + // Thus, check the table existence explicitly + if (tableExists(tablePath)) { + ObjectPath newPath = new ObjectPath(tablePath.getDatabaseName(), newTableName); + // alter_table() doesn't throw a clear exception when new table already exists. + // Thus, check the table existence explicitly + if (tableExists(newPath)) { + throw new TableAlreadyExistException(getName(), newPath); + } else { + Table hiveTable = getHiveTable(tablePath); + + //update hoodie + StorageDescriptor sd = hiveTable.getSd(); + String location = sd.getLocation(); + HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder().setBasePath(location).setConf(hiveConf).build(); + //Init table with new name + HoodieTableMetaClient.withPropertyBuilder().fromProperties(metaClient.getTableConfig().getProps()) + .setTableName(newTableName) + .initTable(hiveConf, location); + + hiveTable.setTableName(newTableName); + client.alter_table( + tablePath.getDatabaseName(), tablePath.getObjectName(), hiveTable); + } + } else if (!ignoreIfNotExists) { + throw new TableNotExistException(getName(), tablePath); + } + } catch (Exception e) { + throw new HoodieCatalogException( + String.format("Failed to rename table %s", tablePath.getFullName()), e); + } + } + + @Override + public void alterTable( + ObjectPath tablePath, CatalogBaseTable newCatalogTable, boolean ignoreIfNotExists) + throws TableNotExistException, CatalogException { + checkNotNull(tablePath, "tablePath cannot be null"); + checkNotNull(newCatalogTable, "newCatalogTable cannot be null"); + + if (!newCatalogTable.getOptions().getOrDefault(CONNECTOR.key(), "").equalsIgnoreCase("hudi")) { + throw new HoodieCatalogException(String.format("The %s is not hoodie table", tablePath.getObjectName())); + } + if (newCatalogTable instanceof CatalogView) { + throw new HoodieCatalogException("Hoodie catalog does not support to ALTER VIEW"); + } + + try { + Table hiveTable = getHiveTable(tablePath); + if (!hiveTable.getParameters().getOrDefault(FlinkOptions.TABLE_TYPE.key(), FlinkOptions.TABLE_TYPE.defaultValue()) + .equalsIgnoreCase(newCatalogTable.getOptions().getOrDefault(FlinkOptions.TABLE_TYPE.key(), FlinkOptions.TABLE_TYPE.defaultValue())) + || !hiveTable.getParameters().getOrDefault(FlinkOptions.INDEX_TYPE.key(), FlinkOptions.INDEX_TYPE.defaultValue()) + .equalsIgnoreCase(newCatalogTable.getOptions().getOrDefault(FlinkOptions.INDEX_TYPE.key(), FlinkOptions.INDEX_TYPE.defaultValue()))) { + throw new HoodieCatalogException("Hoodie catalog does not support to alter table type and index type"); + } + } catch (TableNotExistException e) { + if (!ignoreIfNotExists) { + throw e; + } + return; + } + + try { + boolean isMorTable = newCatalogTable.getOptions().getOrDefault(FlinkOptions.TABLE_TYPE.key(), + FlinkOptions.TABLE_TYPE.defaultValue()).equalsIgnoreCase(FlinkOptions.TABLE_TYPE_MERGE_ON_READ); + Table hiveTable = instantiateHiveTable(tablePath, newCatalogTable, inferTablePath(tablePath, newCatalogTable), isMorTable); + //alter hive table + client.alter_table(tablePath.getDatabaseName(), tablePath.getObjectName(), hiveTable); + } catch (Exception e) { + LOG.error("Failed to alter table {}", tablePath.getObjectName(), e); + throw new HoodieCatalogException(String.format("Failed to alter table %s", tablePath.getObjectName()), e); + } + } + + @Override + public List listPartitions(ObjectPath tablePath) + throws TableNotExistException, TableNotPartitionedException, CatalogException { + throw new HoodieCatalogException("Not supported."); + } + + @Override + public List listPartitions( + ObjectPath tablePath, CatalogPartitionSpec partitionSpec) + throws TableNotExistException, TableNotPartitionedException, + PartitionSpecInvalidException, CatalogException { + throw new HoodieCatalogException("Not supported."); + } + + @Override + public List listPartitionsByFilter( + ObjectPath tablePath, List expressions) + throws TableNotExistException, TableNotPartitionedException, CatalogException { + throw new HoodieCatalogException("Not supported."); + } + + @Override + public CatalogPartition getPartition(ObjectPath tablePath, CatalogPartitionSpec partitionSpec) + throws PartitionNotExistException, CatalogException { + throw new HoodieCatalogException("Not supported."); + } + + @Override + public boolean partitionExists(ObjectPath tablePath, CatalogPartitionSpec partitionSpec) + throws CatalogException { + throw new HoodieCatalogException("Not supported."); + } + + @Override + public void createPartition( + ObjectPath tablePath, + CatalogPartitionSpec partitionSpec, + CatalogPartition partition, + boolean ignoreIfExists) + throws TableNotExistException, TableNotPartitionedException, + PartitionSpecInvalidException, PartitionAlreadyExistsException, + CatalogException { + throw new HoodieCatalogException("Not supported."); + } + + @Override + public void dropPartition( + ObjectPath tablePath, CatalogPartitionSpec partitionSpec, boolean ignoreIfNotExists) + throws PartitionNotExistException, CatalogException { + throw new HoodieCatalogException("Not supported."); + } + + @Override + public void alterPartition( + ObjectPath tablePath, + CatalogPartitionSpec partitionSpec, + CatalogPartition newPartition, + boolean ignoreIfNotExists) + throws PartitionNotExistException, CatalogException { + throw new HoodieCatalogException("Not supported."); + } + + @Override + public List listFunctions(String databaseName) + throws DatabaseNotExistException, CatalogException { + return Collections.emptyList(); + } + + @Override + public CatalogFunction getFunction(ObjectPath functionPath) + throws FunctionNotExistException, CatalogException { + throw new FunctionNotExistException(getName(), functionPath); + } + + @Override + public boolean functionExists(ObjectPath functionPath) throws CatalogException { + return false; + } + + @Override + public void createFunction( + ObjectPath functionPath, CatalogFunction function, boolean ignoreIfExists) + throws FunctionAlreadyExistException, DatabaseNotExistException, CatalogException { + throw new HoodieCatalogException("Not supported."); + } + + @Override + public void alterFunction( + ObjectPath functionPath, CatalogFunction newFunction, boolean ignoreIfNotExists) + throws FunctionNotExistException, CatalogException { + throw new HoodieCatalogException("Not supported."); + } + + @Override + public void dropFunction(ObjectPath functionPath, boolean ignoreIfNotExists) + throws FunctionNotExistException, CatalogException { + throw new HoodieCatalogException("Not supported."); + } + + @Override + public CatalogTableStatistics getTableStatistics(ObjectPath tablePath) + throws TableNotExistException, CatalogException { + return CatalogTableStatistics.UNKNOWN; + } + + @Override + public CatalogColumnStatistics getTableColumnStatistics(ObjectPath tablePath) + throws TableNotExistException, CatalogException { + return CatalogColumnStatistics.UNKNOWN; + } + + @Override + public CatalogTableStatistics getPartitionStatistics( + ObjectPath tablePath, CatalogPartitionSpec partitionSpec) + throws PartitionNotExistException, CatalogException { + return CatalogTableStatistics.UNKNOWN; + } + + @Override + public CatalogColumnStatistics getPartitionColumnStatistics( + ObjectPath tablePath, CatalogPartitionSpec partitionSpec) + throws PartitionNotExistException, CatalogException { + return CatalogColumnStatistics.UNKNOWN; + } + + @Override + public void alterTableStatistics( + ObjectPath tablePath, CatalogTableStatistics tableStatistics, boolean ignoreIfNotExists) + throws TableNotExistException, CatalogException { + throw new HoodieCatalogException("Not supported."); + } + + @Override + public void alterTableColumnStatistics( + ObjectPath tablePath, + CatalogColumnStatistics columnStatistics, + boolean ignoreIfNotExists) + throws TableNotExistException, CatalogException, TablePartitionedException { + throw new HoodieCatalogException("Not supported."); + } + + @Override + public void alterPartitionStatistics( + ObjectPath tablePath, + CatalogPartitionSpec partitionSpec, + CatalogTableStatistics partitionStatistics, + boolean ignoreIfNotExists) + throws PartitionNotExistException, CatalogException { + throw new HoodieCatalogException("Not supported."); + } + + @Override + public void alterPartitionColumnStatistics( + ObjectPath tablePath, + CatalogPartitionSpec partitionSpec, + CatalogColumnStatistics columnStatistics, + boolean ignoreIfNotExists) + throws PartitionNotExistException, CatalogException { + throw new HoodieCatalogException("Not supported."); + } +} 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 ba6ca4efd726c..4f15c4ef0adc6 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 @@ -19,17 +19,42 @@ package org.apache.hudi.table.catalog; import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.model.HoodieTableType; +import org.apache.hudi.common.table.TableSchemaResolver; +import org.apache.hudi.common.util.StringUtils; +import org.apache.hudi.configuration.FlinkOptions; import org.apache.hudi.exception.HoodieIOException; +import org.apache.hudi.sync.common.util.Parquet2SparkSchemaUtils; +import org.apache.hudi.util.AvroSchemaConverter; +import org.apache.avro.Schema; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.catalog.CatalogTable; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.LogicalType; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.metastore.api.FieldSchema; +import org.apache.hadoop.hive.metastore.api.Table; +import org.apache.hadoop.hive.serde2.typeinfo.CharTypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.DecimalTypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.ListTypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.MapTypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.StructTypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils; +import org.apache.hadoop.hive.serde2.typeinfo.VarcharTypeInfo; +import org.apache.parquet.schema.MessageType; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.io.IOException; +import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; import java.util.Date; @@ -37,8 +62,11 @@ import java.util.List; import java.util.Map; import java.util.Properties; +import java.util.TreeMap; import java.util.stream.Collectors; +import static org.apache.flink.table.factories.FactoryUtil.CONNECTOR; +import static org.apache.flink.util.Preconditions.checkNotNull; import static org.apache.hudi.common.table.HoodieTableMetaClient.AUXILIARYFOLDER_NAME; /** @@ -53,9 +81,26 @@ public class TableOptionProperties { public static final String PK_COLUMNS = "pk.columns"; public static final String COMMENT = "comment"; public static final String PARTITION_COLUMNS = "partition.columns"; + public static final String SPARK_SOURCE_PROVIDER = "spark.sql.sources.provider"; + public static final String SPARK_VERSION = "spark.verison"; + public static final String DEFAULT_SPARK_VERSION = "spark2.4.4"; public static final List NON_OPTION_KEYS = Arrays.asList(PK_CONSTRAINT_NAME, PK_COLUMNS, COMMENT, PARTITION_COLUMNS); + private static final Map VALUE_MAPPING = new TreeMap<>(); + private static final Map KEY_MAPPING = new TreeMap<>(); + + static { + VALUE_MAPPING.put("mor", HoodieTableType.MERGE_ON_READ.name()); + VALUE_MAPPING.put("cow", HoodieTableType.COPY_ON_WRITE.name()); + + 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("payloadClass", FlinkOptions.PAYLOAD_CLASS_NAME.key()); + KEY_MAPPING.put(SPARK_SOURCE_PROVIDER, CONNECTOR.key()); + } + /** * Initialize the {@link #FILE_NAME} meta file. */ @@ -128,4 +173,179 @@ public static Map getTableOptions(Map options) { NON_OPTION_KEYS.forEach(copied::remove); return copied; } + + public static Map translateFlinkTableProperties2Spark(CatalogTable catalogTable, Configuration hadoopConf) { + Schema schema = AvroSchemaConverter.convertToSchema(catalogTable.getSchema().toPhysicalRowDataType().getLogicalType()); + MessageType messageType = TableSchemaResolver.convertAvroSchemaToParquet(schema, hadoopConf); + String sparkVersion = catalogTable.getOptions().getOrDefault(SPARK_VERSION, DEFAULT_SPARK_VERSION); + return Parquet2SparkSchemaUtils.getSparkTableProperties(catalogTable.getPartitionKeys(), sparkVersion, 4000, messageType); + } + + public static Map translateSparkTableProperties2Flink(Map options) { + if (options.containsKey(CONNECTOR.key())) { + return options; + } + return options.entrySet().stream().filter(e -> KEY_MAPPING.containsKey(e.getKey())) + .collect(Collectors.toMap(e -> KEY_MAPPING.get(e.getKey()), + e -> e.getKey().equalsIgnoreCase("type") ? VALUE_MAPPING.get(e.getValue()) : e.getValue())); + } + + public static Map translateSparkTableProperties2Flink(Table hiveTable) { + return translateSparkTableProperties2Flink(hiveTable.getParameters()); + } + + /** Get field names from field schemas. */ + public static List getFieldNames(List fieldSchemas) { + List names = new ArrayList<>(fieldSchemas.size()); + for (FieldSchema fs : fieldSchemas) { + names.add(fs.getName()); + } + return names; + } + + public static org.apache.flink.table.api.Schema convertTableSchema(Table hiveTable) { + List allCols = new ArrayList<>(hiveTable.getSd().getCols()); + allCols.addAll(hiveTable.getPartitionKeys()); + + String pkConstraintName = hiveTable.getParameters().get(PK_CONSTRAINT_NAME); + List primaryColNames = StringUtils.isNullOrEmpty(pkConstraintName) + ? Collections.EMPTY_LIST + : StringUtils.split(hiveTable.getParameters().get(PK_COLUMNS),","); + + String[] colNames = new String[allCols.size()]; + DataType[] colTypes = new DataType[allCols.size()]; + + for (int i = 0; i < allCols.size(); i++) { + FieldSchema fs = allCols.get(i); + + colNames[i] = fs.getName(); + colTypes[i] = + toFlinkType(TypeInfoUtils.getTypeInfoFromTypeString(fs.getType())); + if (primaryColNames.contains(colNames[i])) { + colTypes[i] = colTypes[i].notNull(); + } + } + + org.apache.flink.table.api.Schema.Builder builder = org.apache.flink.table.api.Schema.newBuilder().fromFields(colNames, colTypes); + if (!StringUtils.isNullOrEmpty(pkConstraintName)) { + builder.primaryKeyNamed(pkConstraintName, primaryColNames); + } + + return builder.build(); + } + + /** + * Convert Hive data type to a Flink data type. + * + * @param hiveType a Hive data type + * @return the corresponding Flink data type + */ + public static DataType toFlinkType(TypeInfo hiveType) { + checkNotNull(hiveType, "hiveType cannot be null"); + + switch (hiveType.getCategory()) { + case PRIMITIVE: + return toFlinkPrimitiveType((PrimitiveTypeInfo) hiveType); + case LIST: + ListTypeInfo listTypeInfo = (ListTypeInfo) hiveType; + return DataTypes.ARRAY(toFlinkType(listTypeInfo.getListElementTypeInfo())); + case MAP: + MapTypeInfo mapTypeInfo = (MapTypeInfo) hiveType; + return DataTypes.MAP( + toFlinkType(mapTypeInfo.getMapKeyTypeInfo()), + toFlinkType(mapTypeInfo.getMapValueTypeInfo())); + case STRUCT: + StructTypeInfo structTypeInfo = (StructTypeInfo) hiveType; + + List names = structTypeInfo.getAllStructFieldNames(); + List typeInfos = structTypeInfo.getAllStructFieldTypeInfos(); + + DataTypes.Field[] fields = new DataTypes.Field[names.size()]; + + for (int i = 0; i < fields.length; i++) { + fields[i] = DataTypes.FIELD(names.get(i), toFlinkType(typeInfos.get(i))); + } + + return DataTypes.ROW(fields); + default: + throw new UnsupportedOperationException( + String.format("Flink doesn't support Hive data type %s yet.", hiveType)); + } + } + + private static DataType toFlinkPrimitiveType(PrimitiveTypeInfo hiveType) { + checkNotNull(hiveType, "hiveType cannot be null"); + + switch (hiveType.getPrimitiveCategory()) { + case CHAR: + return DataTypes.CHAR(((CharTypeInfo) hiveType).getLength()); + case VARCHAR: + return DataTypes.VARCHAR(((VarcharTypeInfo) hiveType).getLength()); + case STRING: + return DataTypes.STRING(); + case BOOLEAN: + return DataTypes.BOOLEAN(); + case BYTE: + return DataTypes.TINYINT(); + case SHORT: + return DataTypes.SMALLINT(); + case INT: + return DataTypes.INT(); + case LONG: + return DataTypes.BIGINT(); + case FLOAT: + return DataTypes.FLOAT(); + case DOUBLE: + return DataTypes.DOUBLE(); + case DATE: + return DataTypes.DATE(); + case TIMESTAMP: + return DataTypes.TIMESTAMP(9); + case BINARY: + return DataTypes.BYTES(); + case DECIMAL: + DecimalTypeInfo decimalTypeInfo = (DecimalTypeInfo) hiveType; + return DataTypes.DECIMAL( + decimalTypeInfo.getPrecision(), decimalTypeInfo.getScale()); + default: + throw new UnsupportedOperationException( + String.format( + "Flink doesn't support Hive primitive type %s yet", hiveType)); + } + } + + /** Create Hive columns from Flink TableSchema. */ + public static List createHiveColumns(TableSchema schema) { + String[] fieldNames = schema.getFieldNames(); + DataType[] fieldTypes = schema.getFieldDataTypes(); + + List columns = new ArrayList<>(fieldNames.length); + + for (int i = 0; i < fieldNames.length; i++) { + columns.add( + new FieldSchema( + fieldNames[i], + toHiveTypeInfo(fieldTypes[i], true).getTypeName(), + null)); + } + + return columns; + } + + /** + * Convert Flink DataType to Hive TypeInfo. For types with a precision parameter, e.g. + * timestamp, the supported precisions in Hive and Flink can be different. Therefore the + * conversion will fail for those types if the precision is not supported by Hive and + * checkPrecision is true. + * + * @param dataType a Flink DataType + * @param checkPrecision whether to fail the conversion if the precision of the DataType is not + * supported by Hive + * @return the corresponding Hive data type + */ + public static TypeInfo toHiveTypeInfo(DataType dataType, boolean checkPrecision) { + checkNotNull(dataType, "type cannot be null"); + LogicalType logicalType = dataType.getLogicalType(); + return logicalType.accept(new TypeInfoLogicalTypeVisitor(dataType, checkPrecision)); + } } diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/catalog/TypeInfoLogicalTypeVisitor.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/catalog/TypeInfoLogicalTypeVisitor.java new file mode 100644 index 0000000000000..d6cfe3ed723a7 --- /dev/null +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/catalog/TypeInfoLogicalTypeVisitor.java @@ -0,0 +1,236 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.table.catalog; + +import org.apache.hudi.exception.HoodieCatalogException; + +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.ArrayType; +import org.apache.flink.table.types.logical.BigIntType; +import org.apache.flink.table.types.logical.BooleanType; +import org.apache.flink.table.types.logical.CharType; +import org.apache.flink.table.types.logical.DateType; +import org.apache.flink.table.types.logical.DecimalType; +import org.apache.flink.table.types.logical.DoubleType; +import org.apache.flink.table.types.logical.FloatType; +import org.apache.flink.table.types.logical.IntType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.MapType; +import org.apache.flink.table.types.logical.NullType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.table.types.logical.SmallIntType; +import org.apache.flink.table.types.logical.TimestampType; +import org.apache.flink.table.types.logical.TinyIntType; +import org.apache.flink.table.types.logical.VarBinaryType; +import org.apache.flink.table.types.logical.VarCharType; +import org.apache.flink.table.types.logical.utils.LogicalTypeDefaultVisitor; +import org.apache.hadoop.hive.common.type.HiveChar; +import org.apache.hadoop.hive.common.type.HiveVarchar; +import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory; + +import java.util.ArrayList; +import java.util.List; + +/** + * Create a TypeInfoLogicalTypeVisitor for hoodie table. + */ +public class TypeInfoLogicalTypeVisitor extends LogicalTypeDefaultVisitor { + private final LogicalType type; + // whether to check type precision + private final boolean checkPrecision; + + TypeInfoLogicalTypeVisitor(DataType dataType, boolean checkPrecision) { + this(dataType.getLogicalType(), checkPrecision); + } + + TypeInfoLogicalTypeVisitor(LogicalType type, boolean checkPrecision) { + this.type = type; + this.checkPrecision = checkPrecision; + } + + @Override + public TypeInfo visit(CharType charType) { + // Flink and Hive have different length limit for CHAR. Promote it to STRING if it + // exceeds the limits of + // Hive and we're told not to check precision. This can be useful when calling Hive UDF + // to process data. + if (charType.getLength() > HiveChar.MAX_CHAR_LENGTH || charType.getLength() < 1) { + if (checkPrecision) { + throw new HoodieCatalogException( + String.format( + "HiveCatalog doesn't support char type with length of '%d'. " + + "The supported length is [%d, %d]", + charType.getLength(), 1, HiveChar.MAX_CHAR_LENGTH)); + } else { + return TypeInfoFactory.stringTypeInfo; + } + } + return TypeInfoFactory.getCharTypeInfo(charType.getLength()); + } + + @Override + public TypeInfo visit(VarCharType varCharType) { + // Flink's StringType is defined as VARCHAR(Integer.MAX_VALUE) + // We don't have more information in LogicalTypeRoot to distinguish StringType and a + // VARCHAR(Integer.MAX_VALUE) instance + // Thus always treat VARCHAR(Integer.MAX_VALUE) as StringType + if (varCharType.getLength() == Integer.MAX_VALUE) { + return TypeInfoFactory.stringTypeInfo; + } + // Flink and Hive have different length limit for VARCHAR. Promote it to STRING if it + // exceeds the limits of + // Hive and we're told not to check precision. This can be useful when calling Hive UDF + // to process data. + if (varCharType.getLength() > HiveVarchar.MAX_VARCHAR_LENGTH + || varCharType.getLength() < 1) { + if (checkPrecision) { + throw new HoodieCatalogException( + String.format( + "HiveCatalog doesn't support varchar type with length of '%d'. " + + "The supported length is [%d, %d]", + varCharType.getLength(), 1, HiveVarchar.MAX_VARCHAR_LENGTH)); + } else { + return TypeInfoFactory.stringTypeInfo; + } + } + return TypeInfoFactory.getVarcharTypeInfo(varCharType.getLength()); + } + + @Override + public TypeInfo visit(BooleanType booleanType) { + return TypeInfoFactory.booleanTypeInfo; + } + + @Override + public TypeInfo visit(VarBinaryType varBinaryType) { + // Flink's BytesType is defined as VARBINARY(Integer.MAX_VALUE) + // We don't have more information in LogicalTypeRoot to distinguish BytesType and a + // VARBINARY(Integer.MAX_VALUE) instance + // Thus always treat VARBINARY(Integer.MAX_VALUE) as BytesType + if (varBinaryType.getLength() == VarBinaryType.MAX_LENGTH) { + return TypeInfoFactory.binaryTypeInfo; + } + return defaultMethod(varBinaryType); + } + + @Override + public TypeInfo visit(DecimalType decimalType) { + // Flink and Hive share the same precision and scale range + // Flink already validates the type so we don't need to validate again here + return TypeInfoFactory.getDecimalTypeInfo( + decimalType.getPrecision(), decimalType.getScale()); + } + + @Override + public TypeInfo visit(TinyIntType tinyIntType) { + return TypeInfoFactory.byteTypeInfo; + } + + @Override + public TypeInfo visit(SmallIntType smallIntType) { + return TypeInfoFactory.shortTypeInfo; + } + + @Override + public TypeInfo visit(IntType intType) { + return TypeInfoFactory.intTypeInfo; + } + + @Override + public TypeInfo visit(BigIntType bigIntType) { + return TypeInfoFactory.longTypeInfo; + } + + @Override + public TypeInfo visit(FloatType floatType) { + return TypeInfoFactory.floatTypeInfo; + } + + @Override + public TypeInfo visit(DoubleType doubleType) { + return TypeInfoFactory.doubleTypeInfo; + } + + @Override + public TypeInfo visit(DateType dateType) { + return TypeInfoFactory.dateTypeInfo; + } + + @Override + public TypeInfo visit(TimestampType timestampType) { + if (checkPrecision && timestampType.getPrecision() == 9) { + throw new HoodieCatalogException( + "HoodieCatalog currently does not support timestamp of precision 9"); + } + return TypeInfoFactory.timestampTypeInfo; + } + + @Override + public TypeInfo visit(ArrayType arrayType) { + LogicalType elementType = arrayType.getElementType(); + TypeInfo elementTypeInfo = elementType.accept(this); + if (null != elementTypeInfo) { + return TypeInfoFactory.getListTypeInfo(elementTypeInfo); + } else { + return defaultMethod(arrayType); + } + } + + @Override + public TypeInfo visit(MapType mapType) { + LogicalType keyType = mapType.getKeyType(); + LogicalType valueType = mapType.getValueType(); + TypeInfo keyTypeInfo = keyType.accept(this); + TypeInfo valueTypeInfo = valueType.accept(this); + if (null == keyTypeInfo || null == valueTypeInfo) { + return defaultMethod(mapType); + } else { + return TypeInfoFactory.getMapTypeInfo(keyTypeInfo, valueTypeInfo); + } + } + + @Override + public TypeInfo visit(RowType rowType) { + List names = rowType.getFieldNames(); + List typeInfos = new ArrayList<>(names.size()); + for (String name : names) { + TypeInfo typeInfo = rowType.getTypeAt(rowType.getFieldIndex(name)).accept(this); + if (null != typeInfo) { + typeInfos.add(typeInfo); + } else { + return defaultMethod(rowType); + } + } + return TypeInfoFactory.getStructTypeInfo(names, typeInfos); + } + + @Override + public TypeInfo visit(NullType nullType) { + return TypeInfoFactory.voidTypeInfo; + } + + @Override + protected TypeInfo defaultMethod(LogicalType logicalType) { + throw new UnsupportedOperationException( + String.format( + "Flink doesn't support converting type %s to Hive type yet.", + type.toString())); + } +} diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/FilePathUtils.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/FilePathUtils.java index d03c5aac272e8..0da1aca0e243c 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/FilePathUtils.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/FilePathUtils.java @@ -442,4 +442,8 @@ public static String[] extractHivePartitionFields(org.apache.flink.configuration } return conf.getString(FlinkOptions.HIVE_SYNC_PARTITION_FIELDS).split(","); } + + public static boolean isHiveStylePartitioning(String path) { + return HIVE_PARTITION_NAME_PATTERN.matcher(path).matches(); + } } 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 d292b3832ac3b..0db960c32f56d 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 @@ -38,12 +38,14 @@ import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.ReflectionUtils; import org.apache.hudi.common.util.ValidationUtils; +import org.apache.hudi.config.HoodieArchivalConfig; +import org.apache.hudi.config.HoodieCleanConfig; import org.apache.hudi.config.HoodieClusteringConfig; import org.apache.hudi.config.HoodieCompactionConfig; +import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.config.HoodieMemoryConfig; -import org.apache.hudi.config.HoodiePayloadConfig; import org.apache.hudi.config.HoodieStorageConfig; -import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.config.HoodiePayloadConfig; import org.apache.hudi.configuration.FlinkOptions; import org.apache.hudi.configuration.HadoopConfigurations; import org.apache.hudi.configuration.OptionsResolver; @@ -178,23 +180,25 @@ public static HoodieWriteConfig getHoodieClientConfig( .withClusteringSkipPartitionsFromLatest(conf.getInteger(FlinkOptions.CLUSTERING_PLAN_STRATEGY_SKIP_PARTITIONS_FROM_LATEST)) .withAsyncClusteringMaxCommits(conf.getInteger(FlinkOptions.CLUSTERING_DELTA_COMMITS)) .build()) - .withCompactionConfig( - HoodieCompactionConfig.newBuilder() - .withPayloadClass(conf.getString(FlinkOptions.PAYLOAD_CLASS_NAME)) - .withTargetIOPerCompactionInMB(conf.getLong(FlinkOptions.COMPACTION_TARGET_IO)) - .withInlineCompactionTriggerStrategy( - CompactionTriggerStrategy.valueOf(conf.getString(FlinkOptions.COMPACTION_TRIGGER_STRATEGY).toUpperCase(Locale.ROOT))) - .withMaxNumDeltaCommitsBeforeCompaction(conf.getInteger(FlinkOptions.COMPACTION_DELTA_COMMITS)) - .withMaxDeltaSecondsBeforeCompaction(conf.getInteger(FlinkOptions.COMPACTION_DELTA_SECONDS)) - .withAsyncClean(conf.getBoolean(FlinkOptions.CLEAN_ASYNC_ENABLED)) - .retainCommits(conf.getInteger(FlinkOptions.CLEAN_RETAIN_COMMITS)) - .retainFileVersions(conf.getInteger(FlinkOptions.CLEAN_RETAIN_FILE_VERSIONS)) - // override and hardcode to 20, - // actually Flink cleaning is always with parallelism 1 now - .withCleanerParallelism(20) - .archiveCommitsWith(conf.getInteger(FlinkOptions.ARCHIVE_MIN_COMMITS), conf.getInteger(FlinkOptions.ARCHIVE_MAX_COMMITS)) - .withCleanerPolicy(HoodieCleaningPolicy.valueOf(conf.getString(FlinkOptions.CLEAN_POLICY))) - .build()) + .withCleanConfig(HoodieCleanConfig.newBuilder() + .withAsyncClean(conf.getBoolean(FlinkOptions.CLEAN_ASYNC_ENABLED)) + .retainCommits(conf.getInteger(FlinkOptions.CLEAN_RETAIN_COMMITS)) + .retainFileVersions(conf.getInteger(FlinkOptions.CLEAN_RETAIN_FILE_VERSIONS)) + // override and hardcode to 20, + // actually Flink cleaning is always with parallelism 1 now + .withCleanerParallelism(20) + .withCleanerPolicy(HoodieCleaningPolicy.valueOf(conf.getString(FlinkOptions.CLEAN_POLICY))) + .build()) + .withArchivalConfig(HoodieArchivalConfig.newBuilder() + .archiveCommitsWith(conf.getInteger(FlinkOptions.ARCHIVE_MIN_COMMITS), conf.getInteger(FlinkOptions.ARCHIVE_MAX_COMMITS)) + .build()) + .withCompactionConfig(HoodieCompactionConfig.newBuilder() + .withTargetIOPerCompactionInMB(conf.getLong(FlinkOptions.COMPACTION_TARGET_IO)) + .withInlineCompactionTriggerStrategy( + CompactionTriggerStrategy.valueOf(conf.getString(FlinkOptions.COMPACTION_TRIGGER_STRATEGY).toUpperCase(Locale.ROOT))) + .withMaxNumDeltaCommitsBeforeCompaction(conf.getInteger(FlinkOptions.COMPACTION_DELTA_COMMITS)) + .withMaxDeltaSecondsBeforeCompaction(conf.getInteger(FlinkOptions.COMPACTION_DELTA_SECONDS)) + .build()) .withMemoryConfig( HoodieMemoryConfig.newBuilder() .withMaxMemoryMaxSize( @@ -214,8 +218,10 @@ public static HoodieWriteConfig getHoodieClientConfig( .withMaxNumDeltaCommitsBeforeCompaction(conf.getInteger(FlinkOptions.METADATA_COMPACTION_DELTA_COMMITS)) .build()) .withPayloadConfig(HoodiePayloadConfig.newBuilder() + .withPayloadClass(conf.getString(FlinkOptions.PAYLOAD_CLASS_NAME)) .withPayloadOrderingField(conf.getString(FlinkOptions.PRECOMBINE_FIELD)) .withPayloadEventTimeField(conf.getString(FlinkOptions.PRECOMBINE_FIELD)) + .withPayloadClass(conf.getString(FlinkOptions.PAYLOAD_CLASS_NAME)) .build()) .withEmbeddedTimelineServerEnabled(enableEmbeddedTimelineService) .withEmbeddedTimelineServerReuseEnabled(true) // make write client embedded timeline service singleton diff --git a/hudi-flink-datasource/hudi-flink/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory b/hudi-flink-datasource/hudi-flink/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory index 47435c745c461..13668a5590063 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory +++ b/hudi-flink-datasource/hudi-flink/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory @@ -15,4 +15,4 @@ # limitations under the License. org.apache.hudi.table.HoodieTableFactory -org.apache.hudi.table.catalog.HoodieCatalogFactory +org.apache.hudi.table.catalog.HoodieCatalogFactory \ No newline at end of file diff --git a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/compact/ITTestHoodieFlinkCompactor.java b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/compact/ITTestHoodieFlinkCompactor.java index 43e4ed511452d..341a157e86c78 100644 --- a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/compact/ITTestHoodieFlinkCompactor.java +++ b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/compact/ITTestHoodieFlinkCompactor.java @@ -18,11 +18,6 @@ package org.apache.hudi.sink.compact; -import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator; -import org.apache.flink.streaming.api.functions.ProcessFunction; -import org.apache.flink.util.Collector; -import org.apache.flink.util.OutputTag; import org.apache.hudi.avro.model.HoodieCompactionPlan; import org.apache.hudi.client.HoodieFlinkWriteClient; import org.apache.hudi.common.table.HoodieTableMetaClient; @@ -54,8 +49,9 @@ import org.slf4j.LoggerFactory; import java.io.File; -import java.util.Arrays; import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -138,17 +134,7 @@ public void testHoodieFlinkCompactor(boolean enableChangelog) throws Exception { HoodieFlinkWriteClient writeClient = StreamerUtil.createWriteClient(conf); - boolean scheduled = false; - // judge whether have operation - // To compute the compaction instant time and do compaction. - Option compactionInstantTimeOption = CompactionUtil.getCompactionInstantTime(metaClient); - if (compactionInstantTimeOption.isPresent()) { - scheduled = writeClient.scheduleCompactionAtInstant(compactionInstantTimeOption.get(), Option.empty()); - } - - String compactionInstantTime = compactionInstantTimeOption.get(); - - assertTrue(scheduled, "The compaction plan should be scheduled"); + String compactionInstantTime = scheduleCompactionPlan(metaClient, writeClient); HoodieFlinkTable table = writeClient.getHoodieTable(); // generate compaction plan @@ -160,7 +146,7 @@ public void testHoodieFlinkCompactor(boolean enableChangelog) throws Exception { // Mark instant as compaction inflight table.getActiveTimeline().transitionCompactionRequestedToInflight(instant); - env.addSource(new CompactionPlanSourceFunction(compactionPlan, compactionInstantTime)) + env.addSource(new CompactionPlanSourceFunction(Collections.singletonList(Pair.of(compactionInstantTime, compactionPlan)))) .name("compaction_source") .uid("uid_compaction_source") .rebalance() @@ -280,29 +266,18 @@ public void testHoodieFlinkCompactorWithPlanSelectStrategy(boolean enableChangel } table.getMetaClient().reloadActiveTimeline(); - DataStream source = env.addSource(new MultiCompactionPlanSourceFunction(compactionPlans)) + env.addSource(new CompactionPlanSourceFunction(compactionPlans)) .name("compaction_source") - .uid("uid_compaction_source"); - SingleOutputStreamOperator operator = source.rebalance() + .uid("uid_compaction_source") + .rebalance() .transform("compact_task", TypeInformation.of(CompactionCommitEvent.class), new ProcessOperator<>(new CompactFunction(conf))) .setParallelism(1) - .process(new ProcessFunction() { - @Override - public void processElement(CompactionCommitEvent event, ProcessFunction.Context context, Collector out) { - context.output(new OutputTag<>(event.getInstant(), TypeInformation.of(CompactionCommitEvent.class)), event); - } - }) - .name("group_by_compaction_plan") - .uid("uid_group_by_compaction_plan") + .addSink(new CompactionCommitSink(conf)) + .name("compaction_commit") + .uid("uid_compaction_commit") .setParallelism(1); - compactionPlans.forEach(pair -> - operator.getSideOutput(new OutputTag<>(pair.getLeft(), TypeInformation.of(CompactionCommitEvent.class))) - .addSink(new CompactionCommitSink(conf)) - .name("clean_commits " + pair.getLeft()) - .uid("uid_clean_commits_" + pair.getLeft()) - .setParallelism(1)); env.execute("flink_hudi_compaction"); writeClient.close(); @@ -311,8 +286,7 @@ public void processElement(CompactionCommitEvent event, ProcessFunction writeClient) { boolean scheduled = false; - // judge whether have operation - // To compute the compaction instant time and do compaction. + // judge whether there are any compaction operations. Option compactionInstantTimeOption = CompactionUtil.getCompactionInstantTime(metaClient); if (compactionInstantTimeOption.isPresent()) { scheduled = writeClient.scheduleCompactionAtInstant(compactionInstantTimeOption.get(), Option.empty()); diff --git a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/compact/TestCompactionPlanSelectStrategy.java b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/compact/TestCompactionPlanStrategy.java similarity index 74% rename from hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/compact/TestCompactionPlanSelectStrategy.java rename to hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/compact/TestCompactionPlanStrategy.java index 3ac9f6c6663ef..181be26d6dc60 100644 --- a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/compact/TestCompactionPlanSelectStrategy.java +++ b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/compact/TestCompactionPlanStrategy.java @@ -25,18 +25,16 @@ import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; -import org.apache.hudi.sink.compact.strategy.AllPendingCompactionPlanSelectStrategy; -import org.apache.hudi.sink.compact.strategy.CompactionPlanSelectStrategy; -import org.apache.hudi.sink.compact.strategy.InstantCompactionPlanSelectStrategy; -import org.apache.hudi.sink.compact.strategy.MultiCompactionPlanSelectStrategy; -import org.apache.hudi.sink.compact.strategy.SingleCompactionPlanSelectStrategy; +import org.apache.hudi.sink.compact.strategy.CompactionPlanStrategies; +import org.apache.hudi.sink.compact.strategy.CompactionPlanStrategy; + import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; /** - * Test case for every {@link CompactionPlanSelectStrategy} implements + * Test case for every {@link CompactionPlanStrategy} implements */ -public class TestCompactionPlanSelectStrategy { +public class TestCompactionPlanStrategy { private HoodieTimeline timeline; private HoodieTimeline emptyTimeline; private HoodieTimeline allCompleteTimeline; @@ -59,72 +57,75 @@ public void beforeEach() { void testSingleCompactionPlanSelectStrategy() { HoodieTimeline pendingCompactionTimeline = this.timeline.filterPendingCompactionTimeline(); FlinkCompactionConfig compactionConfig = new FlinkCompactionConfig(); + CompactionPlanStrategy strategy = CompactionPlanStrategies.getStrategy(compactionConfig); - SingleCompactionPlanSelectStrategy strategy = new SingleCompactionPlanSelectStrategy(); - assertHoodieInstantsEquals(new HoodieInstant[]{INSTANT_006}, strategy.select(pendingCompactionTimeline, compactionConfig)); + assertHoodieInstantsEquals(new HoodieInstant[]{INSTANT_002}, strategy.select(pendingCompactionTimeline)); - compactionConfig.compactionSeq = FlinkCompactionConfig.SEQ_FIFO; - assertHoodieInstantsEquals(new HoodieInstant[]{INSTANT_002}, strategy.select(pendingCompactionTimeline, compactionConfig)); + compactionConfig.compactionSeq = FlinkCompactionConfig.SEQ_LIFO; + assertHoodieInstantsEquals(new HoodieInstant[]{INSTANT_006}, strategy.select(pendingCompactionTimeline)); HoodieTimeline emptyPendingCompactionTimeline = emptyTimeline.filterPendingCompactionTimeline(); - assertHoodieInstantsEquals(new HoodieInstant[]{}, strategy.select(emptyPendingCompactionTimeline, compactionConfig)); + assertHoodieInstantsEquals(new HoodieInstant[]{}, strategy.select(emptyPendingCompactionTimeline)); HoodieTimeline allCompleteCompactionTimeline = allCompleteTimeline.filterPendingCompactionTimeline(); - assertHoodieInstantsEquals(new HoodieInstant[]{}, strategy.select(allCompleteCompactionTimeline, compactionConfig)); + assertHoodieInstantsEquals(new HoodieInstant[]{}, strategy.select(allCompleteCompactionTimeline)); } @Test void testMultiCompactionPlanSelectStrategy() { HoodieTimeline pendingCompactionTimeline = this.timeline.filterPendingCompactionTimeline(); FlinkCompactionConfig compactionConfig = new FlinkCompactionConfig(); - compactionConfig.compactionPlanMaxSelect = 2; + compactionConfig.maxNumCompactionPlans = 2; - MultiCompactionPlanSelectStrategy strategy = new MultiCompactionPlanSelectStrategy(); - assertHoodieInstantsEquals(new HoodieInstant[]{INSTANT_006, INSTANT_004}, strategy.select(pendingCompactionTimeline, compactionConfig)); + CompactionPlanStrategy strategy = CompactionPlanStrategies.getStrategy(compactionConfig); + assertHoodieInstantsEquals(new HoodieInstant[]{INSTANT_002, INSTANT_003}, strategy.select(pendingCompactionTimeline)); - compactionConfig.compactionSeq = FlinkCompactionConfig.SEQ_FIFO; - assertHoodieInstantsEquals(new HoodieInstant[]{INSTANT_002, INSTANT_003}, strategy.select(pendingCompactionTimeline, compactionConfig)); + compactionConfig.compactionSeq = FlinkCompactionConfig.SEQ_LIFO; + assertHoodieInstantsEquals(new HoodieInstant[]{INSTANT_006, INSTANT_004}, strategy.select(pendingCompactionTimeline)); HoodieTimeline emptyPendingCompactionTimeline = emptyTimeline.filterPendingCompactionTimeline(); - assertHoodieInstantsEquals(new HoodieInstant[]{}, strategy.select(emptyPendingCompactionTimeline, compactionConfig)); + assertHoodieInstantsEquals(new HoodieInstant[]{}, strategy.select(emptyPendingCompactionTimeline)); HoodieTimeline allCompleteCompactionTimeline = allCompleteTimeline.filterPendingCompactionTimeline(); - assertHoodieInstantsEquals(new HoodieInstant[]{}, strategy.select(allCompleteCompactionTimeline, compactionConfig)); + assertHoodieInstantsEquals(new HoodieInstant[]{}, strategy.select(allCompleteCompactionTimeline)); } @Test void testAllPendingCompactionPlanSelectStrategy() { HoodieTimeline pendingCompactionTimeline = this.timeline.filterPendingCompactionTimeline(); FlinkCompactionConfig compactionConfig = new FlinkCompactionConfig(); + compactionConfig.compactionPlanSelectStrategy = CompactionPlanStrategy.ALL; + CompactionPlanStrategy strategy = CompactionPlanStrategies.getStrategy(compactionConfig); - AllPendingCompactionPlanSelectStrategy strategy = new AllPendingCompactionPlanSelectStrategy(); assertHoodieInstantsEquals(new HoodieInstant[]{INSTANT_002, INSTANT_003, INSTANT_004, INSTANT_006}, - strategy.select(pendingCompactionTimeline, compactionConfig)); + strategy.select(pendingCompactionTimeline)); HoodieTimeline emptyPendingCompactionTimeline = emptyTimeline.filterPendingCompactionTimeline(); - assertHoodieInstantsEquals(new HoodieInstant[]{}, strategy.select(emptyPendingCompactionTimeline, compactionConfig)); + assertHoodieInstantsEquals(new HoodieInstant[]{}, strategy.select(emptyPendingCompactionTimeline)); HoodieTimeline allCompleteCompactionTimeline = allCompleteTimeline.filterPendingCompactionTimeline(); - assertHoodieInstantsEquals(new HoodieInstant[]{}, strategy.select(allCompleteCompactionTimeline, compactionConfig)); + assertHoodieInstantsEquals(new HoodieInstant[]{}, strategy.select(allCompleteCompactionTimeline)); } @Test void testInstantCompactionPlanSelectStrategy() { HoodieTimeline pendingCompactionTimeline = this.timeline.filterPendingCompactionTimeline(); FlinkCompactionConfig compactionConfig = new FlinkCompactionConfig(); + + compactionConfig.compactionPlanSelectStrategy = CompactionPlanStrategy.INSTANTS; + CompactionPlanStrategy strategy = CompactionPlanStrategies.getStrategy(compactionConfig); compactionConfig.compactionPlanInstant = "004"; - InstantCompactionPlanSelectStrategy strategy = new InstantCompactionPlanSelectStrategy(); - assertHoodieInstantsEquals(new HoodieInstant[]{INSTANT_004}, strategy.select(pendingCompactionTimeline, compactionConfig)); + assertHoodieInstantsEquals(new HoodieInstant[]{INSTANT_004}, strategy.select(pendingCompactionTimeline)); compactionConfig.compactionPlanInstant = "002,003"; - assertHoodieInstantsEquals(new HoodieInstant[]{INSTANT_002, INSTANT_003}, strategy.select(pendingCompactionTimeline, compactionConfig)); + assertHoodieInstantsEquals(new HoodieInstant[]{INSTANT_002, INSTANT_003}, strategy.select(pendingCompactionTimeline)); compactionConfig.compactionPlanInstant = "002,005"; - assertHoodieInstantsEquals(new HoodieInstant[]{INSTANT_002}, strategy.select(pendingCompactionTimeline, compactionConfig)); + assertHoodieInstantsEquals(new HoodieInstant[]{INSTANT_002}, strategy.select(pendingCompactionTimeline)); compactionConfig.compactionPlanInstant = "005"; - assertHoodieInstantsEquals(new HoodieInstant[]{}, strategy.select(pendingCompactionTimeline, compactionConfig)); + assertHoodieInstantsEquals(new HoodieInstant[]{}, strategy.select(pendingCompactionTimeline)); } private void assertHoodieInstantsEquals(HoodieInstant[] expected, List actual) { 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 a8104efb322f9..a976d483fbc33 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 @@ -23,6 +23,8 @@ import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.configuration.FlinkOptions; +import org.apache.hudi.table.catalog.HoodieHiveCatalog; +import org.apache.hudi.table.catalog.TestHoodieCatalogUtils; import org.apache.hudi.util.StreamerUtil; import org.apache.hudi.utils.TestConfigurations; import org.apache.hudi.utils.TestData; @@ -1287,6 +1289,45 @@ void testWriteAndReadWithDataSkipping() { + "+I[id8, Han, 56, 1970-01-01T00:00:08, par4]]"); } + @Test + void testBuiltinFunctionWithHMSCatalog() { + TableEnvironment tableEnv = batchTableEnv; + + HoodieHiveCatalog hoodieCatalog = TestHoodieCatalogUtils.createHiveCatalog("hudi_catalog"); + + tableEnv.registerCatalog("hudi_catalog", hoodieCatalog); + tableEnv.executeSql("use catalog " + ("hudi_catalog")); + + String dbName = "hudi"; + tableEnv.executeSql("create database " + dbName); + tableEnv.executeSql("use " + dbName); + + String hoodieTableDDL = sql("t1") + .field("f_int int") + .field("f_date DATE") + .pkField("f_int") + .partitionField("f_int") + .option(FlinkOptions.PATH, tempFile.getAbsolutePath() + "/" + dbName + "/" + "t1") + .option(FlinkOptions.RECORD_KEY_FIELD, "f_int") + .option(FlinkOptions.PRECOMBINE_FIELD, "f_date") + .end(); + tableEnv.executeSql(hoodieTableDDL); + + String insertSql = "insert into t1 values (1, TO_DATE('2022-02-02')), (2, DATE '2022-02-02')"; + execInsertSql(tableEnv, insertSql); + + List result = CollectionUtil.iterableToList( + () -> tableEnv.sqlQuery("select * from t1").execute().collect()); + final String expected = "[" + + "+I[1, 2022-02-02], " + + "+I[2, 2022-02-02]]"; + assertRowsEquals(result, expected); + + List partitionResult = CollectionUtil.iterableToList( + () -> tableEnv.sqlQuery("select * from t1 where f_int = 1").execute().collect()); + assertRowsEquals(partitionResult, "[+I[1, 2022-02-02]]"); + } + // ------------------------------------------------------------------------- // Utilities // ------------------------------------------------------------------------- diff --git a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/table/catalog/TestHoodieCatalogFactory.java b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/table/catalog/TestHoodieCatalogFactory.java new file mode 100644 index 0000000000000..a99df0ddd2e4e --- /dev/null +++ b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/table/catalog/TestHoodieCatalogFactory.java @@ -0,0 +1,67 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.table.catalog; + +import org.apache.flink.table.catalog.Catalog; +import org.apache.flink.table.catalog.CommonCatalogOptions; +import org.apache.flink.table.factories.FactoryUtil; +import org.apache.hadoop.hive.conf.HiveConf; +import org.junit.jupiter.api.Test; + +import java.net.URL; +import java.util.HashMap; +import java.util.Map; + +import static org.junit.jupiter.api.Assertions.assertEquals; + +/** + * Test cases for {@link HoodieCatalogFactory}. + */ +public class TestHoodieCatalogFactory { + private static final URL CONF_DIR = + Thread.currentThread().getContextClassLoader().getResource("test-catalog-factory-conf"); + + @Test + public void testCreateHiveCatalog() { + final String catalogName = "mycatalog"; + + final HoodieHiveCatalog expectedCatalog = TestHoodieCatalogUtils.createHiveCatalog(catalogName); + + final Map options = new HashMap<>(); + options.put(CommonCatalogOptions.CATALOG_TYPE.key(), HoodieCatalogFactory.IDENTIFIER); + options.put(HoodieCatalogFactoryOptions.HIVE_CONF_DIR.key(), CONF_DIR.getPath()); + options.put(HoodieCatalogFactoryOptions.MODE.key(), "hms"); + + final Catalog actualCatalog = + FactoryUtil.createCatalog( + catalogName, options, null, Thread.currentThread().getContextClassLoader()); + + assertEquals( + ((HoodieHiveCatalog) actualCatalog) + .getHiveConf() + .getVar(HiveConf.ConfVars.METASTOREURIS), "dummy-hms"); + checkEquals(expectedCatalog, (HoodieHiveCatalog) actualCatalog); + } + + private static void checkEquals(HoodieHiveCatalog c1, HoodieHiveCatalog c2) { + // Only assert a few selected properties for now + assertEquals(c2.getName(), c1.getName()); + assertEquals(c2.getDefaultDatabase(), c1.getDefaultDatabase()); + } +} diff --git a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/table/catalog/TestHoodieCatalogUtils.java b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/table/catalog/TestHoodieCatalogUtils.java new file mode 100644 index 0000000000000..de433d0a3a667 --- /dev/null +++ b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/table/catalog/TestHoodieCatalogUtils.java @@ -0,0 +1,67 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.table.catalog; + +import org.apache.flink.table.catalog.exceptions.CatalogException; +import org.apache.hadoop.hive.conf.HiveConf; + +import java.io.IOException; + +/** Test utils for Hoodie catalog. */ +public class TestHoodieCatalogUtils { + private static final String HIVE_WAREHOUSE_URI_FORMAT = + "jdbc:derby:;databaseName=%s;create=true"; + + private static final String TEST_CATALOG_NAME = "test_catalog"; + + private static final org.junit.rules.TemporaryFolder TEMPORARY_FOLDER = new org.junit.rules.TemporaryFolder(); + + /** Create a HiveCatalog with an embedded Hive Metastore. */ + public static HoodieHiveCatalog createHiveCatalog() { + return createHiveCatalog(TEST_CATALOG_NAME); + } + + public static HoodieHiveCatalog createHiveCatalog(String name) { + return new HoodieHiveCatalog( + name, + null, + createHiveConf(), + true, + true); + } + + public static HiveConf createHiveConf() { + ClassLoader classLoader = TestHoodieCatalogUtils.class.getClassLoader(); + try { + TEMPORARY_FOLDER.create(); + String warehouseDir = TEMPORARY_FOLDER.newFolder().getAbsolutePath() + "/metastore_db"; + String warehouseUri = String.format(HIVE_WAREHOUSE_URI_FORMAT, warehouseDir); + + HiveConf.setHiveSiteLocation(classLoader.getResource(HoodieCatalogFactoryOptions.HIVE_SITE_FILE)); + HiveConf hiveConf = new HiveConf(); + hiveConf.setVar( + HiveConf.ConfVars.METASTOREWAREHOUSE, + TEMPORARY_FOLDER.newFolder("hive_warehouse").getAbsolutePath()); + hiveConf.setVar(HiveConf.ConfVars.METASTORECONNECTURLKEY, warehouseUri); + return hiveConf; + } catch (IOException e) { + throw new CatalogException("Failed to create test HiveConf to HiveCatalog.", e); + } + } +} 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 new file mode 100644 index 0000000000000..ed470c9a96c35 --- /dev/null +++ b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/table/catalog/TestHoodieHiveCatalog.java @@ -0,0 +1,150 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.table.catalog; + +import org.apache.hudi.common.model.HoodieTableType; +import org.apache.hudi.configuration.FlinkOptions; +import org.apache.hudi.exception.HoodieCatalogException; + +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.catalog.CatalogBaseTable; +import org.apache.flink.table.catalog.CatalogTable; +import org.apache.flink.table.catalog.CatalogTableImpl; +import org.apache.flink.table.catalog.ObjectPath; +import org.apache.flink.table.catalog.exceptions.DatabaseNotExistException; +import org.apache.flink.table.catalog.exceptions.TableAlreadyExistException; +import org.apache.flink.table.catalog.exceptions.TableNotExistException; +import org.apache.flink.table.factories.FactoryUtil; +import org.apache.hadoop.hive.metastore.api.Table; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.EnumSource; + +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.apache.flink.table.factories.FactoryUtil.CONNECTOR; +import static org.junit.jupiter.api.Assertions.assertEquals; + +/** + * Test cases for {@link HoodieHiveCatalog}. + */ +public class TestHoodieHiveCatalog { + TableSchema schema = + TableSchema.builder() + .field("uuid", DataTypes.INT().notNull()) + .field("name", DataTypes.STRING()) + .field("age", DataTypes.INT()) + .field("ts", DataTypes.BIGINT()) + .field("par1", DataTypes.STRING()) + .primaryKey("uuid") + .build(); + List partitions = Collections.singletonList("par1"); + private static HoodieHiveCatalog hoodieCatalog; + private final ObjectPath tablePath = new ObjectPath("default", "test"); + + @BeforeAll + public static void createCatalog() { + hoodieCatalog = TestHoodieCatalogUtils.createHiveCatalog(); + hoodieCatalog.open(); + } + + @AfterEach + public void dropTable() throws TableNotExistException { + hoodieCatalog.dropTable(tablePath, true); + } + + @AfterAll + public static void closeCatalog() { + if (hoodieCatalog != null) { + hoodieCatalog.close(); + } + } + + @ParameterizedTest + @EnumSource(value = HoodieTableType.class) + public void testCreateAndGetHoodieTable(HoodieTableType tableType) throws Exception { + Map originOptions = new HashMap<>(); + originOptions.put(FactoryUtil.CONNECTOR.key(), "hudi"); + originOptions.put(FlinkOptions.TABLE_TYPE.key(), tableType.toString()); + + CatalogTable table = + new CatalogTableImpl(schema, partitions, originOptions, "hudi table"); + hoodieCatalog.createTable(tablePath, table, false); + + CatalogBaseTable table1 = hoodieCatalog.getTable(tablePath); + assertEquals(table1.getOptions().get(CONNECTOR.key()), "hudi"); + assertEquals(table1.getOptions().get(FlinkOptions.TABLE_TYPE.key()), tableType.toString()); + assertEquals(table1.getOptions().get(FlinkOptions.RECORD_KEY_FIELD.key()), "uuid"); + assertEquals(table1.getOptions().get(FlinkOptions.PRECOMBINE_FIELD.key()), "ts"); + assertEquals(table1.getUnresolvedSchema().getPrimaryKey().get().getColumnNames(), Collections.singletonList("uuid")); + assertEquals(((CatalogTable)table1).getPartitionKeys(), Collections.singletonList("par1")); + } + + @Test + public void testCreateNonHoodieTable() throws TableAlreadyExistException, DatabaseNotExistException { + CatalogTable table = + new CatalogTableImpl(schema, Collections.emptyMap(), "hudi table"); + try { + hoodieCatalog.createTable(tablePath, table, false); + } catch (HoodieCatalogException e) { + assertEquals(String.format("The %s is not hoodie table", tablePath.getObjectName()), e.getMessage()); + } + } + + @Test + public void testAlterTable() throws Exception { + Map originOptions = new HashMap<>(); + originOptions.put(FactoryUtil.CONNECTOR.key(), "hudi"); + CatalogTable originTable = + new CatalogTableImpl(schema, partitions, originOptions, "hudi table"); + hoodieCatalog.createTable(tablePath, originTable, false); + + Table hiveTable = hoodieCatalog.getHiveTable(tablePath); + Map newOptions = hiveTable.getParameters(); + newOptions.put("k", "v"); + CatalogTable newTable = new CatalogTableImpl(schema, partitions, newOptions, "alter hudi table"); + hoodieCatalog.alterTable(tablePath, newTable, false); + + hiveTable = hoodieCatalog.getHiveTable(tablePath); + assertEquals(hiveTable.getParameters().get(CONNECTOR.key()), "hudi"); + assertEquals(hiveTable.getParameters().get("k"), "v"); + } + + @Test + public void testRenameTable() throws Exception { + Map originOptions = new HashMap<>(); + originOptions.put(FactoryUtil.CONNECTOR.key(), "hudi"); + CatalogTable originTable = + new CatalogTableImpl(schema, partitions, originOptions, "hudi table"); + hoodieCatalog.createTable(tablePath, originTable, false); + + hoodieCatalog.renameTable(tablePath, "test1", false); + + assertEquals(hoodieCatalog.getHiveTable(new ObjectPath("default", "test1")).getTableName(), "test1"); + + hoodieCatalog.renameTable(new ObjectPath("default", "test1"), "test", false); + } +} diff --git a/hudi-flink-datasource/hudi-flink/src/test/resources/hive-site.xml b/hudi-flink-datasource/hudi-flink/src/test/resources/hive-site.xml new file mode 100644 index 0000000000000..b13347f808edd --- /dev/null +++ b/hudi-flink-datasource/hudi-flink/src/test/resources/hive-site.xml @@ -0,0 +1,52 @@ + + + + + + + + + hive.metastore.integral.jdo.pushdown + true + + + + hive.metastore.schema.verification + false + + + + hive.metastore.client.capability.check + false + + + + datanucleus.schema.autoCreateTables + true + + + + datanucleus.schema.autoCreateAll + true + + + + common-key + common-val + + diff --git a/hudi-flink-datasource/hudi-flink/src/test/resources/test-catalog-factory-conf/hive-site.xml b/hudi-flink-datasource/hudi-flink/src/test/resources/test-catalog-factory-conf/hive-site.xml new file mode 100644 index 0000000000000..97424758d172f --- /dev/null +++ b/hudi-flink-datasource/hudi-flink/src/test/resources/test-catalog-factory-conf/hive-site.xml @@ -0,0 +1,58 @@ + + + + + + + + + hive.metastore.integral.jdo.pushdown + true + + + + hive.metastore.schema.verification + false + + + + hive.metastore.client.capability.check + false + + + + datanucleus.schema.autoCreateTables + true + + + + datanucleus.schema.autoCreateAll + true + + + + common-key + common-val + + + + + hive.metastore.uris + dummy-hms + + diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieTestSuiteWriter.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieTestSuiteWriter.java index 7a9e122e86c15..dc711818d710d 100644 --- a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieTestSuiteWriter.java +++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieTestSuiteWriter.java @@ -25,7 +25,6 @@ import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.collection.Pair; -import org.apache.hudi.config.HoodieCompactionConfig; import org.apache.hudi.config.HoodieIndexConfig; import org.apache.hudi.config.HoodiePayloadConfig; import org.apache.hudi.config.HoodieWriteConfig; @@ -95,8 +94,9 @@ private HoodieWriteConfig getHoodieClientConfig(HoodieTestSuiteJob.HoodieTestSui HoodieWriteConfig.Builder builder = HoodieWriteConfig.newBuilder().combineInput(true, true).withPath(cfg.targetBasePath) .withAutoCommit(false) - .withCompactionConfig(HoodieCompactionConfig.newBuilder().withPayloadClass(cfg.payloadClassName).build()) - .withPayloadConfig(HoodiePayloadConfig.newBuilder().withPayloadOrderingField(cfg.sourceOrderingField) + .withPayloadConfig(HoodiePayloadConfig.newBuilder() + .withPayloadOrderingField(cfg.sourceOrderingField) + .withPayloadClass(cfg.payloadClassName) .build()) .forTable(cfg.targetTableName) .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build()) diff --git a/hudi-kafka-connect/demo/config-sink-hive.json b/hudi-kafka-connect/demo/config-sink-hive.json index 214fd1891906f..7c49784cbf6a3 100644 --- a/hudi-kafka-connect/demo/config-sink-hive.json +++ b/hudi-kafka-connect/demo/config-sink-hive.json @@ -21,7 +21,6 @@ "hoodie.datasource.hive_sync.table": "huditesttopic", "hoodie.datasource.hive_sync.partition_fields": "date", "hoodie.datasource.hive_sync.partition_extractor_class": "org.apache.hudi.hive.MultiPartKeysValueExtractor", - "hoodie.datasource.hive_sync.use_jdbc": "false", "hoodie.datasource.hive_sync.mode": "hms", "dfs.client.use.datanode.hostname": "true", "hive.metastore.uris": "thrift://hivemetastore:9083", diff --git a/hudi-kafka-connect/src/main/java/org/apache/hudi/connect/writers/KafkaConnectConfigs.java b/hudi-kafka-connect/src/main/java/org/apache/hudi/connect/writers/KafkaConnectConfigs.java index e4543c692db86..3b51fddfa8db6 100644 --- a/hudi-kafka-connect/src/main/java/org/apache/hudi/connect/writers/KafkaConnectConfigs.java +++ b/hudi-kafka-connect/src/main/java/org/apache/hudi/connect/writers/KafkaConnectConfigs.java @@ -172,7 +172,6 @@ public String getHadoopConfHome() { public static final String HIVE_URL = "hoodie.datasource.hive_sync.jdbcurl"; public static final String HIVE_PARTITION_FIELDS = "hoodie.datasource.hive_sync.partition_fields"; public static final String HIVE_PARTITION_EXTRACTOR_CLASS = "hoodie.datasource.hive_sync.partition_extractor_class"; - public static final String HIVE_USE_JDBC = "hoodie.datasource.hive_sync.use_jdbc"; public static final String HIVE_SYNC_MODE = "hoodie.datasource.hive_sync.mode"; public static final String HIVE_AUTO_CREATE_DATABASE = "hoodie.datasource.hive_sync.auto_create_database"; public static final String HIVE_IGNORE_EXCEPTIONS = "hoodie.datasource.hive_sync.ignore_exceptions"; diff --git a/hudi-kafka-connect/src/main/java/org/apache/hudi/connect/writers/KafkaConnectWriterProvider.java b/hudi-kafka-connect/src/main/java/org/apache/hudi/connect/writers/KafkaConnectWriterProvider.java index 6ab0469738dff..1a33560dc858a 100644 --- a/hudi-kafka-connect/src/main/java/org/apache/hudi/connect/writers/KafkaConnectWriterProvider.java +++ b/hudi-kafka-connect/src/main/java/org/apache/hudi/connect/writers/KafkaConnectWriterProvider.java @@ -27,10 +27,12 @@ import org.apache.hudi.common.model.HoodieAvroPayload; import org.apache.hudi.common.util.ReflectionUtils; import org.apache.hudi.common.util.StringUtils; -import org.apache.hudi.config.HoodieClusteringConfig; -import org.apache.hudi.config.HoodieCompactionConfig; -import org.apache.hudi.config.HoodieIndexConfig; import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.config.HoodieIndexConfig; +import org.apache.hudi.config.HoodieCleanConfig; +import org.apache.hudi.config.HoodieCompactionConfig; +import org.apache.hudi.config.HoodieArchivalConfig; +import org.apache.hudi.config.HoodieClusteringConfig; import org.apache.hudi.connect.KafkaConnectFileIdPrefixProvider; import org.apache.hudi.connect.utils.KafkaConnectUtils; import org.apache.hudi.exception.HoodieException; @@ -87,13 +89,10 @@ public KafkaConnectWriterProvider( .withAutoCommit(false) .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.INMEMORY).build()) // participants should not trigger table services, and leave it to the coordinator - .withCompactionConfig(HoodieCompactionConfig.newBuilder() - .withAutoClean(false) - .withAutoArchive(false) - .withInlineCompaction(false).build()) - .withClusteringConfig(HoodieClusteringConfig.newBuilder() - .withInlineClustering(false) - .build()) + .withArchivalConfig(HoodieArchivalConfig.newBuilder().withAutoArchive(false).build()) + .withCleanConfig(HoodieCleanConfig.newBuilder().withAutoClean(false).build()) + .withCompactionConfig(HoodieCompactionConfig.newBuilder().withInlineCompaction(false).build()) + .withClusteringConfig(HoodieClusteringConfig.newBuilder().withInlineClustering(false).build()) .build(); context = new HoodieJavaEngineContext(hadoopConf); 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 ca1474fa88abc..80be97ebef3a2 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 @@ -180,9 +180,10 @@ public static HoodieWriteConfig createHoodieConfig(String schemaStr, String base return builder.forTable(tblName) .withCompactionConfig(HoodieCompactionConfig.newBuilder() - .withPayloadClass(parameters.get(DataSourceWriteOptions.PAYLOAD_CLASS_NAME().key())) .withInlineCompaction(inlineCompact).build()) - .withPayloadConfig(HoodiePayloadConfig.newBuilder().withPayloadOrderingField(parameters.get(DataSourceWriteOptions.PRECOMBINE_FIELD().key())) + .withPayloadConfig(HoodiePayloadConfig.newBuilder() + .withPayloadClass(parameters.get(DataSourceWriteOptions.PAYLOAD_CLASS_NAME().key())) + .withPayloadOrderingField(parameters.get(DataSourceWriteOptions.PRECOMBINE_FIELD().key())) .build()) // override above with Hoodie configs specified as options. .withProps(parameters).build(); 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 8ff82746b0ceb..feeb57212603d 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 @@ -399,12 +399,7 @@ object DataSourceWriteOptions { .defaultValue(classOf[HiveSyncTool].getName) .withDocumentation("Sync tool class name used to sync to metastore. Defaults to Hive.") - val RECONCILE_SCHEMA: ConfigProperty[Boolean] = ConfigProperty - .key("hoodie.datasource.write.reconcile.schema") - .defaultValue(false) - .withDocumentation("When a new batch of write has records with old schema, but latest table schema got " - + "evolved, this config will upgrade the records to leverage latest table schema(default values will be " - + "injected to missing fields). If not, the write batch would fail.") + val RECONCILE_SCHEMA: ConfigProperty[Boolean] = HoodieCommonConfig.RECONCILE_SCHEMA // HIVE SYNC SPECIFIC CONFIGS // NOTE: DO NOT USE uppercase for the keys as they are internally lower-cased. Using upper-cases causes @@ -419,8 +414,6 @@ object DataSourceWriteOptions { @Deprecated val HIVE_DATABASE: ConfigProperty[String] = HoodieSyncConfig.META_SYNC_DATABASE_NAME @Deprecated - val hiveTableOptKeyInferFunc: JavaFunction[HoodieConfig, Option[String]] = HoodieSyncConfig.TABLE_NAME_INFERENCE_FUNCTION - @Deprecated val HIVE_TABLE: ConfigProperty[String] = HoodieSyncConfig.META_SYNC_TABLE_NAME @Deprecated val HIVE_BASE_FILE_FORMAT: ConfigProperty[String] = HoodieSyncConfig.META_SYNC_BASE_FILE_FORMAT @@ -444,10 +437,6 @@ object DataSourceWriteOptions { val HIVE_ASSUME_DATE_PARTITION: ConfigProperty[String] = HoodieSyncConfig.META_SYNC_ASSUME_DATE_PARTITION @Deprecated val HIVE_USE_PRE_APACHE_INPUT_FORMAT: ConfigProperty[String] = HiveSyncConfigHolder.HIVE_USE_PRE_APACHE_INPUT_FORMAT - - /** @deprecated Use {@link HIVE_SYNC_MODE} instead of this config from 0.9.0 */ - @Deprecated - val HIVE_USE_JDBC: ConfigProperty[String] = HiveSyncConfigHolder.HIVE_USE_JDBC @Deprecated val HIVE_AUTO_CREATE_DATABASE: ConfigProperty[String] = HiveSyncConfigHolder.HIVE_AUTO_CREATE_DATABASE @Deprecated @@ -507,9 +496,6 @@ object DataSourceWriteOptions { /** @deprecated Use {@link HIVE_USE_PRE_APACHE_INPUT_FORMAT} and its methods instead */ @Deprecated val HIVE_USE_PRE_APACHE_INPUT_FORMAT_OPT_KEY = HiveSyncConfigHolder.HIVE_USE_PRE_APACHE_INPUT_FORMAT.key() - /** @deprecated Use {@link HIVE_USE_JDBC} and its methods instead */ - @Deprecated - val HIVE_USE_JDBC_OPT_KEY = HiveSyncConfigHolder.HIVE_USE_JDBC.key() /** @deprecated Use {@link HIVE_AUTO_CREATE_DATABASE} and its methods instead */ @Deprecated val HIVE_AUTO_CREATE_DATABASE_OPT_KEY = HiveSyncConfigHolder.HIVE_AUTO_CREATE_DATABASE.key() @@ -700,9 +686,6 @@ object DataSourceWriteOptions { val DEFAULT_HIVE_ASSUME_DATE_PARTITION_OPT_VAL = HoodieSyncConfig.META_SYNC_ASSUME_DATE_PARTITION.defaultValue() @Deprecated val DEFAULT_USE_PRE_APACHE_INPUT_FORMAT_OPT_VAL = "false" - /** @deprecated Use {@link HIVE_USE_JDBC} and its methods instead */ - @Deprecated - val DEFAULT_HIVE_USE_JDBC_OPT_VAL = HiveSyncConfigHolder.HIVE_USE_JDBC.defaultValue() /** @deprecated Use {@link HIVE_AUTO_CREATE_DATABASE} and its methods instead */ @Deprecated val DEFAULT_HIVE_AUTO_CREATE_DATABASE_OPT_KEY = HiveSyncConfigHolder.HIVE_AUTO_CREATE_DATABASE.defaultValue() 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 510a45899bce5..a90e6b8e8e653 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,6 +40,7 @@ import org.apache.hudi.hive.{HiveSyncConfigHolder, HiveSyncTool} import org.apache.hudi.index.SparkHoodieIndexFactory import org.apache.hudi.internal.DataSourceInternalWriterHelper import org.apache.hudi.internal.schema.InternalSchema +import org.apache.hudi.internal.schema.convert.AvroInternalSchemaConverter import org.apache.hudi.internal.schema.utils.{AvroSchemaEvolutionUtils, SerDeHelper} import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory import org.apache.hudi.keygen.{TimestampBasedAvroKeyGenerator, TimestampBasedKeyGenerator} @@ -242,16 +243,29 @@ object HoodieSparkSqlWriter { classOf[org.apache.avro.Schema])) var schema = AvroConversionUtils.convertStructTypeToAvroSchema(df.schema, structName, nameSpace) val lastestSchema = getLatestTableSchema(fs, basePath, sparkContext, schema) - val internalSchemaOpt = getLatestTableInternalSchema(fs, basePath, sparkContext) + var internalSchemaOpt = getLatestTableInternalSchema(fs, basePath, sparkContext) + if (reconcileSchema && parameters.getOrDefault(DataSourceReadOptions.SCHEMA_EVOLUTION_ENABLED.key(), "false").toBoolean + && internalSchemaOpt.isEmpty) { + // force apply full schema evolution. + internalSchemaOpt = Some(AvroInternalSchemaConverter.convert(schema)) + } if (reconcileSchema) { schema = lastestSchema } if (internalSchemaOpt.isDefined) { - schema = { - val newSparkSchema = AvroConversionUtils.convertAvroSchemaToStructType(AvroSchemaEvolutionUtils.canonicalizeColumnNullability(schema, lastestSchema)) - AvroConversionUtils.convertStructTypeToAvroSchema(newSparkSchema, structName, nameSpace) - + // Apply schema evolution. + val mergedSparkSchema = if (!reconcileSchema) { + AvroConversionUtils.convertAvroSchemaToStructType(AvroSchemaEvolutionUtils.canonicalizeColumnNullability(schema, lastestSchema)) + } else { + // Auto merge write schema and read schema. + val mergedInternalSchema = AvroSchemaEvolutionUtils.reconcileSchema(schema, internalSchemaOpt.get) + AvroConversionUtils.convertAvroSchemaToStructType(AvroInternalSchemaConverter.convert(mergedInternalSchema, lastestSchema.getName)) } + schema = AvroConversionUtils.convertStructTypeToAvroSchema(mergedSparkSchema, structName, nameSpace) + } + + if (reconcileSchema && internalSchemaOpt.isEmpty) { + schema = lastestSchema } validateSchemaForHoodieIsDeleted(schema) sparkContext.getConf.registerAvroSchemas(schema) 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 f9d8a60004a8b..f6473c2b89731 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 @@ -78,7 +78,6 @@ object HoodieWriterUtils { hoodieConfig.setDefaultValue(HoodieSyncConfig.META_SYNC_PARTITION_FIELDS) hoodieConfig.setDefaultValue(HoodieSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS) hoodieConfig.setDefaultValue(HIVE_STYLE_PARTITIONING) - hoodieConfig.setDefaultValue(HiveSyncConfigHolder.HIVE_USE_JDBC) hoodieConfig.setDefaultValue(HiveSyncConfigHolder.HIVE_CREATE_MANAGED_TABLE) hoodieConfig.setDefaultValue(HiveSyncConfigHolder.HIVE_SYNC_AS_DATA_SOURCE_TABLE) hoodieConfig.setDefaultValue(ASYNC_COMPACT_ENABLE) diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/DeDupeType.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/DeDupeType.scala new file mode 100644 index 0000000000000..93cec470ec99c --- /dev/null +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/DeDupeType.scala @@ -0,0 +1,28 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hudi + +object DeDupeType extends Enumeration { + + type dedupeType = Value + + val INSERT_TYPE = Value("insert_type") + val UPDATE_TYPE = Value("update_type") + val UPSERT_TYPE = Value("upsert_type") +} diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/DedupeSparkJob.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/DedupeSparkJob.scala new file mode 100644 index 0000000000000..b6f610e7d7098 --- /dev/null +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/DedupeSparkJob.scala @@ -0,0 +1,245 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hudi + +import org.apache.hadoop.fs.{FileSystem, FileUtil, Path} +import org.apache.hudi.common.fs.FSUtils +import org.apache.hudi.common.model.{HoodieBaseFile, HoodieRecord} +import org.apache.hudi.common.table.HoodieTableMetaClient +import org.apache.hudi.common.table.view.HoodieTableFileSystemView +import org.apache.hudi.exception.HoodieException +import org.apache.log4j.Logger +import org.apache.spark.sql.{DataFrame, Row, SQLContext} + +import java.util.stream.Collectors +import scala.collection.JavaConversions._ +import scala.collection.mutable.{Buffer, HashMap, HashSet, ListBuffer} + +/** + * Spark job to de-duplicate data present in a partition path + */ +class DedupeSparkJob(basePath: String, + duplicatedPartitionPath: String, + repairOutputPath: String, + sqlContext: SQLContext, + fs: FileSystem, + dedupeType: DeDupeType.Value) { + + val sparkHelper = new SparkHelper(sqlContext, fs) + val LOG = Logger.getLogger(this.getClass) + + /** + * + * @param tblName + * @return + */ + def getDupeKeyDF(tblName: String): DataFrame = { + val dupeSql = + s""" + select `${HoodieRecord.RECORD_KEY_METADATA_FIELD}` as dupe_key, + count(*) as dupe_cnt + from ${tblName} + group by `${HoodieRecord.RECORD_KEY_METADATA_FIELD}` + having dupe_cnt > 1 + """ + sqlContext.sql(dupeSql) + } + + /** + * + * Check a given partition for duplicates and suggest the deletions that need to be done in each file, + * in order to set things right. + * + * @return + */ + private def planDuplicateFix(): HashMap[String, HashSet[String]] = { + val tmpTableName = s"htbl_${System.currentTimeMillis()}" + val dedupeTblName = s"${tmpTableName}_dupeKeys" + + val metadata = HoodieTableMetaClient.builder().setConf(fs.getConf).setBasePath(basePath).build() + + val allFiles = fs.listStatus(new org.apache.hadoop.fs.Path(s"$basePath/$duplicatedPartitionPath")) + val fsView = new HoodieTableFileSystemView(metadata, metadata.getActiveTimeline.getCommitTimeline.filterCompletedInstants(), allFiles) + val latestFiles: java.util.List[HoodieBaseFile] = fsView.getLatestBaseFiles().collect(Collectors.toList[HoodieBaseFile]()) + val filteredStatuses = latestFiles.map(f => f.getPath) + LOG.info(s" List of files under partition: ${} => ${filteredStatuses.mkString(" ")}") + + val df = sqlContext.parquetFile(filteredStatuses: _*) + df.registerTempTable(tmpTableName) + val dupeKeyDF = getDupeKeyDF(tmpTableName) + dupeKeyDF.registerTempTable(dedupeTblName) + + // Obtain necessary satellite information for duplicate rows + val dupeDataSql = + s""" + SELECT `_hoodie_record_key`, `_hoodie_partition_path`, `_hoodie_file_name`, `_hoodie_commit_time` + FROM $tmpTableName h + JOIN $dedupeTblName d + ON h.`_hoodie_record_key` = d.dupe_key + """ + val dupeMap = sqlContext.sql(dupeDataSql).collectAsList().groupBy(r => r.getString(0)) + getDedupePlan(dupeMap) + } + + private def getDedupePlan(dupeMap: Map[String, Buffer[Row]]): HashMap[String, HashSet[String]] = { + val fileToDeleteKeyMap = new HashMap[String, HashSet[String]]() + dupeMap.foreach(rt => { + val (key, rows) = rt + + dedupeType match { + case DeDupeType.UPDATE_TYPE => + /* + This corresponds to the case where all duplicates have been updated at least once. + Once updated, duplicates are bound to have same commit time unless forcefully modified. + */ + rows.init.foreach(r => { + val f = r(2).asInstanceOf[String].split("_")(0) + if (!fileToDeleteKeyMap.contains(f)) { + fileToDeleteKeyMap(f) = HashSet[String]() + } + fileToDeleteKeyMap(f).add(key) + }) + + case DeDupeType.INSERT_TYPE => + /* + This corresponds to the case where duplicates got created due to INSERT and have never been updated. + */ + var maxCommit = -1L + + rows.foreach(r => { + val c = r(3).asInstanceOf[String].toLong + if (c > maxCommit) + maxCommit = c + }) + + rows.foreach(r => { + val c = r(3).asInstanceOf[String].toLong + if (c != maxCommit) { + val f = r(2).asInstanceOf[String].split("_")(0) + if (!fileToDeleteKeyMap.contains(f)) { + fileToDeleteKeyMap(f) = HashSet[String]() + } + fileToDeleteKeyMap(f).add(key) + } + }) + + case DeDupeType.UPSERT_TYPE => + /* + This corresponds to the case where duplicates got created as a result of inserts as well as updates, + i.e few duplicate records have been updated, while others were never updated. + */ + var maxCommit = -1L + + rows.foreach(r => { + val c = r(3).asInstanceOf[String].toLong + if (c > maxCommit) + maxCommit = c + }) + val rowsWithMaxCommit = new ListBuffer[Row]() + rows.foreach(r => { + val c = r(3).asInstanceOf[String].toLong + if (c != maxCommit) { + val f = r(2).asInstanceOf[String].split("_")(0) + if (!fileToDeleteKeyMap.contains(f)) { + fileToDeleteKeyMap(f) = HashSet[String]() + } + fileToDeleteKeyMap(f).add(key) + } else { + rowsWithMaxCommit += r + } + }) + + rowsWithMaxCommit.toList.init.foreach(r => { + val f = r(2).asInstanceOf[String].split("_")(0) + if (!fileToDeleteKeyMap.contains(f)) { + fileToDeleteKeyMap(f) = HashSet[String]() + } + fileToDeleteKeyMap(f).add(key) + }) + + case _ => throw new IllegalArgumentException("Please provide valid type for deduping!") + } + }) + LOG.debug(s"fileToDeleteKeyMap size: ${fileToDeleteKeyMap.size}, map: $fileToDeleteKeyMap") + fileToDeleteKeyMap + } + + def fixDuplicates(dryRun: Boolean = true) = { + val metadata = HoodieTableMetaClient.builder().setConf(fs.getConf).setBasePath(basePath).build() + + val allFiles = fs.listStatus(new Path(s"$basePath/$duplicatedPartitionPath")) + val fsView = new HoodieTableFileSystemView(metadata, metadata.getActiveTimeline.getCommitTimeline.filterCompletedInstants(), allFiles) + + val latestFiles: java.util.List[HoodieBaseFile] = fsView.getLatestBaseFiles().collect(Collectors.toList[HoodieBaseFile]()) + + val fileNameToPathMap = latestFiles.map(f => (f.getFileId, new Path(f.getPath))).toMap + val dupeFixPlan = planDuplicateFix() + + // 1. Copy all latest files into the temp fix path + fileNameToPathMap.foreach { case (fileName, filePath) => + val badSuffix = if (dupeFixPlan.contains(fileName)) ".bad" else "" + val dstPath = new Path(s"$repairOutputPath/${filePath.getName}$badSuffix") + LOG.info(s"Copying from $filePath to $dstPath") + FileUtil.copy(fs, filePath, fs, dstPath, false, true, fs.getConf) + } + + // 2. Remove duplicates from the bad files + dupeFixPlan.foreach { case (fileName, keysToSkip) => + val instantTime = FSUtils.getCommitTime(fileNameToPathMap(fileName).getName) + val badFilePath = new Path(s"$repairOutputPath/${fileNameToPathMap(fileName).getName}.bad") + val newFilePath = new Path(s"$repairOutputPath/${fileNameToPathMap(fileName).getName}") + LOG.info(" Skipping and writing new file for : " + fileName) + SparkHelpers.skipKeysAndWriteNewFile(instantTime, fs, badFilePath, newFilePath, dupeFixPlan(fileName)) + fs.delete(badFilePath, true) + } + + // 3. Check that there are no duplicates anymore. + val df = sqlContext.read.parquet(s"$repairOutputPath/*.parquet") + df.registerTempTable("fixedTbl") + val dupeKeyDF = getDupeKeyDF("fixedTbl") + val dupeCnt = dupeKeyDF.count() + if (dupeCnt != 0) { + dupeKeyDF.show() + throw new HoodieException("Still found some duplicates!!.. Inspect output") + } + + // 4. Additionally ensure no record keys are left behind. + val sourceDF = sparkHelper.getDistinctKeyDF(fileNameToPathMap.map(t => t._2.toString).toList) + val fixedDF = sparkHelper.getDistinctKeyDF(fileNameToPathMap.map(t => s"$repairOutputPath/${t._2.getName}").toList) + val missedRecordKeysDF = sourceDF.except(fixedDF) + val missedCnt = missedRecordKeysDF.count() + if (missedCnt != 0) { + missedRecordKeysDF.show() + throw new HoodieException("Some records in source are not found in fixed files. Inspect output!!") + } + + println("No duplicates found & counts are in check!!!! ") + // 5. Prepare to copy the fixed files back. + fileNameToPathMap.foreach { case (_, filePath) => + val srcPath = new Path(s"$repairOutputPath/${filePath.getName}") + val dstPath = new Path(s"$basePath/$duplicatedPartitionPath/${filePath.getName}") + if (dryRun) { + LOG.info(s"[JUST KIDDING!!!] Copying from $srcPath to $dstPath") + } else { + // for real + LOG.info(s"[FOR REAL!!!] Copying from $srcPath to $dstPath") + FileUtil.copy(fs, srcPath, fs, dstPath, false, true, fs.getConf) + } + } + } +} diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/SparkHelpers.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/SparkHelpers.scala new file mode 100644 index 0000000000000..1ed0e5e1a456a --- /dev/null +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/SparkHelpers.scala @@ -0,0 +1,134 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hudi + +import org.apache.avro.Schema +import org.apache.avro.generic.IndexedRecord +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.{FileSystem, Path} +import org.apache.hudi.avro.HoodieAvroWriteSupport +import org.apache.hudi.client.SparkTaskContextSupplier +import org.apache.hudi.common.bloom.{BloomFilter, BloomFilterFactory} +import org.apache.hudi.common.model.{HoodieFileFormat, HoodieRecord} +import org.apache.hudi.common.util.BaseFileUtils +import org.apache.hudi.config.{HoodieIndexConfig, HoodieStorageConfig} +import org.apache.hudi.io.storage.{HoodieAvroParquetWriter, HoodieParquetConfig} +import org.apache.parquet.avro.AvroSchemaConverter +import org.apache.parquet.hadoop.metadata.CompressionCodecName +import org.apache.spark.sql.{DataFrame, SQLContext} + +import scala.collection.JavaConversions._ +import scala.collection.mutable._ + +object SparkHelpers { + @throws[Exception] + def skipKeysAndWriteNewFile(instantTime: String, fs: FileSystem, sourceFile: Path, destinationFile: Path, keysToSkip: Set[String]) { + val sourceRecords = BaseFileUtils.getInstance(HoodieFileFormat.PARQUET).readAvroRecords(fs.getConf, sourceFile) + val schema: Schema = sourceRecords.get(0).getSchema + val filter: BloomFilter = BloomFilterFactory.createBloomFilter(HoodieIndexConfig.BLOOM_FILTER_NUM_ENTRIES_VALUE.defaultValue.toInt, HoodieIndexConfig.BLOOM_FILTER_FPP_VALUE.defaultValue.toDouble, + HoodieIndexConfig.BLOOM_INDEX_FILTER_DYNAMIC_MAX_ENTRIES.defaultValue.toInt, HoodieIndexConfig.BLOOM_FILTER_TYPE.defaultValue); + val writeSupport: HoodieAvroWriteSupport = new HoodieAvroWriteSupport(new AvroSchemaConverter(fs.getConf).convert(schema), schema, org.apache.hudi.common.util.Option.of(filter)) + val parquetConfig: HoodieParquetConfig[HoodieAvroWriteSupport] = new HoodieParquetConfig(writeSupport, CompressionCodecName.GZIP, HoodieStorageConfig.PARQUET_BLOCK_SIZE.defaultValue.toInt, HoodieStorageConfig.PARQUET_PAGE_SIZE.defaultValue.toInt, HoodieStorageConfig.PARQUET_MAX_FILE_SIZE.defaultValue.toInt, fs.getConf, HoodieStorageConfig.PARQUET_COMPRESSION_RATIO_FRACTION.defaultValue.toDouble) + + // Add current classLoad for config, if not will throw classNotFound of 'HoodieWrapperFileSystem'. + parquetConfig.getHadoopConf().setClassLoader(Thread.currentThread.getContextClassLoader) + + val writer = new HoodieAvroParquetWriter[IndexedRecord](destinationFile, parquetConfig, instantTime, new SparkTaskContextSupplier(), true) + for (rec <- sourceRecords) { + val key: String = rec.get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString + if (!keysToSkip.contains(key)) { + + writer.writeAvro(key, rec) + } + } + writer.close + } +} + +/** + * Bunch of Spark Shell/Scala stuff useful for debugging + */ +class SparkHelper(sqlContext: SQLContext, fs: FileSystem) { + + /** + * Print keys from a file + * + * @param file + */ + def printKeysFromFile(file: String) = { + getRowKeyDF(file).collect().foreach(println(_)) + } + + /** + * + * @param file + * @return + */ + def getRowKeyDF(file: String): DataFrame = { + sqlContext.read.parquet(file).select(s"`${HoodieRecord.RECORD_KEY_METADATA_FIELD}`") + } + + /** + * Does the rowKey actually exist in the file. + * + * @param rowKey + * @param file + * @return + */ + def isFileContainsKey(rowKey: String, file: String): Boolean = { + println(s"Checking $file for key $rowKey") + val ff = getRowKeyDF(file).filter(s"`${HoodieRecord.RECORD_KEY_METADATA_FIELD}` = '$rowKey'") + if (ff.count() > 0) true else false + } + + /** + * Number of keys in a given file + * + * @param file + * @param sqlContext + */ + def getKeyCount(file: String, sqlContext: org.apache.spark.sql.SQLContext) = { + val keyCount = getRowKeyDF(file).collect().length + println(keyCount) + keyCount + } + + /** + * + * Checks that all the keys in the file, have been added to the bloom filter + * in the footer + * + * @param conf + * @param sqlContext + * @param file + * @return + */ + def fileKeysAgainstBF(conf: Configuration, sqlContext: SQLContext, file: String): Boolean = { + val bf = BaseFileUtils.getInstance(HoodieFileFormat.PARQUET).readBloomFilterFromMetadata(conf, new Path(file)) + val foundCount = sqlContext.parquetFile(file) + .select(s"`${HoodieRecord.RECORD_KEY_METADATA_FIELD}`") + .collect().count(r => !bf.mightContain(r.getString(0))) + val totalCount = getKeyCount(file, sqlContext) + println(s"totalCount: $totalCount, foundCount: $foundCount") + totalCount == foundCount + } + + def getDistinctKeyDF(paths: List[String]): DataFrame = { + sqlContext.read.parquet(paths: _*).select(s"`${HoodieRecord.RECORD_KEY_METADATA_FIELD}`").distinct() + } +} diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/CopyToTableProcedure.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/CopyToTableProcedure.scala new file mode 100644 index 0000000000000..b49875c94c11c --- /dev/null +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/CopyToTableProcedure.scala @@ -0,0 +1,128 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hudi.command.procedures + +import org.apache.hudi.DataSourceReadOptions +import org.apache.spark.internal.Logging +import org.apache.spark.sql.types.{DataTypes, Metadata, StructField, StructType} +import org.apache.spark.sql.{Row, SaveMode} + +import java.util.function.Supplier + +class CopyToTableProcedure extends BaseProcedure with ProcedureBuilder with Logging { + + + private val PARAMETERS = Array[ProcedureParameter]( + ProcedureParameter.required(0, "table", DataTypes.StringType, None), + ProcedureParameter.optional(1, "query_type", DataTypes.StringType, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL), + ProcedureParameter.required(2, "new_table", DataTypes.StringType, None), + ProcedureParameter.optional(3, "begin_instance_time", DataTypes.StringType, ""), + ProcedureParameter.optional(4, "end_instance_time", DataTypes.StringType, ""), + ProcedureParameter.optional(5, "as_of_instant", DataTypes.StringType, ""), + ProcedureParameter.optional(6, "save_mode", DataTypes.StringType, "overwrite") + ) + + private val OUTPUT_TYPE = new StructType(Array[StructField]( + StructField("status", DataTypes.IntegerType, nullable = true, Metadata.empty)) + ) + + def parameters: Array[ProcedureParameter] = PARAMETERS + + def outputType: StructType = OUTPUT_TYPE + + override def call(args: ProcedureArgs): Seq[Row] = { + super.checkArgs(PARAMETERS, args) + + val tableName = getArgValueOrDefault(args, PARAMETERS(0)) + val queryType = getArgValueOrDefault(args, PARAMETERS(1)).get.asInstanceOf[String] + val newTableName = getArgValueOrDefault(args, PARAMETERS(2)).get.asInstanceOf[String] + val beginInstance = getArgValueOrDefault(args, PARAMETERS(3)).get.asInstanceOf[String] + val endInstance = getArgValueOrDefault(args, PARAMETERS(4)).get.asInstanceOf[String] + val asOfInstant = getArgValueOrDefault(args, PARAMETERS(5)).get.asInstanceOf[String] + val saveModeStr = getArgValueOrDefault(args, PARAMETERS(6)).get.asInstanceOf[String] + + assert(saveModeStr.nonEmpty, "save_mode(append,overwrite) can not be null.") + + val saveMode: Any = saveModeStr.toLowerCase match { + case "overwrite" => SaveMode.Overwrite + case "append" => SaveMode.Append + case _ => assert(assertion = false, s"save_mode not support $saveModeStr.") + } + + + val tablePath = getBasePath(tableName) + + val sourceDataFrame = queryType match { + case DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL => if (asOfInstant.nonEmpty) { + sparkSession.read + .format("org.apache.hudi") + .option(DataSourceReadOptions.QUERY_TYPE.key, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL) + .option(DataSourceReadOptions.TIME_TRAVEL_AS_OF_INSTANT.key, asOfInstant) + .load(tablePath) + } else { + sparkSession.read + .format("org.apache.hudi") + .option(DataSourceReadOptions.QUERY_TYPE.key, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL) + .load(tablePath) + } + case DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL => + assert(beginInstance.nonEmpty && endInstance.nonEmpty, "when the query_type is incremental, begin_instance_time and end_instance_time can not be null.") + sparkSession.read + .format("org.apache.hudi") + .option(DataSourceReadOptions.QUERY_TYPE.key, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL) + .option(DataSourceReadOptions.BEGIN_INSTANTTIME.key, beginInstance) + .option(DataSourceReadOptions.END_INSTANTTIME.key, endInstance) + .load(tablePath) + case DataSourceReadOptions.QUERY_TYPE_READ_OPTIMIZED_OPT_VAL => + sparkSession.read + .format("org.apache.hudi") + .option(DataSourceReadOptions.QUERY_TYPE.key, DataSourceReadOptions.QUERY_TYPE_READ_OPTIMIZED_OPT_VAL) + .load(tablePath) + } + if (sparkSession.catalog.tableExists(newTableName)) { + val schema = sparkSession.read.table(newTableName).schema + val selectColumns = schema.fields.toStream.map(_.name) + sourceDataFrame.selectExpr(selectColumns: _*) + .write + .mode(saveMode.toString) + .saveAsTable(newTableName) + } else { + sourceDataFrame.write + .mode(saveMode.toString) + .saveAsTable(newTableName) + } + + + Seq(Row(0)) + } + + override def build = new CopyToTableProcedure() +} + +object CopyToTableProcedure { + val NAME = "copy_to_table" + + def builder: Supplier[ProcedureBuilder] = new Supplier[ProcedureBuilder] { + override def get() = new CopyToTableProcedure() + } +} + + + + + diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/HoodieProcedures.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/HoodieProcedures.scala index 0545c140bb3e5..1eb82d97c5e95 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/HoodieProcedures.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/HoodieProcedures.scala @@ -74,6 +74,12 @@ object HoodieProcedures { mapBuilder.put(ShowMetadataStatsProcedure.NAME, ShowMetadataStatsProcedure.builder) mapBuilder.put(ValidateMetadataFilesProcedure.NAME, ValidateMetadataFilesProcedure.builder) mapBuilder.put(ShowFsPathDetailProcedure.NAME, ShowFsPathDetailProcedure.builder) + mapBuilder.put(CopyToTableProcedure.NAME, CopyToTableProcedure.builder) + mapBuilder.put(RepairAddpartitionmetaProcedure.NAME, RepairAddpartitionmetaProcedure.builder) + mapBuilder.put(RepairCorruptedCleanFilesProcedure.NAME, RepairCorruptedCleanFilesProcedure.builder) + mapBuilder.put(RepairDeduplicateProcedure.NAME, RepairDeduplicateProcedure.builder) + mapBuilder.put(RepairMigratePartitionMetaProcedure.NAME, RepairMigratePartitionMetaProcedure.builder) + mapBuilder.put(RepairOverwriteHoodiePropsProcedure.NAME, RepairOverwriteHoodiePropsProcedure.builder) mapBuilder.build } } diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/RepairAddpartitionmetaProcedure.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/RepairAddpartitionmetaProcedure.scala new file mode 100644 index 0000000000000..bb65174c4b47b --- /dev/null +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/RepairAddpartitionmetaProcedure.scala @@ -0,0 +1,89 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hudi.command.procedures + +import org.apache.hadoop.fs.Path +import org.apache.hudi.common.fs.FSUtils +import org.apache.hudi.common.model.HoodiePartitionMetadata +import org.apache.hudi.common.table.HoodieTableMetaClient +import org.apache.spark.internal.Logging +import org.apache.spark.sql.Row +import org.apache.spark.sql.types.{DataTypes, Metadata, StructField, StructType} + +import java.util +import java.util.function.Supplier +import scala.collection.JavaConversions._ + +class RepairAddpartitionmetaProcedure extends BaseProcedure with ProcedureBuilder with Logging { + private val PARAMETERS = Array[ProcedureParameter]( + ProcedureParameter.required(0, "table", DataTypes.StringType, None), + ProcedureParameter.optional(1, "dry_run", DataTypes.BooleanType, true) + ) + + private val OUTPUT_TYPE = new StructType(Array[StructField]( + StructField("partition_path", DataTypes.StringType, nullable = true, Metadata.empty), + StructField("metadata_is_present", DataTypes.StringType, nullable = true, Metadata.empty), + StructField("action", DataTypes.StringType, nullable = true, Metadata.empty)) + ) + + def parameters: Array[ProcedureParameter] = PARAMETERS + + def outputType: StructType = OUTPUT_TYPE + + override def call(args: ProcedureArgs): Seq[Row] = { + super.checkArgs(PARAMETERS, args) + + val tableName = getArgValueOrDefault(args, PARAMETERS(0)) + val dryRun = getArgValueOrDefault(args, PARAMETERS(1)).get.asInstanceOf[Boolean] + val tablePath = getBasePath(tableName) + + val metaClient = HoodieTableMetaClient.builder.setConf(jsc.hadoopConfiguration()).setBasePath(tablePath).build + + val latestCommit: String = metaClient.getActiveTimeline.getCommitTimeline.lastInstant.get.getTimestamp + val partitionPaths: util.List[String] = FSUtils.getAllPartitionFoldersThreeLevelsDown(metaClient.getFs, tablePath); + val basePath: Path = new Path(tablePath) + + val rows = new util.ArrayList[Row](partitionPaths.size) + for (partition <- partitionPaths) { + val partitionPath: Path = FSUtils.getPartitionPath(basePath, partition) + var isPresent = "Yes" + var action = "None" + if (!HoodiePartitionMetadata.hasPartitionMetadata(metaClient.getFs, partitionPath)) { + isPresent = "No" + if (!dryRun) { + val partitionMetadata: HoodiePartitionMetadata = new HoodiePartitionMetadata(metaClient.getFs, latestCommit, basePath, partitionPath, metaClient.getTableConfig.getPartitionMetafileFormat) + partitionMetadata.trySave(0) + action = "Repaired" + } + } + rows.add(Row(partition, isPresent, action)) + } + + rows.stream().toArray().map(r => r.asInstanceOf[Row]).toList + } + + override def build: Procedure = new RepairAddpartitionmetaProcedure() +} + +object RepairAddpartitionmetaProcedure { + val NAME = "repair_add_partition_meta" + + def builder: Supplier[ProcedureBuilder] = new Supplier[ProcedureBuilder] { + override def get() = new RepairAddpartitionmetaProcedure() + } +} diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/RepairCorruptedCleanFilesProcedure.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/RepairCorruptedCleanFilesProcedure.scala new file mode 100644 index 0000000000000..ff185d1bdfab1 --- /dev/null +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/RepairCorruptedCleanFilesProcedure.scala @@ -0,0 +1,86 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hudi.command.procedures + +import org.apache.avro.AvroRuntimeException +import org.apache.hudi.common.table.HoodieTableMetaClient +import org.apache.hudi.common.table.timeline.{HoodieActiveTimeline, HoodieInstant} +import org.apache.hudi.common.util.CleanerUtils +import org.apache.hudi.exception.HoodieIOException +import org.apache.spark.internal.Logging +import org.apache.spark.sql.Row +import org.apache.spark.sql.types.{DataTypes, Metadata, StructField, StructType} + +import java.io.IOException +import java.util.function.Supplier +import scala.collection.JavaConverters.asScalaIteratorConverter + +class RepairCorruptedCleanFilesProcedure extends BaseProcedure with ProcedureBuilder with Logging { + private val PARAMETERS = Array[ProcedureParameter]( + ProcedureParameter.required(0, "table", DataTypes.StringType, None) + ) + + private val OUTPUT_TYPE = new StructType(Array[StructField]( + StructField("result", DataTypes.BooleanType, nullable = true, Metadata.empty)) + ) + + def parameters: Array[ProcedureParameter] = PARAMETERS + + def outputType: StructType = OUTPUT_TYPE + + override def call(args: ProcedureArgs): Seq[Row] = { + super.checkArgs(PARAMETERS, args) + + val tableName = getArgValueOrDefault(args, PARAMETERS(0)) + val tablePath = getBasePath(tableName) + + val metaClient = HoodieTableMetaClient.builder.setConf(jsc.hadoopConfiguration()).setBasePath(tablePath).build + + val cleanerTimeline = metaClient.getActiveTimeline.getCleanerTimeline + logInfo("Inspecting pending clean metadata in timeline for corrupted files") + var result = true + cleanerTimeline.filterInflightsAndRequested.getInstants.iterator().asScala.foreach((instant: HoodieInstant) => { + try { + CleanerUtils.getCleanerPlan(metaClient, instant) + } catch { + case e: AvroRuntimeException => + logWarning("Corruption found. Trying to remove corrupted clean instant file: " + instant) + HoodieActiveTimeline.deleteInstantFile(metaClient.getFs, metaClient.getMetaPath, instant) + case ioe: IOException => + if (ioe.getMessage.contains("Not an Avro data file")) { + logWarning("Corruption found. Trying to remove corrupted clean instant file: " + instant) + HoodieActiveTimeline.deleteInstantFile(metaClient.getFs, metaClient.getMetaPath, instant) + } else { + result = false + throw new HoodieIOException(ioe.getMessage, ioe) + } + } + }) + Seq(Row(result)) + } + + override def build: Procedure = new RepairCorruptedCleanFilesProcedure() +} + +object RepairCorruptedCleanFilesProcedure { + val NAME = "repair_corrupted_clean_files" + + def builder: Supplier[ProcedureBuilder] = new Supplier[ProcedureBuilder] { + override def get() = new RepairCorruptedCleanFilesProcedure() + } +} diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/RepairDeduplicateProcedure.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/RepairDeduplicateProcedure.scala new file mode 100644 index 0000000000000..8ee5055e1fdb2 --- /dev/null +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/RepairDeduplicateProcedure.scala @@ -0,0 +1,86 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hudi.command.procedures + +import org.apache.hudi.common.fs.FSUtils +import org.apache.hudi.exception.HoodieException +import org.apache.spark.internal.Logging +import org.apache.spark.sql.Row +import org.apache.spark.sql.types.{DataTypes, Metadata, StructField, StructType} +import java.util.function.Supplier + +import org.apache.spark.sql.hudi.{DeDupeType, DedupeSparkJob} + +import scala.util.{Failure, Success, Try} + +class RepairDeduplicateProcedure extends BaseProcedure with ProcedureBuilder with Logging { + private val PARAMETERS = Array[ProcedureParameter]( + ProcedureParameter.required(0, "table", DataTypes.StringType, None), + ProcedureParameter.required(1, "duplicated_partition_path", DataTypes.StringType, None), + ProcedureParameter.required(2, "repaired_output_path", DataTypes.StringType, None), + ProcedureParameter.optional(3, "dry_run", DataTypes.BooleanType, true), + ProcedureParameter.optional(4, "dedupe_type", DataTypes.StringType, "insert_type") + ) + + private val OUTPUT_TYPE = new StructType(Array[StructField]( + StructField("result", DataTypes.StringType, nullable = true, Metadata.empty)) + ) + + def parameters: Array[ProcedureParameter] = PARAMETERS + + def outputType: StructType = OUTPUT_TYPE + + override def call(args: ProcedureArgs): Seq[Row] = { + super.checkArgs(PARAMETERS, args) + + val tableName = getArgValueOrDefault(args, PARAMETERS(0)) + val duplicatedPartitionPath = getArgValueOrDefault(args, PARAMETERS(1)).get.asInstanceOf[String] + val repairedOutputPath = getArgValueOrDefault(args, PARAMETERS(2)).get.asInstanceOf[String] + val dryRun = getArgValueOrDefault(args, PARAMETERS(3)).get.asInstanceOf[Boolean] + val dedupeType = getArgValueOrDefault(args, PARAMETERS(4)).get.asInstanceOf[String] + + if (!DeDupeType.values.contains(DeDupeType.withName(dedupeType))) { + throw new IllegalArgumentException("Please provide valid dedupe type!") + } + val basePath = getBasePath(tableName) + + Try { + val job = new DedupeSparkJob(basePath, duplicatedPartitionPath, repairedOutputPath, spark.sqlContext, + FSUtils.getFs(basePath, jsc.hadoopConfiguration), DeDupeType.withName(dedupeType)) + job.fixDuplicates(dryRun) + } match { + case Success(_) => + if (dryRun){ + Seq(Row(s"Deduplicated files placed in: $repairedOutputPath.")) + } else { + Seq(Row(s"Deduplicated files placed in: $duplicatedPartitionPath.")) + } + case Failure(e) => + throw new HoodieException(s"Deduplication failed!", e) + } + } + override def build: Procedure = new RepairDeduplicateProcedure() +} + +object RepairDeduplicateProcedure { + val NAME = "repair_deduplicate" + + def builder: Supplier[ProcedureBuilder] = new Supplier[ProcedureBuilder] { + override def get() = new RepairDeduplicateProcedure() + } +} diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/RepairMigratePartitionMetaProcedure.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/RepairMigratePartitionMetaProcedure.scala new file mode 100644 index 0000000000000..7daacb2f184c8 --- /dev/null +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/RepairMigratePartitionMetaProcedure.scala @@ -0,0 +1,112 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hudi.command.procedures + +import org.apache.hadoop.fs.Path +import org.apache.hudi.common.engine.HoodieLocalEngineContext +import org.apache.hudi.common.fs.FSUtils +import org.apache.hudi.common.model.HoodiePartitionMetadata +import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient} +import org.apache.hudi.common.util.Option +import org.apache.hudi.exception.HoodieIOException +import org.apache.spark.internal.Logging +import org.apache.spark.sql.Row +import org.apache.spark.sql.types.{DataTypes, Metadata, StructField, StructType} + +import java.io.IOException +import java.util +import java.util.Properties +import java.util.function.{Consumer, Supplier} +import scala.collection.JavaConversions._ + +class RepairMigratePartitionMetaProcedure extends BaseProcedure with ProcedureBuilder with Logging { + private val PARAMETERS = Array[ProcedureParameter]( + ProcedureParameter.required(0, "table", DataTypes.StringType, None), + ProcedureParameter.optional(1, "dry_run", DataTypes.BooleanType, true) + ) + + private val OUTPUT_TYPE = new StructType(Array[StructField]( + StructField("partition_path", DataTypes.StringType, nullable = true, Metadata.empty), + StructField("text_metafile_present", DataTypes.StringType, nullable = true, Metadata.empty), + StructField("base_metafile_present", DataTypes.StringType, nullable = true, Metadata.empty), + StructField("action", DataTypes.StringType, nullable = true, Metadata.empty)) + ) + + def parameters: Array[ProcedureParameter] = PARAMETERS + + def outputType: StructType = OUTPUT_TYPE + + override def call(args: ProcedureArgs): Seq[Row] = { + super.checkArgs(PARAMETERS, args) + + val tableName = getArgValueOrDefault(args, PARAMETERS(0)) + val dryRun = getArgValueOrDefault(args, PARAMETERS(1)).get.asInstanceOf[Boolean] + val tablePath = getBasePath(tableName) + + val metaClient = HoodieTableMetaClient.builder.setConf(jsc.hadoopConfiguration()).setBasePath(tablePath).build + + val engineContext: HoodieLocalEngineContext = new HoodieLocalEngineContext(metaClient.getHadoopConf) + val partitionPaths: util.List[String] = FSUtils.getAllPartitionPaths(engineContext, tablePath, false, false) + val basePath: Path = new Path(tablePath) + + val rows = new util.ArrayList[Row](partitionPaths.size) + for (partitionPath <- partitionPaths) { + val partition: Path = FSUtils.getPartitionPath(tablePath, partitionPath) + val textFormatFile: Option[Path] = HoodiePartitionMetadata.textFormatMetaPathIfExists(metaClient.getFs, partition) + val baseFormatFile: Option[Path] = HoodiePartitionMetadata.baseFormatMetaPathIfExists(metaClient.getFs, partition) + val latestCommit: String = metaClient.getActiveTimeline.getCommitTimeline.lastInstant.get.getTimestamp + var action = if (textFormatFile.isPresent) "MIGRATE" else "NONE" + if (!dryRun) { + if (!baseFormatFile.isPresent) { + val partitionMetadata: HoodiePartitionMetadata = new HoodiePartitionMetadata(metaClient.getFs, latestCommit, + basePath, partition, Option.of(metaClient.getTableConfig.getBaseFileFormat)) + partitionMetadata.trySave(0) + } + // delete it, in case we failed midway last time. + textFormatFile.ifPresent( + new Consumer[Path] { + override def accept(p: Path): Unit = { + try metaClient.getFs.delete(p, false) + catch { + case e: IOException => + throw new HoodieIOException(e.getMessage, e) + } + } + }) + action = "MIGRATED" + } + rows.add(Row(partitionPath, String.valueOf(textFormatFile.isPresent), + String.valueOf(baseFormatFile.isPresent), action)) + } + val props: Properties = new Properties + props.setProperty(HoodieTableConfig.PARTITION_METAFILE_USE_BASE_FORMAT.key, "true") + HoodieTableConfig.update(metaClient.getFs, new Path(metaClient.getMetaPath), props) + + rows.stream().toArray().map(r => r.asInstanceOf[Row]).toList + } + + override def build: Procedure = new RepairMigratePartitionMetaProcedure() +} + +object RepairMigratePartitionMetaProcedure { + val NAME = "repair_migrate_partition_meta" + + def builder: Supplier[ProcedureBuilder] = new Supplier[ProcedureBuilder] { + override def get() = new RepairMigratePartitionMetaProcedure() + } +} diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/RepairOverwriteHoodiePropsProcedure.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/RepairOverwriteHoodiePropsProcedure.scala new file mode 100644 index 0000000000000..043217cf2df4b --- /dev/null +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/RepairOverwriteHoodiePropsProcedure.scala @@ -0,0 +1,89 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hudi.command.procedures + +import org.apache.hadoop.fs.Path +import org.apache.hudi.common.table.HoodieTableMetaClient.METAFOLDER_NAME +import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient} +import org.apache.spark.internal.Logging +import org.apache.spark.sql.Row +import org.apache.spark.sql.types.{DataTypes, Metadata, StructField, StructType} + +import java.io.FileInputStream +import java.util +import java.util.Properties +import java.util.function.Supplier +import scala.collection.JavaConversions._ +import scala.collection.JavaConverters.asScalaIteratorConverter + +class RepairOverwriteHoodiePropsProcedure extends BaseProcedure with ProcedureBuilder with Logging { + private val PARAMETERS = Array[ProcedureParameter]( + ProcedureParameter.required(0, "table", DataTypes.StringType, None), + ProcedureParameter.required(1, "new_props_file_path", DataTypes.StringType, None) + ) + + private val OUTPUT_TYPE = new StructType(Array[StructField]( + StructField("property", DataTypes.StringType, nullable = true, Metadata.empty), + StructField("old_value", DataTypes.StringType, nullable = true, Metadata.empty), + StructField("new_value", DataTypes.StringType, nullable = true, Metadata.empty)) + ) + + def parameters: Array[ProcedureParameter] = PARAMETERS + + def outputType: StructType = OUTPUT_TYPE + + override def call(args: ProcedureArgs): Seq[Row] = { + super.checkArgs(PARAMETERS, args) + + val tableName = getArgValueOrDefault(args, PARAMETERS(0)) + val overwriteFilePath = getArgValueOrDefault(args, PARAMETERS(1)).get.asInstanceOf[String] + val tablePath = getBasePath(tableName) + + val metaClient = HoodieTableMetaClient.builder.setConf(jsc.hadoopConfiguration()).setBasePath(tablePath).build + + var newProps = new Properties + newProps.load(new FileInputStream(overwriteFilePath)) + val oldProps = metaClient.getTableConfig.propsMap + val metaPathDir = new Path(tablePath, METAFOLDER_NAME) + HoodieTableConfig.create(metaClient.getFs, metaPathDir, newProps) + // reload new props as checksum would have been added + newProps = HoodieTableMetaClient.reload(metaClient).getTableConfig.getProps + + val allPropKeys = new util.TreeSet[String] + allPropKeys.addAll(newProps.keySet.stream.iterator().asScala.map(key => key.toString).toList) + allPropKeys.addAll(oldProps.keySet) + + val rows = new util.ArrayList[Row](allPropKeys.size) + for (propKey <- allPropKeys) { + rows.add(Row(propKey, oldProps.getOrDefault(propKey, "null"), + newProps.getOrDefault(propKey, "null").toString)) + } + + rows.stream().toArray().map(r => r.asInstanceOf[Row]).toList + } + + override def build: Procedure = new RepairOverwriteHoodiePropsProcedure() +} + +object RepairOverwriteHoodiePropsProcedure { + val NAME = "repair_overwrite_hoodie_props" + + def builder: Supplier[ProcedureBuilder] = new Supplier[ProcedureBuilder] { + override def get() = new RepairOverwriteHoodiePropsProcedure() + } +} diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/UpgradeOrDowngradeProcedure.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/UpgradeOrDowngradeProcedure.scala index 792d26b184479..9eeef164a1db3 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/UpgradeOrDowngradeProcedure.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/UpgradeOrDowngradeProcedure.scala @@ -22,7 +22,7 @@ import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion import org.apache.hudi.common.table.{HoodieTableMetaClient, HoodieTableVersion} import org.apache.hudi.common.util.Option -import org.apache.hudi.config.{HoodieCompactionConfig, HoodieIndexConfig, HoodieWriteConfig} +import org.apache.hudi.config.{HoodieIndexConfig, HoodieWriteConfig, HoodieCleanConfig} import org.apache.hudi.index.HoodieIndex import org.apache.hudi.table.upgrade.{SparkUpgradeDowngradeHelper, UpgradeDowngrade} import org.apache.spark.internal.Logging @@ -82,7 +82,7 @@ class UpgradeOrDowngradeProcedure extends BaseProcedure with ProcedureBuilder wi HoodieWriteConfig.newBuilder .withPath(basePath) .withRollbackUsingMarkers(true) - .withCompactionConfig(HoodieCompactionConfig.newBuilder.withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.EAGER).build) + .withCleanConfig(HoodieCleanConfig.newBuilder.withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.EAGER).build) .withIndexConfig(HoodieIndexConfig.newBuilder.withIndexType(HoodieIndex.IndexType.BLOOM).build) .build } diff --git a/hudi-spark-datasource/hudi-spark/src/test/resources/table-config.properties b/hudi-spark-datasource/hudi-spark/src/test/resources/table-config.properties new file mode 100644 index 0000000000000..d74c0444a572c --- /dev/null +++ b/hudi-spark-datasource/hudi-spark/src/test/resources/table-config.properties @@ -0,0 +1,21 @@ +### +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +### +hoodie.table.name=test_table +hoodie.table.type=COPY_ON_WRITE +hoodie.archivelog.folder=archive +hoodie.timeline.layout.version=1 diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieSparkUtils.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieSparkUtils.scala index e71973f94a164..01a81976cf751 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieSparkUtils.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieSparkUtils.scala @@ -199,9 +199,7 @@ class TestHoodieSparkUtils { fail("createRdd should fail, because records don't have a column which is not nullable in the passed in schema") } catch { case e: Exception => - val cause = e.getCause - assertTrue(cause.isInstanceOf[SchemaCompatibilityException]) - assertTrue(e.getMessage.contains("Unable to validate the rewritten record {\"innerKey\": \"innerKey1_2\", \"innerValue\": 2} against schema")) + assertTrue(e.getMessage.contains("null of string in field new_nested_col of test_namespace.test_struct_name.nullableInnerStruct of union")) } spark.stop() } diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestSpark3DDL.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestSpark3DDL.scala index 15fed579bba41..b64d386f1fb4a 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestSpark3DDL.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestSpark3DDL.scala @@ -19,10 +19,13 @@ package org.apache.spark.sql.hudi import org.apache.hadoop.fs.Path import org.apache.hudi.common.model.HoodieRecord -import org.apache.hudi.config.{HoodieClusteringConfig, HoodieWriteConfig} +import org.apache.hudi.common.testutils.HoodieTestDataGenerator +import org.apache.hudi.common.testutils.RawTripTestPayload +import org.apache.hudi.config.HoodieWriteConfig import org.apache.hudi.{DataSourceWriteOptions, HoodieSparkUtils} import org.apache.spark.sql.catalyst.TableIdentifier -import org.apache.spark.sql.{DataFrame, Row, SaveMode, SparkSession} +import org.apache.spark.sql.functions.{arrays_zip, col} +import org.apache.spark.sql.{Row, SaveMode, SparkSession} import scala.collection.JavaConversions._ import scala.collection.JavaConverters._ @@ -460,4 +463,65 @@ class TestSpark3DDL extends HoodieSparkSqlTestBase { } } } + + test("Test schema auto evolution") { + withTempDir { tmp => + Seq("COPY_ON_WRITE", "MERGE_ON_READ").foreach { tableType => + val tableName = generateTableName + val tablePath = s"${new Path(tmp.getCanonicalPath, tableName).toUri.toString}" + if (HoodieSparkUtils.gteqSpark3_1) { + + val dataGen = new HoodieTestDataGenerator + val schema = HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA + val records1 = RawTripTestPayload.recordsToStrings(dataGen.generateInsertsAsPerSchema("001", 1000, schema)).toList + val inputDF1 = spark.read.json(spark.sparkContext.parallelize(records1, 2)) + // drop tip_history.element.amount, city_to_state, distance_in_meters, drivers + val orgStringDf = inputDF1.drop("city_to_state", "distance_in_meters", "drivers") + .withColumn("tip_history", arrays_zip(col("tip_history.currency"))) + spark.sql("set hoodie.schema.on.read.enable=true") + + val hudiOptions = Map[String,String]( + HoodieWriteConfig.TABLE_NAME -> tableName, + DataSourceWriteOptions.TABLE_TYPE_OPT_KEY -> tableType, + DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY -> "_row_key", + DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY -> "partition", + DataSourceWriteOptions.PRECOMBINE_FIELD_OPT_KEY -> "timestamp", + "hoodie.schema.on.read.enable" -> "true", + "hoodie.datasource.write.reconcile.schema" -> "true", + DataSourceWriteOptions.HIVE_STYLE_PARTITIONING_OPT_KEY -> "true" + ) + + orgStringDf.write + .format("org.apache.hudi") + .option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL) + .options(hudiOptions) + .mode(SaveMode.Overwrite) + .save(tablePath) + + val oldView = spark.read.format("hudi").load(tablePath) + oldView.show(false) + + val records2 = RawTripTestPayload.recordsToStrings(dataGen.generateUpdatesAsPerSchema("002", 100, schema)).toList + val inputD2 = spark.read.json(spark.sparkContext.parallelize(records2, 2)) + val updatedStringDf = inputD2.drop("fare").drop("height") + val checkRowKey = inputD2.select("_row_key").collectAsList().map(_.getString(0)).get(0) + + updatedStringDf.write + .format("org.apache.hudi") + .options(hudiOptions) + .option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.UPSERT_OPERATION_OPT_VAL) + .option("hoodie.datasource.write.reconcile.schema", "true") + .mode(SaveMode.Append) + .save(tablePath) + spark.read.format("hudi").load(tablePath).registerTempTable("newView") + val checkResult = spark.sql(s"select tip_history.amount,city_to_state,distance_in_meters,fare,height from newView where _row_key='$checkRowKey' ") + .collect().map(row => (row.isNullAt(0), row.isNullAt(1), row.isNullAt(2), row.isNullAt(3), row.isNullAt(4))) + assertResult((false, false, false, true, true))(checkResult(0)) + checkAnswer(spark.sql(s"select fare,height from newView where _row_key='$checkRowKey'").collect())( + Seq(null, null) + ) + } + } + } + } } 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 new file mode 100644 index 0000000000000..57025ab0b6bcd --- /dev/null +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestCopyToTableProcedure.scala @@ -0,0 +1,353 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hudi.procedure + +import org.apache.spark.sql.Row +import org.apache.spark.sql.hudi.HoodieSparkSqlTestBase + +import java.util + +class TestCopyToTableProcedure extends HoodieSparkSqlTestBase { + + test("Test Call copy_to_table Procedure with default params") { + withTempDir { tmp => + val tableName = generateTableName + // create table + spark.sql( + s""" + |create table $tableName ( + | id int, + | name string, + | price double, + | ts long + |) using hudi + | location '${tmp.getCanonicalPath}/$tableName' + | tblproperties ( + | primaryKey = 'id', + | preCombineField = 'ts' + | ) + """.stripMargin) + + // insert data to table + spark.sql(s"insert into $tableName select 1, 'a1', 10, 1000") + spark.sql(s"insert into $tableName select 2, 'a2', 20, 1500") + spark.sql(s"insert into $tableName select 3, 'a3', 30, 2000") + spark.sql(s"insert into $tableName select 4, 'a4', 40, 2500") + + val copyTableName = generateTableName + // Check required fields + checkExceptionContain(s"call copy_to_table(table=>'$tableName')")(s"Argument: new_table is required") + + val row = spark.sql(s"""call copy_to_table(table=>'$tableName',new_table=>'$copyTableName')""").collectAsList() + assert(row.size() == 1 && row.get(0).get(0) == 0) + val copyTableCount = spark.sql(s"""select count(1) from $copyTableName""").collectAsList() + assert(copyTableCount.size() == 1 && copyTableCount.get(0).get(0) == 4) + } + } + + test("Test Call copy_to_table Procedure with snapshot") { + withTempDir { tmp => + val tableName = generateTableName + // create table + spark.sql( + s""" + |create table $tableName ( + | id int, + | name string, + | price double, + | ts long + |) using hudi + | location '${tmp.getCanonicalPath}/$tableName' + | tblproperties ( + | primaryKey = 'id', + | preCombineField = 'ts' + | ) + """.stripMargin) + + // insert 4 rows data to table + spark.sql(s"insert into $tableName select 1, 'a1', 10, 1000") + spark.sql(s"insert into $tableName select 2, 'a2', 20, 1500") + spark.sql(s"insert into $tableName select 3, 'a3', 30, 2000") + spark.sql(s"insert into $tableName select 4, 'a4', 40, 2500") + + val copyTableName = generateTableName + val row = spark.sql(s"""call copy_to_table(table=>'$tableName',new_table=>'$copyTableName',query_type=>'snapshot')""").collectAsList() + // check exit code + assert(row.size() == 1 && row.get(0).get(0) == 0) + val copyTableCount = spark.sql(s"""select count(1) from $copyTableName""").collectAsList() + assert(copyTableCount.size() == 1 && copyTableCount.get(0).get(0) == 4) + + + // insert 4 rows data to table + spark.sql(s"insert into $tableName select 5, 'a1', 10, 1000") + // mark max instanceTime.total row is 5 + val instanceTime = spark.sql(s"select max(_hoodie_commit_time) from $tableName").collectAsList().get(0).get(0) + spark.sql(s"insert into $tableName select 6, 'a1', 10, 1000") + spark.sql(s"insert into $tableName select 7, 'a1', 10, 1000") + spark.sql(s"insert into $tableName select 8, 'a1', 10, 1000") + + // check row count after twice insert + val finalTableCount = spark.sql(s"select * from $tableName").count() + assert(finalTableCount == 8) + + // check snapshot copy with mark instanceTime + val copyTableName2 = generateTableName + val row2 = spark.sql(s"""call copy_to_table(table=>'$tableName',new_table=>'$copyTableName2',query_type=>'snapshot',as_of_instant=>'$instanceTime')""").collectAsList() + // check exit code + assert(row2.size() == 1 && row2.get(0).get(0) == 0) + val df = spark.sql(s"""select * from $copyTableName2""") + assert(df.count() == 5) + + val ids: util.List[Row] = df.selectExpr("id").collectAsList() + assert(ids.containsAll(util.Arrays.asList(Row(1), Row(2), Row(3), Row(4), Row(5)))) + + } + } + + test("Test Call copy_to_table Procedure with incremental") { + withTempDir { tmp => + val tableName = generateTableName + // create table + spark.sql( + s""" + |create table $tableName ( + | id int, + | name string, + | price double, + | ts long + |) using hudi + | location '${tmp.getCanonicalPath}/$tableName' + | tblproperties ( + | primaryKey = 'id', + | preCombineField = 'ts' + | ) + """.stripMargin) + + // insert 4 rows data to table + spark.sql(s"insert into $tableName select 1, 'a1', 10, 1000") + spark.sql(s"insert into $tableName select 2, 'a2', 20, 1500") + + // mark beginTime + val beginTime = spark.sql(s"select max(_hoodie_commit_time) from $tableName").collectAsList().get(0).get(0) + spark.sql(s"insert into $tableName select 3, 'a3', 30, 2000") + spark.sql(s"insert into $tableName select 4, 'a4', 40, 2500") + val endTime = spark.sql(s"select max(_hoodie_commit_time) from $tableName").collectAsList().get(0).get(0) + + + val copyTableName = generateTableName + // Check required fields + checkExceptionContain(s"call copy_to_table(table=>'$tableName',new_table=>'$copyTableName',query_type=>'incremental')")("begin_instance_time and end_instance_time can not be null") + + //copy from tableName with begin_instance_time、end_instance_time + val copyCmd = spark.sql(s"call copy_to_table" + s"(table=>'$tableName'" + + s",new_table=>'$copyTableName'" + + s",query_type=>'incremental'" + + s",begin_instance_time=>'$beginTime'" + + s",end_instance_time=>'$endTime')").collectAsList() + assert(copyCmd.size() == 1 && copyCmd.get(0).get(0) == 0) + + val df = spark.sql(s"select * from $copyTableName") + assert(df.count() == 2) + val ids = df.selectExpr("id").collectAsList() + assert(ids.containsAll(util.Arrays.asList(Row(3), Row(4)))) + } + } + + test("Test Call copy_to_table Procedure with read_optimized") { + withTempDir { tmp => + val tableName = generateTableName + // create mor table with hoodie.compact.inline.max.delta.commits=5 + spark.sql( + s""" + |create table $tableName ( + | id int, + | name string, + | price double, + | ts long + |) using hudi + | location '${tmp.getCanonicalPath}/$tableName' + | options ( + | type='mor', + | primaryKey = 'id', + | preCombineField = 'ts', + | hoodie.compact.inline.max.delta.commits='5', + | hoodie.compact.inline='true' + | + | ) + """.stripMargin) + + //add 4 delta commit + spark.sql(s"insert into $tableName select 1, 'a1', 10, 1000") + spark.sql(s"update $tableName set ts=2000 where id = 1") + spark.sql(s"update $tableName set ts=3000 where id = 1") + spark.sql(s"update $tableName set ts=4000 where id = 1") + + val copyTableName = generateTableName + + val copyCmd = spark.sql(s"""call copy_to_table(table=>'$tableName',new_table=>'$copyTableName',query_type=>'read_optimized')""").collectAsList() + assert(copyCmd.size() == 1 && copyCmd.get(0).get(0) == 0) + val copyDf = spark.sql(s"select * from $copyTableName") + assert(copyDf.count() == 1) + //check ts + assert(copyDf.selectExpr("ts").collectAsList().contains(Row(1000))) + + // trigger compact (delta_commit==5) + spark.sql(s"update $tableName set ts=5000 where id = 1") + + val copyTableName2 = generateTableName + val copyCmd2 = spark.sql(s"""call copy_to_table(table=>'$tableName',new_table=>'$copyTableName2',query_type=>'read_optimized')""").collectAsList() + assert(copyCmd2.size() == 1 && copyCmd2.get(0).get(0) == 0) + val copyDf2 = spark.sql(s"select * from $copyTableName2") + assert(copyDf2.count() == 1) + //check ts + assert(copyDf2.selectExpr("ts").collectAsList().contains(Row(5000))) + } + } + + test("Test Call copy_to_table Procedure with append_mode") { + withTempDir { tmp => + val tableName = generateTableName + // create table + spark.sql( + s""" + |create table $tableName ( + | id int, + | name string, + | price double, + | ts long + |) using hudi + | location '${tmp.getCanonicalPath}/$tableName' + | tblproperties ( + | primaryKey = 'id', + | preCombineField = 'ts' + | ) + """.stripMargin) + + // insert data to table + spark.sql(s"insert into $tableName select 1, 'a1', 10, 1000") + spark.sql(s"insert into $tableName select 2, 'a2', 20, 1500") + spark.sql(s"insert into $tableName select 3, 'a3', 30, 2000") + spark.sql(s"insert into $tableName select 4, 'a4', 40, 2500") + + val copyTableName = generateTableName + // Check required fields + checkExceptionContain(s"call copy_to_table(table=>'$tableName')")(s"Argument: new_table is required") + + val copyCmd = spark.sql(s"call copy_to_table(table=>'$tableName',new_table=>'$copyTableName')").collectAsList() + assert(copyCmd.size() == 1 && copyCmd.get(0).get(0) == 0) + val copyTableCount = spark.sql(s"""select count(1) from $copyTableName""").collectAsList() + assert(copyTableCount.size() == 1 && copyTableCount.get(0).get(0) == 4) + + // add 2 rows + spark.sql(s"insert into $tableName select 5, 'a1', 10, 1000") + spark.sql(s"insert into $tableName select 6, 'a2', 20, 1500") + + val copyCmd2 = spark.sql(s"call copy_to_table(table=>'$tableName',new_table=>'$copyTableName',save_mode=>'append')").collectAsList() + assert(copyCmd2.size() == 1 && copyCmd2.get(0).get(0) == 0) + + val df2 = spark.sql(s"""select * from $copyTableName""") + // total insert 4+6=10 rows + assert(df2.count() == 10) + val ids2 = df2.selectExpr("id").collectAsList() + assert(ids2.containsAll(util.Arrays.asList(Row(1), Row(2), Row(3), Row(4), Row(5), Row(6)))) + + } + } + + test("Test Call copy_to_table Procedure with overwrite_mode") { + withTempDir { tmp => + val tableName = generateTableName + // create table + spark.sql( + s""" + |create table $tableName ( + | id int, + | name string, + | price double, + | ts long + |) using hudi + | location '${tmp.getCanonicalPath}/$tableName' + | tblproperties ( + | primaryKey = 'id', + | preCombineField = 'ts' + | ) + """.stripMargin) + + // insert data to table + spark.sql(s"insert into $tableName select 1, 'a1', 10, 1000") + spark.sql(s"insert into $tableName select 2, 'a2', 20, 1500") + spark.sql(s"insert into $tableName select 3, 'a3', 30, 2000") + spark.sql(s"insert into $tableName select 4, 'a4', 40, 2500") + + val copyTableName = generateTableName + // Check required fields + checkExceptionContain(s"call copy_to_table(table=>'$tableName')")(s"Argument: new_table is required") + + val copyCmd = spark.sql(s"call copy_to_table(table=>'$tableName',new_table=>'$copyTableName')").collectAsList() + assert(copyCmd.size() == 1 && copyCmd.get(0).get(0) == 0) + val copyTableCount = spark.sql(s"""select count(1) from $copyTableName""").collectAsList() + assert(copyTableCount.size() == 1 && copyTableCount.get(0).get(0) == 4) + + // add 2 rows + spark.sql(s"insert into $tableName select 5, 'a1', 10, 1000") + spark.sql(s"insert into $tableName select 6, 'a2', 20, 1500") + + val copyCmd2 = spark.sql(s"call copy_to_table(table=>'$tableName',new_table=>'$copyTableName',save_mode=>'Overwrite')").collectAsList() + assert(copyCmd2.size() == 1 && copyCmd2.get(0).get(0) == 0) + + val df2 = spark.sql(s"""select * from $copyTableName""") + // total insert 6 rows + assert(df2.count() == 6) + val ids2 = df2.selectExpr("id").collectAsList() + assert(ids2.containsAll(util.Arrays.asList(Row(1), Row(2), Row(3), Row(4), Row(5), Row(6)))) + + } + } + + test("Test Call copy_to_table Procedure with not support mode") { + withTempDir { tmp => + val tableName = generateTableName + // create table + spark.sql( + s""" + |create table $tableName ( + | id int, + | name string, + | price double, + | ts long + |) using hudi + | location '${tmp.getCanonicalPath}/$tableName' + | tblproperties ( + | primaryKey = 'id', + | preCombineField = 'ts' + | ) + """.stripMargin) + + // insert 4 rows data to table + spark.sql(s"insert into $tableName select 1, 'a1', 10, 1000") + spark.sql(s"insert into $tableName select 2, 'a2', 20, 1500") + spark.sql(s"insert into $tableName select 3, 'a3', 30, 2000") + spark.sql(s"insert into $tableName select 4, 'a4', 40, 2500") + + val copyTableName = generateTableName + // Check required fields + checkExceptionContain(s"call copy_to_table(table=>'$tableName',new_table=>'$copyTableName',save_mode=>'append1')")(s"save_mode not support append1") + + } + } +} 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 new file mode 100644 index 0000000000000..587f7a4bdda64 --- /dev/null +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestRepairsProcedure.scala @@ -0,0 +1,507 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hudi.procedure + +import org.apache.avro.Schema +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.{FileStatus, Path} +import org.apache.hudi.avro.HoodieAvroUtils +import org.apache.hudi.common.fs.FSUtils +import org.apache.hudi.common.model.HoodieFileFormat +import org.apache.hudi.common.table.HoodieTableMetaClient +import org.apache.hudi.common.table.timeline.HoodieTimeline +import org.apache.hudi.common.table.view.HoodieTableFileSystemView +import org.apache.hudi.common.testutils.{HoodieTestDataGenerator, SchemaTestUtil} +import org.apache.hudi.testutils.HoodieSparkWriteableTestTable +import org.apache.spark.api.java.JavaSparkContext +import org.apache.spark.sql.hudi.HoodieSparkSqlTestBase + +import java.io.IOException +import java.net.URL +import java.nio.file.{Files, Paths} +import scala.collection.JavaConverters.asScalaIteratorConverter + +class TestRepairsProcedure extends HoodieSparkSqlTestBase { + + test("Test Call repair_add_partition_meta Procedure") { + withTempDir { tmp => + val tableName = generateTableName + val tablePath = s"${tmp.getCanonicalPath}/$tableName" + // create table + spark.sql( + s""" + |create table $tableName ( + | id int, + | name string, + | price double, + | ts long + |) using hudi + | location '$tablePath' + | tblproperties ( + | primaryKey = 'id', + | preCombineField = 'ts' + | ) + """.stripMargin) + // create commit instant + Files.createFile(Paths.get(tablePath, ".hoodie", "100.commit")) + + val metaClient = HoodieTableMetaClient.builder + .setConf(new JavaSparkContext(spark.sparkContext).hadoopConfiguration()) + .setBasePath(tablePath) + .build + + // create partition path + val partition1 = Paths.get(tablePath, "2016/03/15").toString + val partition2 = Paths.get(tablePath, "2015/03/16").toString + val partition3 = Paths.get(tablePath, "2015/03/17").toString + assertResult(metaClient.getFs.mkdirs(new Path(partition1))) {true} + assertResult(metaClient.getFs.mkdirs(new Path(partition2))) {true} + assertResult(metaClient.getFs.mkdirs(new Path(partition3))) {true} + + // default is dry run + val dryResult = spark.sql(s"""call repair_add_partition_meta(table => '$tableName')""").collect() + assertResult(3) { + dryResult.length + } + + // real run + val realRunResult = spark.sql(s"""call repair_add_partition_meta(table => '$tableName', dry_run => false)""").collect() + assertResult(3) { + realRunResult.length + } + } + } + + test("Test Call repair_overwrite_hoodie_props Procedure") { + withTempDir { tmp => + val tableName = generateTableName + val tablePath = s"${tmp.getCanonicalPath}/$tableName" + // create table + spark.sql( + s""" + |create table $tableName ( + | id int, + | name string, + | price double, + | ts long + |) using hudi + | location '$tablePath' + | tblproperties ( + | primaryKey = 'id', + | preCombineField = 'ts' + | ) + """.stripMargin) + // create commit instant + val newProps: URL = this.getClass.getClassLoader.getResource("table-config.properties") + + // overwrite hoodie props + val Result = spark.sql(s"""call repair_overwrite_hoodie_props(table => '$tableName', new_props_file_path => '${newProps.getPath}')""").collect() + assertResult(15) { + Result.length + } + } + } + + test("Test Call repair_corrupted_clean_files Procedure") { + withTempDir { tmp => + val tableName = generateTableName + val tablePath = s"${tmp.getCanonicalPath}/$tableName" + // create table + spark.sql( + s""" + |create table $tableName ( + | id int, + | name string, + | price double, + | ts long + |) using hudi + | location '$tablePath' + | tblproperties ( + | primaryKey = 'id', + | preCombineField = 'ts' + | ) + """.stripMargin) + var metaClient = HoodieTableMetaClient.builder + .setConf(new JavaSparkContext(spark.sparkContext).hadoopConfiguration()) + .setBasePath(tablePath) + .build + + // Create four requested files + for (i <- 100 until 104) { + val timestamp = String.valueOf(i) + // Write corrupted requested Clean File + createEmptyCleanRequestedFile(tablePath, timestamp, metaClient.getHadoopConf) + } + + // reload meta client + metaClient = HoodieTableMetaClient.reload(metaClient) + // first, there are four instants + assertResult(4) { + metaClient.getActiveTimeline.filterInflightsAndRequested.getInstants.count + } + + checkAnswer(s"""call repair_corrupted_clean_files(table => '$tableName')""")(Seq(true)) + + // reload meta client + metaClient = HoodieTableMetaClient.reload(metaClient) + // after clearing, there should be 0 instant + assertResult(0) { + metaClient.getActiveTimeline.filterInflightsAndRequested.getInstants.count + } + } + } + + private var duplicatedPartitionPath: String = null + private var duplicatedPartitionPathWithUpdates: String = null + private var duplicatedPartitionPathWithUpserts: String = null + private var repairedOutputPath: String = null + private var fileFormat: HoodieFileFormat = null + + test("Test Call repair_deduplicate Procedure with insert") { + withTempDir { tmp => + val tableName = generateTableName + val bashPath = tmp.getCanonicalPath + val tablePath = s"$bashPath/$tableName" + // create table + spark.sql( + s""" + |create table $tableName ( + | name string, + | favorite_number int, + | favorite_color string + |) using hudi + | location '$tablePath' + | tblproperties ( + | primaryKey = 'name', + | type = 'cow' + | ) + """.stripMargin) + var metaClient = HoodieTableMetaClient.builder + .setConf(new JavaSparkContext(spark.sparkContext).hadoopConfiguration()) + .setBasePath(tablePath) + .build + + generateRecords(tablePath, bashPath, metaClient) + + // reload meta client + metaClient = HoodieTableMetaClient.reload(metaClient) + + // get fs and check number of latest files + val fsView = new HoodieTableFileSystemView(metaClient, metaClient.getActiveTimeline.getCommitTimeline.filterCompletedInstants, + metaClient.getFs.listStatus(new Path(duplicatedPartitionPath))) + val filteredStatuses = fsView.getLatestBaseFiles.iterator().asScala.map(value => value.getPath).toList + // there should be 3 files + assertResult(3) { + filteredStatuses.size + } + + // before deduplicate, all files contain 210 records + var files = filteredStatuses.toArray + var recordCount = getRecordCount(files) + assertResult(210){recordCount} + + val partitionPath = HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH + val result = spark.sql( + s"""call repair_deduplicate(table => '$tableName', + | duplicated_partition_path => '$partitionPath', + | repaired_output_path => '$repairedOutputPath')""".stripMargin).collect() + assertResult(1) { + result.length + } + + // after deduplicate, there are 200 records + val fileStatus = metaClient.getFs.listStatus(new Path(repairedOutputPath)) + files = fileStatus.map((status: FileStatus) => status.getPath.toString) + recordCount = getRecordCount(files) + assertResult(200){recordCount} + } + } + + test("Test Call repair_deduplicate Procedure with update") { + withTempDir { tmp => + val tableName = generateTableName + val bashPath = tmp.getCanonicalPath + val tablePath = s"$bashPath/$tableName" + // create table + spark.sql( + s""" + |create table $tableName ( + | name string, + | favorite_number int, + | favorite_color string + |) using hudi + | location '$tablePath' + | tblproperties ( + | primaryKey = 'name', + | type = 'cow' + | ) + """.stripMargin) + var metaClient = HoodieTableMetaClient.builder + .setConf(new JavaSparkContext(spark.sparkContext).hadoopConfiguration()) + .setBasePath(tablePath) + .build + + generateRecords(tablePath, bashPath, metaClient) + + // reload meta client + metaClient = HoodieTableMetaClient.reload(metaClient) + + // get fs and check number of latest files + val fsView = new HoodieTableFileSystemView(metaClient, metaClient.getActiveTimeline.getCommitTimeline.filterCompletedInstants, + metaClient.getFs.listStatus(new Path(duplicatedPartitionPathWithUpdates))) + val filteredStatuses = fsView.getLatestBaseFiles.iterator().asScala.map(value => value.getPath).toList + // there should be 2 files + assertResult(2) { + filteredStatuses.size + } + + // before deduplicate, all files contain 110 records + var files = filteredStatuses.toArray + var recordCount = getRecordCount(files) + assertResult(110){recordCount} + + val partitionPath = HoodieTestDataGenerator.DEFAULT_SECOND_PARTITION_PATH + val result = spark.sql( + s"""call repair_deduplicate(table => '$tableName', + | duplicated_partition_path => '$partitionPath', + | repaired_output_path => '$repairedOutputPath', + | dedupe_type => 'update_type')""".stripMargin).collect() + assertResult(1) { + result.length + } + + // after deduplicate, there are 100 records + val fileStatus = metaClient.getFs.listStatus(new Path(repairedOutputPath)) + files = fileStatus.map((status: FileStatus) => status.getPath.toString) + recordCount = getRecordCount(files) + assertResult(100){recordCount} + } + } + + test("Test Call repair_deduplicate Procedure with upsert") { + withTempDir { tmp => + val tableName = generateTableName + val bashPath = tmp.getCanonicalPath + val tablePath = s"$bashPath/$tableName" + // create table + spark.sql( + s""" + |create table $tableName ( + | name string, + | favorite_number int, + | favorite_color string + |) using hudi + | location '$tablePath' + | tblproperties ( + | primaryKey = 'name', + | type = 'cow' + | ) + """.stripMargin) + var metaClient = HoodieTableMetaClient.builder + .setConf(new JavaSparkContext(spark.sparkContext).hadoopConfiguration()) + .setBasePath(tablePath) + .build + + generateRecords(tablePath, bashPath, metaClient) + + // reload meta client + metaClient = HoodieTableMetaClient.reload(metaClient) + + // get fs and check number of latest files + val fsView = new HoodieTableFileSystemView(metaClient, metaClient.getActiveTimeline.getCommitTimeline.filterCompletedInstants, + metaClient.getFs.listStatus(new Path(duplicatedPartitionPathWithUpserts))) + val filteredStatuses = fsView.getLatestBaseFiles.iterator().asScala.map(value => value.getPath).toList + // there should be 3 files + assertResult(3) { + filteredStatuses.size + } + + // before deduplicate, all files contain 120 records + var files = filteredStatuses.toArray + var recordCount = getRecordCount(files) + assertResult(120){recordCount} + + val partitionPath = HoodieTestDataGenerator.DEFAULT_THIRD_PARTITION_PATH + val result = spark.sql( + s"""call repair_deduplicate(table => '$tableName', + | duplicated_partition_path => '$partitionPath', + | repaired_output_path => '$repairedOutputPath', + | dedupe_type => 'upsert_type')""".stripMargin).collect() + assertResult(1) { + result.length + } + + // after deduplicate, there are 100 records + val fileStatus = metaClient.getFs.listStatus(new Path(repairedOutputPath)) + files = fileStatus.map((status: FileStatus) => status.getPath.toString) + recordCount = getRecordCount(files) + assertResult(100){recordCount} + } + } + + test("Test Call repair_deduplicate Procedure with real") { + withTempDir { tmp => + val tableName = generateTableName + val bashPath = tmp.getCanonicalPath + val tablePath = s"$bashPath/$tableName" + // create table + spark.sql( + s""" + |create table $tableName ( + | name string, + | favorite_number int, + | favorite_color string + |) using hudi + | location '$tablePath' + | tblproperties ( + | primaryKey = 'name', + | type = 'cow' + | ) + """.stripMargin) + var metaClient = HoodieTableMetaClient.builder + .setConf(new JavaSparkContext(spark.sparkContext).hadoopConfiguration()) + .setBasePath(tablePath) + .build + + generateRecords(tablePath, bashPath, metaClient) + + // reload meta client + metaClient = HoodieTableMetaClient.reload(metaClient) + + // get fs and check number of latest files + val fsView = new HoodieTableFileSystemView(metaClient, metaClient.getActiveTimeline.getCommitTimeline.filterCompletedInstants, + metaClient.getFs.listStatus(new Path(duplicatedPartitionPath))) + val filteredStatuses = fsView.getLatestBaseFiles.iterator().asScala.map(value => value.getPath).toList + // there should be 3 files + assertResult(3) { + filteredStatuses.size + } + + // before deduplicate, all files contain 210 records + var files = filteredStatuses.toArray + var recordCount = getRecordCount(files) + assertResult(210){recordCount} + + val partitionPath = HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH + val result = spark.sql( + s"""call repair_deduplicate(table => '$tableName', + | duplicated_partition_path => '$partitionPath', + | repaired_output_path => '$repairedOutputPath', + | dry_run => false)""".stripMargin).collect() + assertResult(1) { + result.length + } + + // after deduplicate, there are 200 records + val fileStatus = metaClient.getFs.listStatus(new Path(duplicatedPartitionPath)) + files = fileStatus.map((status: FileStatus) => status.getPath.toString).filter(p => p.endsWith(".parquet")) + recordCount = getRecordCount(files) + assertResult(200){recordCount} + } + } + + test("Test Call repair_migrate_partition_meta Procedure") { + withTempDir { tmp => + val tableName = generateTableName + // create table + spark.sql( + s""" + |create table $tableName ( + | id int, + | name string, + | price double, + | ts long + |) using hudi + | location '${tmp.getCanonicalPath}/$tableName' + | partitioned by (ts) + | tblproperties ( + | primaryKey = 'id', + | preCombineField = 'ts' + | ) + """.stripMargin) + // insert data to table + spark.sql(s"insert into $tableName select 1, 'a1', 10, 1000") + spark.sql(s"insert into $tableName select 2, 'a2', 20, 1500") + + // default is dry run + var result = spark.sql(s"""call repair_migrate_partition_meta(table => '$tableName')""").collect() + assertResult(2) { + result.length + } + + // real run + result = spark.sql(s"""call repair_migrate_partition_meta(table => '$tableName', dry_run => false)""").collect() + assertResult(2) { + result.length + } + } + } + + private def generateRecords(tablePath: String, bashpath: String, metaClient: HoodieTableMetaClient): Unit ={ + duplicatedPartitionPath = Paths.get(tablePath, HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH).toString + duplicatedPartitionPathWithUpdates = Paths.get(tablePath, HoodieTestDataGenerator.DEFAULT_SECOND_PARTITION_PATH).toString + duplicatedPartitionPathWithUpserts = Paths.get(tablePath, HoodieTestDataGenerator.DEFAULT_THIRD_PARTITION_PATH).toString + repairedOutputPath = Paths.get(bashpath, "tmp").toString + + // generate 200 records + val schema: Schema = HoodieAvroUtils.addMetadataFields(SchemaTestUtil.getSimpleSchema) + val testTable: HoodieSparkWriteableTestTable = HoodieSparkWriteableTestTable.of(metaClient, schema) + + val hoodieRecords1 = SchemaTestUtil.generateHoodieTestRecords(0, 100, schema) + val hoodieRecords2 = SchemaTestUtil.generateHoodieTestRecords(100, 100, schema) + testTable.addCommit("20160401010101") + .withInserts(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, "1", hoodieRecords1) + testTable.withInserts(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, "2", hoodieRecords2) + testTable.getFileIdWithLogFile(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH) + + testTable.withInserts(HoodieTestDataGenerator.DEFAULT_SECOND_PARTITION_PATH, "4", hoodieRecords1) + testTable.withInserts(HoodieTestDataGenerator.DEFAULT_THIRD_PARTITION_PATH, "6", hoodieRecords1) + + // read records and get 10 to generate duplicates + val dupRecords = hoodieRecords1.subList(0, 10) + testTable.withInserts(HoodieTestDataGenerator.DEFAULT_SECOND_PARTITION_PATH, "5", dupRecords) + testTable.addCommit("20160401010202") + .withInserts(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, "3", dupRecords) + testTable.withInserts(HoodieTestDataGenerator.DEFAULT_THIRD_PARTITION_PATH, "7", dupRecords) + testTable.withInserts(HoodieTestDataGenerator.DEFAULT_THIRD_PARTITION_PATH, "8", dupRecords) + + fileFormat = metaClient.getTableConfig.getBaseFileFormat + } + + private def getRecordCount(files: Array[String]): Long = { + var recordCount: Long = 0 + for (file <- files){ + if (HoodieFileFormat.PARQUET == fileFormat){ + recordCount += spark.sqlContext.read.parquet(file).count() + } else if (HoodieFileFormat.ORC == fileFormat) { + recordCount += spark.sqlContext.read.orc(file).count() + } else { + throw new UnsupportedOperationException(fileFormat.name + " format not supported yet.") + } + } + recordCount + } + + @throws[IOException] + def createEmptyCleanRequestedFile(basePath: String, instantTime: String, configuration: Configuration): Unit = { + val commitFilePath = new Path(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + HoodieTimeline.makeRequestedCleanerFileName(instantTime)) + val fs = FSUtils.getFs(basePath, configuration) + val os = fs.create(commitFilePath, true) + os.close() + } +} diff --git a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HiveSyncConfig.java b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HiveSyncConfig.java index 6f2cc50a0af69..cdb192f9fedd5 100644 --- a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HiveSyncConfig.java +++ b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HiveSyncConfig.java @@ -45,7 +45,6 @@ import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_TABLE_SERDE_PROPERTIES; import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_URL; import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_USER; -import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_USE_JDBC; import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_USE_PRE_APACHE_INPUT_FORMAT; import static org.apache.hudi.hive.HiveSyncConfigHolder.METASTORE_URIS; @@ -95,9 +94,6 @@ public static class HiveSyncConfigParams { + "com.uber.hoodie to org.apache.hudi. Stop using this after you migrated the table definition to " + "org.apache.hudi input format.") public Boolean usePreApacheInputFormat; - @Deprecated - @Parameter(names = {"--use-jdbc"}, description = "Hive jdbc connect url") - public Boolean useJdbc; @Parameter(names = {"--metastore-uris"}, description = "Hive metastore uris") public String metastoreUris; @Parameter(names = {"--sync-mode"}, description = "Mode to choose for Hive ops. Valid values are hms,glue,jdbc and hiveql") @@ -142,7 +138,6 @@ public TypedProperties toProps() { props.setPropertyIfNonNull(HIVE_PASS.key(), hivePass); props.setPropertyIfNonNull(HIVE_URL.key(), jdbcUrl); props.setPropertyIfNonNull(HIVE_USE_PRE_APACHE_INPUT_FORMAT.key(), usePreApacheInputFormat); - props.setPropertyIfNonNull(HIVE_USE_JDBC.key(), useJdbc); props.setPropertyIfNonNull(HIVE_SYNC_MODE.key(), syncMode); props.setPropertyIfNonNull(METASTORE_URIS.key(), metastoreUris); props.setPropertyIfNonNull(HIVE_AUTO_CREATE_DATABASE.key(), autoCreateDatabase); diff --git a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HiveSyncConfigHolder.java b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HiveSyncConfigHolder.java index 3877782c92026..8c14cdfe29beb 100644 --- a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HiveSyncConfigHolder.java +++ b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HiveSyncConfigHolder.java @@ -52,15 +52,6 @@ public class HiveSyncConfigHolder { .withDocumentation("Flag to choose InputFormat under com.uber.hoodie package instead of org.apache.hudi package. " + "Use this when you are in the process of migrating from " + "com.uber.hoodie to org.apache.hudi. Stop using this after you migrated the table definition to org.apache.hudi input format"); - /** - * @deprecated Use {@link #HIVE_SYNC_MODE} instead of this config from 0.9.0 - */ - @Deprecated - public static final ConfigProperty HIVE_USE_JDBC = ConfigProperty - .key("hoodie.datasource.hive_sync.use_jdbc") - .defaultValue("true") - .deprecatedAfter("0.9.0") - .withDocumentation("Use JDBC when hive synchronization is enabled"); public static final ConfigProperty METASTORE_URIS = ConfigProperty .key("hoodie.datasource.hive_sync.metastore.uris") .defaultValue("thrift://localhost:9083") @@ -109,7 +100,7 @@ public class HiveSyncConfigHolder { .withDocumentation("The number of partitions one batch when synchronous partitions to hive."); public static final ConfigProperty HIVE_SYNC_MODE = ConfigProperty .key("hoodie.datasource.hive_sync.mode") - .noDefaultValue() + .defaultValue("jdbc") .withDocumentation("Mode to choose for Hive ops. Valid values are hms, jdbc and hiveql."); public static final ConfigProperty HIVE_SYNC_BUCKET_SYNC = ConfigProperty .key("hoodie.datasource.hive_sync.bucket_sync") diff --git a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HoodieHiveSyncClient.java b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HoodieHiveSyncClient.java index d5a85adcbacc2..26ba4ae8e1b50 100644 --- a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HoodieHiveSyncClient.java +++ b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HoodieHiveSyncClient.java @@ -21,7 +21,6 @@ import org.apache.hudi.common.table.TableSchemaResolver; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.util.Option; -import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.hive.ddl.DDLExecutor; import org.apache.hudi.hive.ddl.HMSDDLExecutor; @@ -49,7 +48,6 @@ import static org.apache.hudi.hadoop.utils.HoodieHiveUtils.GLOBALLY_CONSISTENT_READ_TIMESTAMP; import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_SYNC_MODE; -import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_USE_JDBC; import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_DATABASE_NAME; import static org.apache.hudi.sync.common.util.TableUtils.tableId; @@ -72,23 +70,19 @@ public HoodieHiveSyncClient(HiveSyncConfig config) { // Support JDBC, HiveQL and metastore based implementations for backwards compatibility. Future users should // disable jdbc and depend on metastore client for all hive registrations try { - if (!StringUtils.isNullOrEmpty(config.getString(HIVE_SYNC_MODE))) { - HiveSyncMode syncMode = HiveSyncMode.of(config.getString(HIVE_SYNC_MODE)); - switch (syncMode) { - case HMS: - ddlExecutor = new HMSDDLExecutor(config); - break; - case HIVEQL: - ddlExecutor = new HiveQueryDDLExecutor(config); - break; - case JDBC: - ddlExecutor = new JDBCExecutor(config); - break; - default: - throw new HoodieHiveSyncException("Invalid sync mode given " + config.getString(HIVE_SYNC_MODE)); - } - } else { - ddlExecutor = config.getBoolean(HIVE_USE_JDBC) ? new JDBCExecutor(config) : new HiveQueryDDLExecutor(config); + HiveSyncMode syncMode = HiveSyncMode.of(config.getStringOrDefault(HIVE_SYNC_MODE)); + switch (syncMode) { + case HMS: + ddlExecutor = new HMSDDLExecutor(config); + break; + case HIVEQL: + ddlExecutor = new HiveQueryDDLExecutor(config); + break; + case JDBC: + ddlExecutor = new JDBCExecutor(config); + break; + default: + throw new HoodieHiveSyncException("Invalid sync mode given " + config.getString(HIVE_SYNC_MODE)); } this.client = Hive.get(config.getHiveConf()).getMSC(); } catch (Exception e) { diff --git a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/ddl/HiveQueryDDLExecutor.java b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/ddl/HiveQueryDDLExecutor.java index 90efd2701c793..619a417c331f3 100644 --- a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/ddl/HiveQueryDDLExecutor.java +++ b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/ddl/HiveQueryDDLExecutor.java @@ -47,7 +47,7 @@ import static org.apache.hudi.sync.common.util.TableUtils.tableId; /** - * This class offers DDL executor backed by the hive.ql Driver This class preserves the old useJDBC = false way of doing things. + * This class offers DDL executor backed by the HiveQL Driver. */ public class HiveQueryDDLExecutor extends QueryBasedDDLExecutor { diff --git a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/ddl/JDBCExecutor.java b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/ddl/JDBCExecutor.java index 2673e46a9f745..1fc8f082d8c0f 100644 --- a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/ddl/JDBCExecutor.java +++ b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/ddl/JDBCExecutor.java @@ -43,7 +43,7 @@ import static org.apache.hudi.hive.util.HiveSchemaUtil.HIVE_ESCAPE_CHARACTER; /** - * This class offers DDL executor backed by the jdbc This class preserves the old useJDBC = true way of doing things. + * This class offers DDL executor backed by the jdbc. */ public class JDBCExecutor extends QueryBasedDDLExecutor { diff --git a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/replication/HiveSyncGlobalCommitParams.java b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/replication/HiveSyncGlobalCommitParams.java index 58188f578e1bf..b8b2de73e7062 100644 --- a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/replication/HiveSyncGlobalCommitParams.java +++ b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/replication/HiveSyncGlobalCommitParams.java @@ -32,6 +32,7 @@ import java.io.InputStream; import java.util.Properties; +import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_SYNC_MODE; import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_URL; import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_BASE_PATH; @@ -92,6 +93,7 @@ Properties mkGlobalHiveSyncProps(boolean forRemote) { String jdbcUrl = forRemote ? loadedProps.getProperty(REMOTE_HIVE_SERVER_JDBC_URLS) : loadedProps.getProperty(LOCAL_HIVE_SERVER_JDBC_URLS, loadedProps.getProperty(HIVE_URL.key())); props.setPropertyIfNonNull(HIVE_URL.key(), jdbcUrl); + props.setProperty(HIVE_SYNC_MODE.key(), "jdbc"); LOG.info("building hivesync config forRemote: " + forRemote + " " + jdbcUrl + " " + basePath); return props; diff --git a/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/TestHiveSyncTool.java b/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/TestHiveSyncTool.java index 072feeb663160..7abeed480c639 100644 --- a/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/TestHiveSyncTool.java +++ b/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/TestHiveSyncTool.java @@ -120,7 +120,7 @@ private static Iterable syncModeAndSchemaFromCommitMetadataAndManagedT return opts; } - // (useJdbc, useSchemaFromCommitMetadata, syncAsDataSource) + // (useSchemaFromCommitMetadata, syncAsDataSource, syncMode) private static Iterable syncDataSourceTableParams() { List opts = new ArrayList<>(); for (Object mode : SYNC_MODES) { diff --git a/hudi-sync/hudi-sync-common/src/main/java/org/apache/hudi/sync/common/HoodieSyncConfig.java b/hudi-sync/hudi-sync-common/src/main/java/org/apache/hudi/sync/common/HoodieSyncConfig.java index ba763ddc14bb7..429bb93aa3284 100644 --- a/hudi-sync/hudi-sync-common/src/main/java/org/apache/hudi/sync/common/HoodieSyncConfig.java +++ b/hudi-sync/hudi-sync-common/src/main/java/org/apache/hudi/sync/common/HoodieSyncConfig.java @@ -23,7 +23,6 @@ import org.apache.hudi.common.config.HoodieMetadataConfig; import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.fs.FSUtils; -import org.apache.hudi.common.table.HoodieTableConfig; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.keygen.constant.KeyGeneratorOptions; @@ -37,6 +36,10 @@ import java.util.Properties; import java.util.function.Function; +import static org.apache.hudi.common.table.HoodieTableConfig.DATABASE_NAME; +import static org.apache.hudi.common.table.HoodieTableConfig.HOODIE_TABLE_NAME_KEY; +import static org.apache.hudi.common.table.HoodieTableConfig.HOODIE_WRITE_TABLE_NAME_KEY; + /** * Configs needed to sync data into external meta stores, catalogs, etc. */ @@ -56,22 +59,14 @@ public class HoodieSyncConfig extends HoodieConfig { public static final ConfigProperty META_SYNC_DATABASE_NAME = ConfigProperty .key("hoodie.datasource.hive_sync.database") .defaultValue("default") + .withInferFunction(cfg -> Option.ofNullable(cfg.getString(DATABASE_NAME))) .withDocumentation("The name of the destination database that we should sync the hudi table to."); - // If the table name for the metastore destination is not provided, pick it up from write or table configs. - public static final Function> TABLE_NAME_INFERENCE_FUNCTION = cfg -> { - if (cfg.contains(HoodieTableConfig.HOODIE_WRITE_TABLE_NAME_KEY)) { - return Option.of(cfg.getString(HoodieTableConfig.HOODIE_WRITE_TABLE_NAME_KEY)); - } else if (cfg.contains(HoodieTableConfig.HOODIE_TABLE_NAME_KEY)) { - return Option.of(cfg.getString(HoodieTableConfig.HOODIE_TABLE_NAME_KEY)); - } else { - return Option.empty(); - } - }; public static final ConfigProperty META_SYNC_TABLE_NAME = ConfigProperty .key("hoodie.datasource.hive_sync.table") .defaultValue("unknown") - .withInferFunction(TABLE_NAME_INFERENCE_FUNCTION) + .withInferFunction(cfg -> Option.ofNullable(cfg.getString(HOODIE_WRITE_TABLE_NAME_KEY)) + .or(() -> Option.ofNullable(cfg.getString(HOODIE_TABLE_NAME_KEY)))) .withDocumentation("The name of the destination table that we should sync the hudi table to."); public static final ConfigProperty META_SYNC_BASE_FILE_FORMAT = ConfigProperty @@ -148,6 +143,7 @@ public HoodieSyncConfig(Properties props) { public HoodieSyncConfig(Properties props, Configuration hadoopConf) { super(props); + setDefaults(getClass().getName()); this.hadoopConf = hadoopConf; } @@ -173,9 +169,9 @@ public String toString() { } public static class HoodieSyncConfigParams { - @Parameter(names = {"--database"}, description = "name of the target database in meta store", required = true) + @Parameter(names = {"--database"}, description = "name of the target database in meta store") public String databaseName; - @Parameter(names = {"--table"}, description = "name of the target table in meta store", required = true) + @Parameter(names = {"--table"}, description = "name of the target table in meta store") public String tableName; @Parameter(names = {"--base-path"}, description = "Base path of the hoodie table to sync", required = true) public String basePath; diff --git a/hudi-sync/hudi-sync-common/src/main/java/org/apache/hudi/sync/common/util/Parquet2SparkSchemaUtils.java b/hudi-sync/hudi-sync-common/src/main/java/org/apache/hudi/sync/common/util/Parquet2SparkSchemaUtils.java index c5b98c17eb4a1..3edec8314b1f9 100644 --- a/hudi-sync/hudi-sync-common/src/main/java/org/apache/hudi/sync/common/util/Parquet2SparkSchemaUtils.java +++ b/hudi-sync/hudi-sync-common/src/main/java/org/apache/hudi/sync/common/util/Parquet2SparkSchemaUtils.java @@ -18,12 +18,21 @@ package org.apache.hudi.sync.common.util; +import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.common.util.ValidationUtils; import org.apache.parquet.schema.GroupType; +import org.apache.parquet.schema.MessageType; import org.apache.parquet.schema.OriginalType; import org.apache.parquet.schema.PrimitiveType; import org.apache.parquet.schema.Type; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.apache.parquet.schema.OriginalType.UTF8; +import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.BINARY; import static org.apache.parquet.schema.Type.Repetition.OPTIONAL; /** @@ -33,6 +42,68 @@ */ public class Parquet2SparkSchemaUtils { + /** + * Get Spark Sql related table properties. This is used for spark datasource table. + * @param schema The schema to write to the table. + * @return A new parameters added the spark's table properties. + */ + public static Map getSparkTableProperties(List partitionNames, String sparkVersion, + int schemaLengthThreshold, MessageType schema) { + // Convert the schema and partition info used by spark sql to hive table properties. + // The following code refers to the spark code in + // https://github.com/apache/spark/blob/master/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala + GroupType originGroupType = schema.asGroupType(); + List partitionCols = new ArrayList<>(); + List dataCols = new ArrayList<>(); + Map column2Field = new HashMap<>(); + + for (Type field : originGroupType.getFields()) { + column2Field.put(field.getName(), field); + } + // Get partition columns and data columns. + for (String partitionName : partitionNames) { + // Default the unknown partition fields to be String. + // Keep the same logical with HiveSchemaUtil#getPartitionKeyType. + partitionCols.add(column2Field.getOrDefault(partitionName, + new PrimitiveType(Type.Repetition.REQUIRED, BINARY, partitionName, UTF8))); + } + + for (Type field : originGroupType.getFields()) { + if (!partitionNames.contains(field.getName())) { + dataCols.add(field); + } + } + + List reOrderedFields = new ArrayList<>(); + reOrderedFields.addAll(dataCols); + reOrderedFields.addAll(partitionCols); + GroupType reOrderedType = new GroupType(originGroupType.getRepetition(), originGroupType.getName(), reOrderedFields); + + Map sparkProperties = new HashMap<>(); + sparkProperties.put("spark.sql.sources.provider", "hudi"); + if (!StringUtils.isNullOrEmpty(sparkVersion)) { + sparkProperties.put("spark.sql.create.version", sparkVersion); + } + // Split the schema string to multi-parts according the schemaLengthThreshold size. + String schemaString = Parquet2SparkSchemaUtils.convertToSparkSchemaJson(reOrderedType); + int numSchemaPart = (schemaString.length() + schemaLengthThreshold - 1) / schemaLengthThreshold; + sparkProperties.put("spark.sql.sources.schema.numParts", String.valueOf(numSchemaPart)); + // Add each part of schema string to sparkProperties + for (int i = 0; i < numSchemaPart; i++) { + int start = i * schemaLengthThreshold; + int end = Math.min(start + schemaLengthThreshold, schemaString.length()); + sparkProperties.put("spark.sql.sources.schema.part." + i, schemaString.substring(start, end)); + } + // Add partition columns + if (!partitionNames.isEmpty()) { + sparkProperties.put("spark.sql.sources.schema.numPartCols", String.valueOf(partitionNames.size())); + for (int i = 0; i < partitionNames.size(); i++) { + sparkProperties.put("spark.sql.sources.schema.partCol." + i, partitionNames.get(i)); + } + } + return sparkProperties; + } + public static String convertToSparkSchemaJson(GroupType parquetSchema) { String fieldsJsonString = parquetSchema.getFields().stream().map(field -> { switch (field.getRepetition()) { diff --git a/hudi-sync/hudi-sync-common/src/test/java/org/apache/hudi/sync/common/TestHoodieSyncConfig.java b/hudi-sync/hudi-sync-common/src/test/java/org/apache/hudi/sync/common/TestHoodieSyncConfig.java new file mode 100644 index 0000000000000..1f6c05cd12423 --- /dev/null +++ b/hudi-sync/hudi-sync-common/src/test/java/org/apache/hudi/sync/common/TestHoodieSyncConfig.java @@ -0,0 +1,55 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.hudi.sync.common; + +import org.apache.hudi.common.table.HoodieTableConfig; + +import org.apache.hadoop.conf.Configuration; +import org.junit.jupiter.api.Test; + +import java.util.Properties; + +import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_DATABASE_NAME; +import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_TABLE_NAME; +import static org.junit.jupiter.api.Assertions.assertEquals; + +class TestHoodieSyncConfig { + + @Test + void testInferDatabaseAndTableNames() { + Properties props1 = new Properties(); + props1.setProperty(HoodieTableConfig.DATABASE_NAME.key(), "db1"); + props1.setProperty(HoodieTableConfig.HOODIE_TABLE_NAME_KEY, "tbl1"); + HoodieSyncConfig config1 = new HoodieSyncConfig(props1, new Configuration()); + assertEquals("db1", config1.getString(META_SYNC_DATABASE_NAME)); + assertEquals("tbl1", config1.getString(META_SYNC_TABLE_NAME)); + + Properties props2 = new Properties(); + props2.setProperty(HoodieTableConfig.DATABASE_NAME.key(), "db2"); + props2.setProperty(HoodieTableConfig.HOODIE_WRITE_TABLE_NAME_KEY, "tbl2"); + HoodieSyncConfig config2 = new HoodieSyncConfig(props2, new Configuration()); + assertEquals("db2", config2.getString(META_SYNC_DATABASE_NAME)); + assertEquals("tbl2", config2.getString(META_SYNC_TABLE_NAME)); + + HoodieSyncConfig config3 = new HoodieSyncConfig(new Properties(), new Configuration()); + assertEquals("default", config3.getString(META_SYNC_DATABASE_NAME)); + assertEquals("unknown", config3.getString(META_SYNC_TABLE_NAME)); + } +} diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieDropPartitionsTool.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieDropPartitionsTool.java index 95e84e413cd57..fc16bcaa1bc69 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieDropPartitionsTool.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieDropPartitionsTool.java @@ -174,8 +174,6 @@ public static class Config implements Serializable { public String hiveURL = "jdbc:hive2://localhost:10000"; @Parameter(names = {"--hive-partition-field"}, description = "Comma separated list of field in the hive table to use for determining hive partition columns.", required = false) public String hivePartitionsField = ""; - @Parameter(names = {"--hive-sync-use-jdbc"}, description = "Use JDBC when hive synchronization.", required = false) - public boolean hiveUseJdbc = true; @Parameter(names = {"--hive-metastore-uris"}, description = "hive meta store uris to use.", required = false) public String hiveHMSUris = null; @Parameter(names = {"--hive-sync-mode"}, description = "Mode to choose for Hive ops. Valid values are hms, jdbc and hiveql.", required = false) @@ -215,7 +213,7 @@ public String toString() { + " --hive-pass-word " + "Masked" + ", \n" + " --hive-jdbc-url " + hiveURL + ", \n" + " --hive-partition-field " + hivePartitionsField + ", \n" - + " --hive-sync-use-jdbc " + hiveUseJdbc + ", \n" + + " --hive-sync-mode " + hiveSyncMode + ", \n" + " --hive-metastore-uris " + hiveHMSUris + ", \n" + " --hive-sync-ignore-exception " + hiveSyncIgnoreException + ", \n" + " --hive-partition-value-extractor-class " + partitionValueExtractorClass + ", \n" @@ -247,7 +245,7 @@ public boolean equals(Object o) { && Objects.equals(hivePassWord, config.hivePassWord) && Objects.equals(hiveURL, config.hiveURL) && Objects.equals(hivePartitionsField, config.hivePartitionsField) - && Objects.equals(hiveUseJdbc, config.hiveUseJdbc) + && Objects.equals(hiveSyncMode, config.hiveSyncMode) && Objects.equals(hiveHMSUris, config.hiveHMSUris) && Objects.equals(partitionValueExtractorClass, config.partitionValueExtractorClass) && Objects.equals(sparkMaster, config.sparkMaster) @@ -261,7 +259,7 @@ public boolean equals(Object o) { public int hashCode() { return Objects.hash(basePath, runningMode, tableName, partitions, instantTime, syncToHive, hiveDataBase, hiveTableName, hiveUserName, hivePassWord, hiveURL, - hivePartitionsField, hiveUseJdbc, hiveHMSUris, partitionValueExtractorClass, + hivePartitionsField, hiveSyncMode, hiveHMSUris, partitionValueExtractorClass, sparkMaster, sparkMemory, propsFilePath, configs, hiveSyncIgnoreException, help); } } @@ -350,7 +348,6 @@ private HiveSyncConfig buildHiveSyncProps() { props.put(DataSourceWriteOptions.HIVE_PASS().key(), cfg.hivePassWord); props.put(DataSourceWriteOptions.HIVE_URL().key(), cfg.hiveURL); props.put(DataSourceWriteOptions.HIVE_PARTITION_FIELDS().key(), cfg.hivePartitionsField); - props.put(DataSourceWriteOptions.HIVE_USE_JDBC().key(), cfg.hiveUseJdbc); props.put(DataSourceWriteOptions.HIVE_SYNC_MODE().key(), cfg.hiveSyncMode); props.put(DataSourceWriteOptions.HIVE_IGNORE_EXCEPTIONS().key(), cfg.hiveSyncIgnoreException); props.put(DataSourceWriteOptions.HIVE_PASS().key(), cfg.hivePassWord); diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/DeltaSync.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/DeltaSync.java index 5940ab1d46f25..badd3ab6275c7 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/DeltaSync.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/DeltaSync.java @@ -783,12 +783,12 @@ private HoodieWriteConfig getHoodieClientConfig(Schema schema) { .combineInput(cfg.filterDupes, combineBeforeUpsert) .withCompactionConfig( HoodieCompactionConfig.newBuilder() - .withPayloadClass(cfg.payloadClassName) .withInlineCompaction(cfg.isInlineCompactionEnabled()) .build() ) .withPayloadConfig( HoodiePayloadConfig.newBuilder() + .withPayloadClass(cfg.payloadClassName) .withPayloadOrderingField(cfg.sourceOrderingField) .build()) .forTable(cfg.targetTableName) diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieDeltaStreamer.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieDeltaStreamer.java index 45d8a427c0436..dde0e5f73fc4d 100644 --- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieDeltaStreamer.java +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieDeltaStreamer.java @@ -46,6 +46,8 @@ import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.config.HoodieClusteringConfig; +import org.apache.hudi.config.HoodieCleanConfig; +import org.apache.hudi.config.HoodieArchivalConfig; import org.apache.hudi.config.HoodieCompactionConfig; import org.apache.hudi.config.HoodieLockConfig; import org.apache.hudi.config.HoodieWriteConfig; @@ -766,7 +768,7 @@ private void testUpsertsContinuousMode(HoodieTableType tableType, String tempDir } cfg.tableType = tableType.name(); cfg.configs.add(String.format("%s=%d", SourceConfigs.MAX_UNIQUE_RECORDS_PROP, totalRecords)); - cfg.configs.add(String.format("%s=false", HoodieCompactionConfig.AUTO_CLEAN.key())); + cfg.configs.add(String.format("%s=false", HoodieCleanConfig.AUTO_CLEAN.key())); HoodieDeltaStreamer ds = new HoodieDeltaStreamer(cfg, jsc); deltaStreamerTestRunner(ds, cfg, (r) -> { if (tableType.equals(HoodieTableType.MERGE_ON_READ)) { @@ -946,16 +948,16 @@ public void testCleanerDeleteReplacedDataWithArchive(Boolean asyncClean) throws // Step 4 : Insert 1 record and trigger sync/async cleaner and archive. List configs = getAsyncServicesConfigs(1, "true", "true", "2", "", ""); - configs.add(String.format("%s=%s", HoodieCompactionConfig.CLEANER_POLICY.key(), "KEEP_LATEST_COMMITS")); - configs.add(String.format("%s=%s", HoodieCompactionConfig.CLEANER_COMMITS_RETAINED.key(), "1")); - configs.add(String.format("%s=%s", HoodieCompactionConfig.MIN_COMMITS_TO_KEEP.key(), "2")); - configs.add(String.format("%s=%s", HoodieCompactionConfig.MAX_COMMITS_TO_KEEP.key(), "3")); - configs.add(String.format("%s=%s", HoodieCompactionConfig.ASYNC_CLEAN.key(), asyncClean)); + configs.add(String.format("%s=%s", HoodieCleanConfig.CLEANER_POLICY.key(), "KEEP_LATEST_COMMITS")); + configs.add(String.format("%s=%s", HoodieCleanConfig.CLEANER_COMMITS_RETAINED.key(), "1")); + configs.add(String.format("%s=%s", HoodieArchivalConfig.MIN_COMMITS_TO_KEEP.key(), "2")); + configs.add(String.format("%s=%s", HoodieArchivalConfig.MAX_COMMITS_TO_KEEP.key(), "3")); + configs.add(String.format("%s=%s", HoodieCleanConfig.ASYNC_CLEAN.key(), asyncClean)); configs.add(String.format("%s=%s", HoodieMetadataConfig.COMPACT_NUM_DELTA_COMMITS.key(), "1")); if (asyncClean) { configs.add(String.format("%s=%s", HoodieWriteConfig.WRITE_CONCURRENCY_MODE.key(), WriteConcurrencyMode.OPTIMISTIC_CONCURRENCY_CONTROL.name())); - configs.add(String.format("%s=%s", HoodieCompactionConfig.FAILED_WRITES_CLEANER_POLICY.key(), + configs.add(String.format("%s=%s", HoodieCleanConfig.FAILED_WRITES_CLEANER_POLICY.key(), HoodieFailedWritesCleaningPolicy.LAZY.name())); configs.add(String.format("%s=%s", HoodieLockConfig.LOCK_PROVIDER_CLASS_NAME.key(), InProcessLockProvider.class.getName())); @@ -987,7 +989,7 @@ private List getAsyncServicesConfigs(int totalRecords, String autoClean, List configs = new ArrayList<>(); configs.add(String.format("%s=%d", SourceConfigs.MAX_UNIQUE_RECORDS_PROP, totalRecords)); if (!StringUtils.isNullOrEmpty(autoClean)) { - configs.add(String.format("%s=%s", HoodieCompactionConfig.AUTO_CLEAN.key(), autoClean)); + configs.add(String.format("%s=%s", HoodieCleanConfig.AUTO_CLEAN.key(), autoClean)); } if (!StringUtils.isNullOrEmpty(inlineCluster)) { configs.add(String.format("%s=%s", HoodieClusteringConfig.INLINE_CLUSTERING.key(), inlineCluster)); @@ -1462,7 +1464,7 @@ public void testFilterDupes() throws Exception { cfg2.filterDupes = false; cfg2.sourceLimit = 2000; cfg2.operation = WriteOperationType.UPSERT; - cfg2.configs.add(String.format("%s=false", HoodieCompactionConfig.AUTO_CLEAN.key())); + cfg2.configs.add(String.format("%s=false", HoodieCleanConfig.AUTO_CLEAN.key())); HoodieDeltaStreamer ds2 = new HoodieDeltaStreamer(cfg2, jsc); ds2.sync(); mClient = HoodieTableMetaClient.builder().setConf(jsc.hadoopConfiguration()).setBasePath(tableBasePath).setLoadActiveTimelineOnLoad(true).build(); diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieDeltaStreamerWithMultiWriter.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieDeltaStreamerWithMultiWriter.java index 13f5ad97cfc92..abe1994a3dd87 100644 --- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieDeltaStreamerWithMultiWriter.java +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieDeltaStreamerWithMultiWriter.java @@ -26,7 +26,7 @@ import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.HoodieTimeline; -import org.apache.hudi.config.HoodieCompactionConfig; +import org.apache.hudi.config.HoodieCleanConfig; import org.apache.hudi.execution.bulkinsert.BulkInsertSortMode; import org.apache.hudi.testutils.SparkClientFunctionalTestHarness; import org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer; @@ -95,7 +95,7 @@ void testUpsertsContinuousModeWithMultipleWritersForConflicts(HoodieTableType ta propsFilePath, Collections.singletonList(TestHoodieDeltaStreamer.TripsWithDistanceTransformer.class.getName())); prepJobConfig.continuousMode = true; prepJobConfig.configs.add(String.format("%s=%d", SourceConfigs.MAX_UNIQUE_RECORDS_PROP, totalRecords)); - prepJobConfig.configs.add(String.format("%s=false", HoodieCompactionConfig.AUTO_CLEAN.key())); + prepJobConfig.configs.add(String.format("%s=false", HoodieCleanConfig.AUTO_CLEAN.key())); HoodieDeltaStreamer prepJob = new HoodieDeltaStreamer(prepJobConfig, jsc()); // Prepare base dataset with some commits @@ -115,7 +115,7 @@ void testUpsertsContinuousModeWithMultipleWritersForConflicts(HoodieTableType ta propsFilePath, Collections.singletonList(TestHoodieDeltaStreamer.TripsWithDistanceTransformer.class.getName())); cfgIngestionJob.continuousMode = true; cfgIngestionJob.configs.add(String.format("%s=%d", SourceConfigs.MAX_UNIQUE_RECORDS_PROP, totalRecords)); - cfgIngestionJob.configs.add(String.format("%s=false", HoodieCompactionConfig.AUTO_CLEAN.key())); + cfgIngestionJob.configs.add(String.format("%s=false", HoodieCleanConfig.AUTO_CLEAN.key())); // create a backfill job HoodieDeltaStreamer.Config cfgBackfillJob = getDeltaStreamerConfig(tableBasePath, tableType.name(), WriteOperationType.UPSERT, @@ -127,7 +127,7 @@ void testUpsertsContinuousModeWithMultipleWritersForConflicts(HoodieTableType ta .fromBytes(timeline.getInstantDetails(timeline.firstInstant().get()).get(), HoodieCommitMetadata.class); cfgBackfillJob.checkpoint = commitMetadata.getMetadata(CHECKPOINT_KEY); cfgBackfillJob.configs.add(String.format("%s=%d", SourceConfigs.MAX_UNIQUE_RECORDS_PROP, totalRecords)); - cfgBackfillJob.configs.add(String.format("%s=false", HoodieCompactionConfig.AUTO_CLEAN.key())); + cfgBackfillJob.configs.add(String.format("%s=false", HoodieCleanConfig.AUTO_CLEAN.key())); HoodieDeltaStreamer backfillJob = new HoodieDeltaStreamer(cfgBackfillJob, jsc()); // re-init ingestion job to start sync service @@ -157,7 +157,7 @@ void testUpsertsContinuousModeWithMultipleWritersWithoutConflicts(HoodieTableTyp propsFilePath, Collections.singletonList(TestHoodieDeltaStreamer.TripsWithDistanceTransformer.class.getName())); prepJobConfig.continuousMode = true; prepJobConfig.configs.add(String.format("%s=%d", SourceConfigs.MAX_UNIQUE_RECORDS_PROP, totalRecords)); - prepJobConfig.configs.add(String.format("%s=false", HoodieCompactionConfig.AUTO_CLEAN.key())); + prepJobConfig.configs.add(String.format("%s=false", HoodieCleanConfig.AUTO_CLEAN.key())); HoodieDeltaStreamer prepJob = new HoodieDeltaStreamer(prepJobConfig, jsc()); // Prepare base dataset with some commits @@ -188,13 +188,13 @@ void testUpsertsContinuousModeWithMultipleWritersWithoutConflicts(HoodieTableTyp .fromBytes(timeline.getInstantDetails(timeline.firstInstant().get()).get(), HoodieCommitMetadata.class); cfgBackfillJob2.checkpoint = commitMetadata.getMetadata(CHECKPOINT_KEY); cfgBackfillJob2.configs.add(String.format("%s=%d", SourceConfigs.MAX_UNIQUE_RECORDS_PROP, totalRecords)); - cfgBackfillJob2.configs.add(String.format("%s=false", HoodieCompactionConfig.AUTO_CLEAN.key())); + cfgBackfillJob2.configs.add(String.format("%s=false", HoodieCleanConfig.AUTO_CLEAN.key())); HoodieDeltaStreamer.Config cfgIngestionJob2 = getDeltaStreamerConfig(tableBasePath, tableType.name(), WriteOperationType.UPSERT, propsFilePath, Collections.singletonList(TestHoodieDeltaStreamer.TestIdentityTransformer.class.getName())); cfgIngestionJob2.continuousMode = true; cfgIngestionJob2.configs.add(String.format("%s=%d", SourceConfigs.MAX_UNIQUE_RECORDS_PROP, totalRecords)); - cfgIngestionJob2.configs.add(String.format("%s=false", HoodieCompactionConfig.AUTO_CLEAN.key())); + cfgIngestionJob2.configs.add(String.format("%s=false", HoodieCleanConfig.AUTO_CLEAN.key())); // re-init ingestion job HoodieDeltaStreamer ingestionJob3 = new HoodieDeltaStreamer(cfgIngestionJob2, jsc()); // re-init backfill job @@ -225,7 +225,7 @@ void testLatestCheckpointCarryOverWithMultipleWriters(HoodieTableType tableType) propsFilePath, Collections.singletonList(TestHoodieDeltaStreamer.TripsWithDistanceTransformer.class.getName())); prepJobConfig.continuousMode = true; prepJobConfig.configs.add(String.format("%s=%d", SourceConfigs.MAX_UNIQUE_RECORDS_PROP, totalRecords)); - prepJobConfig.configs.add(String.format("%s=false", HoodieCompactionConfig.AUTO_CLEAN.key())); + prepJobConfig.configs.add(String.format("%s=false", HoodieCleanConfig.AUTO_CLEAN.key())); HoodieDeltaStreamer prepJob = new HoodieDeltaStreamer(prepJobConfig, jsc()); // Prepare base dataset with some commits @@ -263,7 +263,7 @@ void testLatestCheckpointCarryOverWithMultipleWriters(HoodieTableType tableType) // Set checkpoint to the last successful position cfgBackfillJob.checkpoint = commitMetadataForLastInstant.getMetadata(CHECKPOINT_KEY); cfgBackfillJob.configs.add(String.format("%s=%d", SourceConfigs.MAX_UNIQUE_RECORDS_PROP, totalRecords)); - cfgBackfillJob.configs.add(String.format("%s=false", HoodieCompactionConfig.AUTO_CLEAN.key())); + cfgBackfillJob.configs.add(String.format("%s=false", HoodieCleanConfig.AUTO_CLEAN.key())); HoodieDeltaStreamer backfillJob = new HoodieDeltaStreamer(cfgBackfillJob, jsc()); backfillJob.sync(); diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/TestHoodieIncrSource.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/TestHoodieIncrSource.java index fa5cba446f928..57270bdf812d3 100644 --- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/TestHoodieIncrSource.java +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/TestHoodieIncrSource.java @@ -28,7 +28,8 @@ import org.apache.hudi.common.testutils.HoodieTestDataGenerator; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.collection.Pair; -import org.apache.hudi.config.HoodieCompactionConfig; +import org.apache.hudi.config.HoodieArchivalConfig; +import org.apache.hudi.config.HoodieCleanConfig; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.testutils.SparkClientFunctionalTestHarness; import org.apache.hudi.utilities.schema.SchemaProvider; @@ -64,8 +65,8 @@ public void setUp() throws IOException { @Test public void testHoodieIncrSource() throws IOException { HoodieWriteConfig writeConfig = getConfigBuilder(basePath(), metaClient) - .withCompactionConfig(HoodieCompactionConfig.newBuilder() - .archiveCommitsWith(2, 3).retainCommits(1).build()) + .withArchivalConfig(HoodieArchivalConfig.newBuilder().archiveCommitsWith(2, 3).build()) + .withCleanConfig(HoodieCleanConfig.newBuilder().retainCommits(1).build()) .withMetadataConfig(HoodieMetadataConfig.newBuilder() .withMaxNumDeltaCommitsBeforeCompaction(1).build()) .build(); diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/testutils/UtilitiesTestBase.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/testutils/UtilitiesTestBase.java index 8e7bce944f91a..67a002c3bac79 100644 --- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/testutils/UtilitiesTestBase.java +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/testutils/UtilitiesTestBase.java @@ -124,11 +124,14 @@ public static void setLogLevel() { } public static void initTestServices(boolean needsHive, boolean needsZookeeper) throws Exception { - hdfsTestService = new HdfsTestService(); - dfsCluster = hdfsTestService.start(true); - dfs = dfsCluster.getFileSystem(); - dfsBasePath = dfs.getWorkingDirectory().toString(); - dfs.mkdirs(new Path(dfsBasePath)); + + if (hdfsTestService == null) { + hdfsTestService = new HdfsTestService(); + dfsCluster = hdfsTestService.start(true); + dfs = dfsCluster.getFileSystem(); + dfsBasePath = dfs.getWorkingDirectory().toString(); + dfs.mkdirs(new Path(dfsBasePath)); + } if (needsHive) { hiveTestService = new HiveTestService(hdfsTestService.getHadoopConf()); hiveServer = hiveTestService.start(); diff --git a/pom.xml b/pom.xml index 7ff6add7b189c..9c5fe24aabe0c 100644 --- a/pom.xml +++ b/pom.xml @@ -728,7 +728,7 @@ xalan xalan - 2.7.1 + 2.7.2 diff --git a/rfc/README.md b/rfc/README.md index 3d33b82c29925..f567d2beb16cd 100644 --- a/rfc/README.md +++ b/rfc/README.md @@ -66,12 +66,12 @@ The list of all RFCs can be found here. | 28 | [Support Z-order curve](https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=181307144) | `COMPLETED` | | 29 | [Hash Index](https://cwiki.apache.org/confluence/display/HUDI/RFC+-+29%3A+Hash+Index) | `ONGOING` | | 30 | [Batch operation](https://cwiki.apache.org/confluence/display/HUDI/RFC+-+30%3A+Batch+operation) | `UNDER REVIEW` | -| 31 | [Hive integration Improvement](https://cwiki.apache.org/confluence/display/HUDI/RFC+-+31%3A+Hive+integration+Improvment)| `UNDER REVIEW` | +| 31 | [Hive integration Improvement](https://cwiki.apache.org/confluence/display/HUDI/RFC+-+31%3A+Hive+integration+Improvment)| `ONGOING` | | 32 | [Kafka Connect Sink for Hudi](https://cwiki.apache.org/confluence/display/HUDI/RFC-32+Kafka+Connect+Sink+for+Hudi)| `ONGOING` | | 33 | [Hudi supports more comprehensive Schema Evolution](https://cwiki.apache.org/confluence/display/HUDI/RFC+-+33++Hudi+supports+more+comprehensive+Schema+Evolution)| `ONGOING` | | 34 | [Hudi BigQuery Integration](./rfc-34/rfc-34.md) | `COMPLETED` | | 35 | [Make Flink MOR table writing streaming friendly](https://cwiki.apache.org/confluence/display/HUDI/RFC-35%3A+Make+Flink+MOR+table+writing+streaming+friendly)| `UNDER REVIEW` | -| 36 | [HUDI Metastore Server](https://cwiki.apache.org/confluence/display/HUDI/%5BWIP%5D+RFC-36%3A+HUDI+Metastore+Server)| `UNDER REVIEW` | +| 36 | [HUDI Metastore Server](https://cwiki.apache.org/confluence/display/HUDI/%5BWIP%5D+RFC-36%3A+HUDI+Metastore+Server)| `IN PROGRESS` | | 37 | [Hudi Metadata based Bloom Index](rfc-37/rfc-37.md) | `ONGOING` | | 38 | [Spark Datasource V2 Integration](./rfc-38/rfc-38.md) | `IN PROGRESS` | | 39 | [Incremental source for Debezium](./rfc-39/rfc-39.md) | `ONGOING` | @@ -79,16 +79,16 @@ The list of all RFCs can be found here. | 41 | [Hudi Snowflake Integration] | `UNDER REVIEW`| | 42 | [Consistent Hashing Index](./rfc-42/rfc-42.md) | `IN PROGRESS` | | 43 | [Compaction / Clustering Service](./rfc-43/rfc-43.md) | `UNDER REVIEW` | -| 44 | [Hudi Connector for Presto](./rfc-44/rfc-44.md) | `IN PROGRESS` | +| 44 | [Hudi Connector for Presto](./rfc-44/rfc-44.md) | `ONGOING` | | 45 | [Asynchronous Metadata Indexing](./rfc-45/rfc-45.md) | `ONGOING` | -| 46 | [Optimizing Record Payload Handling](./rfc-46/rfc-46.md) | `UNDER REVIEW` | -| 47 | [Add Call Produce Command for Spark SQL](./rfc-47/rfc-47.md) | `UNDER REVIEW` | +| 46 | [Optimizing Record Payload Handling](./rfc-46/rfc-46.md) | `IN PROGRESS` | +| 47 | [Add Call Produce Command for Spark SQL](./rfc-47/rfc-47.md) | `ONGOING` | | 48 | [LogCompaction for MOR tables](./rfc-48/rfc-48.md) | `UNDER REVIEW` | | 49 | [Support sync with DataHub](./rfc-49/rfc-49.md) | `ONGOING` | -| 50 | [Improve Timeline Server](./rfc-50/rfc-50.md) | `UNDER REVIEW` | +| 50 | [Improve Timeline Server](./rfc-50/rfc-50.md) | `IN PROGRESS` | | 51 | [Change Data Capture](./rfc-51/rfc-51.md) | `UNDER REVIEW` | | 52 | [Introduce Secondary Index to Improve HUDI Query Performance](./rfc-52/rfc-52.md) | `UNDER REVIEW` | -| 53 | [Use Lock-Free Message Queue Improving Hoodie Writing Efficiency](./rfc-53/rfc-53.md) | `UNDER REVIEW` | +| 53 | [Use Lock-Free Message Queue Improving Hoodie Writing Efficiency](./rfc-53/rfc-53.md) | `IN PROGRESS` | | 54 | [New Table APIs and Streamline Hudi Configs](./rfc-54/rfc-54.md) | `UNDER REVIEW` | -| 55 | [Improve Hive/Meta sync class design and hierachies](./rfc-55/rfc-55.md) | `UNDER REVIEW` | +| 55 | [Improve Hive/Meta sync class design and hierachies](./rfc-55/rfc-55.md) | `ONGOING` | | 56 | [Early Conflict Detection For Multi-Writer](./rfc-56/rfc-56.md) | `UNDER REVIEW` | diff --git a/rfc/rfc-55/hudi-sync-class-diagram.png b/rfc/rfc-55/hudi-sync-class-diagram.png new file mode 100644 index 0000000000000..11acf4bffcbc6 Binary files /dev/null and b/rfc/rfc-55/hudi-sync-class-diagram.png differ diff --git a/rfc/rfc-55/hudi-sync-flows.png b/rfc/rfc-55/hudi-sync-flows.png new file mode 100644 index 0000000000000..068adcc6a5d97 Binary files /dev/null and b/rfc/rfc-55/hudi-sync-flows.png differ diff --git a/rfc/rfc-55/rfc-55.md b/rfc/rfc-55/rfc-55.md new file mode 100644 index 0000000000000..3e8d130216647 --- /dev/null +++ b/rfc/rfc-55/rfc-55.md @@ -0,0 +1,157 @@ + +# RFC-55: Improve hudi-sync classes design and simplify configs + +## Proposers + +- @ +- @ + +## Approvers + + - @ + - @ + +## Status + +JIRA: [HUDI-3730](https://issues.apache.org/jira/browse/HUDI-3730) + +## Abstract + +![hudi-sync-flows.png](hudi-sync-flows.png) + +Hudi support sync to various metastores via different processing framework like Spark, Flink, and Kafka connect. + +There are some room for improvement + +* The way to generate Sync configs are inconsistent in different framework +* The abstraction of SyncClasses was designed for HiveSync, hence there are duplicated code, unused method, and parameters. + +We need a standard way to run hudi sync. We also need a unified abstraction of XXXSyncTool , XXXSyncClient and XXXSyncConfig to handle supported metastores, like hive metastore, bigquery, datahub, etc. + +## Classes design + +![hudi-sync-class-diagram.png](hudi-sync-class-diagram.png) + +Below are the proposed key classes to handle the main sync logic. They are extensible for different metastores. + +### `HoodieSyncTool` + +*Renamed from `AbstractSyncTool`.* + +```java +public abstract class HoodieSyncTool implements AutoCloseable { + + protected HoodieSyncClient syncClient; + + /** + * Sync tool class is the entrypoint to run meta sync. + * + * @param props A bag of properties passed by users. It can contain all hoodie.* and any other config. + * @param hadoopConf Hadoop specific configs. + */ + public HoodieSyncTool(Properties props, Configuration hadoopConf); + + public abstract void syncHoodieTable(); + + public static void main(String[] args) { + // instantiate HoodieSyncConfig and concrete sync tool, and run sync. + } +} +``` + +### `HoodieSyncConfig` + +```java +public class HoodieSyncConfig extends HoodieConfig { + + public static class HoodieSyncConfigParams { + // POJO class to take command line parameters + @Parameter() + private String basePath; // common essential parameters + + public Properties toProps(); + } + + /** + * XXXSyncConfig is meant to be created and used by XXXSyncTool exclusively and internally. + * + * @param props passed from XXXSyncTool. + * @param hadoopConf passed from XXXSyncTool. + */ + public HoodieSyncConfig(Properties props, Configuration hadoopConf); +} + +public class HiveSyncConfig extends HoodieSyncConfig { + + public static class HiveSyncConfigParams { + + @Parameter() + private String syncMode; + + // delegate common parameters to other XXXParams class + // this overcomes single-inheritance's inconvenience + // see https://jcommander.org/#_parameter_delegates + @ParametersDelegate() + private HoodieSyncConfigParams hoodieSyncConfigParams = new HoodieSyncConfigParams(); + + public Properties toProps(); + } + + public HoodieSyncConfig(Properties props); +} +``` + +### `HoodieSyncClient` + +*Renamed from `AbstractSyncHoodieClient`.* + +```java +public abstract class HoodieSyncClient implements AutoCloseable { + // metastore-agnostic APIs +} +``` + +## Config simplification + +- rename all sync related configs to suffix as `hoodie.sync.*` + - no more `hoodie.meta.sync.*` or `hoodie.meta_sync.*` + - no more variable name or class name like `metaSyncEnabled` or `metaSyncTool`; standardize as `hoodieSync*` to align with module name `hudi-sync` +- remove all sync related option constants from `DataSourceOptions` +- `database` and `table` should not be required by sync tool; they should be inferred from table properties +- users should not need to set PartitionValueExtractor; partition values should be inferred automatically +- remove `USE_JDBC` and fully adopt `SYNC_MODE` +- remove `HIVE_SYNC_ENDABLED` and related arguments from sync tools and delta streamers. Use `SYNC_ENABLED` +- migrate repeated sync config to original config + - `META_SYNC_BASE_FILE_FORMAT` -> `org.apache.hudi.common.table.HoodieTableConfig.BASE_FILE_FORMAT` + - `META_SYNC_PARTITION_FIELDS` -> `org.apache.hudi.common.table.HoodieTableConfig.PARTITION_FIELDS` + - `META_SYNC_ASSUME_DATE_PARTITION` -> `org.apache.hudi.common.config.HoodieMetadataConfig.ASSUME_DATE_PARTITIONING` + - `META_SYNC_DECODE_PARTITION` -> `org.apache.hudi.common.table.HoodieTableConfig.URL_ENCODE_PARTITIONING` + - `META_SYNC_USE_FILE_LISTING_FROM_METADATA` -> `org.apache.hudi.common.config.HoodieMetadataConfig.ENABLE` + +## Rollout/Adoption Plan + +- Users who set `USE_JDBC` will need to change to set `SYNC_MODE=jdbc` +- Users who set `--enable-hive-sync` or `HIVE_SYNC_ENABLED` will need to drop the argument or config and change to `--enable-sync` or `SYNC_ENABLED`. +- Users who import from `DataSourceOptions` for meta sync constants will need to import relevant configs from `HoodieSyncConfig` and subclasses. +- Users who set `AwsGlueCatalogSyncTool` as sync tool class need to update the class name to `AWSGlueCatalogSyncTool` + +## Test Plan + +- CI covers most operations for Hive sync with HMS +- end-to-end testing with setup for Glue Catalog, BigQuery, DataHub instance +- manual testing with partitions added and removed