diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieTableServiceClient.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieTableServiceClient.java index 5245c4bba2145..54241756ec4c0 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieTableServiceClient.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieTableServiceClient.java @@ -18,12 +18,16 @@ package org.apache.hudi.client; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; import org.apache.hudi.async.AsyncArchiveService; import org.apache.hudi.async.AsyncCleanerService; import org.apache.hudi.avro.model.HoodieCleanMetadata; import org.apache.hudi.avro.model.HoodieCleanerPlan; import org.apache.hudi.avro.model.HoodieClusteringPlan; import org.apache.hudi.avro.model.HoodieCompactionPlan; +import org.apache.hudi.avro.model.HoodieInstantInfo; import org.apache.hudi.avro.model.HoodieRollbackMetadata; import org.apache.hudi.avro.model.HoodieRollbackPlan; import org.apache.hudi.client.embedded.EmbeddedTimelineService; @@ -33,6 +37,7 @@ import org.apache.hudi.common.model.ActionType; import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy; +import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.model.TableServiceType; import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.common.table.HoodieTableMetaClient; @@ -63,6 +68,7 @@ import javax.annotation.Nullable; import java.io.IOException; +import java.util.Arrays; import java.util.Collections; import java.util.HashMap; import java.util.LinkedHashMap; @@ -87,6 +93,7 @@ public abstract class BaseHoodieTableServiceClient extends BaseHoodieClient i protected transient AsyncArchiveService asyncArchiveService; protected Set pendingInflightAndRequestedInstants; + protected Set userSpecificPartitions; protected BaseHoodieTableServiceClient(HoodieEngineContext context, HoodieWriteConfig clientConfig, @@ -282,7 +289,6 @@ protected void completeLogCompaction(HoodieCommitMetadata metadata, HoodieTable throw new UnsupportedOperationException("Log compaction is not supported yet."); } - /** * Schedules a new compaction instant with passed-in instant time. * @@ -358,20 +364,38 @@ protected void runTableServicesInline(HoodieTable table, HoodieCommitMetadata me metadata.addMetadata(HoodieCompactionConfig.INLINE_LOG_COMPACT.key(), "false"); } - // Do an inline clustering if enabled - if (config.inlineClusteringEnabled()) { - metadata.addMetadata(HoodieClusteringConfig.INLINE_CLUSTERING.key(), "true"); - inlineClustering(table, extraMetadata); + if (table.getMetaClient().getTableConfig().isLSMBasedLogFormat()) { + // Do an inline lsm clustering if enabled + if (config.inlineLSMClusteringEnabled()) { + metadata.addMetadata(HoodieClusteringConfig.LSM_INLINE_CLUSTERING.key(), "true"); + inlineClustering(table, extraMetadata); + } else { + metadata.addMetadata(HoodieClusteringConfig.LSM_INLINE_CLUSTERING.key(), "false"); + } + + // if just inline schedule lsm clustering is enabled + if (!config.inlineLSMClusteringEnabled() && config.scheduleInlineLSMClustering() + && table.getActiveTimeline().filterPendingReplaceTimeline().empty()) { + // proceed only if there are no pending clustering + metadata.addMetadata(HoodieClusteringConfig.LSM_SCHEDULE_INLINE_CLUSTERING.key(), "true"); + inlineScheduleClustering(extraMetadata); + } } else { - metadata.addMetadata(HoodieClusteringConfig.INLINE_CLUSTERING.key(), "false"); - } + // Do an inline clustering if enabled + if (config.inlineClusteringEnabled()) { + metadata.addMetadata(HoodieClusteringConfig.INLINE_CLUSTERING.key(), "true"); + inlineClustering(table, extraMetadata); + } else { + metadata.addMetadata(HoodieClusteringConfig.INLINE_CLUSTERING.key(), "false"); + } - // if just inline schedule is enabled - if (!config.inlineClusteringEnabled() && config.scheduleInlineClustering() - && table.getActiveTimeline().filterPendingReplaceTimeline().empty()) { - // proceed only if there are no pending clustering - metadata.addMetadata(HoodieClusteringConfig.SCHEDULE_INLINE_CLUSTERING.key(), "true"); - inlineScheduleClustering(extraMetadata); + // if just inline schedule is enabled + if (!config.inlineClusteringEnabled() && config.scheduleInlineClustering() + && table.getActiveTimeline().filterPendingReplaceTimeline().empty()) { + // proceed only if there are no pending clustering + metadata.addMetadata(HoodieClusteringConfig.SCHEDULE_INLINE_CLUSTERING.key(), "true"); + inlineScheduleClustering(extraMetadata); + } } } @@ -417,8 +441,8 @@ protected Option scheduleTableServiceInternal(String instantTime, Option break; case COMPACT: LOG.info("Scheduling compaction at instant time :" + instantTime); - Option compactionPlan = table - .scheduleCompaction(context, instantTime, extraMetadata); + Option compactionPlan = userSpecificPartitions != null ? table.scheduleCompaction(context, instantTime, extraMetadata, userSpecificPartitions) : + table.scheduleCompaction(context, instantTime, extraMetadata); option = compactionPlan.isPresent() ? Option.of(instantTime) : Option.empty(); break; case LOG_COMPACT: @@ -429,8 +453,9 @@ protected Option scheduleTableServiceInternal(String instantTime, Option break; case CLEAN: LOG.info("Scheduling cleaning at instant time :" + instantTime); - Option cleanerPlan = table - .scheduleCleaning(context, instantTime, extraMetadata); + Option cleanerPlan = table.getMetaClient().getTableConfig().isLSMBasedLogFormat() + ? table.scheduleLSMCleaning(context, instantTime, extraMetadata) + : table.scheduleCleaning(context, instantTime, extraMetadata); option = cleanerPlan.isPresent() ? Option.of(instantTime) : Option.empty(); break; default: @@ -535,8 +560,12 @@ public HoodieCleanMetadata clean(String cleanInstantTime, boolean scheduleInline return null; } final Timer.Context timerContext = metrics.getCleanCtx(); - CleanerUtils.rollbackFailedWrites(config.getFailedWritesCleanPolicy(), - HoodieTimeline.CLEAN_ACTION, () -> rollbackFailedWrites()); + if (config.getAsyncRollbackEnable()) { + LOG.info("Rollback will not be executed when clean, the rolllback async enable."); + } else { + CleanerUtils.rollbackFailedWrites(config.getFailedWritesCleanPolicy(), + HoodieTimeline.CLEAN_ACTION, () -> rollbackFailedWrites()); + } HoodieTable table = createTable(config, hadoopConf); if (config.allowMultipleCleans() || !table.getActiveTimeline().getCleanerTimeline().filterInflightsAndRequested().firstInstant().isPresent()) { @@ -609,7 +638,7 @@ public Option getPendingRollbackInfo(HoodieTableMetaC return getPendingRollbackInfos(metaClient, ignoreCompactionAndClusteringInstants).getOrDefault(commitToRollback, Option.empty()); } - protected Map> getPendingRollbackInfos(HoodieTableMetaClient metaClient) { + public Map> getPendingRollbackInfos(HoodieTableMetaClient metaClient) { return getPendingRollbackInfos(metaClient, true); } @@ -621,6 +650,8 @@ protected Map> getPendingRollbackInfos protected Map> getPendingRollbackInfos(HoodieTableMetaClient metaClient, boolean ignoreCompactionAndClusteringInstants) { List instants = metaClient.getActiveTimeline().filterPendingRollbackTimeline().getInstants(); Map> infoMap = new HashMap<>(); + Map tempRollbackInstantFileMap = getTempRollbackInstantFile(fs, metaClient); + for (HoodieInstant rollbackInstant : instants) { HoodieRollbackPlan rollbackPlan; try { @@ -643,15 +674,43 @@ protected Map> getPendingRollbackInfos } try { - String action = rollbackPlan.getInstantToRollback().getAction(); + HoodieInstantInfo instantToRollback = rollbackPlan.getInstantToRollback(); + + if (tempRollbackInstantFileMap.containsKey(rollbackInstant.getTimestamp())) { + // rollback.uuid 文件存在 + Path tmpPath = tempRollbackInstantFileMap.get(rollbackInstant.getTimestamp()); + if (metaClient.getActiveTimeline().containsInstant(instantToRollback.getCommitTime())) { + // instantToRollback 元数据文件存在, rollback未完成, 需要重新执行, 删除该 rollback.uuid 文件 + LOG.info("Rollback for " + rollbackInstant + " is uncompleted. Should re-execute this rollback. " + + "File path: " + tmpPath); + try { + fs.delete(tmpPath); + } catch (IOException e) { + throw new HoodieIOException("Delete temp rollback instant file failed for " + rollbackInstant, e); + } + } else { + // instantToRollback 元数据文件不存在, rollback已经完成, 不需要重新执行, 重命名该 rollback.uuid 文件 + LOG.info("Rollback for " + rollbackInstant + " is completed. Just rename this temp file. " + + "File path: " + tmpPath); + HoodieInstant commitInstant = new HoodieInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.ROLLBACK_ACTION, rollbackInstant.getTimestamp()); + Path targetPath = new Path(tmpPath.getParent(), commitInstant.getFileName()); + try { + fs.rename(tmpPath, targetPath); + } catch (IOException e) { + LOG.warn("Rename temp rollback instant file failed for " + rollbackInstant); + } + continue; + } + } + + String action = instantToRollback.getAction(); if (ignoreCompactionAndClusteringInstants) { if (!HoodieTimeline.COMPACTION_ACTION.equals(action)) { boolean isClustering = HoodieTimeline.REPLACE_COMMIT_ACTION.equals(action) && ClusteringUtils.getClusteringPlan(metaClient, new HoodieInstant(true, rollbackPlan.getInstantToRollback().getAction(), rollbackPlan.getInstantToRollback().getCommitTime())).isPresent(); if (!isClustering) { - String instantToRollback = rollbackPlan.getInstantToRollback().getCommitTime(); - infoMap.putIfAbsent(instantToRollback, Option.of(new HoodiePendingRollbackInfo(rollbackInstant, rollbackPlan))); + infoMap.putIfAbsent(instantToRollback.getCommitTime(), Option.of(new HoodiePendingRollbackInfo(rollbackInstant, rollbackPlan))); } } } else { @@ -664,6 +723,35 @@ protected Map> getPendingRollbackInfos return infoMap; } + /** + * 获取.hoodie目录下的临时 rollback instant 文件 (instant.rollback.uuid) + * + * @param fs + * @param metaClient + * @return tempRollbackInstantFileMap [instant timestamp, file_path] + */ + private Map getTempRollbackInstantFile(FileSystem fs, HoodieTableMetaClient metaClient) { + Map tempRollbackInstantFileMap = new HashMap<>(); + try { + FileStatus[] fileStatuses = HoodieTableMetaClient.scanFiles(fs, new Path(metaClient.getMetaPath()), path -> { + return path.getName().contains(HoodieTimeline.ROLLBACK_ACTION) + && !path.getName().endsWith(HoodieTimeline.REQUESTED_ROLLBACK_EXTENSION) + && !path.getName().endsWith(HoodieTimeline.INFLIGHT_ROLLBACK_EXTENSION) + && !path.getName().endsWith(HoodieTimeline.ROLLBACK_EXTENSION); + }); + + Arrays.stream(fileStatuses).forEach(fileStatus -> { + int dotIndex = fileStatus.getPath().getName().indexOf("."); + String instantTimestamp = fileStatus.getPath().getName().substring(0, dotIndex); + tempRollbackInstantFileMap.put(instantTimestamp, fileStatus.getPath()); + }); + } catch (IOException e) { + throw new HoodieIOException("Failed to scan temp rollback instant meta file", e); + } + LOG.info("Find " + tempRollbackInstantFileMap.keySet().size() + " temp rollback instant files: " + tempRollbackInstantFileMap.values()); + return tempRollbackInstantFileMap; + } + /** * Rolls back the failed delta commits corresponding to the indexing action. * Such delta commits are identified based on the suffix `METADATA_INDEXER_TIME_SUFFIX` ("004"). @@ -758,13 +846,7 @@ protected List getInstantsToRollback(HoodieTableMetaClient metaClient, H } }).collect(Collectors.toList()); } else if (cleaningPolicy.isLazy()) { - return inflightInstantsStream.filter(instant -> { - try { - return heartbeatClient.isHeartbeatExpired(instant.getTimestamp()); - } catch (IOException io) { - throw new HoodieException("Failed to check heartbeat for instant " + instant, io); - } - }).map(HoodieInstant::getTimestamp).collect(Collectors.toList()); + return getInstantsToRollbackForLazyCleanPolicy(metaClient, inflightInstantsStream); } else if (cleaningPolicy.isNever()) { return Collections.emptyList(); } else { @@ -772,6 +854,42 @@ protected List getInstantsToRollback(HoodieTableMetaClient metaClient, H } } + private List getInstantsToRollbackForLazyCleanPolicy(HoodieTableMetaClient metaClient, + Stream inflightInstantsStream) { + // Get expired instants, must store them into list before double-checking + List expiredInstants = inflightInstantsStream.filter(instant -> { + try { + // An instant transformed from inflight to completed have no heartbeat file and will be detected as expired instant here + return heartbeatClient.isHeartbeatExpired(instant.getTimestamp()); + } catch (IOException io) { + throw new HoodieException("Failed to check heartbeat for instant " + instant, io); + } + }).map(HoodieInstant::getTimestamp).collect(Collectors.toList()); + + if (!expiredInstants.isEmpty()) { + // Only return instants that haven't been completed by other writers + metaClient.reloadActiveTimeline(); + List allWriteInstants = getAllWriteInstants(metaClient); + if (!allWriteInstants.isEmpty() && allWriteInstants.get(0).equals(expiredInstants.get(0))) { + expiredInstants.remove(0); + } + HoodieTimeline refreshedInflightTimeline = getInflightTimelineExcludeCompactionAndClustering(metaClient); + return expiredInstants.stream().filter(refreshedInflightTimeline::containsInstant).collect(Collectors.toList()); + } else { + return Collections.emptyList(); + } + } + + private List getAllWriteInstants(HoodieTableMetaClient metaClient) { + HoodieTableType tableType = metaClient.getTableConfig().getTableType(); + if (tableType.equals(HoodieTableType.COPY_ON_WRITE)) { + return metaClient.getActiveTimeline().getCommitTimeline().filter(instant -> instant.getAction().equals(HoodieTimeline.COMMIT_ACTION)) + .getReverseOrderedInstants().map(HoodieInstant::getTimestamp).collect(Collectors.toList()); + } + return metaClient.getActiveTimeline().getDeltaCommitTimeline().getReverseOrderedInstants() + .map(HoodieInstant::getTimestamp).collect(Collectors.toList()); + } + /** * @Deprecated * Rollback the inflight record changes with the given commit time. This @@ -783,7 +901,8 @@ protected List getInstantsToRollback(HoodieTableMetaClient metaClient, H * @throws HoodieRollbackException if rollback cannot be performed successfully */ @Deprecated - public boolean rollback(final String commitInstantTime, Option pendingRollbackInfo, boolean skipLocking) throws HoodieRollbackException { + public Pair> rollback(final String commitInstantTime, + Option pendingRollbackInfo, boolean skipLocking) throws HoodieRollbackException { LOG.info("Begin rollback of instant " + commitInstantTime); final String rollbackInstantTime = pendingRollbackInfo.map(entry -> entry.getRollbackInstant().getTimestamp()).orElse(HoodieActiveTimeline.createNewInstantTime()); final Timer.Context timerContext = this.metrics.getRollbackCtx(); @@ -814,13 +933,13 @@ public boolean rollback(final String commitInstantTime, Option userSpecificPartitions) { + this.userSpecificPartitions = userSpecificPartitions; + } } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieClusteringConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieClusteringConfig.java index bfcd4315d2927..30d2635e9187b 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieClusteringConfig.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieClusteringConfig.java @@ -24,6 +24,7 @@ import org.apache.hudi.common.config.HoodieConfig; import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.engine.EngineType; +import org.apache.hudi.common.table.HoodieTableConfig; import org.apache.hudi.common.util.TypeUtils; import org.apache.hudi.common.util.ValidationUtils; import org.apache.hudi.exception.HoodieException; @@ -50,6 +51,9 @@ public class HoodieClusteringConfig extends HoodieConfig { // Any strategy specific params can be saved with this prefix public static final String CLUSTERING_STRATEGY_PARAM_PREFIX = "hoodie.clustering.plan.strategy."; + public static final String LSM_CLUSTERING_STRATEGY_PARAM_PREFIX = "hoodie.clustering.lsm.plan.strategy."; + public static final String LSM_CLUSTERING_USING_STREAMING_COPY = "hoodie.clustering.lsm.using.streaming.copy"; + public static final String LSM_CLUSTERING_OUT_PUT_LEVEL = "hoodie.clustering.lsm.output.level"; public static final String SPARK_SIZED_BASED_CLUSTERING_PLAN_STRATEGY = "org.apache.hudi.client.clustering.plan.strategy.SparkSizeBasedClusteringPlanStrategy"; public static final String FLINK_SIZED_BASED_CLUSTERING_PLAN_STRATEGY = @@ -58,8 +62,14 @@ public class HoodieClusteringConfig extends HoodieConfig { "org.apache.hudi.client.clustering.plan.strategy.SparkConsistentBucketClusteringPlanStrategy"; public static final String JAVA_SIZED_BASED_CLUSTERING_PLAN_STRATEGY = "org.apache.hudi.client.clustering.plan.strategy.JavaSizeBasedClusteringPlanStrategy"; + public static final String LSM_BASE_CLUSTERING_PLAN_STRATEGY = + "org.apache.hudi.table.action.cluster.strategy.LsmBaseClusteringPlanStrategy"; + public static final String SPARK_LSM_BASE_CLUSTERING_PLAN_STRATEGY = + "org.apache.hudi.client.clustering.plan.strategy.SparkLsmBaseClusteringPlanStrategy"; public static final String SPARK_SORT_AND_SIZE_EXECUTION_STRATEGY = "org.apache.hudi.client.clustering.run.strategy.SparkSortAndSizeExecutionStrategy"; + public static final String SPARK_LSM_EXECUTION_STRATEGY = + "org.apache.hudi.client.clustering.run.strategy.LsmSparkClusteringExecutionStrategy"; public static final String SPARK_CONSISTENT_BUCKET_EXECUTION_STRATEGY = "org.apache.hudi.client.clustering.run.strategy.SparkConsistentBucketClusteringExecutionStrategy"; public static final String JAVA_SORT_AND_SIZE_EXECUTION_STRATEGY = @@ -96,6 +106,12 @@ public class HoodieClusteringConfig extends HoodieConfig { .sinceVersion("0.7.0") .withDocumentation("Files smaller than the size in bytes specified here are candidates for clustering"); + public static final ConfigProperty LSM_PLAN_STRATEGY_SMALL_FILE_LIMIT = ConfigProperty + .key(LSM_CLUSTERING_STRATEGY_PARAM_PREFIX + "small.file.limit") + .defaultValue(String.valueOf(300 * 1024 * 1024L)) + .sinceVersion("0.13.1") + .withDocumentation("Files smaller than the size in bytes specified here are candidates for lsm clustering"); + public static final ConfigProperty PARTITION_REGEX_PATTERN = ConfigProperty .key(CLUSTERING_STRATEGY_PARAM_PREFIX + "partition.regex.pattern") .noDefaultValue() @@ -116,6 +132,14 @@ public class HoodieClusteringConfig extends HoodieConfig { + "i.e select what file groups are being clustered. Default strategy, looks at the clustering small file size limit (determined by " + PLAN_STRATEGY_SMALL_FILE_LIMIT.key() + ") to pick the small file slices within partitions for clustering."); + public static final ConfigProperty LSM_PLAN_STRATEGY_CLASS_NAME = ConfigProperty + .key("hoodie.clustering.lsm.plan.strategy.class") + .defaultValue(SPARK_LSM_BASE_CLUSTERING_PLAN_STRATEGY) + .sinceVersion("0.13.1") + .withDocumentation("Config to provide a strategy class (subclass of ClusteringPlanStrategy) to create clustering plan " + + "i.e select what file groups are being clustered. Default strategy, looks at the clustering small file size limit (determined by " + + LSM_PLAN_STRATEGY_SMALL_FILE_LIMIT.key() + ") to pick the small file slices within partitions for clustering."); + public static final ConfigProperty EXECUTION_STRATEGY_CLASS_NAME = ConfigProperty .key("hoodie.clustering.execution.strategy.class") .defaultValue(SPARK_SORT_AND_SIZE_EXECUTION_STRATEGY) @@ -124,6 +148,12 @@ public class HoodieClusteringConfig extends HoodieConfig { + " clustering plan is executed. By default, we sort the file groups in th plan by the specified columns, while " + " meeting the configured target file sizes."); + public static final ConfigProperty LSM_EXECUTION_STRATEGY_CLASS_NAME = ConfigProperty + .key("hoodie.clustering.lsm.execution.strategy.class") + .defaultValue(SPARK_LSM_EXECUTION_STRATEGY) + .sinceVersion("0.13.1") + .withDocumentation("Used for LSM."); + public static final ConfigProperty INLINE_CLUSTERING = ConfigProperty .key("hoodie.clustering.inline") .defaultValue("false") @@ -137,12 +167,55 @@ public class HoodieClusteringConfig extends HoodieConfig { .sinceVersion("0.7.0") .withDocumentation("Config to control frequency of clustering planning"); + public static final ConfigProperty LSM_INLINE_CLUSTERING_MAX_COMMITS = ConfigProperty + .key("hoodie.clustering.lsm.inline.max.commits") + .defaultValue("4") + .sinceVersion("0.13.1") + .withDocumentation("Config to control frequency of lsm clustering planning"); + public static final ConfigProperty ASYNC_CLUSTERING_MAX_COMMITS = ConfigProperty .key("hoodie.clustering.async.max.commits") .defaultValue("4") .sinceVersion("0.9.0") .withDocumentation("Config to control frequency of async clustering"); + public static final ConfigProperty LSM_ASYNC_CLUSTERING_MAX_COMMITS = ConfigProperty + .key("hoodie.clustering.lsm.async.max.commits") + .defaultValue("4") + .sinceVersion("0.13.1") + .withDocumentation("Config to control frequency of lsm async clustering"); + + public static final ConfigProperty PENDING_CLUSTERING_MAX_COMMITS = ConfigProperty + .key("hoodie.clustering.pending.max.commits") + .defaultValue("2") + .withDocumentation("The maximum number of scheduled clustering at the same time."); + + public static final ConfigProperty LSM_PENDING_CLUSTERING_MAX_COMMITS = ConfigProperty + .key("hoodie.clustering.lsm.pending.max.commits") + .defaultValue("2") + .sinceVersion("0.13.1") + .withDocumentation("The maximum number of scheduled lsm clustering at the same time."); + + public static final ConfigProperty CLUSTERING_SKIP_SORT = ConfigProperty + .key("hoodie.clustering.skip.sort") + .defaultValue(false) + .withDocumentation("When set to true, skip sorting during clustering, even if the clustering plan contains sorting fields."); + + public static final ConfigProperty CLUSTERING_MAX_PARALLELISM = ConfigProperty + .key("hoodie.clustering.max.parallelism") + .defaultValue(5) + .sinceVersion("0.14.0") + .withDocumentation("Maximum number of parallelism jobs submitted in clustering operation. " + + "If the resource is sufficient(Like Spark engine has enough idle executors), increasing this " + + "value will let the clustering job run faster, while it will give additional pressure to the " + + "execution engines to manage more concurrent running jobs."); + + public static final ConfigProperty CLUSTERING_EXECUTION_SEPARATE = ConfigProperty + .key("hoodie.clustering.execution.separate") + .defaultValue(true) + .sinceVersion("0.14.0") + .withDocumentation(""); + public static final ConfigProperty PLAN_STRATEGY_SKIP_PARTITIONS_FROM_LATEST = ConfigProperty .key(CLUSTERING_STRATEGY_PARAM_PREFIX + "daybased.skipfromlatest.partitions") .defaultValue("0") @@ -170,18 +243,54 @@ public class HoodieClusteringConfig extends HoodieConfig { + " is defined by below two properties (CLUSTERING_MAX_BYTES_PER_GROUP * CLUSTERING_MAX_NUM_GROUPS)." + " Max amount of data to be included in one group"); + public static final ConfigProperty LSM_PLAN_STRATEGY_MAX_BYTES_PER_OUTPUT_FILEGROUP = ConfigProperty + .key(LSM_CLUSTERING_STRATEGY_PARAM_PREFIX + "max.bytes.per.group") + .defaultValue(String.valueOf(2 * 1024 * 1024 * 1024L)) + .sinceVersion("0.13.1") + .withDocumentation("Each clustering operation can create multiple output file groups. Total amount of data processed by clustering operation" + + " is defined by below two properties (CLUSTERING_MAX_BYTES_PER_GROUP * CLUSTERING_MAX_NUM_GROUPS)." + + " Max amount of data to be included in one group"); + public static final ConfigProperty PLAN_STRATEGY_MAX_GROUPS = ConfigProperty .key(CLUSTERING_STRATEGY_PARAM_PREFIX + "max.num.groups") .defaultValue("30") .sinceVersion("0.7.0") .withDocumentation("Maximum number of groups to create as part of ClusteringPlan. Increasing groups will increase parallelism"); + public static final ConfigProperty LSM_PLAN_STRATEGY_MIN_GROUPS = ConfigProperty + .key(LSM_CLUSTERING_STRATEGY_PARAM_PREFIX + "min.num.groups") + .defaultValue("10") + .sinceVersion("0.13.1") + .withDocumentation("Maximum number of groups to create as part of ClusteringPlan. Increasing groups will increase parallelism"); + + public static final ConfigProperty PLAN_STRATEGY_INSTANT_LIMIT = ConfigProperty + .key("hoodie.clustering.plan.instants.limit") + .defaultValue(0) + .withDocumentation("给JDFlinkSizeBasedClusteringPlanStrategyRecently使用,从active timeline中选择最新的LimitN个instant来获取partition"); + public static final ConfigProperty PLAN_STRATEGY_TARGET_FILE_MAX_BYTES = ConfigProperty .key(CLUSTERING_STRATEGY_PARAM_PREFIX + "target.file.max.bytes") .defaultValue(String.valueOf(1024 * 1024 * 1024L)) .sinceVersion("0.7.0") .withDocumentation("Each group can produce 'N' (CLUSTERING_MAX_GROUP_SIZE/CLUSTERING_TARGET_FILE_SIZE) output file groups"); + public static final ConfigProperty PLAN_STRATEGY_SINGLE_CLUSTERING_GROUP_PER_PARTITION = ConfigProperty + .key(CLUSTERING_STRATEGY_PARAM_PREFIX + "single.clustering.group.per.partition") + .defaultValue(false) + .withDocumentation("用于标识生成Plan时将每个分区下的小文件作为一个clustering group."); + + public static final ConfigProperty PLAN_STRATEGY_OUT_FILE_GROUP_EXPANSION = ConfigProperty + .key(CLUSTERING_STRATEGY_PARAM_PREFIX + "out.filegroup.expansion") + .defaultValue("") + .withDocumentation("application#others/business#K:10@application#others/business#Z:3(#替换=), " + + "用@分隔不同分区,用冒号分隔分区与放大系数,分区中用井号替换等号(切割问题), for SparkSizeBasedClusteringPlanStrategyWithStorageStrategy and JDFlinkPartitionSizeBasedClusteringPlanStrategyWithCacheLayer"); + + public static final ConfigProperty CLUSTER_EXECUTE_OUT_FILE_GROUP_EXPANSION = ConfigProperty + .key("hoodie.clustering.execution.out.filegroup.expansion") + .defaultValue("") + .withDocumentation("application#others/business#K:10@application#others/business#Z:3(#替换=), " + + "用@分隔不同分区,用冒号分隔分区与放大系数,分区中用井号替换等号(切割问题), for SparkSizeBasedClusteringPlanStrategyWithStorageStrategy and JDFlinkPartitionSizeBasedClusteringPlanStrategyWithCacheLayer"); + public static final ConfigProperty PLAN_STRATEGY_SORT_COLUMNS = ConfigProperty .key(CLUSTERING_STRATEGY_PARAM_PREFIX + "sort.columns") .noDefaultValue() @@ -212,6 +321,12 @@ public class HoodieClusteringConfig extends HoodieConfig { .withDocumentation("Enable running of clustering service, asynchronously as inserts happen on the table.") .withAlternatives("hoodie.datasource.clustering.async.enable"); + public static final ConfigProperty LSM_ASYNC_CLUSTERING_SCHEDULE_ENABLE = ConfigProperty + .key("hoodie.clustering.lsm.async.schedule.enabled") + .defaultValue("false") + .sinceVersion("0.13.1") + .withDocumentation("Enable lsm async clustering schedule by delta commits"); + public static final ConfigProperty PRESERVE_COMMIT_METADATA = ConfigProperty .key("hoodie.clustering.preserve.commit.metadata") .defaultValue(true) @@ -238,7 +353,7 @@ public class HoodieClusteringConfig extends HoodieConfig { *
  • Z-order: orders records along Z-order spatial-curve
  • *
  • Hilbert: orders records along Hilbert's spatial-curve
  • * - * + *

    * NOTE: "z-order", "hilbert" strategies may consume considerably more compute, than "linear". * Make sure to perform small-scale local testing for your dataset before applying globally. */ @@ -251,8 +366,8 @@ public class HoodieClusteringConfig extends HoodieConfig { /** * NOTE: This setting only has effect if {@link #LAYOUT_OPTIMIZE_STRATEGY} value is set to - * either "z-order" or "hilbert" (ie leveraging space-filling curves) - * + * either "z-order" or "hilbert" (ie leveraging space-filling curves) + *

    * Currently, two methods to order records along the curve are supported "build" and "sample": * *

      @@ -261,12 +376,12 @@ public class HoodieClusteringConfig extends HoodieConfig { *
    • Sample: leverages boundary-base interleaved index method (described in more details in * Amazon DynamoDB blog [1])
    • *
    - * + *

    * NOTE: Boundary-based interleaved Index method has better generalization, * but is slower than direct method. - * + *

    * Please refer to RFC-28 for specific elaboration on both flows. - * + *

    * [1] https://aws.amazon.com/cn/blogs/database/tag/z-order/ */ public static final ConfigProperty LAYOUT_OPTIMIZE_SPATIAL_CURVE_BUILD_METHOD = ConfigProperty @@ -279,8 +394,8 @@ public class HoodieClusteringConfig extends HoodieConfig { /** * NOTE: This setting only has effect if {@link #LAYOUT_OPTIMIZE_SPATIAL_CURVE_BUILD_METHOD} value - * is set to "sample" - * + * is set to "sample" + *

    * Determines target sample size used by the Boundary-based Interleaved Index method. * Larger sample size entails better layout optimization outcomes, at the expense of higher memory * footprint. @@ -312,6 +427,39 @@ public class HoodieClusteringConfig extends HoodieConfig { + "Please exercise caution while setting this config, especially when clustering is done very frequently. This could lead to race condition in " + "rare scenarios, for example, when the clustering completes after instants are fetched but before rollback completed."); + public static final ConfigProperty LSM_INLINE_CLUSTERING = ConfigProperty + .key("hoodie.clustering.lsm.inline") + .defaultValue("false") + .withDocumentation("Turn on lsm inline clustering - clustering will be run after each write operation is complete, defaults to enable inline clustering"); + + public static final ConfigProperty LSM_SCHEDULE_INLINE_CLUSTERING = ConfigProperty + .key("hoodie.clustering.lsm.schedule.inline") + .defaultValue("false") + .withDocumentation("When set to true, clustering service will be attempted for lsm inline scheduling after each write."); + + public static final ConfigProperty NUM_RUN_CLUSTERING_TRIGGER = ConfigProperty + .key("hoodie.clustering.lsm.num.sorted.run.trigger") + .defaultValue(5) + .withDocumentation("The sorted run number to trigger compaction. " + + "Includes level0 files (one file one sorted run) and level1 runs (one level one sorted run one file)."); + + public static final ConfigProperty MAX_SIZE_AMP = ConfigProperty + .key("hoodie.clustering.lsm.maxsize.amplification.percent") + .defaultValue(200) + .withDocumentation("The size amplification is defined as the amount (in percentage) of " + + "additional storage needed to store a single byte of data in the merge tree for changelog mode table."); + + public static final ConfigProperty LSM_CLUSTERING_READFOOTER_ENABLED = ConfigProperty + .key("hoodie.clustering.lsm.readfooter.enabled") + .defaultValue(false) + .withDocumentation("Whether read footer when clustering schedule"); + + public static final ConfigProperty LSM_CLUSTERING_READ_FOOTER_TASKS = ConfigProperty + .key("hoodie.clustering.lsm.readfooter.tasks") + .defaultValue(200) + .withDocumentation("This config controls the behavior of reading footers during LSM clustering. " + + "The final parallelism is the minimum of the number of file slices and this config."); + /** * @deprecated Use {@link #PLAN_STRATEGY_CLASS_NAME} and its methods instead */ @@ -422,7 +570,9 @@ public class HoodieClusteringConfig extends HoodieConfig { */ @Deprecated public static final String ASYNC_CLUSTERING_ENABLE_OPT_KEY = ASYNC_CLUSTERING_ENABLE.key(); - /** @deprecated Use {@link #ASYNC_CLUSTERING_ENABLE} and its methods instead */ + /** + * @deprecated Use {@link #ASYNC_CLUSTERING_ENABLE} and its methods instead + */ @Deprecated public static final String DEFAULT_ASYNC_CLUSTERING_ENABLE_OPT_VAL = ASYNC_CLUSTERING_ENABLE.defaultValue(); @@ -440,7 +590,7 @@ public boolean isInlineClusteringEnabled() { } public static HoodieClusteringConfig from(TypedProperties props) { - return HoodieClusteringConfig.newBuilder().fromProperties(props).build(); + return HoodieClusteringConfig.newBuilder().fromProperties(props).build(); } public static Builder newBuilder() { @@ -469,6 +619,31 @@ public Builder withClusteringPlanStrategyClass(String clusteringStrategyClass) { return this; } + public Builder withLsmClusteringPlanStrategyClass(String clusteringStrategyClass) { + clusteringConfig.setValue(LSM_PLAN_STRATEGY_CLASS_NAME, clusteringStrategyClass); + return this; + } + + public Builder withInlineLSMClustering(Boolean inlineLSMClustering) { + clusteringConfig.setValue(LSM_INLINE_CLUSTERING, String.valueOf(inlineLSMClustering)); + return this; + } + + public Builder withLsmClusteringPlanNumSortRunTrigger(int number) { + clusteringConfig.setValue(NUM_RUN_CLUSTERING_TRIGGER, String.valueOf(number)); + return this; + } + + public Builder withLsmClusteringPlanMaxSizeAmp(int amp) { + clusteringConfig.setValue(MAX_SIZE_AMP, String.valueOf(amp)); + return this; + } + + public Builder withLsmClusteringReadFooterEnabled(boolean readFooterEnabled) { + clusteringConfig.setValue(LSM_CLUSTERING_READFOOTER_ENABLED, String.valueOf(readFooterEnabled)); + return this; + } + public Builder withClusteringPlanPartitionFilterMode(ClusteringPlanPartitionFilterMode mode) { clusteringConfig.setValue(PLAN_PARTITION_FILTER_MODE_NAME.key(), mode.toString()); return this; @@ -479,6 +654,11 @@ public Builder withClusteringExecutionStrategyClass(String runClusteringStrategy return this; } + public Builder withLsmClusteringExecutionStrategyClass(String runClusteringStrategyClass) { + clusteringConfig.setValue(LSM_EXECUTION_STRATEGY_CLASS_NAME, runClusteringStrategyClass); + return this; + } + public Builder withClusteringTargetPartitions(int clusteringTargetPartitions) { clusteringConfig.setValue(DAYBASED_LOOKBACK_PARTITIONS, String.valueOf(clusteringTargetPartitions)); return this; @@ -513,27 +693,72 @@ public Builder withClusteringPlanSmallFileLimit(long clusteringSmallFileLimit) { clusteringConfig.setValue(PLAN_STRATEGY_SMALL_FILE_LIMIT, String.valueOf(clusteringSmallFileLimit)); return this; } - + + public Builder withLsmClusteringPlanSmallFileLimit(long clusteringSmallFileLimit) { + clusteringConfig.setValue(LSM_PLAN_STRATEGY_SMALL_FILE_LIMIT, String.valueOf(clusteringSmallFileLimit)); + return this; + } + public Builder withClusteringSortColumns(String sortColumns) { clusteringConfig.setValue(PLAN_STRATEGY_SORT_COLUMNS, sortColumns); return this; } + public Builder withClusteringSkipSort(Boolean skipSort) { + clusteringConfig.setValue(CLUSTERING_SKIP_SORT, String.valueOf(skipSort)); + return this; + } + + public Builder withClusteringMaxParallelism(int parallelism) { + clusteringConfig.setValue(CLUSTERING_MAX_PARALLELISM, String.valueOf(parallelism)); + return this; + } + + public Builder withClusteringExecutionSeparate(Boolean executionSeparate) { + clusteringConfig.setValue(CLUSTERING_EXECUTION_SEPARATE, String.valueOf(executionSeparate)); + return this; + } + public Builder withClusteringMaxBytesInGroup(long clusteringMaxGroupSize) { clusteringConfig.setValue(PLAN_STRATEGY_MAX_BYTES_PER_OUTPUT_FILEGROUP, String.valueOf(clusteringMaxGroupSize)); return this; } + public Builder withLsmClusteringMaxBytesInGroup(long clusteringMaxGroupSize) { + clusteringConfig.setValue(LSM_PLAN_STRATEGY_MAX_BYTES_PER_OUTPUT_FILEGROUP, String.valueOf(clusteringMaxGroupSize)); + return this; + } + public Builder withClusteringMaxNumGroups(int maxNumGroups) { clusteringConfig.setValue(PLAN_STRATEGY_MAX_GROUPS, String.valueOf(maxNumGroups)); return this; } + public Builder withLsmClusteringMinNumGroups(int minNumGroups) { + clusteringConfig.setValue(LSM_PLAN_STRATEGY_MIN_GROUPS, String.valueOf(minNumGroups)); + return this; + } + public Builder withClusteringTargetFileMaxBytes(long targetFileSize) { clusteringConfig.setValue(PLAN_STRATEGY_TARGET_FILE_MAX_BYTES, String.valueOf(targetFileSize)); return this; } + public Builder withClusteringPlanFileGroupExpansion(String expansion) { + clusteringConfig.setValue(PLAN_STRATEGY_OUT_FILE_GROUP_EXPANSION, expansion); + return this; + } + + public Builder withSingleClusteringGroupPerPartition(boolean singleClusteringGroupPerPartition) { + clusteringConfig.setValue(PLAN_STRATEGY_SINGLE_CLUSTERING_GROUP_PER_PARTITION, String.valueOf(singleClusteringGroupPerPartition)); + return this; + } + + public Builder withClusteringExecuteFileGroupExpansion(String expansion) { + clusteringConfig.setValue(CLUSTER_EXECUTE_OUT_FILE_GROUP_EXPANSION, expansion); + return this; + } + public Builder withInlineClustering(Boolean inlineClustering) { clusteringConfig.setValue(INLINE_CLUSTERING, String.valueOf(inlineClustering)); return this; @@ -554,6 +779,21 @@ public Builder withAsyncClusteringMaxCommits(int numCommits) { return this; } + public Builder withLsmAsyncClusteringMaxCommits(int numCommits) { + clusteringConfig.setValue(LSM_ASYNC_CLUSTERING_MAX_COMMITS, String.valueOf(numCommits)); + return this; + } + + public Builder withMaxOfPendingClustering(int maxNumPendingClustering) { + clusteringConfig.setValue(PENDING_CLUSTERING_MAX_COMMITS, String.valueOf(maxNumPendingClustering)); + return this; + } + + public Builder withLsmMaxOfPendingClustering(int maxNumPendingClustering) { + clusteringConfig.setValue(LSM_PENDING_CLUSTERING_MAX_COMMITS, String.valueOf(maxNumPendingClustering)); + return this; + } + public Builder fromProperties(Properties props) { // TODO this should cherry-pick only clustering properties this.clusteringConfig.getProps().putAll(props); @@ -570,6 +810,11 @@ public Builder withAsyncClustering(Boolean asyncClustering) { return this; } + public Builder withLsmAsyncClusteringSchedule(Boolean asyncClustering) { + clusteringConfig.setValue(LSM_ASYNC_CLUSTERING_SCHEDULE_ENABLE, String.valueOf(asyncClustering)); + return this; + } + public Builder withPreserveHoodieCommitMetadata(Boolean preserveHoodieCommitMetadata) { clusteringConfig.setValue(PRESERVE_COMMIT_METADATA, String.valueOf(preserveHoodieCommitMetadata)); return this; @@ -611,6 +856,11 @@ private void setDefaults() { clusteringConfig.setDefaultValue(PLAN_STRATEGY_CLASS_NAME, SPARK_CONSISTENT_BUCKET_CLUSTERING_PLAN_STRATEGY); clusteringConfig.setDefaultValue(EXECUTION_STRATEGY_CLASS_NAME, SPARK_CONSISTENT_BUCKET_EXECUTION_STRATEGY); } else { + String logFormat = clusteringConfig.props.getProperty(HoodieTableConfig.HOODIE_LOG_FORMAT.key()); + if (logFormat != null && logFormat.equalsIgnoreCase(HoodieTableConfig.LSM_HOODIE_TABLE_LOG_FORMAT)) { + clusteringConfig.setDefaultValue( + LSM_PLAN_STRATEGY_CLASS_NAME, getLsmDefaultPlanStrategyClassName(engineType)); + } clusteringConfig.setDefaultValue( PLAN_STRATEGY_CLASS_NAME, getDefaultPlanStrategyClassName(engineType)); clusteringConfig.setDefaultValue( @@ -636,6 +886,27 @@ private void validate() { ValidationUtils.checkArgument(clusteringConfig.getString(EXECUTION_STRATEGY_CLASS_NAME).equals(SPARK_CONSISTENT_BUCKET_EXECUTION_STRATEGY), "Consistent hashing bucket index only supports clustering execution strategy : " + SPARK_CONSISTENT_BUCKET_EXECUTION_STRATEGY); } + + // validate lsm inline clustering + String logFormat = clusteringConfig.props.getProperty(HoodieTableConfig.HOODIE_LOG_FORMAT.key()); + if (logFormat != null && logFormat.equalsIgnoreCase(HoodieTableConfig.LSM_HOODIE_TABLE_LOG_FORMAT)) { + boolean inlineLSMCluster = clusteringConfig.getBoolean(HoodieClusteringConfig.LSM_INLINE_CLUSTERING); + boolean inlineLSMClusterSchedule = clusteringConfig.getBoolean(HoodieClusteringConfig.LSM_SCHEDULE_INLINE_CLUSTERING); + ValidationUtils.checkArgument(!(inlineLSMCluster && inlineLSMClusterSchedule), String.format("Either of inline lsm clustering (%s) or " + + "schedule inline lsm clustering (%s) can be enabled. Both can't be set to true at the same time. %s,%s", HoodieClusteringConfig.LSM_INLINE_CLUSTERING.key(), + HoodieClusteringConfig.LSM_SCHEDULE_INLINE_CLUSTERING.key(), inlineCluster, inlineClusterSchedule)); + } + } + + private String getLsmDefaultPlanStrategyClassName(EngineType engineType) { + switch (engineType) { + case SPARK: + return SPARK_LSM_BASE_CLUSTERING_PLAN_STRATEGY; + case FLINK: + return LSM_BASE_CLUSTERING_PLAN_STRATEGY; + default: + throw new HoodieNotSupportedException("Unsupported engine " + engineType); + } } private String getDefaultPlanStrategyClassName(EngineType engineType) { @@ -739,7 +1010,7 @@ public enum ClusteringOperator { SCHEDULE_AND_EXECUTE("scheduleandexecute"); private static final Map VALUE_TO_ENUM_MAP = - TypeUtils.getValueToEnumMap(ClusteringOperator.class, e -> e.value); + TypeUtils.getValueToEnumMap(ClusteringOperator.class, e -> e.value); private final String value; 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 f61ba16a33c6c..8b1d9f30b3df8 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 @@ -35,14 +35,18 @@ import org.apache.hudi.common.engine.EngineType; import org.apache.hudi.common.fs.ConsistencyGuardConfig; import org.apache.hudi.common.fs.FileSystemRetryConfig; +import org.apache.hudi.common.model.DefaultHoodieRecordPayload; import org.apache.hudi.common.model.HoodieAvroRecordMerger; import org.apache.hudi.common.model.HoodieCleaningPolicy; import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy; import org.apache.hudi.common.model.HoodieFileFormat; import org.apache.hudi.common.model.HoodieRecordMerger; import org.apache.hudi.common.model.HoodieTableType; +import org.apache.hudi.common.model.ModelUtils; import org.apache.hudi.common.model.OverwriteWithLatestAvroPayload; import org.apache.hudi.common.model.WriteConcurrencyMode; +import org.apache.hudi.common.model.WriteOperationType; +import org.apache.hudi.common.storage.HoodieStorageStrategyType; import org.apache.hudi.common.table.HoodieTableConfig; import org.apache.hudi.common.table.cdc.HoodieCDCSupplementalLoggingMode; import org.apache.hudi.common.table.log.block.HoodieLogBlock; @@ -56,6 +60,7 @@ import org.apache.hudi.common.util.VisibleForTesting; import org.apache.hudi.common.util.queue.DisruptorWaitStrategyType; import org.apache.hudi.common.util.queue.ExecutorType; +import org.apache.hudi.config.metrics.HoodieMetricsBaizeConfig; import org.apache.hudi.config.metrics.HoodieMetricsCloudWatchConfig; import org.apache.hudi.config.metrics.HoodieMetricsConfig; import org.apache.hudi.config.metrics.HoodieMetricsDatadogConfig; @@ -78,6 +83,7 @@ import org.apache.hudi.table.action.compact.strategy.CompactionStrategy; import org.apache.hudi.table.storage.HoodieStorageLayout; +import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.io.compress.Compression; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; @@ -91,6 +97,7 @@ import java.io.IOException; import java.io.InputStream; import java.util.Arrays; +import java.util.HashMap; import java.util.List; import java.util.Locale; import java.util.Map; @@ -102,6 +109,7 @@ import static org.apache.hudi.common.util.ValidationUtils.checkArgument; import static org.apache.hudi.common.util.queue.ExecutorType.SIMPLE; import static org.apache.hudi.config.HoodieCleanConfig.CLEANER_POLICY; +import static org.apache.hudi.config.metrics.HoodieMetricsBaizeConfig.PUSHGATEWAY_ENV_TEST; import static org.apache.hudi.table.marker.ConflictDetectionUtils.getDefaultEarlyConflictDetectionStrategy; /** @@ -134,6 +142,18 @@ public class HoodieWriteConfig extends HoodieConfig { .withDocumentation("Field used in preCombining before actual write. When two records have the same key value, " + "we will pick the one with the largest value for the precombine field, determined by Object.compareTo(..)"); + public static final ConfigProperty OPERATION = ConfigProperty + .key("hoodie.datasource.write.operation") + .defaultValue(WriteOperationType.UPSERT.value()) + .withDocumentation("Whether to do upsert, insert or bulk_insert for the write operation. " + + "Use bulk_insert to load new data into a table, and there on use upsert/insert. " + + "bulk insert uses a disk based write path to scale to load large inputs without need to cache it."); + + public static final ConfigProperty REPLACE_PARTITION_RECORDS = ConfigProperty + .key("hoodie.datasource.replace.partition.records") + .defaultValue("true") + .withDocumentation("When set to false, insert overwrite will not replace the partition records."); + public static final ConfigProperty WRITE_PAYLOAD_CLASS_NAME = ConfigProperty .key("hoodie.datasource.write.payload.class") .defaultValue(OverwriteWithLatestAvroPayload.class.getName()) @@ -144,6 +164,16 @@ public class HoodieWriteConfig extends HoodieConfig { .key("hoodie.datasource.write.record.merger.impls") .defaultValue(HoodieAvroRecordMerger.class.getName()) .sinceVersion("0.13.0") + .withInferFunction(cfg -> { + if (cfg.contains(HoodieTableConfig.HOODIE_LOG_FORMAT.key()) + && cfg.getString(HoodieTableConfig.HOODIE_LOG_FORMAT.key()) + .equalsIgnoreCase(HoodieTableConfig.LSM_HOODIE_TABLE_LOG_FORMAT)) { + return Option.of(ModelUtils.getSupportedRecordMerger()); + } else { + return Option.of(HoodieAvroRecordMerger.class.getName()); + } + }) + .sinceVersion("0.13.0") .withDocumentation("List of HoodieMerger implementations constituting Hudi's merging strategy -- based on the engine used. " + "These merger impls will filter by hoodie.datasource.write.record.merger.strategy " + "Hudi will pick most efficient implementation to perform merging/combining of the records (during update, reading MOR table, etc)"); @@ -207,6 +237,24 @@ public class HoodieWriteConfig extends HoodieConfig { + "Hudi stores all the main meta-data about commits, savepoints, cleaning audit logs " + "etc in .hoodie directory under this base path directory."); + public static final ConfigProperty BULK_INSERT_FLUSH_CONCURRENCY = ConfigProperty + .key("hoodie.bulkinsert.flush.concurrency") + .defaultValue(30) + .withDocumentation("Base path on lake storage, under which all the table data is stored. " + + "Always prefix it explicitly with the storage scheme (e.g hdfs://, s3:// etc). " + + "Hudi stores all the main meta-data about commits, savepoints, cleaning audit logs " + + "etc in .hoodie directory under this base path directory."); + + public static final ConfigProperty STORAGE_PATH = ConfigProperty + .key(HoodieTableConfig.HOODIE_STORAGE_PATH_KEY) + .defaultValue("") + .withDocumentation("Base storage path for Hudi table"); + + public static final ConfigProperty STORAGE_STRATEGY_CLASS_NAME = ConfigProperty + .key(HoodieTableConfig.HOODIE_STORAGE_STRATEGY_CLASS_NAME_KEY) + .defaultValue(HoodieStorageStrategyType.DEFAULT.value) + .withDocumentation("Class that provides storage file locations"); + public static final ConfigProperty AVRO_SCHEMA_STRING = ConfigProperty .key("hoodie.avro.schema") .noDefaultValue() @@ -266,6 +314,38 @@ public class HoodieWriteConfig extends HoodieConfig { + "operation, we suggest configuring this shuffle parallelism explicitly, so that the " + "parallelism is around total_input_data_size/120MB."); + public static final ConfigProperty LSM_SHUFFLE_FACTOR_VALUE = ConfigProperty + .key("hoodie.lsm.shuffle.factor") + .defaultValue("1") + .withDocumentation(""); + + public static final ConfigProperty SHUFFLE_PARTITIONER = ConfigProperty + .key("hoodie.shuffle.remote.partitioner.enable") + .defaultValue(true) + .withDocumentation(""); + + public static final ConfigProperty LSM_FLUSH_CONCURRENCY_VALUE = ConfigProperty + .key("hoodie.lsm.flush.concurrency") + .defaultValue("10") + .withDocumentation(""); + + public static final ConfigProperty HOODIE_LSM_WRITE_L1 = ConfigProperty + .key("hoodie.lsm.write.force.l1") + .defaultValue("false") + .withDocumentation("When set to true, LSM Bulk Insert will write to L1."); + + public static final ConfigProperty LSM_SORT_MERGE_SPILL_THRESHOLD = ConfigProperty + .key("hoodie.lsm.sort.merge.spill.threshold") + .defaultValue(16) + .withDocumentation("Keep how many files in memory and spill to disk for remaining (default value 16 for 32mb block size)"); + + public static final ConfigProperty BULKINSERT_USE_FIRST_PARTITION_FOR_STATISTIC_WHEN_COMMIT = ConfigProperty + .key("hoodie.bulkinsert.use.first.partition.when.commit") + .defaultValue(false) + .withDocumentation("when submitting a commit, the file id information of all partitions will be obtained. " + + "If the partitions are the same, it will obviously have an impact on performance. " + + "So, in this situation, we only take one element as the partition value"); + public static final ConfigProperty BULKINSERT_USER_DEFINED_PARTITIONER_SORT_COLUMNS = ConfigProperty .key("hoodie.bulkinsert.user.defined.partitioner.sort.columns") .noDefaultValue() @@ -519,7 +599,7 @@ public class HoodieWriteConfig extends HoodieConfig { public static final ConfigProperty CLIENT_HEARTBEAT_NUM_TOLERABLE_MISSES = ConfigProperty .key("hoodie.client.heartbeat.tolerable.misses") - .defaultValue(2) + .defaultValue(180) .withDocumentation("Number of heartbeat misses, before a writer is deemed not alive and all pending writes are aborted."); public static final ConfigProperty WRITE_CONCURRENCY_MODE = ConfigProperty @@ -656,6 +736,36 @@ public class HoodieWriteConfig extends HoodieConfig { + "will not print any configuration which contains the configured filter. For example with " + "a configured filter `ssl`, value for config `ssl.trustore.location` would be masked."); + public static final ConfigProperty RECORD_EVENT_TIME_IN_SAVEPOINT = ConfigProperty + .key("hoodie.savepoint.write.record.eventtime") + .defaultValue(false) + .sinceVersion("") + .withDocumentation(""); + + public static final ConfigProperty MUTLIPLE_WRITE_ENABLE = ConfigProperty + .key("hoodie.multiple.write.enable") + .defaultValue(false) + .sinceVersion("") + .withDocumentation("Only support to do multi-streaming appending for COW table."); + + public static final ConfigProperty MULTIPLE_WRITE_JOB_ID = ConfigProperty + .key("hoodie.multiple.write.job.id") + .defaultValue("") + .sinceVersion("") + .withDocumentation("Only support to do multi-streaming appending for COW table."); + + public static final ConfigProperty IGNORE_FAILED = ConfigProperty + .key("hoodie.write.ignore.failed") + .defaultValue(false) + .sinceVersion("") + .withDocumentation("Flag to indicate whether to ignore any non exception error (e.g. writestatus error)."); + + public static final ConfigProperty ROLLBACK_AYSNC_ENABLE = ConfigProperty + .key("hoodie.rollback.async.enable") + .defaultValue(false) + .sinceVersion("") + .withDocumentation(""); + private ConsistencyGuardConfig consistencyGuardConfig; private FileSystemRetryConfig fileSystemRetryConfig; @@ -1076,6 +1186,14 @@ public String getBasePath() { return getString(BASE_PATH); } + public int getFlushConcurrency() { + return getInt(BULK_INSERT_FLUSH_CONCURRENCY); + } + + public String getStoragePath() { + return getStringOrDefault(STORAGE_PATH); + } + public HoodieRecordMerger getRecordMerger() { List mergers = StringUtils.split(getStringOrDefault(RECORD_MERGER_IMPLS), ",").stream() .map(String::trim) @@ -1099,9 +1217,9 @@ public void setRecordMergerClass(String recordMergerStrategy) { /** * Returns schema used for writing records - * + *

    * NOTE: This method respects {@link HoodieWriteConfig#WRITE_SCHEMA_OVERRIDE} being - * specified overriding original writing schema + * specified overriding original writing schema */ public String getWriteSchema() { if (props.containsKey(WRITE_SCHEMA_OVERRIDE.key())) { @@ -1155,6 +1273,18 @@ public String getPreCombineField() { return getString(PRECOMBINE_FIELD_NAME); } + public String getOperationType() { + return getString(OPERATION); + } + + public String getBulkInsertOverWriteOperationType() { + return getString(HoodieInternalConfig.BULKINSERT_OVERWRITE_OPERATION_TYPE); + } + + public boolean getReplacePartitionRecords() { + return getBooleanOrDefault(REPLACE_PARTITION_RECORDS); + } + public String getWritePayloadClass() { return getString(WRITE_PAYLOAD_CLASS_NAME); } @@ -1196,6 +1326,22 @@ public int getBulkInsertShuffleParallelism() { return getInt(BULKINSERT_PARALLELISM_VALUE); } + public int getLsmShuffleFactor() { + return getInt(LSM_SHUFFLE_FACTOR_VALUE); + } + + public boolean isUsingRemotePartitioner() { + return getBoolean(SHUFFLE_PARTITIONER); + } + + public int getLsmFlushConcurrency() { + return getInt(LSM_FLUSH_CONCURRENCY_VALUE); + } + + public boolean isForcingWriteToL1() { + return getBoolean(HOODIE_LSM_WRITE_L1); + } + public String getUserDefinedBulkInsertPartitionerClass() { return getString(BULKINSERT_USER_DEFINED_PARTITIONER_CLASS_NAME); } @@ -1390,6 +1536,18 @@ public int getMinCommitsToKeep() { return getInt(HoodieArchivalConfig.MIN_COMMITS_TO_KEEP); } + public int getSavepointInstantToKeep() { + return getInt(HoodieArchivalConfig.SAVEPOINT_COMMITS_TO_KEEP); + } + + public int getMaxCommitsToKeepClean() { + return getInt(HoodieArchivalConfig.MAX_COMMITS_TO_KEEP_CLEAN); + } + + public int getMinCommitsToKeepClean() { + return getInt(HoodieArchivalConfig.MIN_COMMITS_TO_KEEP_CLEAN); + } + public int getArchiveMergeFilesBatchSize() { return getInt(HoodieArchivalConfig.ARCHIVE_MERGE_FILES_BATCH_SIZE); } @@ -1478,6 +1636,14 @@ public CompactionTriggerStrategy getInlineCompactTriggerStrategy() { return CompactionTriggerStrategy.valueOf(getString(HoodieCompactionConfig.INLINE_COMPACT_TRIGGER_STRATEGY)); } + public boolean getInlineCompactScheduleIncrementalPartitions() { + return getBoolean(HoodieCompactionConfig.INLINE_COMPACT_SCHEDULE_INCREMENTAL_PARTITIONS); + } + + public int getInlineCompactScheduleInstantLimit() { + return getInt(HoodieCompactionConfig.INLINE_COMPACT_SCHEDULE_INSTANT_LIMIT); + } + public int getInlineCompactDeltaCommitMax() { return getInt(HoodieCompactionConfig.INLINE_COMPACT_NUM_DELTA_COMMITS); } @@ -1486,6 +1652,18 @@ public int getInlineCompactDeltaSecondsMax() { return getInt(HoodieCompactionConfig.INLINE_COMPACT_TIME_DELTA_SECONDS); } + public boolean getInlineCompactByEventTime() { + return getBoolean(HoodieCompactionConfig.INLINE_COMPACT_BY_EVENT_TIME); + } + + public String getInlineCompactTimeExpression() { + return getString(HoodieCompactionConfig.INLINE_COMPACT_TIME_EXPRESSION); + } + + public int getMaxOfPendingCompaction() { + return getInt(HoodieCompactionConfig.PENDING_COMPACT_MAX_COMMITS); + } + public CompactionStrategy getCompactionStrategy() { return ReflectionUtils.loadClass(getString(HoodieCompactionConfig.COMPACTION_STRATEGY)); } @@ -1526,6 +1704,10 @@ public boolean isAsyncClusteringEnabled() { return getBoolean(HoodieClusteringConfig.ASYNC_CLUSTERING_ENABLE); } + public boolean isLsmAsyncClusteringScheduleEnabled() { + return getBoolean(HoodieClusteringConfig.LSM_ASYNC_CLUSTERING_SCHEDULE_ENABLE); + } + public boolean isPreserveHoodieCommitMetadataForClustering() { return getBoolean(HoodieClusteringConfig.PRESERVE_COMMIT_METADATA); } @@ -1547,10 +1729,26 @@ public int getInlineClusterMaxCommits() { return getInt(HoodieClusteringConfig.INLINE_CLUSTERING_MAX_COMMITS); } + public int getLsmInlineClusterMaxCommits() { + return getInt(HoodieClusteringConfig.LSM_INLINE_CLUSTERING_MAX_COMMITS); + } + public int getAsyncClusterMaxCommits() { return getInt(HoodieClusteringConfig.ASYNC_CLUSTERING_MAX_COMMITS); } + public int getLsmAsyncClusterMaxCommits() { + return getInt(HoodieClusteringConfig.LSM_ASYNC_CLUSTERING_MAX_COMMITS); + } + + public int getMaxOfPendingClustering() { + return getInt(HoodieClusteringConfig.PENDING_CLUSTERING_MAX_COMMITS); + } + + public int getLsmMaxOfPendingClustering() { + return getInt(HoodieClusteringConfig.LSM_PENDING_CLUSTERING_MAX_COMMITS); + } + public String getPayloadClass() { return getString(HoodiePayloadConfig.PAYLOAD_CLASS_NAME); } @@ -1559,6 +1757,18 @@ public int getTargetPartitionsPerDayBasedCompaction() { return getInt(HoodieCompactionConfig.TARGET_PARTITIONS_PER_DAYBASED_COMPACTION); } + public boolean isCompactionReadSplitByDatasketchEnabled() { + return getBoolean(HoodieCompactionConfig.COMPACTION_READ_SPLIT_BY_DATASKETCH_ENABLED); + } + + public Long getCompactionReadSplitTargetSize() { + return getLong(HoodieCompactionConfig.COMPACTION_READ_SPLIT_TARGET_SIZE); + } + + public String getCompactionReadMergeType() { + return getString(HoodieCompactionConfig.COMPACTION_READ_MERGE_TYPE); + } + public int getCommitArchivalBatchSize() { return getInt(HoodieArchivalConfig.COMMITS_ARCHIVAL_BATCH_SIZE); } @@ -1583,6 +1793,26 @@ public String getClusteringPlanStrategyClass() { return getString(HoodieClusteringConfig.PLAN_STRATEGY_CLASS_NAME); } + public String getLsmClusteringPlanStrategyClass() { + return getString(HoodieClusteringConfig.LSM_PLAN_STRATEGY_CLASS_NAME); + } + + public int getClusteringMaxParallelism() { + return getInt(HoodieClusteringConfig.CLUSTERING_MAX_PARALLELISM); + } + + public Boolean getClusteringExecutionSeparate() { + return getBoolean(HoodieClusteringConfig.CLUSTERING_EXECUTION_SEPARATE); + } + + public boolean getClusteringSkipSort() { + return getBoolean(HoodieClusteringConfig.CLUSTERING_SKIP_SORT); + } + + public boolean getSingleClusteringGroupPerPartition() { + return getBoolean(HoodieClusteringConfig.PLAN_STRATEGY_SINGLE_CLUSTERING_GROUP_PER_PARTITION); + } + public ClusteringPlanPartitionFilterMode getClusteringPlanPartitionFilterMode() { String mode = getString(HoodieClusteringConfig.PLAN_PARTITION_FILTER_MODE_NAME); return ClusteringPlanPartitionFilterMode.valueOf(mode); @@ -1600,14 +1830,26 @@ public String getClusteringExecutionStrategyClass() { return getString(HoodieClusteringConfig.EXECUTION_STRATEGY_CLASS_NAME); } + public String getLSMClusteringExecutionStrategyClass() { + return getString(HoodieClusteringConfig.LSM_EXECUTION_STRATEGY_CLASS_NAME); + } + public long getClusteringMaxBytesInGroup() { return getLong(HoodieClusteringConfig.PLAN_STRATEGY_MAX_BYTES_PER_OUTPUT_FILEGROUP); } + public long getLsmClusteringMaxBytesInGroup() { + return getLong(HoodieClusteringConfig.LSM_PLAN_STRATEGY_MAX_BYTES_PER_OUTPUT_FILEGROUP); + } + public long getClusteringSmallFileLimit() { return getLong(HoodieClusteringConfig.PLAN_STRATEGY_SMALL_FILE_LIMIT); } + public long getLsmClusteringSmallFileLimit() { + return getLong(HoodieClusteringConfig.LSM_PLAN_STRATEGY_SMALL_FILE_LIMIT); + } + public String getClusteringPartitionSelected() { return getString(HoodieClusteringConfig.PARTITION_SELECTED); } @@ -1620,10 +1862,41 @@ public int getClusteringMaxNumGroups() { return getInt(HoodieClusteringConfig.PLAN_STRATEGY_MAX_GROUPS); } + public int getLsmClusteringMinNumGroups() { + return getInt(HoodieClusteringConfig.LSM_PLAN_STRATEGY_MIN_GROUPS); + } + + public int getClusteringPlanInstantsLimit() { + return getInt(HoodieClusteringConfig.PLAN_STRATEGY_INSTANT_LIMIT); + } + public long getClusteringTargetFileMaxBytes() { return getLong(HoodieClusteringConfig.PLAN_STRATEGY_TARGET_FILE_MAX_BYTES); } + /** + * application#others/business#K:10@application#others/business#Z:3 + * ==> + * [application=others/business=K, 10] + * [application=others/business=Z, 3] + */ + public Option> getOutFilegroupExpansion(ConfigProperty outFileGroupExpansion) { + String expansion = getString(outFileGroupExpansion); + try { + if (!StringUtils.isNullOrEmpty(expansion)) { + HashMap res = new HashMap<>(); + Arrays.stream(expansion.split("@")).forEach(ele -> { + String[] split = ele.split(":"); + res.put(split[0].replace("#", "="), Integer.valueOf(split[1])); + }); + return Option.of(res); + } + } catch (Exception ex) { + LOG.warn("Failed to getOutFilegroupExpansion " + expansion, ex); + } + return Option.empty(); + } + public int getTargetPartitionsForClustering() { return getInt(HoodieClusteringConfig.DAYBASED_LOOKBACK_PARTITIONS); } @@ -1658,6 +1931,30 @@ public HoodieIndex.IndexType getIndexType() { return HoodieIndex.IndexType.valueOf(getString(HoodieIndexConfig.INDEX_TYPE)); } + public boolean inlineLSMClusteringEnabled() { + return getBoolean(HoodieClusteringConfig.LSM_INLINE_CLUSTERING); + } + + public boolean scheduleInlineLSMClustering() { + return getBoolean(HoodieClusteringConfig.LSM_SCHEDULE_INLINE_CLUSTERING); + } + + public int getLsmNunRunClusteringTrigger() { + return getInt(HoodieClusteringConfig.NUM_RUN_CLUSTERING_TRIGGER); + } + + public int getLsmMaxSizeAMP() { + return getInt(HoodieClusteringConfig.MAX_SIZE_AMP); + } + + public boolean isLsmReadFooterEnabled() { + return getBoolean(HoodieClusteringConfig.LSM_CLUSTERING_READFOOTER_ENABLED); + } + + public int getLsmReadFooterParallelism() { + return getIntOrDefault(HoodieClusteringConfig.LSM_CLUSTERING_READ_FOOTER_TASKS); + } + public String getIndexClass() { return getString(HoodieIndexConfig.INDEX_CLASS_NAME); } @@ -1871,6 +2168,39 @@ public int getBucketIndexNumBuckets() { return getIntOrDefault(HoodieIndexConfig.BUCKET_INDEX_NUM_BUCKETS); } + public String getBucketStrategistClassName() { + return getString(HoodieIndexConfig.BUCKET_STRATEGIST_CLASS_NAME); + } + + public boolean isBucketIndexAtPartitionLevel() { + return getBoolean(HoodieIndexConfig.BUCKET_INDEX_AT_PARTITION_LEVEL); + } + + public Option getBucketIndexCacheMaxAgeMills() { + if (props.containsKey(HoodieIndexConfig.BUCKET_INDEX_CACHE_MAX_AGE_MILLS_KEY)) { + return Option.of(props.getLong(HoodieIndexConfig.BUCKET_INDEX_CACHE_MAX_AGE_MILLS_KEY)); + } else { + return Option.empty(); + } + } + + public Option getBucketIndexCacheMaxCapacity() { + if (props.containsKey(HoodieIndexConfig.BUCKET_INDEX_CACHE_MAX_CAPACITY_KEY)) { + return Option.of(props.getInteger(HoodieIndexConfig.BUCKET_INDEX_CACHE_MAX_CAPACITY_KEY)); + } else { + return Option.empty(); + } + } + + public void setBucketIndexPartitionBucketExpr(String expr) { + setValue(HoodieIndexConfig.BUCKET_INDEX_PARTITION_BUCKET_EXPR.key(), expr); + } + + public Option getBucketIndexPartitionBucketExpr() { + String expr = props.getString(HoodieIndexConfig.BUCKET_INDEX_PARTITION_BUCKET_EXPR.key()); + return StringUtils.isNullOrEmpty(expr) ? Option.empty() : Option.of(expr); + } + public int getBucketIndexMaxNumBuckets() { return getInt(HoodieIndexConfig.BUCKET_INDEX_MAX_NUM_BUCKETS); } @@ -1891,6 +2221,10 @@ public String getBucketIndexHashField() { return getString(HoodieIndexConfig.BUCKET_INDEX_HASH_FIELD); } + public String getBucketIndexHashFieldWithDefault() { + return getStringOrDefault(HoodieIndexConfig.BUCKET_INDEX_HASH_FIELD, getString(KeyGeneratorOptions.RECORDKEY_FIELD_NAME)); + } + /** * storage properties. */ @@ -1948,6 +2282,22 @@ public String parquetFieldIdWriteEnabled() { return getString(HoodieStorageConfig.PARQUET_FIELD_ID_WRITE_ENABLED); } + public boolean parquetRecordkeyFieldsBloomFilterEnabled() { + return getBoolean(HoodieStorageConfig.PARQUET_RECORDKEY_BLOOM_FILTER_ENABLED); + } + + public boolean parquetRecordkeyClusteringBloomFilterEnabled() { + return getBoolean(HoodieStorageConfig.PARQUET_RECORDKEY_CLUSTERING_BLOOM_FILTER_ENABLED); + } + + public boolean parquetRecordkeyCompactionBloomFilterEnabled() { + return getBoolean(HoodieStorageConfig.PARQUET_RECORDKEY_COMPACTION_BLOOM_FILTER_ENABLED); + } + + public boolean parquetRecordkeyInsertOverwriteBloomFilterEnabled() { + return getBoolean(HoodieStorageConfig.PARQUET_RECORDKEY_INSERT_OVERWRITE_BLOOM_FILTER_ENABLED); + } + public Option getLogDataBlockFormat() { return Option.ofNullable(getString(HoodieStorageConfig.LOGFILE_DATA_BLOCK_FORMAT)) .map(HoodieLogBlock.HoodieLogBlockType::fromId); @@ -1957,6 +2307,10 @@ public long getLogFileMaxSize() { return getLong(HoodieStorageConfig.LOGFILE_MAX_SIZE); } + public boolean isLogFileRolloverDirectly() { + return getBoolean(HoodieStorageConfig.LOGFILE_ROLLOVER_DIRECTLY); + } + public double getLogFileToParquetCompressionRatio() { return getDouble(HoodieStorageConfig.LOGFILE_TO_PARQUET_COMPRESSION_RATIO_FRACTION); } @@ -1989,6 +2343,14 @@ public CompressionKind getOrcCompressionCodec() { return CompressionKind.valueOf(getString(HoodieStorageConfig.ORC_COMPRESSION_CODEC_NAME)); } + public String getStorageStrategyClass() { + return getStringOrDefault(STORAGE_STRATEGY_CLASS_NAME); + } + + public boolean isDataSketchEnabled() { + return getBoolean(HoodieStorageConfig.DATASKETCH_ENABLED); + } + /** * metrics properties. */ @@ -2105,6 +2467,18 @@ public int getPushGatewayPort() { return getInt(HoodieMetricsPrometheusConfig.PUSHGATEWAY_PORT_NUM); } + public String getBaizeAddr() { + return getString(HoodieMetricsBaizeConfig.PUSHGATEWAY_ADDR); + } + + public String getBaizeServiceId() { + return getString(HoodieMetricsBaizeConfig.PUSHGATEWAY_SERVICEID); + } + + public boolean isPushGatewayTestEnv() { + return getBoolean(PUSHGATEWAY_ENV_TEST); + } + public int getPushGatewayReportPeriodSeconds() { return getInt(HoodieMetricsPrometheusConfig.PUSHGATEWAY_REPORT_PERIOD_IN_SECONDS); } @@ -2284,6 +2658,7 @@ public int getMetadataCleanerCommitsRetained() { /** * Hoodie Client Lock Configs. + * * @return */ public boolean isAutoAdjustLockConfigs() { @@ -2334,6 +2709,10 @@ public boolean earlyConflictDetectionCheckCommitConflict() { return getBoolean(EARLY_CONFLICT_DETECTION_CHECK_COMMIT_CONFLICT); } + public boolean needRecordEventTimeInSavepoint() { + return getBoolean(RECORD_EVENT_TIME_IN_SAVEPOINT); + } + // misc configs public Boolean doSkipDefaultPartitionValidation() { return getBoolean(SKIP_DEFAULT_PARTITION_VALIDATION); @@ -2445,6 +2824,20 @@ public boolean isTableServiceManagerEnabled() { return tableServiceManagerConfig.isTableServiceManagerEnabled(); } + /** + * Whether to ignore the write failed. + */ + public boolean getIgnoreWriteFailed() { + return getBooleanOrDefault(IGNORE_FAILED); + } + + /** + * Whether to execute async rollback. + */ + public boolean getAsyncRollbackEnable() { + return getBooleanOrDefault(ROLLBACK_AYSNC_ENABLE); + } + public static class Builder { protected final HoodieWriteConfig writeConfig = new HoodieWriteConfig(); @@ -2468,6 +2861,7 @@ public static class Builder { private boolean isPreCommitValidationConfigSet = false; private boolean isMetricsJmxConfigSet = false; private boolean isMetricsGraphiteConfigSet = false; + private boolean isMetricsBaizeConfigSet = false; private boolean isLayoutConfigSet = false; public Builder withEngineType(EngineType engineType) { @@ -2501,6 +2895,21 @@ public Builder withPath(String basePath) { return this; } + public Builder withFlushConcurrency(Integer concurrency) { + writeConfig.setValue(BULK_INSERT_FLUSH_CONCURRENCY, String.valueOf(concurrency)); + return this; + } + + public Builder withStoragePath(String storagePath) { + writeConfig.setValue(STORAGE_PATH, storagePath); + return this; + } + + public Builder withStorageStrategyClass(String storageStrategyClass) { + writeConfig.setValue(STORAGE_STRATEGY_CLASS_NAME, storageStrategyClass); + return this; + } + public Builder withSchema(String schemaStr) { writeConfig.setValue(AVRO_SCHEMA_STRING, schemaStr); return this; @@ -2536,6 +2945,11 @@ public Builder withPreCombineField(String preCombineField) { return this; } + public Builder withOperation(String operation) { + writeConfig.setValue(OPERATION, operation); + return this; + } + public Builder withWritePayLoad(String payload) { writeConfig.setValue(WRITE_PAYLOAD_CLASS_NAME, payload); return this; @@ -2571,6 +2985,11 @@ public Builder withBulkInsertParallelism(int bulkInsertParallelism) { return this; } + public Builder withShufflePartitioner(boolean shufflePartitioner) { + writeConfig.setValue(SHUFFLE_PARTITIONER, String.valueOf(shufflePartitioner)); + return this; + } + public Builder withUserDefinedBulkInsertPartitionerClass(String className) { writeConfig.setValue(BULKINSERT_USER_DEFINED_PARTITIONER_CLASS_NAME, className); return this; @@ -2586,7 +3005,7 @@ public Builder withDeleteParallelism(int parallelism) { return this; } - public Builder withFailureOnInlineTableServiceException(boolean fail) { + public Builder withFailureOnInlineTableServiceException(boolean fail) { writeConfig.setValue(FAIL_ON_INLINE_TABLE_SERVICE_EXCEPTION, String.valueOf(fail)); return this; } @@ -2662,6 +3081,11 @@ public Builder withCleanConfig(HoodieCleanConfig cleanConfig) { return this; } + public Builder withAysncRollback(Boolean enableAysncRollback) { + writeConfig.setValue(ROLLBACK_AYSNC_ENABLE, String.valueOf(enableAysncRollback)); + return this; + } + public Builder withArchivalConfig(HoodieArchivalConfig cleanConfig) { writeConfig.getProps().putAll(cleanConfig.getProps()); isArchivalConfigSet = true; @@ -2686,6 +3110,12 @@ public Builder withMetricsJmxConfig(HoodieMetricsJmxConfig metricsJmxConfig) { return this; } + public Builder withMetricsBaizeConfig(HoodieMetricsBaizeConfig metricsBaizeConfig) { + writeConfig.getProps().putAll(metricsBaizeConfig.getProps()); + isMetricsBaizeConfigSet = true; + return this; + } + public Builder withMetricsGraphiteConfig(HoodieMetricsGraphiteConfig mericsGraphiteConfig) { writeConfig.getProps().putAll(mericsGraphiteConfig.getProps()); isMetricsGraphiteConfigSet = true; @@ -2852,6 +3282,11 @@ public Builder withPopulateMetaFields(boolean populateMetaFields) { return this; } + public Builder withLogFormatType(String format) { + writeConfig.setValue(HoodieTableConfig.HOODIE_LOG_FORMAT, format); + return this; + } + public Builder withAllowOperationMetadataField(boolean allowOperationMetadataField) { writeConfig.setValue(ALLOW_OPERATION_METADATA_FIELD, Boolean.toString(allowOperationMetadataField)); return this; @@ -2907,11 +3342,21 @@ public Builder withEarlyConflictDetectionCheckCommitConflict(boolean enable) { return this; } + public Builder withRecordEventTimeInCommitEnable(boolean enable) { + writeConfig.setValue(RECORD_EVENT_TIME_IN_SAVEPOINT, String.valueOf(enable)); + return this; + } + public Builder withEarlyConflictDetectionStrategy(String className) { writeConfig.setValue(EARLY_CONFLICT_DETECTION_STRATEGY_CLASS_NAME, className); return this; } + public Builder withWriteIgnoreFailed(boolean ignoreFailedWriteData) { + writeConfig.setValue(IGNORE_FAILED, String.valueOf(ignoreFailedWriteData)); + return this; + } + protected void setDefaults() { writeConfig.setDefaultValue(MARKERS_TYPE, getDefaultMarkersType(engineType)); // Check for mandatory properties @@ -3041,6 +3486,21 @@ private void validate() { 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)); + if (writeConfig.needRecordEventTimeInSavepoint()) { + checkArgument(writeConfig.getPayloadClass().equals(DefaultHoodieRecordPayload.class.getName()), + String.format("Need to use %s as payload class if users want to record event time in savepoint commit", DefaultHoodieRecordPayload.class.getName())); + } + + if (!writeConfig.getStorageStrategyClass().equalsIgnoreCase(HoodieStorageStrategyType.DEFAULT.value) + && WriteOperationType.fromValue(writeConfig.getOperationType()) == WriteOperationType.INSERT) { + checkArgument((writeConfig.isClusteringEnabled() || writeConfig.scheduleInlineClustering()) && writeConfig.getClusteringSmallFileLimit() >= 1 * 1024 * 1024 * 1024, + "对于使用缓存层feature的 insert 任务,请开启clustering功能!且要求hoodie.clustering.plan.strategy.small.file.limit大于1GB!"); + String baseName = new Path(writeConfig.getBasePath()).getName(); + Arrays.stream(writeConfig.getStoragePath().split(",")).forEach(path -> { + Path storagePath = new Path(path); + checkArgument(storagePath.getName().equalsIgnoreCase(baseName), "storage path的路径的最后一级应为表名" + writeConfig.getTableName()); + }); + } } public HoodieWriteConfig build() { diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/lsm/ExternalRecordReader.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/lsm/ExternalRecordReader.java new file mode 100644 index 0000000000000..3e54d4a986893 --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/lsm/ExternalRecordReader.java @@ -0,0 +1,34 @@ +/* + * 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.io.lsm; + +import java.io.IOException; +import java.util.Iterator; + +public interface ExternalRecordReader extends RecordReader { + /** + * Spill all record to disk and return a external record iterator + * + * @param iterator + * @return A External record iterator + * @throws IOException + */ + Iterator spillToDisk(Iterator iterator) throws IOException; + +} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/lsm/FieldsComparator.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/lsm/FieldsComparator.java new file mode 100644 index 0000000000000..3b79399d4b156 --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/lsm/FieldsComparator.java @@ -0,0 +1,26 @@ +/* + * 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.io.lsm; + +import java.util.Comparator; + +public interface FieldsComparator extends Comparator { + + int[] compareFields(); +} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/lsm/MergeFunction.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/lsm/MergeFunction.java new file mode 100644 index 0000000000000..f214d5f5af4ba --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/lsm/MergeFunction.java @@ -0,0 +1,46 @@ +/* + * 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.io.lsm; + +import org.apache.hudi.common.model.HoodieRecord; + +/** + * Merge function to merge multiple {@link KeyValue}s. + * + *

    IMPORTANT, Object reusing inside the kv of the {@link #add} input: + * + *

      + *
    • Please don't save KeyValue and InternalRow references to the List: the KeyValue of the + * first two objects and the InternalRow object inside them are safe, but the reference of the + * third object may overwrite the reference of the first object. + *
    • You can save fields references: fields don't reuse their objects. + *
    + * + * @param result type + */ +public interface MergeFunction { + /** Reset the merge function to its default state. */ + void reset(); + + /** Add the given {@link KeyValue} to the merge function. */ + void add(HoodieRecord kv); + + /** Get current merged value. */ + T getResult(); +} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/lsm/MergeSorter.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/lsm/MergeSorter.java new file mode 100644 index 0000000000000..8ea16dd9b33cf --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/lsm/MergeSorter.java @@ -0,0 +1,37 @@ +/* + * 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.io.lsm; + +import org.apache.hudi.common.model.HoodieRecord; +import java.util.Comparator; +import java.util.List; + +public class MergeSorter { + private final SortEngine sortEngine; + + public MergeSorter() { + this.sortEngine = SortEngine.LOSER_TREE; + } + + public RecordReader mergeSort(List> readers, + RecordMergeWrapper mergeFunctionWrapper, + Comparator comparator) { + return SortMergeReader.createSortMergeReader(readers, comparator, mergeFunctionWrapper, sortEngine); + } +} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/lsm/RecordMergeWrapper.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/lsm/RecordMergeWrapper.java new file mode 100644 index 0000000000000..297702af4b43d --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/lsm/RecordMergeWrapper.java @@ -0,0 +1,56 @@ +/* + * 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.io.lsm; + +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.util.Option; + +import java.util.Iterator; +import java.util.List; + +public interface RecordMergeWrapper { + + /** + * merge a group of HoodieRecords with the same record key + * @param recordGroup + * @return + */ + Option merge(List recordGroup); + + /** + * merge a iterator of HoodieRecords with the same record key + * @param sameKeyIterator + * @return + */ + Option merge(Iterator sameKeyIterator); + + /** + * Sequentially merge HoodieRecords with the same record key + * @param record + */ + void merge(HoodieRecord record); + + /** + * Obtain the sequentially merged results of HoodieRecords + * @return Option + */ + Option getMergedResult(); + + void reset(); +} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/lsm/RecordReader.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/lsm/RecordReader.java new file mode 100644 index 0000000000000..cc6d4161d7e5b --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/lsm/RecordReader.java @@ -0,0 +1,41 @@ +/* + * 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.io.lsm; + +import javax.annotation.Nullable; + +import java.io.Closeable; +import java.io.IOException; +import java.util.Iterator; + +public interface RecordReader extends Closeable { + + /** + * Reads one batch. The method should return null when reaching the end of the input. + * + *

    The returned iterator object and any contained objects may be held onto by the source for + * some time, so it should not be immediately reused by the reader. + */ + @Nullable + Iterator read() throws IOException; + + /** Closes the reader and should release all resources. */ + @Override + void close() throws IOException; +} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/lsm/SortEngine.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/lsm/SortEngine.java new file mode 100644 index 0000000000000..bf578e1d5cd94 --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/lsm/SortEngine.java @@ -0,0 +1,39 @@ +/* + * 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.io.lsm; + +public enum SortEngine { + MIN_HEAP("min-heap", "Use min-heap for multiway sorting."), + LOSER_TREE( + "loser-tree", + "Use loser-tree for multiway sorting. Compared with heapsort, loser-tree has fewer comparisons and is more efficient."); + + private final String value; + private final String description; + + SortEngine(String value, String description) { + this.value = value; + this.description = description; + } + + @Override + public String toString() { + return value; + } +} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/lsm/SortMergeReader.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/lsm/SortMergeReader.java new file mode 100644 index 0000000000000..692a4e9daf254 --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/lsm/SortMergeReader.java @@ -0,0 +1,41 @@ +/* + * 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.io.lsm; + +import org.apache.hudi.common.model.HoodieRecord; + +import java.util.Comparator; +import java.util.List; + +public interface SortMergeReader extends RecordReader { + + // todo zhangyue143 实现 LOSER_TREE + static SortMergeReader createSortMergeReader(List> readers, + Comparator userKeyComparator, + RecordMergeWrapper mergeFunctionWrapper, + SortEngine sortEngine) { + switch (sortEngine) { + case LOSER_TREE: + return new SortMergeReaderLoserTreeStateMachine<>(readers, userKeyComparator, mergeFunctionWrapper); + + default: + throw new UnsupportedOperationException("Unsupported sort engine: " + sortEngine); + } + } +} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/lsm/SortMergeReaderLoserTreeStateMachine.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/lsm/SortMergeReaderLoserTreeStateMachine.java new file mode 100644 index 0000000000000..6d65184717a39 --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/lsm/SortMergeReaderLoserTreeStateMachine.java @@ -0,0 +1,402 @@ +/* + * 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.io.lsm; + +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.exception.HoodieIOException; + +import java.io.IOException; +import java.util.Arrays; +import java.util.Comparator; +import java.util.Iterator; +import java.util.List; +import java.util.NoSuchElementException; +import java.util.stream.Collectors; + +public class SortMergeReaderLoserTreeStateMachine implements SortMergeReader { + + private final Comparator comparator; + private final RecordMergeWrapper mergeFunctionWrapper; + private final List> readers; + private final List> iterables; + + public SortMergeReaderLoserTreeStateMachine(List> readers, + Comparator userKeyComparator, + RecordMergeWrapper mergeFunctionWrapper) { + this.comparator = userKeyComparator; + this.readers = readers; + this.mergeFunctionWrapper = mergeFunctionWrapper; + + this.iterables = readers.stream().map(reader -> { + try { + return reader.read(); + } catch (IOException e) { + throw new HoodieIOException(e.getMessage(), e); + } + }).collect(Collectors.toList()); + } + + @Override + public Iterator read() { + return new SortMergeReaderLoserTreeIterator(iterables, comparator); + } + + @Override + public void close() throws IOException { + for (RecordReader reader : readers) { + reader.close(); + } + } + + // 败者树节点状态 + protected enum State { + WINNER_WITH_NEW_KEY, + WINNER_WITH_SAME_KEY, + WINNER_POPPED, + LOSER_WITH_NEW_KEY, + LOSER_WITH_SAME_KEY, + LOSER_POPPED + } + + // 败者树节点结构 + protected static class LoserTreeLeafNode { + HoodieRecord record; + int sourceIndex; // 输入流Index + State state; + int firstSameKeyIndex = -1; + + public LoserTreeLeafNode(HoodieRecord record, int sourceIndex, State state, int firstSameKeyIndex) { + this.record = record; + this.sourceIndex = sourceIndex; + this.state = state; + this.firstSameKeyIndex = firstSameKeyIndex; + } + + public void setFirstSameKeyIndex(int index) { + if (firstSameKeyIndex == -1) { + firstSameKeyIndex = index; + } + } + } + + // 败者树 + protected static class LoserTree { + protected final int[] tree; // tree[0]存储冠军下标,tree[1...k-1]存储败者 + protected final int k; // 叶子节点(即输入序列)的数量 + protected final Comparator comparator; + protected final LoserTreeLeafNode[] leaves; + + public LoserTree(List> iterators, Comparator comparator) { + this.k = iterators.size(); + this.leaves = new LoserTreeLeafNode[k]; // 叶子当前元素 + this.tree = new int[k]; + this.comparator = comparator; + + // 初始化叶子,取出第一个元素,包装成 `LoserTreeNode`(元素值和序列编号) + // 叶子节点状态初始化为 `WINNER_WITH_NEW_KEY` + for (int i = 0; i < k; i++) { + Iterator iter = iterators.get(i); + if (iter != null && iter.hasNext()) { + leaves[i] = new LoserTreeLeafNode(iter.next(), i, State.WINNER_WITH_NEW_KEY, -1); + } else { + leaves[i] = null; + } + } + + // 初始化节点 + Arrays.fill(tree, -1); + for (int i = 0; i < k; i++) { + adjust(i); + } + } + + // 调整第index个叶子节点 + // 叶子节点和父节点比较, 败者留在父节点位置, 胜者继续和父节点的父节点比较,直到整棵树的根节点 + protected void adjust(int index) { + int parent = (index + k) >> 1; // 父节点下标 + int winner = index; // winner 需要向上调整的叶子结点索引 + LoserTreeLeafNode parentNode; + while (parent > 0) { + if (tree[parent] == -1) { // 父节点还未初始化 + tree[parent] = winner; + if (leaves[winner] != null) { + leaves[winner].state = State.LOSER_WITH_NEW_KEY; // 更改对应叶子结点的state + } + winner = -1; + break; // 父节点为空 + } else { + parentNode = leaves[tree[parent]]; + // null值判断 + if (parentNode == null && leaves[winner] == null) { + parent >>= 1; + continue; + } else if (leaves[winner] == null && parentNode != null) { // parent胜出 + if (parentNode.state == State.LOSER_POPPED) { // 相当于照搬adjustWinnerWithNewKey逻辑,提前设置state + parentNode.state = State.WINNER_POPPED; + parentNode.firstSameKeyIndex = -1; + } else { + if (parentNode.state != State.WINNER_WITH_NEW_KEY && parentNode.state != State.WINNER_WITH_SAME_KEY && parentNode.state != State.WINNER_POPPED) { + parentNode.state = State.WINNER_WITH_NEW_KEY; + } + } + } else if (leaves[winner] != null && parentNode == null) { // winner胜出 + if (leaves[winner].state != State.WINNER_WITH_NEW_KEY && leaves[winner].state != State.WINNER_WITH_SAME_KEY && leaves[winner].state != State.WINNER_POPPED) { + leaves[winner].state = State.WINNER_WITH_NEW_KEY; + } + } else { + switch (leaves[winner].state) { + case WINNER_WITH_NEW_KEY: + adjustWinnerWithNewKey(parent, winner, parentNode, leaves[winner]); + break; + case WINNER_WITH_SAME_KEY: + adjustWinnerWithSameKey(parent, winner, parentNode, leaves[winner]); + break; + case WINNER_POPPED: // 是否有相同的未处理 UserKey 节点 + if (leaves[winner].firstSameKeyIndex < 0) { + // 当前 winnerNode 没有另一个相同 key、尚未处理完的叶子节点 + parent = -1; // 结束循环 + } else { + // 为“不同叶子”的同 key 做快速跳转 + parent = leaves[winner].firstSameKeyIndex; + parentNode = leaves[tree[parent]]; + leaves[winner].state = State.LOSER_POPPED; + parentNode.state = State.WINNER_WITH_SAME_KEY; + } + break; + default: + throw new UnsupportedOperationException( + "unknown state for " + leaves[winner].state.name()); + } + } + } + + if (parent > 0) { + int loser = tree[parent]; + // if the winner loses, exchange nodes. + if (leaves[winner] == null || leaves[winner].state == State.LOSER_WITH_SAME_KEY || leaves[winner].state == State.LOSER_WITH_NEW_KEY || leaves[winner].state == State.LOSER_POPPED) { + int tmp = winner; + winner = loser; + tree[parent] = tmp; + } + parent >>= 1; + } + } + tree[0] = winner; + } + + protected void adjustWinnerWithNewKey(int parent, int winner, LoserTreeLeafNode parentNode, LoserTreeLeafNode winnerNode) { + switch (parentNode.state) { + case LOSER_WITH_NEW_KEY: + int loser = tree[parent]; + int compareResult = compare(winner, loser); + if (compareResult < 0) { // 子节点胜 + parentNode.state = State.LOSER_WITH_NEW_KEY; + } else if (compareResult > 0) { // 父节点胜 + winnerNode.state = State.LOSER_WITH_NEW_KEY; + parentNode.state = State.WINNER_WITH_NEW_KEY; + } else { // key相等,比较叶子索引,小的优先 + if (winner > loser) { // 索引大、判为loser + winnerNode.state = State.LOSER_WITH_SAME_KEY; + parentNode.state = State.WINNER_WITH_NEW_KEY; + parentNode.setFirstSameKeyIndex(parent); + } else { + parentNode.state = State.LOSER_WITH_SAME_KEY; + winnerNode.setFirstSameKeyIndex(parent); + } + } + return; + case LOSER_WITH_SAME_KEY: + throw new RuntimeException( + "This is a bug. Please file an issue. A node in the WINNER_WITH_NEW_KEY " + + "state cannot encounter a node in the LOSER_WITH_SAME_KEY state."); + case LOSER_POPPED: + parentNode.state = State.WINNER_POPPED; + parentNode.firstSameKeyIndex = -1; + winnerNode.state = State.LOSER_WITH_NEW_KEY; + return; + default: + throw new UnsupportedOperationException( + "unknown state for " + parentNode.state.name()); + } + } + + protected void adjustWinnerWithSameKey(int parent, int winner, LoserTreeLeafNode parentNode, LoserTreeLeafNode winnerNode) { + switch (parentNode.state) { + case LOSER_WITH_NEW_KEY: + return; + case LOSER_POPPED: + return; + case LOSER_WITH_SAME_KEY: // Key相同,只需要比较两个节点的索引 + int loser = tree[parent]; + if (winner > loser) { // 索引大、判为loser + winnerNode.state = State.LOSER_WITH_SAME_KEY; + parentNode.state = State.WINNER_WITH_SAME_KEY; + parentNode.setFirstSameKeyIndex(parent); + } else { + parentNode.state = State.LOSER_WITH_SAME_KEY; // 可删除,因状态没变 + winnerNode.setFirstSameKeyIndex(parent); + } + return; + default: + throw new UnsupportedOperationException( + "unknown state for " + parentNode.state.name()); + } + } + + // 返回结果 正数为index2胜出 负数为index1胜出 0为相等 + private int compare(int index1, int index2) { + if (index1 == -1) { + return 1; + } + if (index2 == -1) { + return -1; + } + if (leaves[index1] == null) { + return 1; // 1为空,2胜出 + } + if (leaves[index2] == null) { + return -1; + } + return comparator.compare(leaves[index1].record, leaves[index2].record); + } + + // pop获胜节点,补充对应元素 + protected void popAdvance(List> iterators) { + LoserTreeLeafNode winner = leaves[tree[0]]; // 获胜节点 + while (winner != null && winner.state == State.WINNER_POPPED) { + if (tree[0] == -1) { + return; + } + if (iterators.get(tree[0]).hasNext()) { // 输入序列有值 + leaves[tree[0]] = new LoserTreeLeafNode(iterators.get(tree[0]).next(), tree[0], State.WINNER_WITH_NEW_KEY, -1); + } else { + leaves[tree[0]] = null; + } + adjust(tree[0]); // 调整 + winner = leaves[tree[0]]; + } + } + + protected HoodieRecord popWinner() { + LoserTreeLeafNode winner = leaves[tree[0]]; // 获胜节点 + if (winner == null) { + return null; + } + if (winner.state == State.WINNER_POPPED) { + // 如果winner的状态已经是WINNER_POPPED,说明本组key已经处理过了,直接返回null + return null; + } + winner.state = State.WINNER_POPPED; + HoodieRecord winnerRecord = winner.record; + + // 重新调整获胜的叶子节点,检查是否还有相同key未处理 + adjust(tree[0]); + return winnerRecord; + } + + // 查看但不弹出当前败者树的胜者 + protected HoodieRecord peekWinner() { + if (tree[0] == -1) { + return null; + } + LoserTreeLeafNode winner = leaves[tree[0]]; + if (winner == null || winner.state == State.WINNER_POPPED) { + return null; + } + return winner.record; + } + + public boolean peekNull() { + LoserTreeLeafNode winner = leaves[tree[0]]; + return winner != null; + } + } + + private class SortMergeReaderLoserTreeIterator implements Iterator { + private final List> iterators; + private final LoserTree loserTree; + private T nextRecord; + + // Merge迭代器 + public SortMergeReaderLoserTreeIterator(List> iterables, Comparator comparator) { + this.iterators = iterables; + this.loserTree = new LoserTree(iterators, comparator); + } + + @Override + public boolean hasNext() { + nextRecord = getNextRecord(); + return nextRecord != null; + } + + @Override + public T next() { + // 如果 nextRecord 没值,说明没有调用过hasNext方法,需要提前抛出异常 + if (nextRecord == null) { + throw new NoSuchElementException(); + } + T resultRecord = nextRecord; + nextRecord = null; + return resultRecord; + } + + public T getNextRecord() { + if (!hasNextWinner()) { + return null; + } + + Option result = null; + while (hasNextWinner()) { + // 判断是否是WINNER_POPPED,是的话,补充弹出后的叶子值 + loserTree.popAdvance(iterators); + // 判断是否处理相同key,如WINNER_POPPED,则已经处理过了 + HoodieRecord winnerRecord = loserTree.popWinner(); + if (winnerRecord == null) { + break; + } + + mergeFunctionWrapper.merge(winnerRecord); + while (loserTree.peekWinner() != null) { // 当前胜者没有相同key待处理,不需要增加sameKeyRecords + mergeFunctionWrapper.merge(loserTree.popWinner()); + } + result = mergeFunctionWrapper.getMergedResult(); + mergeFunctionWrapper.reset(); + + if (result.isPresent()) { + break; + } + } + + if (!result.isPresent()) { + // 如果最后真的没有这条数据 说明来自delete数据 + return null; + } + + return result.get(); + + } + + private boolean hasNextWinner() { + loserTree.popAdvance(iterators); + return loserTree.peekNull(); + } + } +} + diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/LSMCleanPlanActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/LSMCleanPlanActionExecutor.java new file mode 100644 index 0000000000000..46a74b2c50195 --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/LSMCleanPlanActionExecutor.java @@ -0,0 +1,96 @@ +/* + * 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.action.clean; + +import org.apache.hudi.avro.model.HoodieActionInstant; +import org.apache.hudi.avro.model.HoodieCleanFileInfo; +import org.apache.hudi.avro.model.HoodieCleanerPlan; +import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.model.HoodieCleaningPolicy; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.util.CollectionUtils; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.exception.HoodieIOException; +import org.apache.hudi.table.HoodieTable; + +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +public class LSMCleanPlanActionExecutor extends CleanPlanActionExecutor { + + private static final Logger LOG = LogManager.getLogger(CleanPlanner.class); + + private final Option> extraMetadata; + private final HoodieTableMetaClient metaClient; + + public LSMCleanPlanActionExecutor(HoodieEngineContext context, + HoodieWriteConfig config, + HoodieTable table, + String instantTime, + Option> extraMetadata) { + super(context, config, table, instantTime, extraMetadata); + this.extraMetadata = extraMetadata; + this.metaClient = table.getMetaClient(); + } + + /** + * Generates List of files to be cleaned. + * + * @param context HoodieEngineContext + * @return Cleaner Plan + */ + @Override + HoodieCleanerPlan requestClean(HoodieEngineContext context) { + try { + LSMCleanPlanner planner = new LSMCleanPlanner<>(context, table, config); + Option earliestInstant = planner.getEarliestCommitToRetain(); + context.setJobStatus(this.getClass().getSimpleName(), "Obtaining list of partitions to be cleaned: " + config.getTableName()); + if (metaClient.getActiveTimeline().getCompletedReplaceTimeline().getInstants().isEmpty()) { + LOG.info("Nothing to clean here. It is already clean"); + return HoodieCleanerPlan.newBuilder().setPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS.name()).build(); + } + + LOG.info("Earliest commit to retain for clean : " + (earliestInstant.isPresent() ? earliestInstant.get().getTimestamp() : "null")); + Map> cleanOps = planner.getDeletePaths(); + return new HoodieCleanerPlan(earliestInstant + .map(x -> new HoodieActionInstant(x.getTimestamp(), x.getAction(), x.getState().name())).orElse(null), + planner.getLastCompletedCommitTimestamp(), + config.getCleanerPolicy().name(), CollectionUtils.createImmutableMap(), + CleanPlanner.LATEST_CLEAN_PLAN_VERSION, cleanOps, new ArrayList<>()); + } catch (IOException e) { + throw new HoodieIOException("Failed to schedule lsm clean operation", e); + } + } + + @Override + public Option execute() { + if (!needsCleaning(config.getCleaningTriggerStrategy())) { + return Option.empty(); + } + // Plan a new clean action + return requestClean(instantTime); + } +} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/LSMCleanPlanner.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/LSMCleanPlanner.java new file mode 100644 index 0000000000000..029c849ee1757 --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/LSMCleanPlanner.java @@ -0,0 +1,215 @@ +/* + * 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.action.clean; + +import org.apache.hudi.avro.model.HoodieCleanFileInfo; +import org.apache.hudi.avro.model.HoodieCleanMetadata; +import org.apache.hudi.common.data.HoodieData; +import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.fs.HoodieWrapperFileSystem; +import org.apache.hudi.common.model.CleanFileInfo; +import org.apache.hudi.common.model.HoodieCleaningPolicy; +import org.apache.hudi.common.model.HoodieReplaceCommitMetadata; +import org.apache.hudi.common.storage.HoodieStorageStrategy; +import org.apache.hudi.common.storage.HoodieStorageStrategyFactory; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.table.timeline.TimelineMetadataUtils; +import org.apache.hudi.common.table.view.SyncableFileSystemView; +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.HoodieWriteConfig; +import org.apache.hudi.exception.HoodieIOException; +import org.apache.hudi.table.HoodieTable; + +import org.apache.hadoop.fs.Path; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; + +import java.io.FileNotFoundException; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Comparator; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; + +public class LSMCleanPlanner extends CleanPlanner { + + private static final Logger LOG = LogManager.getLogger(LSMCleanPlanner.class); + // List + private final List pendingReplaceFiles; + private final HoodieStorageStrategy storageStrategy; + private final HoodieTableMetaClient metaClient; + private final int cleanParallelism; + + public LSMCleanPlanner(HoodieEngineContext context, HoodieTable hoodieTable, HoodieWriteConfig config) { + super(context, hoodieTable, config); + ValidationUtils.checkArgument(!config.getCleanerPolicy().equals(HoodieCleaningPolicy.KEEP_LATEST_BY_HOURS), + "Not support KEEP_LATEST_BY_HOURS clean policy for now"); + SyncableFileSystemView fileSystemView = (SyncableFileSystemView) hoodieTable.getSliceView(); + this.pendingReplaceFiles = fileSystemView.getFileGroupsInPendingClustering().map(group -> { + String partitionPath = group.getLeft().getPartitionPath(); + String filename = group.getLeft().getFileId(); + return partitionPath + filename; + }).collect(Collectors.toList()); + this.metaClient = hoodieTable.getMetaClient(); + this.storageStrategy = HoodieStorageStrategyFactory.getInstant(metaClient); + this.cleanParallelism = config.getCleanerParallelism(); + } + + // partitionPath -> List + // TODO zhangyue143 only support KEEP_LATEST_COMMITS for now + public Map> getDeletePaths() throws IOException { + HoodieCleaningPolicy policy = config.getCleanerPolicy(); + Map> deletePaths; + if (policy == HoodieCleaningPolicy.KEEP_LATEST_COMMITS) { + deletePaths = getFilesToCleanKeepingLatestCommits(); + } else { + throw new IllegalArgumentException("Unknown cleaning policy : " + policy.name()); + } + return deletePaths; + } + + // TODO zhangyue143 take care of savepoint + private Map> getFilesToCleanKeepingLatestCommits() throws IOException { + int commitsRetained = config.getCleanerCommitsRetained(); + LOG.info("Retaining latest " + commitsRetained + " commits."); + Map> maps = new HashMap<>(); + if (commitTimeline.countInstants() > commitsRetained) { + // List>>> + List>>>> replacedGroups = getFileGroupsReplaced(metaClient.getActiveTimeline()); + replacedGroups.stream().flatMap(ele -> ele.getRight().stream()).forEach(ele -> { + String partitionPath = ele.getLeft(); + List infos = ele.getRight().stream().filter(path -> { + String key = partitionPath + (new Path(path)).getName(); + return !pendingReplaceFiles.contains(key); + }).map(path -> { + return (new CleanFileInfo(path, false)).toHoodieFileCleanInfo(); + }).collect(Collectors.toList()); + if (maps.containsKey(partitionPath)) { + maps.get(partitionPath).addAll(infos); + } else { + maps.put(partitionPath, infos); + } + }); + } + return maps; + } + + // List>>>> + private List>>>> getFileGroupsReplaced(HoodieTimeline timeline) throws IOException { + Option earliestCommitToRetainOption = getEarliestCommitToRetain(); + if (!earliestCommitToRetainOption.isPresent()) { + return new ArrayList<>(); + } + + // for each REPLACE instant, get map of (partitionPath -> deleteFileGroup) + HoodieTimeline replacedTimeline = timeline.getCompletedReplaceTimeline(); + List backtrackInstances = metaClient.getBacktrackInstances(); + List replaceCommitInstantToClean = getReplaceInstantsToClean(replacedTimeline, earliestCommitToRetainOption.get(), backtrackInstances); + + if (replaceCommitInstantToClean.size() == 0) { + return new ArrayList<>(); + } + + List> metas = replaceCommitInstantToClean.stream().map(instant -> { + HoodieReplaceCommitMetadata replaceMetadata = null; + try { + replaceMetadata = HoodieReplaceCommitMetadata.fromBytes(metaClient.getActiveTimeline().getInstantDetails(instant).get(), + HoodieReplaceCommitMetadata.class); + } catch (IOException e) { + throw new HoodieIOException(e.getMessage(), e); + } + return Pair.of(instant, replaceMetadata); + }).collect(Collectors.toList()); + + //TODO zhangyue143 clean 与缓冲层适配测试 + HoodieData>>>> finalGroups = context.parallelize(metas, Math.max(metas.size(), cleanParallelism)).mapPartitions(iterator -> { + List>>>> res = new ArrayList<>(); + iterator.forEachRemaining(pair -> { + HoodieWrapperFileSystem innerFS = metaClient.getFs(); + HoodieInstant instant = pair.getLeft(); + HoodieReplaceCommitMetadata meta = pair.getRight(); + List>> collect = meta.getPartitionToReplaceFileIds().entrySet().stream().map(entry -> { + String partitionPath = entry.getKey(); + List groups = entry.getValue().stream().map(name -> { + return storageStrategy.getAllLocations(partitionPath + "/" + name, false); + }).flatMap(Set::stream).filter(path -> { + // 过滤不存在的文件 + try { + // getFileStatus 比list调用更轻量,不会对NN造成太大压力 + innerFS.getFileStatus(path); + return true; + } catch (FileNotFoundException e) { + return false; + } catch (Exception ioe) { + // 若发生任何其他异常,则直接判定文件是存在的,后续尝试删除(宁错杀不放过) + return true; + } + }).map(Path::toString).collect(Collectors.toList()); + return Pair.of(partitionPath, groups); + }).collect(Collectors.toList()); + + res.add(Pair.of(instant, collect)); + }); + return res.iterator(); + }, true); + List>>>> list = new ArrayList<>(finalGroups.collectAsList()); + list.sort((new Comparator>>>>() { + @Override + public int compare(Pair>>> o1, Pair>>> o2) { + return o1.getLeft().compareTo(o2.getLeft()); + } + }).reversed()); + return list; + } + + public List getReplaceInstantsToClean(HoodieTimeline replacedTimeline, HoodieInstant earliestCommitToRetain, List backtrackInstances) throws IOException { + if (config.incrementalCleanerModeEnabled()) { + Option lastClean = metaClient.getActiveTimeline().getCleanerTimeline().filterCompletedInstants().lastInstant(); + if (lastClean.isPresent()) { + if (metaClient.getActiveTimeline().isEmpty(lastClean.get())) { + metaClient.getActiveTimeline().deleteEmptyInstantIfExists(lastClean.get()); + } else { + HoodieCleanMetadata cleanMetadata = TimelineMetadataUtils + .deserializeHoodieCleanMetadata(metaClient.getActiveTimeline().getInstantDetails(lastClean.get()).get()); + String earliestCommitToRetainFromLastClean = cleanMetadata.getEarliestCommitToRetain(); + if ((earliestCommitToRetainFromLastClean != null) + && (earliestCommitToRetainFromLastClean.length() > 0) + && !metaClient.getActiveTimeline().isBeforeTimelineStarts(earliestCommitToRetainFromLastClean)) { + return replacedTimeline.getReverseOrderedInstants().filter(instant -> { + return HoodieTimeline.compareTimestamps(instant.getTimestamp(), HoodieTimeline.GREATER_THAN_OR_EQUALS, earliestCommitToRetainFromLastClean) + && instant.compareTo(earliestCommitToRetain) < 0 + && !backtrackInstances.contains(instant.getTimestamp()); + }).collect(Collectors.toList()); + } + } + } + } + + return replacedTimeline.getReverseOrderedInstants().filter(instant -> { + return instant.compareTo(earliestCommitToRetain) < 0 && !backtrackInstances.contains(instant.getTimestamp()); + }).collect(Collectors.toList()); + } +} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/ClusteringPlanActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/ClusteringPlanActionExecutor.java index 2445043e07c82..d324f81e42a73 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/ClusteringPlanActionExecutor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/ClusteringPlanActionExecutor.java @@ -38,6 +38,7 @@ import java.io.IOException; import java.util.Collections; +import java.util.List; import java.util.Map; public class ClusteringPlanActionExecutor extends BaseActionExecutor> { @@ -57,35 +58,90 @@ public ClusteringPlanActionExecutor(HoodieEngineContext context, protected Option createClusteringPlan() { LOG.info("Checking if clustering needs to be run on " + config.getBasePath()); + + List backtrackInstances = table.getMetaClient().getBacktrackInstances(); + + // get the pending insert overwrite instant + List pendingBackTrackInstances = table.getMetaClient().getPendingBacktrackInstances(); + int pendingClusteringNum = table.getActiveTimeline() + .filter(s -> s.getAction().equalsIgnoreCase(HoodieTimeline.REPLACE_COMMIT_ACTION) + && !backtrackInstances.contains(s.getTimestamp()) + && !pendingBackTrackInstances.contains(s.getTimestamp()) + && !s.isCompleted()) + .filter(this::canExecute) + .countInstants(); + + if (table.getMetaClient().getTableConfig().isLSMBasedLogFormat()) { + if (pendingClusteringNum >= config.getLsmMaxOfPendingClustering()) { + LOG.info(String.format("The num of lsm pending clustering is %s >= %s.", pendingClusteringNum, config.getLsmMaxOfPendingClustering())); + return Option.empty(); + } + } else { + if (pendingClusteringNum >= config.getMaxOfPendingClustering()) { + LOG.info(String.format("The num of pending clustering is %s >= %s.", pendingClusteringNum, config.getMaxOfPendingClustering())); + return Option.empty(); + } + } + Option lastClusteringInstant = table.getActiveTimeline() - .filter(s -> s.getAction().equalsIgnoreCase(HoodieTimeline.REPLACE_COMMIT_ACTION)).lastInstant(); + .filter(s -> s.getAction().equalsIgnoreCase(HoodieTimeline.REPLACE_COMMIT_ACTION) + && !backtrackInstances.contains(s.getTimestamp()) + && !pendingBackTrackInstances.contains(s.getTimestamp()) + && canExecute(s)) + .lastInstant(); int commitsSinceLastClustering = table.getActiveTimeline().getCommitsTimeline().filterCompletedInstants() .findInstantsAfter(lastClusteringInstant.map(HoodieInstant::getTimestamp).orElse("0"), Integer.MAX_VALUE) .countInstants(); - if (config.inlineClusteringEnabled() && config.getInlineClusterMaxCommits() > commitsSinceLastClustering) { - LOG.info("Not scheduling inline clustering as only " + commitsSinceLastClustering - + " commits was found since last clustering " + lastClusteringInstant + ". Waiting for " - + config.getInlineClusterMaxCommits()); - return Option.empty(); - } + ClusteringPlanStrategy strategy; + if (table.getMetaClient().getTableConfig().isLSMBasedLogFormat()) { + if (config.inlineLSMClusteringEnabled() && config.getLsmInlineClusterMaxCommits() > commitsSinceLastClustering) { + LOG.info("Not scheduling lsm inline clustering as only " + commitsSinceLastClustering + + " commits was found since last clustering " + lastClusteringInstant + ". Waiting for " + + config.getLsmInlineClusterMaxCommits()); + return Option.empty(); + } - if (config.isAsyncClusteringEnabled() && config.getAsyncClusterMaxCommits() > commitsSinceLastClustering) { - LOG.info("Not scheduling async clustering as only " + commitsSinceLastClustering - + " commits was found since last clustering " + lastClusteringInstant + ". Waiting for " - + config.getAsyncClusterMaxCommits()); - return Option.empty(); - } + if (config.isLsmAsyncClusteringScheduleEnabled() && config.getLsmAsyncClusterMaxCommits() > commitsSinceLastClustering) { + LOG.info("Not scheduling lsm async clustering as only " + commitsSinceLastClustering + + " commits was found since last clustering " + lastClusteringInstant + ". Waiting for " + + config.getLsmAsyncClusterMaxCommits()); + return Option.empty(); + } - LOG.info("Generating clustering plan for table " + config.getBasePath()); - ClusteringPlanStrategy strategy = (ClusteringPlanStrategy) ReflectionUtils.loadClass( - ClusteringPlanStrategy.checkAndGetClusteringPlanStrategy(config), - new Class[] {HoodieTable.class, HoodieEngineContext.class, HoodieWriteConfig.class}, table, context, config); + LOG.info("Generating clustering plan for table " + config.getBasePath()); + strategy = (ClusteringPlanStrategy) ReflectionUtils.loadClass( + config.getLsmClusteringPlanStrategyClass(), new Class[] {HoodieTable.class, HoodieEngineContext.class, HoodieWriteConfig.class}, + table, context, config); + } else { + if (config.inlineClusteringEnabled() && config.getInlineClusterMaxCommits() > commitsSinceLastClustering) { + LOG.info("Not scheduling inline clustering as only " + commitsSinceLastClustering + + " commits was found since last clustering " + lastClusteringInstant + ". Waiting for " + + config.getInlineClusterMaxCommits()); + return Option.empty(); + } + + if (config.isAsyncClusteringEnabled() && config.getAsyncClusterMaxCommits() > commitsSinceLastClustering) { + LOG.info("Not scheduling async clustering as only " + commitsSinceLastClustering + + " commits was found since last clustering " + lastClusteringInstant + ". Waiting for " + + config.getAsyncClusterMaxCommits()); + return Option.empty(); + } + + LOG.info("Generating clustering plan for table " + config.getBasePath()); + strategy = (ClusteringPlanStrategy) ReflectionUtils.loadClass( + ClusteringPlanStrategy.checkAndGetClusteringPlanStrategy(config), + new Class[] {HoodieTable.class, HoodieEngineContext.class, HoodieWriteConfig.class}, table, context, config); + } return strategy.generateClusteringPlan(); } + protected boolean canExecute(HoodieInstant instant) { + return true; + } + @Override public Option execute() { Option planOption = createClusteringPlan(); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/SparkClusteringPlanActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/SparkClusteringPlanActionExecutor.java new file mode 100644 index 0000000000000..9ebcca7fa579e --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/SparkClusteringPlanActionExecutor.java @@ -0,0 +1,73 @@ +/* + * 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.action.cluster; + +import org.apache.hudi.avro.model.HoodieRequestedReplaceMetadata; +import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.table.timeline.TimelineMetadataUtils; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.config.HoodieClusteringConfig; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.table.HoodieTable; + +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; + +import java.io.IOException; +import java.util.Map; + +public class SparkClusteringPlanActionExecutor extends ClusteringPlanActionExecutor { + private static final Logger LOG = LogManager.getLogger(SparkClusteringPlanActionExecutor.class); + private static final String SPARK_PERSIST_STORAGE_FULL_CLASS_NAME + = "org.apache.hudi.client.clustering.plan.strategy.SparkSizeBasedClusteringPlanStrategyWithPersistStorage"; + public SparkClusteringPlanActionExecutor(HoodieEngineContext context, HoodieWriteConfig config, HoodieTable table, + String instantTime, Option> extraMetadata) { + super(context, config, table, instantTime, extraMetadata); + } + + @Override + protected boolean canExecute(HoodieInstant instant) { + boolean persistStorageConf = isPersistStorageConf(config); + return persistStorageConf == isPersistInstant(instant, table.getMetaClient()); + } + + private boolean isPersistInstant(HoodieInstant instant, HoodieTableMetaClient metaClient) { + try { + HoodieRequestedReplaceMetadata replaceMetadata = TimelineMetadataUtils + .deserializeRequestedReplaceMetadata(metaClient.getActiveTimeline().getInstantDetails(HoodieTimeline.getReplaceCommitRequestedInstant(instant.getTimestamp())).get()); + if (replaceMetadata.getClusteringPlan() == null) { + return false; + } + Map strategyParams = replaceMetadata.getClusteringPlan().getStrategy().getStrategyParams(); + return strategyParams.containsKey(HoodieClusteringConfig.PLAN_STRATEGY_CLASS_NAME.key()) + && SPARK_PERSIST_STORAGE_FULL_CLASS_NAME.equals(strategyParams.get(HoodieClusteringConfig.PLAN_STRATEGY_CLASS_NAME.key())); + } catch (IOException e) { + LOG.warn("failed to deserialize replacecommit metadata.", e); + return false; + } + } + + private boolean isPersistStorageConf(HoodieWriteConfig config) { + return config.contains(HoodieClusteringConfig.PLAN_STRATEGY_CLASS_NAME) + && SPARK_PERSIST_STORAGE_FULL_CLASS_NAME.equals(config.getString(HoodieClusteringConfig.PLAN_STRATEGY_CLASS_NAME)); + } +} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/lsm/DataFile.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/lsm/DataFile.java new file mode 100644 index 0000000000000..66720d295592d --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/lsm/DataFile.java @@ -0,0 +1,76 @@ +/* + * 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.action.cluster.lsm; + +import org.apache.hudi.common.model.HoodieLogFile; + +public class DataFile { + + private final String min; + + private final String max; + private final String fileFullPath; + private final long dataSize; + + private final int levelNumber; + private final HoodieLogFile logFile; + + public DataFile(int levelNumber, String min, String max, long dataSize, String fileFullPath, HoodieLogFile logfile) { + this.levelNumber = levelNumber; + this.min = min; + this.max = max; + this.dataSize = dataSize; + this.fileFullPath = fileFullPath; + this.logFile = logfile; + } + + public String getMax() { + return max; + } + + public String getMin() { + return min; + } + + public String getPath() { + return fileFullPath; + } + + public long getDataSize() { + return dataSize; + } + + public int getLevelNumber() { + return levelNumber; + } + + public HoodieLogFile getLogFile() { + return this.logFile; + } + + @Override + public String toString() { + return String.format( + "{filePath: %s, fileSize: %d, min: %s, max: %s}", + fileFullPath, + dataSize, + min, + max); + } +} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/lsm/LsmClusteringUtils.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/lsm/LsmClusteringUtils.java new file mode 100644 index 0000000000000..45520320cde47 --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/lsm/LsmClusteringUtils.java @@ -0,0 +1,78 @@ +/* + * 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.action.cluster.lsm; + +import org.apache.hudi.common.model.lsm.HoodieLSMLogFile; +import org.apache.hudi.common.util.BaseFileUtils; + +import org.apache.avro.Schema; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.Path; + +import java.util.EnumSet; + +public class LsmClusteringUtils { + + private static final EnumSet NUMERIC_TYPES = EnumSet.of( + Schema.Type.INT, + Schema.Type.LONG, + Schema.Type.FLOAT, + Schema.Type.DOUBLE + ); + + public static DataFile buildDataFile(BaseFileUtils parquetUtils, HoodieLSMLogFile logFile, Configuration conf) { + if (logFile.getMax() != null) { + return new DataFile(logFile.getLevelNumber(), logFile.getMin(), logFile.getMax(), logFile.getFileSize(), logFile.getPath().toString(), logFile); + } else { + FileStatus logStatus = logFile.getFileStatus(); + Path logPath = logStatus.getPath(); + String[] minMax = parquetUtils.readMinMaxRecordKeys(conf, logPath); + int levelNumber = logFile.getLevelNumber(); + return new DataFile(levelNumber, minMax[0], minMax[1], logStatus.getLen(), logPath.toUri().toString(), logFile); + } + } + + public static boolean isNumericType(Schema.Field field) { + if (field == null) { + return false; + } + + Schema fieldSchema = field.schema(); + Schema.Type fieldType = fieldSchema.getType(); + + // 处理单类型和union类型 + if (fieldType == Schema.Type.UNION) { + // 检查union中的所有可能类型 + for (Schema unionType : fieldSchema.getTypes()) { + if (isPrimitiveNumericType(unionType.getType())) { + return true; + } + } + return false; + } else { + // 处理非union类型 + return isPrimitiveNumericType(fieldType); + } + } + + private static boolean isPrimitiveNumericType(Schema.Type type) { + return NUMERIC_TYPES.contains(type); + } +} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/strategy/ClusteringPlanStrategy.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/strategy/ClusteringPlanStrategy.java index e55e900c3cedb..c8bb7ac71adae 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/strategy/ClusteringPlanStrategy.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/strategy/ClusteringPlanStrategy.java @@ -54,7 +54,7 @@ public abstract class ClusteringPlanStrategy implements Serializable { public static final int CLUSTERING_PLAN_VERSION_1 = 1; - private final HoodieTable hoodieTable; + protected final HoodieTable hoodieTable; private final transient HoodieEngineContext engineContext; private final HoodieWriteConfig writeConfig; diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/strategy/LsmBaseClusteringPlanStrategy.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/strategy/LsmBaseClusteringPlanStrategy.java new file mode 100644 index 0000000000000..4ae2cb4f471f6 --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/strategy/LsmBaseClusteringPlanStrategy.java @@ -0,0 +1,716 @@ +/* + * 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.action.cluster.strategy; + +import org.apache.hudi.avro.HoodieAvroUtils; +import org.apache.hudi.avro.model.HoodieClusteringGroup; +import org.apache.hudi.avro.model.HoodieClusteringPlan; +import org.apache.hudi.avro.model.HoodieClusteringStrategy; +import org.apache.hudi.avro.model.HoodieSliceInfo; +import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.model.FileSlice; +import org.apache.hudi.common.model.HoodieCommitMetadata; +import org.apache.hudi.common.model.HoodieFileFormat; +import org.apache.hudi.common.model.HoodieFileGroupId; +import org.apache.hudi.common.model.HoodieLogFile; +import org.apache.hudi.common.model.HoodieWriteStat; +import org.apache.hudi.common.model.lsm.HoodieLSMLogFile; +import org.apache.hudi.common.table.HoodieTableMetaClient; +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.common.table.timeline.TimelineUtils; +import org.apache.hudi.common.table.view.SyncableFileSystemView; +import org.apache.hudi.common.util.BaseFileUtils; +import org.apache.hudi.common.util.ClusteringUtils; +import org.apache.hudi.common.util.CollectionUtils; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.StringUtils; +import org.apache.hudi.common.util.ValidationUtils; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.config.HoodieClusteringConfig; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.exception.HoodieIOException; +import org.apache.hudi.table.HoodieTable; +import org.apache.hudi.table.action.cluster.lsm.DataFile; +import org.apache.hudi.table.action.cluster.lsm.LsmClusteringUtils; + +import org.apache.avro.Schema; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.Comparator; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.TreeSet; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static org.apache.hudi.common.table.timeline.HoodieTimeline.DELTA_COMMIT_ACTION; + +/** + * Scheduling strategy with restriction that clustering groups can only contain files from same partition. + */ +public class LsmBaseClusteringPlanStrategy extends ClusteringPlanStrategy { + private static final Logger LOG = LogManager.getLogger(LsmBaseClusteringPlanStrategy.class); + private HoodieTableMetaClient metaClient; + public static final BaseFileUtils PARQUET_UTILS = BaseFileUtils.getInstance(HoodieFileFormat.PARQUET); + protected final transient Configuration conf; + private final int numRunClusteringTrigger; + private final int maxSizeAmp; + protected boolean skipOP; + protected final long clusteringSmallFileLimit; + protected final long clusteringMaxBytesInGroup; + private final boolean numeric; + + protected Option plan; + protected transient Schema schema; + + public LsmBaseClusteringPlanStrategy(HoodieTable table, HoodieEngineContext engineContext, HoodieWriteConfig writeConfig) { + super(table, engineContext, writeConfig); + this.metaClient = table.getMetaClient(); + this.conf = engineContext.getHadoopConf().get(); + this.numRunClusteringTrigger = getWriteConfig().getLsmNunRunClusteringTrigger(); + this.maxSizeAmp = getWriteConfig().getLsmMaxSizeAMP(); + this.skipOP = !getWriteConfig().isLsmReadFooterEnabled(); + this.clusteringSmallFileLimit = getWriteConfig().getLsmClusteringSmallFileLimit(); + this.clusteringMaxBytesInGroup = getWriteConfig().getLsmClusteringMaxBytesInGroup(); + // get schema from write config + String schemaStr = getWriteConfig().getSchema(); + this.schema = StringUtils.isNullOrEmpty(schemaStr) ? null : HoodieAvroUtils.createHoodieWriteSchema(schemaStr); + this.numeric = this.isHoodieRecordKeyNumeric(); + } + + public LsmBaseClusteringPlanStrategy(HoodieTable table, HoodieEngineContext engineContext, HoodieWriteConfig writeConfig, Schema schema) { + super(table, engineContext, writeConfig); + this.metaClient = table.getMetaClient(); + this.conf = engineContext.getHadoopConf().get(); + this.numRunClusteringTrigger = getWriteConfig().getLsmNunRunClusteringTrigger(); + this.maxSizeAmp = getWriteConfig().getLsmMaxSizeAMP(); + this.skipOP = !getWriteConfig().isLsmReadFooterEnabled(); + this.clusteringSmallFileLimit = getWriteConfig().getLsmClusteringSmallFileLimit(); + this.clusteringMaxBytesInGroup = getWriteConfig().getLsmClusteringMaxBytesInGroup(); + this.schema = schema; + this.numeric = this.isHoodieRecordKeyNumeric(); + } + + /** + * Create Clustering group based on files eligible for clustering in the partition. + * @param fileSlices all file slice of the partition + * @return Pair of missing partition and clustering groups + */ + public Stream> buildClusteringGroupsForPartition(String partitionPath, + List fileSlices, + Set level1, + Set level1InPending, + boolean skipOP) { + Stream> hoodieClusteringGroupStream = fileSlices.stream().flatMap(fileSlice -> { + Pair> partitionClusteringGroupPair = + buildClusteringGroups(fileSlice.getLogFiles().collect(Collectors.toList()), level1.contains(fileSlice.getFileId()), + level1InPending.contains(fileSlice.getFileId()), partitionPath, skipOP); + return partitionClusteringGroupPair.getRight().map(group -> { + return partitionClusteringGroupPair.getLeft() ? Pair.of(partitionPath, group) : Pair.of("", group); + }); + }); + return hoodieClusteringGroupStream; + } + + /** + * Create Clustering group based on files slice for clustering in the file slice. + * @return Pair of partial schedule and clustering groups + */ + public Pair> buildClusteringGroups(List logfiles, boolean level1Existed, boolean level1InPending, String partitionPath, boolean skipOP) { + // Add a Boolean variable to indicate whether all log files on the partition are participating in clustering. + List amp = pickForSizeAmp(logfiles, level1Existed, level1InPending); + if (amp != null) { + return Pair.of(false, majorClustering(amp, partitionPath)); + } + + List filesWithoutLevel1 = pickMinorClustering(logfiles); + if (filesWithoutLevel1 != null) { + return Pair.of(false, minorClustering(filesWithoutLevel1, partitionPath, skipOP)); + } + + return Pair.of(false, Stream.empty()); + } + + protected Stream minorClustering(List filesWithoutLevel1, String partitionPath, boolean skipOP) { + if (skipOP) { + return minorClusteringWithoutGroup(filesWithoutLevel1, partitionPath); + } + + List files = filesWithoutLevel1.stream().map(hoodieLsmLogFile -> { + return LsmClusteringUtils.buildDataFile(PARQUET_UTILS, (HoodieLSMLogFile) hoodieLsmLogFile, metaClient.getHadoopConf()); + }).collect(Collectors.toList()); + + List> newSortedRunGroup = groupFiles(files, numeric); + + ArrayList res = new ArrayList<>(); + Comparator dataFileComparator = new Comparator() { + private Comparator internalComparator = HoodieLogFile.getLogFileComparator(); + + @Override + public int compare(DataFile o1, DataFile o2) { + return internalComparator.compare(o1.getLogFile(), o2.getLogFile()); + } + }; + TreeSet dataFileGroup = new TreeSet<>(dataFileComparator); + long totalFileSize = 0L; + // each data file is a 'slice' + // clustering at clustering group level + for (List group : newSortedRunGroup) { + if (group.size() > 1) { + List infos = group.stream() + .sorted(dataFileComparator) + .map(dataFile -> { + return buildHoodieSliceInfo(dataFile, partitionPath); + }).collect(Collectors.toList()); + res.add(buildHoodieClusteringGroup(infos, true, 0)); + } else if (group.size() == 1) { + DataFile dataFile = group.get(0); + if (totalFileSize < this.clusteringMaxBytesInGroup) { + dataFileGroup.add(dataFile); + totalFileSize = totalFileSize + dataFile.getDataSize(); + } else { + if (dataFileGroup.size() > 0) { + List infos = dataFileGroup.stream().map(d -> { + return buildHoodieSliceInfo(d, partitionPath); + }).collect(Collectors.toList()); + res.add(buildHoodieClusteringGroup(infos, false, 0)); + } + totalFileSize = 0L; + dataFileGroup.clear(); + + dataFileGroup.add(dataFile); + totalFileSize = totalFileSize + dataFile.getDataSize(); + } + } + } + + if (!dataFileGroup.isEmpty()) { + List infos = dataFileGroup.stream().map(d -> { + return buildHoodieSliceInfo(d, partitionPath); + }).collect(Collectors.toList()); + res.add(buildHoodieClusteringGroup(infos, false, 0)); + totalFileSize = 0L; + dataFileGroup.clear(); + } + + return res.stream(); + } + + protected Stream minorClusteringWithoutGroup(List filesWithoutLevel1, String partitionPath) { + if (filesWithoutLevel1.isEmpty()) { + return null; + } + + ArrayList res = new ArrayList<>(); + long totalFileSize = 0L; + TreeSet dataFileGroup = new TreeSet<>(HoodieLogFile.getLogFileComparator()); + for (HoodieLogFile hoodieLogFile : filesWithoutLevel1) { + HoodieLSMLogFile dataFile = (HoodieLSMLogFile) hoodieLogFile; + if (totalFileSize < this.clusteringMaxBytesInGroup) { + dataFileGroup.add(dataFile); + totalFileSize = totalFileSize + dataFile.getFileSize(); + } else { + if (!dataFileGroup.isEmpty()) { + List infos = dataFileGroup.stream().map(d -> { + return buildHoodieSliceInfo(d, partitionPath); + }).collect(Collectors.toList()); + res.add(buildHoodieClusteringGroup(infos, false, 0)); + } + totalFileSize = 0L; + dataFileGroup.clear(); + + dataFileGroup.add(dataFile); + totalFileSize = totalFileSize + dataFile.getFileSize(); + } + } + + if (!dataFileGroup.isEmpty()) { + List infos = dataFileGroup.stream().map(d -> { + return buildHoodieSliceInfo(d, partitionPath); + }).collect(Collectors.toList()); + res.add(buildHoodieClusteringGroup(infos, false, 0)); + totalFileSize = 0L; + dataFileGroup.clear(); + } + + return res.stream(); + } + + // Major Clustering : merge all sorted runs into one + private Stream majorClustering(List files, String partitionPath) { + List infos = files.stream() + .sorted(HoodieLogFile.getLogFileComparator()) + .map(file -> { + return buildHoodieSliceInfo(file, partitionPath); + }).collect(Collectors.toList()); + + return Stream.of(buildHoodieClusteringGroup(infos, false, 1)); + } + + protected HoodieSliceInfo buildHoodieSliceInfo(HoodieLogFile logFile, String partitionPath) { + return HoodieSliceInfo.newBuilder() + .setPartitionPath(partitionPath) + .setFileId(logFile.getFileName()) // use full file name as fileID + .setDataFilePath(logFile.getPath().toString()) + .setDeltaFilePaths(new ArrayList<>()) + .setBootstrapFilePath(StringUtils.EMPTY_STRING) + .build(); + } + + private HoodieSliceInfo buildHoodieSliceInfo(DataFile dataFile, String partitionPath) { + return HoodieSliceInfo.newBuilder() + .setPartitionPath(partitionPath) + .setFileId(new Path(dataFile.getPath()).getName()) // use full file name as fileID + .setDataFilePath(dataFile.getPath()) + .setDeltaFilePaths(new ArrayList<>()) + .setBootstrapFilePath(StringUtils.EMPTY_STRING) + .build(); + } + + protected HoodieClusteringGroup buildHoodieClusteringGroup(List infos, boolean usingStreamingCopy, int outputLevel) { + HashMap extraMeta = new HashMap<>(); + extraMeta.put(HoodieClusteringConfig.LSM_CLUSTERING_USING_STREAMING_COPY, String.valueOf(usingStreamingCopy)); + extraMeta.put(HoodieClusteringConfig.LSM_CLUSTERING_OUT_PUT_LEVEL, String.valueOf(outputLevel)); + + return HoodieClusteringGroup.newBuilder() + .setSlices(infos) + .setNumOutputFileGroups(1) + .setExtraMetadata(extraMeta) + .build(); + } + + // Minor Clustering + public List pickMinorClustering(List logfiles) { + List smallLogfiles = logfiles.stream().filter(logfile -> { + return logfile.getFileSize() <= this.clusteringSmallFileLimit && ((HoodieLSMLogFile) logfile).getLevelNumber() != 1; + }).collect(Collectors.toList()); + + if (smallLogfiles.size() < numRunClusteringTrigger) { + return null; + } + + return smallLogfiles; + } + + // Major Clustering + protected List pickForSizeAmp(List logFiles, boolean level1Existed, boolean level1InPending) { + if (level1InPending) { + LOG.info("Already exists Level 1 operation in pending clustering. Try to schedule minor clustering."); + return null; + } + + if (!level1Existed) { + // 控制第一次 Major Clustering的频率 + long totalSize = logFiles.stream().mapToLong(HoodieLogFile::getFileSize).sum(); + if (totalSize > clusteringMaxBytesInGroup) { + return logFiles; + } + return null; + } + + // level1 existed + if (logFiles.size() < numRunClusteringTrigger) { + return null; + } + + long level1Size = 0; + long level0SizeSum = 0; + for (HoodieLogFile file : logFiles) { + HoodieLSMLogFile logFile = (HoodieLSMLogFile) file; + int levelNumber = logFile.getLevelNumber(); + if (levelNumber == 1) { + level1Size = level1Size + logFile.getFileSize(); + } else { + level0SizeSum = level0SizeSum + logFile.getFileSize(); + } + } + // size amplification = percentage of additional size + if (level0SizeSum * 100 > maxSizeAmp * level1Size) { + return logFiles; + } + return null; + } + + @Override + public Option generateClusteringPlan() { + if (plan != null && plan.isPresent()) { + return plan; + } + + if (!checkPrecondition()) { + return Option.empty(); + } + + HoodieTableMetaClient metaClient = getHoodieTable().getMetaClient(); + LOG.info("Scheduling clustering for " + metaClient.getBasePath()); + Pair, Pair, Set>> partitionsAndInstantsPair = getPartitionPathsToCluster(); + List partitionPaths = partitionsAndInstantsPair.getLeft(); + Set missingPartitions = new HashSet<>(); + + if (partitionPaths.isEmpty()) { + // In case no partitions could be picked, return no clustering plan + return Option.empty(); + } + // Pair + List> partitionClusteringGroupPair = getEngineContext() + .flatMap( + partitionPaths, + partitionPath -> { + // get and plan at file level + Pair, Set> res = getFilesEligibleForClustering(partitionPath); + List fileSlicesEligible = res.getLeft(); + Set level1 = res.getRight(); + Set level1InPendingClustering = getLevel1InPendingClustering(partitionPath); + return buildClusteringGroupsForPartition(partitionPath, fileSlicesEligible, level1, level1InPendingClustering, skipOP); + }, + partitionPaths.size()); + + List clusteringGroups = partitionClusteringGroupPair.stream().map(pair -> { + String missingPartition = pair.getLeft(); + if (!StringUtils.isNullOrEmpty(missingPartition)) { + // missingPartition value is not empty, which means it related candidate fileSlices all not all processed. + // so that we need to mark this kind of partition as missing partition. + missingPartitions.add(missingPartition); + } + return pair.getRight(); + }).collect(Collectors.toList()); + + if (clusteringGroups.isEmpty()) { + LOG.info("No data available to cluster"); + return Option.empty(); + } + + return buildClusteringPlan(metaClient, + clusteringGroups, getWriteConfig(), + partitionsAndInstantsPair.getRight().getLeft(), + partitionsAndInstantsPair.getRight().getRight(), + missingPartitions, + getStrategyParams(), + getExtraMetadata(), + getPlanVersion()); + } + + public static Option buildClusteringPlan(HoodieTableMetaClient metaClient, + List clusteringGroups, + HoodieWriteConfig config, + Set missingInstants, + Set completedInstants, + Set missingSchedulePartitions, + Map para, + Map extraMeta, + int version) { + //Avoid missing inflight instants generated during plan building + HoodieActiveTimeline hoodieActiveTimeline = metaClient.reloadActiveTimeline(); + Option clusteringInstant = hoodieActiveTimeline.filterCompletedInstants().getCompletedReplaceTimeline().lastInstant(); + hoodieActiveTimeline.getDeltaCommitTimeline().getInstants().forEach(instant -> { + if (!clusteringInstant.isPresent() || instant.getTimestamp().compareToIgnoreCase(clusteringInstant.get().getTimestamp()) > 0) { + if (instant.isCompleted() && !completedInstants.contains(instant.getTimestamp())) { + missingInstants.add(instant.getTimestamp()); + } + if (!instant.isCompleted() && !missingInstants.contains(instant.getTimestamp())) { + missingInstants.add(instant.getTimestamp()); + } + } + }); + HoodieClusteringStrategy strategy = HoodieClusteringStrategy.newBuilder() + .setStrategyClassName(config.getLSMClusteringExecutionStrategyClass()) + .setStrategyParams(para) + .build(); + + return Option.of(HoodieClusteringPlan.newBuilder() + .setStrategy(strategy) + .setInputGroups(clusteringGroups) + .setMissingInstants(new ArrayList<>(missingInstants)) + .setMissingSchedulePartitions(new ArrayList<>(missingSchedulePartitions)) + .setExtraMetadata(extraMeta) + .setVersion(version) + .setPreserveHoodieMetadata(config.isPreserveHoodieCommitMetadataForClustering()) + .build()); + } + + @Override + protected Map getStrategyParams() { + return new HashMap<>(); + } + + public Pair, Set> getFilesEligibleForClustering(String partition) { + SyncableFileSystemView fileSystemView = (SyncableFileSystemView) getHoodieTable().getSliceView(); + + Set pendingClusteringFiles = fileSystemView.getFileGroupsInPendingClustering() + .map(Pair::getKey).map(HoodieFileGroupId::getFileId).collect(Collectors.toSet()); + + HashSet level1 = new HashSet<>(); + List fileSlices = hoodieTable.getSliceView().getLatestFileSlices(partition).map(slice -> { + List newLogFiles = dropPending(slice.getLogFiles(), level1, pendingClusteringFiles); + + return new FileSlice(slice.getFileGroupId(), slice.getBaseInstantTime(), null, newLogFiles); + }).collect(Collectors.toList()); + + return Pair.of(fileSlices, level1); + } + + public Set getLevel1InPendingClustering(String partition) { + return ((SyncableFileSystemView) getHoodieTable().getSliceView()).getLevel1FileIdInPendingClustering(partition); + } + + public List dropPending(Stream stream, HashSet levelMap, Set pendingClusteringFiles) { + return stream.filter(logfile -> { + if (!pendingClusteringFiles.contains(logfile.getFileName())) { + int levelNumber = ((HoodieLSMLogFile) logfile).getLevelNumber(); + if (levelNumber == 1) { + levelMap.add(logfile.getFileId()); + } + return true; + } + return false; + }).collect(Collectors.toList()); + } + + /** + * Get the partitions that need to be scheduled for clustering + * @return Pair(List of partitions, Pair(Set of missing instants, Set of completed instants)) + */ + public Pair, Pair, Set>> getPartitionPathsToCluster() { + if (getWriteConfig().getClusteringPlanInstantsLimit() > 0) { + // When setting the limit, missing partitions and missing instants are not considered. + return Pair.of(getLimitedInstantsRelatedPartitions(), Pair.of(Collections.emptySet(), Collections.emptySet())); + } + + List replaceCommitInstant = hoodieTable.getActiveTimeline().filterCompletedInstants().getCompletedReplaceTimeline().getReverseOrderedInstants().collect(Collectors.toList()); + Option> lastCompetedClusteringPlan = getLastCompetedClusteringPlan(replaceCommitInstant); + if (lastCompetedClusteringPlan.isPresent()) { + return getIncrementalPartitions(lastCompetedClusteringPlan.get()); + } else { + Set currentMissingInstants = new HashSet<>(); + Set completedInstants = new HashSet<>(); + hoodieTable.getActiveTimeline().getDeltaCommitTimeline().getInstants().stream().forEach(instant -> { + if (instant.isCompleted()) { + completedInstants.add(instant.getTimestamp()); + } else { + currentMissingInstants.add(instant.getTimestamp()); + } + }); + return Pair.of(FSUtils.getAllPartitionPaths(getEngineContext(), getWriteConfig().getMetadataConfig(), metaClient.getBasePath()), + Pair.of(currentMissingInstants, completedInstants)); + } + } + + /** + * take care of missing instants from plan caused by multi-write + * @return Pair(List of partitions, Pair(Set of missing instants, Set of completed instants)) + */ + protected Pair, Pair, Set>> getIncrementalPartitions(Pair pair) { + HoodieInstant clusteringInstant = pair.getLeft(); + HoodieClusteringPlan clusteringPlan = pair.getRight(); + List missingPartitionsFromPlan = clusteringPlan.getMissingSchedulePartitions() == null + ? Collections.emptyList() : clusteringPlan.getMissingSchedulePartitions(); + List missingInstantsFromPlan = clusteringPlan.getMissingInstants() == null + ? Collections.emptyList() : clusteringPlan.getMissingInstants(); + + HoodieActiveTimeline activeTimeline = hoodieTable.getActiveTimeline(); + + Set currentMissingInstants = new HashSet<>(); + Set completedInstants = new HashSet<>(); + Set partitions = new HashSet<>(missingPartitionsFromPlan); + ArrayList res = hoodieTable.getActiveTimeline().getDeltaCommitTimeline().getInstants().stream().filter(instant -> { + boolean isMissing = missingInstantsFromPlan.contains(instant.getTimestamp()); + boolean afterLastClustering = instant.getTimestamp().compareToIgnoreCase(clusteringInstant.getTimestamp()) > 0; + if (isMissing || afterLastClustering) { + if (instant.isCompleted()) { + completedInstants.add(instant.getTimestamp()); + return true; + } else { + currentMissingInstants.add(instant.getTimestamp()); + return false; + } + } + return false; + }).flatMap(instant -> { + try { + HoodieCommitMetadata metadata = TimelineUtils.getCommitMetadata(instant, activeTimeline); + return metadata.getWriteStats().stream().map(HoodieWriteStat::getPartitionPath); + } catch (IOException e) { + throw new HoodieIOException(e.getMessage(), e); + } + }).distinct().collect(Collectors.toCollection(ArrayList::new)); + partitions.addAll(res); + + LOG.info("Get Missing Instants : " + currentMissingInstants); + return Pair.of(new ArrayList<>(partitions), Pair.of(currentMissingInstants, completedInstants)); + } + + protected Option> getLastCompetedClusteringPlan(List replaceCommitInstant) { + for (HoodieInstant hoodieInstant : replaceCommitInstant) { + Option> clusteringPlan = + ClusteringUtils.getClusteringPlan(metaClient, hoodieInstant, false); + if (clusteringPlan.isPresent()) { + return clusteringPlan; + } + } + return Option.empty(); + } + + protected List getLimitedInstantsRelatedPartitions() { + HoodieWriteConfig writeConfig = getWriteConfig(); + int limit = writeConfig.getClusteringPlanInstantsLimit(); + ValidationUtils.checkArgument(limit > 0); + HashSet partitions = new HashSet<>(); + HoodieTimeline lsmDeltaCommitTimeline = hoodieTable.getActiveTimeline().getTimelineOfActions(CollectionUtils.createSet(DELTA_COMMIT_ACTION)).filterCompletedInstants(); + lsmDeltaCommitTimeline.getInstantsAsStream().sorted(Comparator.comparing(HoodieInstant::getTimestamp).reversed()).limit(limit).forEach(instant -> { + try { + HoodieCommitMetadata metadata = + HoodieCommitMetadata.fromBytes(lsmDeltaCommitTimeline.getInstantDetails(instant).get(), HoodieCommitMetadata.class); + partitions.addAll(metadata.getWritePartitionPaths()); + } catch (IOException e) { + // ignore Exception here + } + }); + + if (!partitions.isEmpty()) { + return new ArrayList<>(partitions); + } else { + return FSUtils.getAllPartitionPaths(getEngineContext(), getWriteConfig().getMetadataConfig(), metaClient.getBasePath()); + } + } + + /** + * 将Files分组 List> res,要求统一组内的DataFile:1.min max没有交集 2.小的放在前面 + */ + public List> groupFiles(List files, boolean numeric) { + // 按 min 升序排序 + List sortedFiles = new ArrayList<>(files); + sortedFiles.sort((file1, file2) -> { + String min1 = file1.getMin(); + String min2 = file2.getMin(); + String max1 = file1.getMax(); + String max2 = file2.getMax(); + + if (numeric) { + double min1Double = Double.parseDouble(min1); + double min2Double = Double.parseDouble(min2); + double max1Double = Double.parseDouble(max1); + double max2Double = Double.parseDouble(max2); + int i = Double.compare(min1Double, min2Double); + if (i == 0) { + return Double.compare(max1Double, max2Double); + } + return i; + } else { + int i = min1.compareToIgnoreCase(min2); + if (i == 0) { + return max1.compareToIgnoreCase(max2); + } + return i; + } + }); + + List groups = new ArrayList<>(); + + for (DataFile file : sortedFiles) { + boolean added = false; + for (Group group : groups) { + if (group.canAdd(file, clusteringMaxBytesInGroup)) { + group.addFile(file); + added = true; + break; + } + } + if (!added) { + groups.add(new Group(file)); + } + } + + // 转换为 List> + List> result = new ArrayList<>(); + for (Group group : groups) { + result.add(group.getFiles()); + } + return result; + } + + // 辅助类,用于维护分组信息 + private static class Group { + private final List files; + private long totalSize; + private String lastMax; + + public Group(DataFile file) { + this.files = new ArrayList<>(); + this.files.add(file); + this.totalSize = file.getDataSize(); + this.lastMax = file.getMax(); + } + + public boolean canAdd(DataFile file, long maxSize) { + // 检查区间是否无交集 + String currentMin = file.getMin(); + String groupLastMax = this.lastMax; + boolean noOverlap = groupLastMax.compareToIgnoreCase(currentMin) < 0; + + // 检查总大小是否超限 + boolean sizeWithinLimit = (this.totalSize + file.getDataSize()) <= maxSize; + + return noOverlap && sizeWithinLimit; + } + + public void addFile(DataFile file) { + this.files.add(file); + this.totalSize += file.getDataSize(); + this.lastMax = file.getMax(); + } + + public List getFiles() { + return new ArrayList<>(this.files); + } + } + + private boolean isHoodieRecordKeyNumeric() { + if (schema == null) { + return false; + } + Option partitionFields = metaClient.getTableConfig().getPartitionFields(); + if (partitionFields.isPresent() && partitionFields.get().length > 1) { + return false; + } + Option recordKeyFields = metaClient.getTableConfig().getRecordKeyFields(); + if (recordKeyFields.isPresent() && recordKeyFields.get().length == 1) { + // 只关心单一主键 + String recordKeyName = recordKeyFields.get()[0]; + Schema.Field field = schema.getField(recordKeyName); + return LsmClusteringUtils.isNumericType(field); + } + return false; + } + + public boolean getNumeric() { + return numeric; + } +} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseCommitActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseCommitActionExecutor.java index 10be7e7be06ac..812e33da6a164 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseCommitActionExecutor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseCommitActionExecutor.java @@ -103,7 +103,7 @@ public BaseCommitActionExecutor(HoodieEngineContext context, HoodieWriteConfig c this.lastCompletedTxn = Option.empty(); this.pendingInflightAndRequestedInstants = Collections.emptySet(); } - if (!table.getStorageLayout().writeOperationSupported(operationType)) { + if (!table.getStorageLayout().writeOperationSupported(operationType) && !table.getMetaClient().getTableConfig().isLSMBasedLogFormat()) { throw new UnsupportedOperationException("Executor " + this.getClass().getSimpleName() + " is not compatible with table layout " + table.getStorageLayout().getClass().getSimpleName()); } @@ -250,9 +250,11 @@ protected HoodieWriteMetadata> executeClustering(HoodieC config.setValue(HoodieWriteConfig.AUTO_COMMIT_ENABLE, Boolean.FALSE.toString()); final Schema schema = HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(config.getSchema())); + boolean lsmBasedLogFormat = table.getMetaClient().getTableConfig().isLSMBasedLogFormat(); + String className = lsmBasedLogFormat ? config.getLSMClusteringExecutionStrategyClass() : config.getClusteringExecutionStrategyClass(); HoodieWriteMetadata> writeMetadata = ( (ClusteringExecutionStrategy>, HoodieData, HoodieData>) - ReflectionUtils.loadClass(config.getClusteringExecutionStrategyClass(), + ReflectionUtils.loadClass(className, new Class[] {HoodieTable.class, HoodieEngineContext.class, HoodieWriteConfig.class}, table, context, config)) .performClustering(clusteringPlan, schema, instantTime); HoodieData writeStatusList = writeMetadata.getWriteStatuses(); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/ListingBasedRollbackStrategy.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/ListingBasedRollbackStrategy.java index 10b02d7fff200..898f697db0a28 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/ListingBasedRollbackStrategy.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/ListingBasedRollbackStrategy.java @@ -18,14 +18,18 @@ package org.apache.hudi.table.action.rollback; +import java.io.FileNotFoundException; import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; import java.util.List; import java.util.Map; +import java.util.Set; import java.util.function.Function; import java.util.stream.Collectors; +import java.util.stream.Stream; + import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -39,6 +43,8 @@ import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.model.HoodieWriteStat; import org.apache.hudi.common.model.WriteOperationType; +import org.apache.hudi.common.storage.HoodieStorageStrategy; +import org.apache.hudi.common.storage.HoodieStorageStrategyFactory; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; @@ -52,6 +58,7 @@ import org.jetbrains.annotations.NotNull; import static org.apache.hudi.client.utils.MetadataConversionUtils.getHoodieCommitMetadata; +import static org.apache.hudi.common.fs.FSUtils.LSM_TEMP_FILE_SUFFIX; import static org.apache.hudi.common.util.ValidationUtils.checkArgument; import static org.apache.hudi.table.action.rollback.BaseRollbackHelper.EMPTY_STRING; @@ -70,6 +77,8 @@ public class ListingBasedRollbackStrategy implements BaseRollbackPlanActionExecu protected final String instantTime; + protected final HoodieStorageStrategy hoodieStorageStrategy; + public ListingBasedRollbackStrategy(HoodieTable table, HoodieEngineContext context, HoodieWriteConfig config, @@ -78,6 +87,7 @@ public ListingBasedRollbackStrategy(HoodieTable table, this.context = context; this.config = config; this.instantTime = instantTime; + this.hoodieStorageStrategy = HoodieStorageStrategyFactory.getInstant(table.getMetaClient()); } @Override @@ -99,8 +109,7 @@ public List getRollbackRequests(HoodieInstant instantToRo List hoodieRollbackRequests = new ArrayList<>(partitionPaths.size()); FileStatus[] filesToDelete = fetchFilesFromInstant(instantToRollback, partitionPath, metaClient.getBasePath(), baseFileExtension, - metaClient.getFs(), commitMetadataOptional, isCommitMetadataCompleted); - + metaClient.getFs(), commitMetadataOptional, isCommitMetadataCompleted, hoodieStorageStrategy, metaClient.getTableConfig().isLSMBasedLogFormat()); if (HoodieTableType.COPY_ON_WRITE == tableType) { hoodieRollbackRequests.add(getHoodieRollbackRequest(partitionPath, filesToDelete)); } else if (HoodieTableType.MERGE_ON_READ == tableType) { @@ -193,21 +202,27 @@ private String getBaseFileExtension(HoodieTableMetaClient metaClient) { @NotNull private HoodieRollbackRequest getHoodieRollbackRequest(String partitionPath, FileStatus[] filesToDeletedStatus) { - List filesToDelete = getFilesToBeDeleted(filesToDeletedStatus); + List filesToDelete = getFilesToBeDeleted(Arrays.stream(filesToDeletedStatus)); + return new HoodieRollbackRequest( + partitionPath, EMPTY_STRING, EMPTY_STRING, filesToDelete, Collections.emptyMap()); + } + + private HoodieRollbackRequest getHoodieRollbackRequest(String partitionPath, List filesToDeletedStatus) { + List filesToDelete = getFilesToBeDeleted(filesToDeletedStatus.stream()); return new HoodieRollbackRequest( partitionPath, EMPTY_STRING, EMPTY_STRING, filesToDelete, Collections.emptyMap()); } @NotNull - private List getFilesToBeDeleted(FileStatus[] dataFilesToDeletedStatus) { - return Arrays.stream(dataFilesToDeletedStatus).map(fileStatus -> { + private List getFilesToBeDeleted(Stream filesToDeletedStatus) { + return filesToDeletedStatus.map(fileStatus -> { String dataFileToBeDeleted = fileStatus.getPath().toString(); // strip scheme E.g: file:/var/folders - return dataFileToBeDeleted.substring(dataFileToBeDeleted.indexOf(":") + 1); + return dataFileToBeDeleted; }).collect(Collectors.toList()); } - private FileStatus[] listFilesToBeDeleted(String commit, String basefileExtension, String partitionPath, + private List listFilesToBeDeleted(String commit, String basefileExtension, String partitionPath, FileSystem fs) throws IOException { LOG.info("Collecting files to be cleaned/rolledback up for path " + partitionPath + " and commit " + commit); PathFilter filter = (path) -> { @@ -217,27 +232,40 @@ private FileStatus[] listFilesToBeDeleted(String commit, String basefileExtensio } return false; }; - return fs.listStatus(FSUtils.getPartitionPath(config.getBasePath(), partitionPath), filter); + + List files = new ArrayList<>(); + Set allLocations = hoodieStorageStrategy.getAllLocations(partitionPath, true); + for (Path fullPath : allLocations) { + try { + FileStatus[] res = fs.listStatus(fullPath, filter); + files.addAll(Arrays.stream(res).collect(Collectors.toList())); + } catch (FileNotFoundException e) { + // ignore FileNotFoundException here. + } + } + + return files; } private FileStatus[] fetchFilesFromInstant(HoodieInstant instantToRollback, String partitionPath, String basePath, String baseFileExtension, HoodieWrapperFileSystem fs, Option commitMetadataOptional, - Boolean isCommitMetadataCompleted) throws IOException { + Boolean isCommitMetadataCompleted, HoodieStorageStrategy strategy, boolean isLSMFormat) throws IOException { if (isCommitMetadataCompleted) { return fetchFilesFromCommitMetadata(instantToRollback, partitionPath, basePath, commitMetadataOptional.get(), - baseFileExtension, fs); + baseFileExtension, fs, strategy, isLSMFormat); } else { - return fetchFilesFromListFiles(instantToRollback, partitionPath, basePath, baseFileExtension, fs); + return fetchFilesFromListFiles(instantToRollback, partitionPath, basePath, baseFileExtension, fs, isLSMFormat); } } private FileStatus[] fetchFilesFromCommitMetadata(HoodieInstant instantToRollback, String partitionPath, String basePath, HoodieCommitMetadata commitMetadata, - String baseFileExtension, HoodieWrapperFileSystem fs) + String baseFileExtension, HoodieWrapperFileSystem fs, + HoodieStorageStrategy strategy, boolean isLSMFormat) throws IOException { - SerializablePathFilter pathFilter = getSerializablePathFilter(baseFileExtension, instantToRollback.getTimestamp()); - Path[] filePaths = getFilesFromCommitMetadata(basePath, commitMetadata, partitionPath); + SerializablePathFilter pathFilter = getSerializablePathFilter(baseFileExtension, instantToRollback.getTimestamp(), isLSMFormat); + Path[] filePaths = getFilesFromCommitMetadata(instantToRollback.getTimestamp(), commitMetadata, partitionPath, strategy); return fs.listStatus(Arrays.stream(filePaths).filter(entry -> { try { @@ -251,12 +279,11 @@ private FileStatus[] fetchFilesFromCommitMetadata(HoodieInstant instantToRollbac } private FileStatus[] fetchFilesFromListFiles(HoodieInstant instantToRollback, String partitionPath, String basePath, - String baseFileExtension, HoodieWrapperFileSystem fs) + String baseFileExtension, HoodieWrapperFileSystem fs, boolean isLSMFormat) throws IOException { - SerializablePathFilter pathFilter = getSerializablePathFilter(baseFileExtension, instantToRollback.getTimestamp()); - Path[] filePaths = listFilesToBeDeleted(basePath, partitionPath); - - return fs.listStatus(filePaths, pathFilter); + SerializablePathFilter pathFilter = getSerializablePathFilter(baseFileExtension, instantToRollback.getTimestamp(), isLSMFormat); + Path path = hoodieStorageStrategy.storageLocation(partitionPath, instantToRollback.getTimestamp()); + return fs.listStatus(path, pathFilter); } private Boolean checkCommitMetadataCompleted(HoodieInstant instantToRollback, @@ -265,17 +292,14 @@ private Boolean checkCommitMetadataCompleted(HoodieInstant instantToRollback, && !WriteOperationType.UNKNOWN.equals(commitMetadataOptional.get().getOperationType()); } - private static Path[] listFilesToBeDeleted(String basePath, String partitionPath) { - return new Path[] {FSUtils.getPartitionPath(basePath, partitionPath)}; - } - - private static Path[] getFilesFromCommitMetadata(String basePath, HoodieCommitMetadata commitMetadata, String partitionPath) { - List fullPaths = commitMetadata.getFullPathsByPartitionPath(basePath, partitionPath); + private Path[] getFilesFromCommitMetadata(String instantTime, HoodieCommitMetadata commitMetadata, String partitionPath, + HoodieStorageStrategy strategy) { + List fullPaths = commitMetadata.getFullPathsByPartitionPath(instantTime, partitionPath, strategy); return fullPaths.stream().map(Path::new).toArray(Path[]::new); } @NotNull - private static SerializablePathFilter getSerializablePathFilter(String basefileExtension, String commit) { + private static SerializablePathFilter getSerializablePathFilter(String basefileExtension, String commit, boolean isLSMFormat) { return (path) -> { if (path.toString().endsWith(basefileExtension)) { String fileCommitTime = FSUtils.getCommitTime(path.getName()); @@ -284,12 +308,16 @@ private static SerializablePathFilter getSerializablePathFilter(String basefileE // Since the baseCommitTime is the only commit for new log files, it's okay here String fileCommitTime = FSUtils.getBaseCommitTimeFromLogPath(path); return commit.equals(fileCommitTime); + } else if (isLSMFormat && path.toString().endsWith(basefileExtension + LSM_TEMP_FILE_SUFFIX)) { + // With LSM, get the CommitTime of files ending with parquet.temp + String fileCommitTime = FSUtils.getCommitTime(path.getName().substring(0, path.getName().indexOf(LSM_TEMP_FILE_SUFFIX))); + return commit.equals(fileCommitTime); } return false; }; } - public static List getRollbackRequestToAppend(String partitionPath, HoodieInstant rollbackInstant, + public List getRollbackRequestToAppend(String partitionPath, HoodieInstant rollbackInstant, HoodieCommitMetadata commitMetadata, HoodieTable table) { List hoodieRollbackRequests = new ArrayList<>(); checkArgument(rollbackInstant.getAction().equals(HoodieTimeline.DELTA_COMMIT_ACTION)); @@ -336,8 +364,7 @@ public static List getRollbackRequestToAppend(String part String fileId = writeStat.getFileId(); String latestBaseInstant = latestFileSlice.getBaseInstantTime(); - Path fullLogFilePath = FSUtils.getPartitionPath(table.getConfig().getBasePath(), writeStat.getPath()); - + Path fullLogFilePath = hoodieStorageStrategy.storageLocation(writeStat.getPath(), rollbackInstant.getTimestamp()); Map logFilesWithBlocksToRollback = Collections.singletonMap(fullLogFilePath.toString(), writeStat.getTotalWriteBytes()); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/MarkerBasedRollbackStrategy.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/MarkerBasedRollbackStrategy.java index 45870501cab20..94362806f9dd1 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/MarkerBasedRollbackStrategy.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/MarkerBasedRollbackStrategy.java @@ -24,6 +24,8 @@ import org.apache.hudi.common.model.HoodieFileFormat; import org.apache.hudi.common.model.HoodieLogFile; import org.apache.hudi.common.model.IOType; +import org.apache.hudi.common.storage.HoodieStorageStrategy; +import org.apache.hudi.common.storage.HoodieStorageStrategyFactory; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieWriteConfig; @@ -61,12 +63,15 @@ public class MarkerBasedRollbackStrategy implements BaseRollbackPlan protected final String instantTime; + protected final HoodieStorageStrategy hoodieStorageStrategy; + public MarkerBasedRollbackStrategy(HoodieTable table, HoodieEngineContext context, HoodieWriteConfig config, String instantTime) { this.table = table; this.context = context; this.basePath = table.getMetaClient().getBasePath(); this.config = config; this.instantTime = instantTime; + this.hoodieStorageStrategy = HoodieStorageStrategyFactory.getInstant(table.getMetaClient()); } @Override @@ -82,8 +87,8 @@ public List getRollbackRequests(HoodieInstant instantToRo case MERGE: case CREATE: String fileToDelete = WriteMarkers.stripMarkerSuffix(markerFilePath); - Path fullDeletePath = new Path(basePath, fileToDelete); - String partitionPath = FSUtils.getRelativePartitionPath(new Path(basePath), fullDeletePath.getParent()); + Path fullDeletePath = hoodieStorageStrategy.storageLocation(fileToDelete, instantToRollback.getTimestamp()); + String partitionPath = hoodieStorageStrategy.getRelativePath(fullDeletePath.getParent()); return new HoodieRollbackRequest(partitionPath, EMPTY_STRING, EMPTY_STRING, Collections.singletonList(fullDeletePath.toString()), Collections.emptyMap()); @@ -104,11 +109,11 @@ public List getRollbackRequests(HoodieInstant instantToRo } protected HoodieRollbackRequest getRollbackRequestForAppend(HoodieInstant instantToRollback, String markerFilePath) throws IOException { - Path baseFilePathForAppend = new Path(basePath, markerFilePath); + Path baseFilePathForAppend = hoodieStorageStrategy.storageLocation(markerFilePath, instantToRollback.getTimestamp()); String fileId = FSUtils.getFileIdFromFilePath(baseFilePathForAppend); String baseCommitTime = FSUtils.getCommitTime(baseFilePathForAppend.getName()); - String relativePartitionPath = FSUtils.getRelativePartitionPath(new Path(basePath), baseFilePathForAppend.getParent()); - Path partitionPath = FSUtils.getPartitionPath(config.getBasePath(), relativePartitionPath); + String relativePartitionPath = hoodieStorageStrategy.getRelativePath(baseFilePathForAppend.getParent()); + Path partitionPath = hoodieStorageStrategy.storageLocation(relativePartitionPath, instantToRollback.getTimestamp()); // NOTE: Since we're rolling back incomplete Delta Commit, it only could have appended its // block to the latest log-file diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/model/AbstractHoodieRowData.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/model/AbstractHoodieRowData.java index 37d100fa78653..c88d8862fb193 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/model/AbstractHoodieRowData.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/model/AbstractHoodieRowData.java @@ -59,6 +59,14 @@ public AbstractHoodieRowData(String commitTime, this.row = row; } + // wrap row data for lsm + public AbstractHoodieRowData(RowData row, int metaColumnsNum, boolean withOperation) { + this.metaColumnsNum = withOperation ? metaColumnsNum + 1 : metaColumnsNum; + // use StringData[] instead of String[] in child classes + this.metaColumns = new String[0]; + this.row = row; + } + @Override public RowKind getRowKind() { return row.getRowKind(); @@ -155,7 +163,7 @@ public MapData getMap(int ordinal) { return row.getMap(rebaseOrdinal(ordinal)); } - private String getMetaColumnVal(int ordinal) { + protected String getMetaColumnVal(int ordinal) { return this.metaColumns[ordinal]; } diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/model/CommitTimeFlinkRecordMerger.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/model/CommitTimeFlinkRecordMerger.java new file mode 100644 index 0000000000000..2efad38773ecf --- /dev/null +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/model/CommitTimeFlinkRecordMerger.java @@ -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.hudi.client.model; + +import org.apache.hudi.common.config.TypedProperties; +import org.apache.hudi.common.model.HoodieRecord; +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.avro.Schema; + +import java.io.IOException; + +/** + * Record merger for Flink HoodieRecord that implements commit time based merging strategy. + */ +public class CommitTimeFlinkRecordMerger extends HoodieFlinkRecordMerger { + + @Override + public String getMergingStrategy() { + return COMMIT_TIME_BASED_MERGE_STRATEGY_UUID; + } + + @Override + public Option> merge( + HoodieRecord older, + Schema oldSchema, + HoodieRecord newer, + Schema newSchema, + TypedProperties props) throws IOException { + // Note: can be removed if we can ensure the type from invoker. + ValidationUtils.checkArgument(older.getRecordType() == HoodieRecord.HoodieRecordType.FLINK); + ValidationUtils.checkArgument(newer.getRecordType() == HoodieRecord.HoodieRecordType.FLINK); + + + if (oldSchema.getField(HoodieRecord.COMMIT_TIME_METADATA_FIELD) != null + && newSchema.getField(HoodieRecord.COMMIT_TIME_METADATA_FIELD) != null) { + Comparable olderFlinkCommitTime = (Comparable) older.getColumnValues(oldSchema, + new String[] {HoodieRecord.COMMIT_TIME_METADATA_FIELD}, false)[0]; + Comparable newerFlinkCommitTime = (Comparable) newer.getColumnValues(newSchema, + new String[] {HoodieRecord.COMMIT_TIME_METADATA_FIELD}, false)[0]; + + if (olderFlinkCommitTime.compareTo(newerFlinkCommitTime) > 0) { + return Option.of(Pair.of(older, oldSchema)); + } else if (olderFlinkCommitTime.compareTo(newerFlinkCommitTime) < 0) { + return Option.of(Pair.of(newer, newSchema)); + } else if (newSchema.getField(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD) != null + && oldSchema.getField(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD) != null) { + return compareCommitSeq(older, oldSchema, newer, newSchema, props, HoodieRecord.COMMIT_SEQNO_METADATA_FIELD); + } + } else if (newSchema.getField(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD) != null + && oldSchema.getField(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD) != null) { + return compareCommitSeq(older, oldSchema, newer, newSchema, props, HoodieRecord.COMMIT_SEQNO_METADATA_FIELD); + } + return Option.of(Pair.of(newer, newSchema)); + } + + private static Option> compareCommitSeq(HoodieRecord older, Schema oldSchema, + HoodieRecord newer, Schema newSchema, + TypedProperties props, String fieldName) throws IOException { + Long olderCommitSeq = Long.valueOf((String) older.getColumnValues(oldSchema, + new String[] {fieldName}, false)[0]); + Long newerCommitSeq = Long.valueOf((String) newer.getColumnValues(newSchema, + new String[] {fieldName}, false)[0]); + if (olderCommitSeq.compareTo(newerCommitSeq) > 0) { + return Option.of(Pair.of(older, oldSchema)); + } else { + return Option.of(Pair.of(newer, newSchema)); + } + } +} diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/model/EventTimeFlinkRecordMerger.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/model/EventTimeFlinkRecordMerger.java new file mode 100644 index 0000000000000..8e746b269e9fb --- /dev/null +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/model/EventTimeFlinkRecordMerger.java @@ -0,0 +1,90 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.hudi.client.model; + +import org.apache.hudi.common.config.TypedProperties; +import org.apache.hudi.common.model.HoodieRecord; +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.avro.Schema; + +import java.io.IOException; + +/** + * Record merger for Flink HoodieRecord that implements event time based merging strategy. + */ +public class EventTimeFlinkRecordMerger extends HoodieFlinkRecordMerger { + @Override + public String getMergingStrategy() { + return EVENT_TIME_BASED_MERGE_STRATEGY_UUID; + } + + @Override + public Option> merge( + HoodieRecord older, + Schema oldSchema, + HoodieRecord newer, + Schema newSchema, + TypedProperties props) throws IOException { + // Note: can be removed if we can ensure the type from invoker. + ValidationUtils.checkArgument(older.getRecordType() == HoodieRecord.HoodieRecordType.FLINK); + ValidationUtils.checkArgument(newer.getRecordType() == HoodieRecord.HoodieRecordType.FLINK); + + if (older.getOrderingValue(oldSchema, props).compareTo(newer.getOrderingValue(newSchema, props)) > 0) { + return Option.of(Pair.of(older, oldSchema)); + } else if (older.getOrderingValue(oldSchema, props).compareTo(newer.getOrderingValue(newSchema, props)) < 0) { + return Option.of(Pair.of(newer, newSchema)); + } else if (newSchema.getField(HoodieRecord.COMMIT_TIME_METADATA_FIELD) != null + && oldSchema.getField(HoodieRecord.COMMIT_TIME_METADATA_FIELD) != null) { + Comparable olderCommitTime = (Comparable) older.getColumnValues(oldSchema, + new String[] {HoodieRecord.COMMIT_TIME_METADATA_FIELD}, false)[0]; + Comparable newerCommitTime = (Comparable) newer.getColumnValues(newSchema, + new String[] {HoodieRecord.COMMIT_TIME_METADATA_FIELD}, false)[0]; + if (olderCommitTime.compareTo(newerCommitTime) > 0) { + return Option.of(Pair.of(older, oldSchema)); + } else if (olderCommitTime.compareTo(newerCommitTime) < 0) { + return Option.of(Pair.of(newer, newSchema)); + } else if (newSchema.getField(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD) != null + && oldSchema.getField(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD) != null) { + return compareCommitSeq(older, oldSchema, newer, newSchema, props, HoodieRecord.COMMIT_SEQNO_METADATA_FIELD); + } + } else if (newSchema.getField(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD) != null + && oldSchema.getField(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD) != null) { + return compareCommitSeq(older, oldSchema, newer, newSchema, props, HoodieRecord.COMMIT_SEQNO_METADATA_FIELD); + } + return Option.of(Pair.of(newer, newSchema)); + } + + private static Option> compareCommitSeq(HoodieRecord older, Schema oldSchema, + HoodieRecord newer, Schema newSchema, + TypedProperties props, String fieldName) throws IOException { + Long olderCommitSeq = Long.valueOf((String) older.getColumnValues(oldSchema, + new String[] {fieldName}, false)[0]); + Long newerCommitSeq = Long.valueOf((String) newer.getColumnValues(newSchema, + new String[] {fieldName}, false)[0]); + if (olderCommitSeq.compareTo(newerCommitSeq) > 0) { + return Option.of(Pair.of(older, oldSchema)); + } else { + return Option.of(Pair.of(newer, newSchema)); + } + } +} diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/model/HoodieFlinkRecord.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/model/HoodieFlinkRecord.java new file mode 100644 index 0000000000000..3b16e0185340b --- /dev/null +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/model/HoodieFlinkRecord.java @@ -0,0 +1,226 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.client.model; + +import org.apache.hudi.common.model.HoodieAvroIndexedRecord; +import org.apache.hudi.common.model.HoodieKey; +import org.apache.hudi.common.model.HoodieOperation; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.MetadataValues; +import org.apache.hudi.common.util.ConfigUtils; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.keygen.BaseKeyGenerator; +import org.apache.hudi.util.RowDataAvroQueryContexts; + +import com.esotericsoftware.kryo.Kryo; +import com.esotericsoftware.kryo.io.Input; +import com.esotericsoftware.kryo.io.Output; +import org.apache.avro.Schema; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.data.utils.JoinedRowData; +import org.apache.flink.table.runtime.typeutils.RowDataSerializer; + +import java.io.IOException; +import java.util.Map; +import java.util.Properties; + +import static org.apache.hudi.common.util.StringUtils.isNullOrEmpty; + +/** + * Flink Engine-specific Implementations of `HoodieRecord`, which is expected to hold {@code RowData} as payload. + */ +public class HoodieFlinkRecord extends HoodieRecord { + private RowDataSerializer rowDataSerializer; + protected transient Comparable orderingValue; + private boolean copy = false; + public static final int DEFAULT_ORDERING_VALUE = 0; + private StringData recordKeyStringData; + + public HoodieFlinkRecord(RowData rowData) { + super(null, rowData); + } + + public HoodieFlinkRecord(HoodieKey key, HoodieOperation op, RowData rowData) { + super(key, rowData, op, Option.empty()); + } + + public HoodieFlinkRecord(HoodieKey key, HoodieOperation op, Comparable orderingValue, RowData rowData) { + super(key, rowData, op, Option.empty()); + this.orderingValue = orderingValue; + } + + public HoodieFlinkRecord(HoodieKey key, RowData rowData, RowDataSerializer rowDataSerializer) { + super(key, rowData); + this.rowDataSerializer = rowDataSerializer; + } + + public HoodieFlinkRecord(StringData recordKey, RowData record) { + super(null, record); + this.recordKeyStringData = recordKey; + } + + @Override + public HoodieRecord newInstance() { + return new HoodieFlinkRecord(key, operation, orderingValue, data); + } + + @Override + public HoodieRecord newInstance(HoodieKey key, HoodieOperation op) { + return new HoodieFlinkRecord(key, op, orderingValue, this.data); + } + + @Override + public HoodieRecord newInstance(HoodieKey key) { + throw new UnsupportedOperationException("Not supported for " + this.getClass().getSimpleName()); + } + + @Override + public Comparable doGetOrderingValue(Schema recordSchema, Properties props) { + String orderingField = ConfigUtils.getOrderingField(props); + if (isNullOrEmpty(orderingField)) { + return DEFAULT_ORDERING_VALUE; + } else { + return (Comparable) getColumnValueAsJava(recordSchema, orderingField, props, false); + } + } + + @Override + public HoodieRecordType getRecordType() { + return HoodieRecordType.FLINK; + } + + @Override + public String getRecordKey(Schema recordSchema, Option keyGeneratorOpt) { + return getRecordKey(); + } + + public StringData getRecordKeyStringData() { + return this.recordKeyStringData; + } + + @Override + public String getRecordKey(Schema recordSchema, String keyFieldName) { + return getRecordKey(); + } + + @Override + protected void writeRecordPayload(RowData payload, Kryo kryo, Output output) { + + } + + @Override + protected RowData readRecordPayload(Kryo kryo, Input input) { + throw new UnsupportedOperationException("Not supported for " + this.getClass().getSimpleName()); + } + + @Override + public Object[] getColumnValues(Schema recordSchema, String[] columns, boolean consistentLogicalTimestampEnabled) { + Object[] fieldValues = new Object[columns.length]; + for (int i = 0; i < columns.length; i++) { + fieldValues[i] = getColumnValueAsJava(recordSchema, columns[i], new Properties(), true); + } + return fieldValues; + } + + @Override + public HoodieRecord joinWith(HoodieRecord other, Schema targetSchema) { + throw new UnsupportedOperationException("Not supported for " + this.getClass().getSimpleName()); + } + + @Override + public HoodieRecord prependMetaFields(Schema recordSchema, Schema targetSchema, MetadataValues metadataValues, Properties props) { + int metaFieldSize = targetSchema.getFields().size() - recordSchema.getFields().size(); + GenericRowData metaRow = new GenericRowData(metaFieldSize); + String[] metaVals = metadataValues.getValues(); + for (int i = 0; i < metaVals.length; i++) { + if (metaVals[i] != null) { + metaRow.setField(i, StringData.fromString(metaVals[i])); + } + } + return new HoodieFlinkRecord(key, operation, orderingValue, new JoinedRowData(data.getRowKind(), metaRow, data)); + } + + @Override + public HoodieRecord rewriteRecordWithNewSchema(Schema recordSchema, Properties props, Schema newSchema, Map renameCols) { + throw new UnsupportedOperationException("Not supported for " + this.getClass().getSimpleName()); + } + + @Override + public boolean isDelete(Schema recordSchema, Properties props) throws IOException { + if (data == null) { + return true; + } + + if (HoodieOperation.isDelete(getOperation())) { + return true; + } + + // Use data field to decide. + Schema.Field deleteField = recordSchema.getField(HOODIE_IS_DELETED_FIELD); + return deleteField != null && data.getBoolean(deleteField.pos()); + } + + @Override + public boolean shouldIgnore(Schema recordSchema, Properties props) throws IOException { + return false; + } + + @Override + public HoodieFlinkRecord copy() { + if (!copy) { + this.data = rowDataSerializer.copy(data); + this.copy = true; + } + return this; + } + + @Override + public Option> getMetadata() { + return Option.empty(); + } + + @Override + public HoodieRecord wrapIntoHoodieRecordPayloadWithParams(Schema recordSchema, Properties props, Option> simpleKeyGenFieldsOpt, Boolean withOperation, + Option partitionNameOp, Boolean populateMetaFieldsOp) throws IOException { + throw new UnsupportedOperationException("Not supported for " + this.getClass().getSimpleName()); + } + + @Override + public HoodieRecord wrapIntoHoodieRecordPayloadWithKeyGen(Schema recordSchema, Properties props, Option keyGen) { + throw new UnsupportedOperationException("Not supported for " + this.getClass().getSimpleName()); + } + + @Override + public HoodieRecord truncateRecordKey(Schema recordSchema, Properties props, String keyFieldName) throws IOException { + throw new UnsupportedOperationException("Not supported for " + this.getClass().getSimpleName()); + } + + @Override + public Option toIndexedRecord(Schema recordSchema, Properties props) throws IOException { + throw new UnsupportedOperationException("Not supported for " + this.getClass().getSimpleName()); + } + + private Object getColumnValueAsJava(Schema recordSchema, String column, Properties props, boolean allowsNull) { + RowDataAvroQueryContexts.RowDataQueryContext rowDataQueryContext = RowDataAvroQueryContexts.fromAvroSchema(recordSchema); + return rowDataQueryContext.getFieldQueryContext(column).getValAsJava(data, allowsNull); + } +} diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/model/HoodieFlinkRecordMerger.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/model/HoodieFlinkRecordMerger.java new file mode 100644 index 0000000000000..37fdf5d0d9fd1 --- /dev/null +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/model/HoodieFlinkRecordMerger.java @@ -0,0 +1,33 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.hudi.client.model; + +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecordMerger; + +/** + * Base class for {@link HoodieRecordMerger } of HoodieFlinkRecord. + */ +public abstract class HoodieFlinkRecordMerger implements HoodieRecordMerger { + @Override + public HoodieRecord.HoodieRecordType getRecordType() { + return HoodieRecord.HoodieRecordType.FLINK; + } +} diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/model/HoodieRowDataCreation.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/model/HoodieRowDataCreation.java index b4fd5cd74ae5e..ba37f859acb58 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/model/HoodieRowDataCreation.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/model/HoodieRowDataCreation.java @@ -19,6 +19,7 @@ package org.apache.hudi.client.model; import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; /** * The factory clazz for hoodie row data. @@ -40,4 +41,28 @@ public static AbstractHoodieRowData create( ? new HoodieRowDataWithMetaFields(commitTime, commitSeqNumber, recordKey, partitionPath, fileName, row, withOperation) : new HoodieRowData(commitTime, commitSeqNumber, recordKey, partitionPath, fileName, row, withOperation); } + + public static AbstractHoodieRowData createLSMHoodieInternalRowData( + String recordKey, + String seqId, + RowData row) { + return new LSMHoodieInternalRowData( + StringData.fromString(recordKey), + StringData.fromString(seqId), + row); + } + + public static AbstractHoodieRowData createLSMHoodieRowData( + String commitTime, + String partitionPath, + String fileName, + RowData row, + boolean withOperation) { + return new LSMHoodieRowData( + StringData.fromString(commitTime), + StringData.fromString(partitionPath), + StringData.fromString(fileName), + row, + withOperation); + } } diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/model/LSMHoodieInternalRowData.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/model/LSMHoodieInternalRowData.java new file mode 100644 index 0000000000000..a3dfea91add1f --- /dev/null +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/model/LSMHoodieInternalRowData.java @@ -0,0 +1,73 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.client.model; + +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; + +/** + * RowData implementation for Hoodie Row. It wraps an {@link RowData} and keeps meta columns locally. But the {@link RowData} + * does include the meta columns as well just that {@link LSMHoodieInternalRowData} will intercept queries for meta columns and serve from its + * copy rather than fetching from {@link RowData}. + * + *

    The wrapped {@link RowData} does not contain hoodie metadata fields. + */ +public class LSMHoodieInternalRowData extends AbstractHoodieRowData { + private final StringData[] lsmInternalMetaColumns; + + public static int LSM_INTERNAL_RECORD_KEY_META_FIELD_ORD = 0; + public static int LSM_INTERNAL_COMMIT_SEQNO_META_FIELD_ORD = 1; + + public LSMHoodieInternalRowData(StringData recordKey, StringData seqId, RowData row) { + super(row, 2, false); + lsmInternalMetaColumns = new StringData[metaColumnsNum]; + lsmInternalMetaColumns[0] = recordKey; + lsmInternalMetaColumns[1] = seqId; + } + + @Override + public int getArity() { + return metaColumnsNum + row.getArity(); + } + + @Override + public boolean isNullAt(int ordinal) { + if (ordinal < metaColumnsNum) { + return null == getLsmInternalMetaColumnVal(ordinal); + } + return row.isNullAt(rebaseOrdinal(ordinal)); + } + + @Override + public StringData getString(int ordinal) { + if (ordinal < metaColumnsNum) { + return getLsmInternalMetaColumnVal(ordinal); + } + return row.getString(rebaseOrdinal(ordinal)); + } + + protected StringData getLsmInternalMetaColumnVal(int ordinal) { + return this.lsmInternalMetaColumns[ordinal]; + } + + @Override + protected int rebaseOrdinal(int ordinal) { + return ordinal - metaColumnsNum; + } +} diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/model/LSMHoodieRowData.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/model/LSMHoodieRowData.java new file mode 100644 index 0000000000000..0b72d229b73a5 --- /dev/null +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/model/LSMHoodieRowData.java @@ -0,0 +1,140 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.client.model; + +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.hudi.common.model.HoodieOperation; +import org.apache.hudi.common.util.CollectionUtils; +import org.apache.hudi.common.util.collection.Pair; + +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; +import java.util.stream.IntStream; + +import static org.apache.hudi.common.model.HoodieRecord.COMMIT_SEQNO_METADATA_FIELD; +import static org.apache.hudi.common.model.HoodieRecord.COMMIT_SEQNO_METADATA_FIELD_ORD; +import static org.apache.hudi.common.model.HoodieRecord.COMMIT_TIME_METADATA_FIELD; +import static org.apache.hudi.common.model.HoodieRecord.COMMIT_TIME_METADATA_FIELD_ORD; +import static org.apache.hudi.common.model.HoodieRecord.FILENAME_METADATA_FIELD; +import static org.apache.hudi.common.model.HoodieRecord.FILENAME_META_FIELD_ORD; +import static org.apache.hudi.common.model.HoodieRecord.OPERATION_METADATA_FIELD; +import static org.apache.hudi.common.model.HoodieRecord.PARTITION_PATH_METADATA_FIELD; +import static org.apache.hudi.common.model.HoodieRecord.PARTITION_PATH_META_FIELD_ORD; +import static org.apache.hudi.common.model.HoodieRecord.RECORD_KEY_METADATA_FIELD; +import static org.apache.hudi.common.model.HoodieRecord.RECORD_KEY_META_FIELD_ORD; + +/** + * RowData implementation for Hoodie Row with meta data columns + * + *

    The wrapped {@link RowData} only contain two metadata fields (recordkey and sequenceId). + */ +public class LSMHoodieRowData extends AbstractHoodieRowData { + + private final StringData[] lsmMetaColumns; + private final boolean withOperation; + + private static final List LSM_HOODIE_META_COLUMNS = + CollectionUtils.createImmutableList(COMMIT_TIME_METADATA_FIELD, PARTITION_PATH_METADATA_FIELD, + FILENAME_METADATA_FIELD, RECORD_KEY_METADATA_FIELD, COMMIT_SEQNO_METADATA_FIELD); + + private static final List LSM_HOODIE_META_COLUMNS_WITH_OPERATION = + CollectionUtils.createImmutableList(COMMIT_TIME_METADATA_FIELD, PARTITION_PATH_METADATA_FIELD, + FILENAME_METADATA_FIELD, OPERATION_METADATA_FIELD, RECORD_KEY_METADATA_FIELD, COMMIT_SEQNO_METADATA_FIELD); + + private static final Map LSM_HOODIE_META_COLUMNS_NAME_TO_POS = + IntStream.range(0, LSM_HOODIE_META_COLUMNS.size()) + .mapToObj(idx -> Pair.of(LSM_HOODIE_META_COLUMNS.get(idx), idx)) + .collect(Collectors.toMap(Pair::getKey, Pair::getValue)); + + private static final Map INTERNAL_METADATA_FIELD_ORD_MAP = CollectionUtils.createImmutableMap( + Pair.of(COMMIT_TIME_METADATA_FIELD_ORD, LSM_HOODIE_META_COLUMNS_NAME_TO_POS.get(COMMIT_TIME_METADATA_FIELD)), + Pair.of(COMMIT_SEQNO_METADATA_FIELD_ORD, LSM_HOODIE_META_COLUMNS_NAME_TO_POS.get(COMMIT_SEQNO_METADATA_FIELD)), + Pair.of(RECORD_KEY_META_FIELD_ORD, LSM_HOODIE_META_COLUMNS_NAME_TO_POS.get(RECORD_KEY_METADATA_FIELD)), + Pair.of(PARTITION_PATH_META_FIELD_ORD, LSM_HOODIE_META_COLUMNS_NAME_TO_POS.get(PARTITION_PATH_METADATA_FIELD)), + Pair.of(FILENAME_META_FIELD_ORD, LSM_HOODIE_META_COLUMNS_NAME_TO_POS.get(FILENAME_METADATA_FIELD)) + ); + + private static final Map LSM_HOODIE_META_COLUMNS_NAME_WITH_OPERATION_TO_POS = + IntStream.range(0, LSM_HOODIE_META_COLUMNS_WITH_OPERATION.size()) + .mapToObj(idx -> Pair.of(LSM_HOODIE_META_COLUMNS_WITH_OPERATION.get(idx), idx)) + .collect(Collectors.toMap(Pair::getKey, Pair::getValue)); + + private static final int OPERATION_METADATA_FIELD_ORD = 5; + + private static final Map INTERNAL_METADATA_FIELD_WITH_OPERATION_ORD_MAP = CollectionUtils.createImmutableMap( + Pair.of(COMMIT_TIME_METADATA_FIELD_ORD, LSM_HOODIE_META_COLUMNS_NAME_WITH_OPERATION_TO_POS.get(COMMIT_TIME_METADATA_FIELD)), + Pair.of(COMMIT_SEQNO_METADATA_FIELD_ORD, LSM_HOODIE_META_COLUMNS_NAME_WITH_OPERATION_TO_POS.get(COMMIT_SEQNO_METADATA_FIELD)), + Pair.of(RECORD_KEY_META_FIELD_ORD, LSM_HOODIE_META_COLUMNS_NAME_WITH_OPERATION_TO_POS.get(RECORD_KEY_METADATA_FIELD)), + Pair.of(PARTITION_PATH_META_FIELD_ORD, LSM_HOODIE_META_COLUMNS_NAME_WITH_OPERATION_TO_POS.get(PARTITION_PATH_METADATA_FIELD)), + Pair.of(FILENAME_META_FIELD_ORD, LSM_HOODIE_META_COLUMNS_NAME_WITH_OPERATION_TO_POS.get(FILENAME_METADATA_FIELD)), + Pair.of(OPERATION_METADATA_FIELD_ORD, LSM_HOODIE_META_COLUMNS_NAME_WITH_OPERATION_TO_POS.get(OPERATION_METADATA_FIELD)) + ); + + public LSMHoodieRowData(StringData commitTime, + StringData partitionPath, + StringData fileName, + RowData row, + boolean withOperation) { + super(row, 3, withOperation); + this.withOperation = withOperation; + this.lsmMetaColumns = new StringData[metaColumnsNum]; + lsmMetaColumns[0] = commitTime; + lsmMetaColumns[1] = partitionPath; + lsmMetaColumns[2] = fileName; + if (withOperation) { + lsmMetaColumns[3] = StringData.fromString(HoodieOperation.fromValue(row.getRowKind().toByteValue()).getName()); + } + } + + @Override + public int getArity() { + return metaColumnsNum + row.getArity(); + } + + @Override + public boolean isNullAt(int ordinal) { + if (ordinal < metaColumnsNum) { + return null == getLsmMetaColumnVal(ordinal); + } + return row.isNullAt(rebaseOrdinal(ordinal)); + } + + @Override + public StringData getString(int ordinal) { + if (ordinal < metaColumnsNum + 2) { + int realPos = withOperation ? INTERNAL_METADATA_FIELD_WITH_OPERATION_ORD_MAP.get(ordinal) : INTERNAL_METADATA_FIELD_ORD_MAP.get(ordinal); + if (realPos < metaColumnsNum) { + return getLsmMetaColumnVal(realPos); + } + return row.getString(rebaseOrdinal(realPos)); + } + return row.getString(rebaseOrdinal(ordinal)); + } + + protected StringData getLsmMetaColumnVal(int ordinal) { + return this.lsmMetaColumns[ordinal]; + } + + @Override + protected int rebaseOrdinal(int ordinal) { + return ordinal - metaColumnsNum; + } +} diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/model/ModelUtils.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/model/ModelUtils.java new file mode 100644 index 0000000000000..50baf45e6a531 --- /dev/null +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/model/ModelUtils.java @@ -0,0 +1,33 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.client.model; + +import org.apache.hudi.common.model.HoodieAvroRecordMerger; + +public class ModelUtils { + public static String getSupportedRecordMerger() { + StringBuilder builder = new StringBuilder(); + builder.append(HoodieAvroRecordMerger.class.getName()).append(","); + builder.append(CommitTimeFlinkRecordMerger.class.getName()).append(","); + builder.append(EventTimeFlinkRecordMerger.class.getName()).append(","); + builder.append(PartialUpdateFlinkRecordMerger.class.getName()).append(","); + builder.append(PartialUpdateSupportMultiTsFlinkRecordMerger.class.getName()); + return builder.toString(); + } +} diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/storage/row/HoodieLSMRowDataParquetWriteSupport.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/storage/row/HoodieLSMRowDataParquetWriteSupport.java new file mode 100644 index 0000000000000..999cd41c7a990 --- /dev/null +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/storage/row/HoodieLSMRowDataParquetWriteSupport.java @@ -0,0 +1,56 @@ +/* + * 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.io.storage.row; + +import org.apache.hudi.avro.HoodieMinMaxWriteSupport; +import org.apache.hudi.common.util.Option; + +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.logical.RowType; +import org.apache.hadoop.conf.Configuration; + +import java.util.Collections; +import java.util.Map; + +/** + * Hoodie Write Support for directly writing {@link RowData} to Parquet. + */ +public class HoodieLSMRowDataParquetWriteSupport extends HoodieRowDataParquetWriteSupport { + private final Option> minMaxWriteSupportOpt; + + public HoodieLSMRowDataParquetWriteSupport(Configuration conf, RowType rowType) { + super(conf, rowType, null, false); + this.minMaxWriteSupportOpt = Option.of(new HoodieMinMaxWriteSupport<>()); + } + + @Override + public FinalizedWriteContext finalizeWrite() { + Map extraMetadata = + minMaxWriteSupportOpt.map(HoodieMinMaxWriteSupport::finalizeMetadata) + .orElse(Collections.emptyMap()); + + return new FinalizedWriteContext(extraMetadata); + } + + @Override + public void add(String recordKey) { + this.minMaxWriteSupportOpt.ifPresent(writeSupport -> + writeSupport.addKey(recordKey)); + } +} diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowDataCreateHandle.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowDataCreateHandle.java index 231dee2c7d4bc..609bdff252faa 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowDataCreateHandle.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowDataCreateHandle.java @@ -24,18 +24,28 @@ import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodiePartitionMetadata; import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.model.HoodieWriteStat; import org.apache.hudi.common.model.IOType; +import org.apache.hudi.common.storage.HoodieStorageStrategy; +import org.apache.hudi.common.storage.HoodieStorageStrategyFactory; import org.apache.hudi.common.table.HoodieTableConfig; import org.apache.hudi.common.util.HoodieTimer; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.config.HoodiePayloadConfig; import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.exception.HoodieException; import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.exception.HoodieInsertException; +import org.apache.hudi.index.HoodieIndex; +import org.apache.hudi.index.bucket.BucketStrategist; +import org.apache.hudi.index.bucket.BucketStrategistFactory; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.marker.WriteMarkers; import org.apache.hudi.table.marker.WriteMarkersFactory; import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.logical.LogicalType; import org.apache.flink.table.types.logical.RowType; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -46,6 +56,8 @@ import java.io.Serializable; import java.util.concurrent.atomic.AtomicLong; +import static org.apache.hudi.common.config.HoodieStorageConfig.DATASKETCH_ENABLED; + /** * Create handle with RowData for datasource implemention of bulk insert. */ @@ -69,12 +81,27 @@ public class HoodieRowDataCreateHandle implements Serializable { private final FileSystem fs; protected final HoodieInternalWriteStatus writeStatus; private final HoodieTimer currTimer; + private final int fieldIndex; + private LogicalType fieldType; + private final boolean isUpdateEventTime; + private final HoodieStorageStrategy hoodieStorageStrategy; public HoodieRowDataCreateHandle(HoodieTable table, HoodieWriteConfig writeConfig, String partitionPath, String fileId, String instantTime, int taskPartitionId, long taskId, long taskEpochId, RowType rowType, boolean preserveHoodieMetadata) { + this.hoodieStorageStrategy = HoodieStorageStrategyFactory.getInstant(table.getMetaClient()); + Option bucketStrategist; + if (writeConfig.getIndexType().equals(HoodieIndex.IndexType.BUCKET) && writeConfig.isBucketIndexAtPartitionLevel()) { + bucketStrategist = Option.of(BucketStrategistFactory.getInstant(writeConfig, table.getMetaClient().getFs())); + } else { + bucketStrategist = Option.empty(); + } this.partitionPath = partitionPath; this.table = table; + if (HoodieTableType.COPY_ON_WRITE.equals(table.getMetaClient().getTableType())) { + writeConfig.getStorageConfig().setValue(DATASKETCH_ENABLED.key(), "false"); + writeConfig.setValue(DATASKETCH_ENABLED.key(), "false"); + } this.writeConfig = writeConfig; this.instantTime = instantTime; this.taskPartitionId = taskPartitionId; @@ -85,6 +112,9 @@ public HoodieRowDataCreateHandle(HoodieTable table, HoodieWriteConfig writeConfi this.currTimer = HoodieTimer.start(); this.fs = table.getMetaClient().getFs(); this.path = makeNewPath(partitionPath); + this.fieldIndex = rowType.getFieldIndex(writeConfig.getPayloadConfig().getString(HoodiePayloadConfig.EVENT_TIME_FIELD.key())); + this.fieldType = fieldIndex != -1 ? rowType.getTypeAt(fieldIndex) : null; + this.isUpdateEventTime = supportUpdateEventTime(); this.writeStatus = new HoodieInternalWriteStatus(!table.getIndex().isImplicitWithStorage(), writeConfig.getWriteStatusFailureFraction()); writeStatus.setPartitionPath(partitionPath); @@ -96,8 +126,10 @@ public HoodieRowDataCreateHandle(HoodieTable table, HoodieWriteConfig writeConfi fs, instantTime, new Path(writeConfig.getBasePath()), - FSUtils.getPartitionPath(writeConfig.getBasePath(), partitionPath), - table.getPartitionMetafileFormat()); + hoodieStorageStrategy.storageLocation(partitionPath, instantTime), + table.getPartitionMetafileFormat(), + hoodieStorageStrategy, + bucketStrategist.map(strategist -> strategist.computeBucketNumber(partitionPath))); partitionMetadata.trySave(taskPartitionId); createMarkerFile(partitionPath, FSUtils.makeBaseFileName(this.instantTime, getWriteToken(), this.fileId, table.getBaseFileExtension())); this.fileWriter = createNewFileWriter(path, table, writeConfig, rowType); @@ -129,9 +161,18 @@ public void write(String recordKey, String partitionPath, RowData record) throws record, writeConfig.allowOperationMetadataField(), preserveHoodieMetadata); try { fileWriter.writeRow(recordKey, rowData); - writeStatus.markSuccess(recordKey); + if (isUpdateEventTime) { + // set min/max + writeStatus.markSuccess(recordKey, String.valueOf(getValueFromRowData(rowData))); + } else { + writeStatus.markSuccess(recordKey); + } } catch (Throwable t) { + LOG.error("Failed to write : key is " + recordKey + ", data is " + rowData, t); writeStatus.markFailure(recordKey, t); + if (!writeConfig.getIgnoreWriteFailed()) { + throw new HoodieException(t.getMessage(), t); + } } } catch (Throwable ge) { writeStatus.setGlobalError(ge); @@ -139,6 +180,34 @@ public void write(String recordKey, String partitionPath, RowData record) throws } } + private Object getValueFromRowData(RowData rowData) { + if (rowData.isNullAt(this.fieldIndex)) { + return null; + } + switch (this.fieldType.getTypeRoot()) { + case BIGINT: + return rowData.getLong(this.fieldIndex); + case VARCHAR: + return rowData.getString(this.fieldIndex); + default: + return null; + } + } + + private boolean supportUpdateEventTime() { + if (fieldIndex == -1) { + return false; + } + switch (this.fieldType.getTypeRoot()) { + case BIGINT: + case VARCHAR: + return true; + default: + // other types are not supported + return false; + } + } + /** * Returns {@code true} if this handle can take in more writes. else {@code false}. */ @@ -162,7 +231,7 @@ public HoodieInternalWriteStatus close() throws IOException { stat.setNumInserts(writeStatus.getTotalRecords()); stat.setPrevCommit(HoodieWriteStat.NULL_COMMIT); stat.setFileId(fileId); - stat.setPath(new Path(writeConfig.getBasePath()), path); + stat.setPath(hoodieStorageStrategy.getRelativePath(path)); long fileSizeInBytes = FSUtils.getFileSize(table.getMetaClient().getFs(), path); stat.setTotalWriteBytes(fileSizeInBytes); stat.setFileSizeInBytes(fileSizeInBytes); @@ -178,7 +247,7 @@ public String getFileName() { } private Path makeNewPath(String partitionPath) { - Path path = FSUtils.getPartitionPath(writeConfig.getBasePath(), partitionPath); + Path path = hoodieStorageStrategy.storageLocation(partitionPath, instantTime); try { if (!fs.exists(path)) { fs.mkdirs(path); // create a new partition as needed. diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowDataFileWriterFactory.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowDataFileWriterFactory.java index 816cec4f906c9..92bc24c0bcd5a 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowDataFileWriterFactory.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowDataFileWriterFactory.java @@ -60,22 +60,43 @@ public static HoodieRowDataFileWriter getRowDataFileWriter( private static HoodieRowDataFileWriter newParquetInternalRowFileWriter( Path path, HoodieWriteConfig writeConfig, RowType rowType, HoodieTable table) throws IOException { - BloomFilter filter = BloomFilterFactory.createBloomFilter( - writeConfig.getBloomFilterNumEntries(), - writeConfig.getBloomFilterFPP(), - writeConfig.getDynamicBloomFilterMaxNumEntries(), - writeConfig.getBloomFilterType()); - HoodieRowDataParquetWriteSupport writeSupport = - new HoodieRowDataParquetWriteSupport(table.getHadoopConf(), rowType, filter); - return new HoodieRowDataParquetWriter( - path, new HoodieParquetConfig<>( - writeSupport, - writeConfig.getParquetCompressionCodec(), - writeConfig.getParquetBlockSize(), - writeConfig.getParquetPageSize(), - writeConfig.getParquetMaxFileSize(), - writeSupport.getHadoopConf(), - writeConfig.getParquetCompressionRatio(), - writeConfig.parquetDictionaryEnabled())); + if (table.getMetaClient().getTableConfig().isLSMBasedLogFormat()) { + HoodieLSMRowDataParquetWriteSupport lsmWriteSupport = + new HoodieLSMRowDataParquetWriteSupport(table.getHadoopConf(), rowType); + return new HoodieRowDataParquetWriter( + path, new HoodieParquetConfig<>( + lsmWriteSupport, + writeConfig.getParquetCompressionCodec(), + writeConfig.getParquetBlockSize(), + writeConfig.getParquetPageSize(), + writeConfig.getParquetMaxFileSize(), + lsmWriteSupport.getHadoopConf(), + writeConfig.getParquetCompressionRatio(), + writeConfig.parquetDictionaryEnabled(), + writeConfig, + rowType.getFieldCount() + )); + } else { + BloomFilter filter = BloomFilterFactory.createBloomFilter( + writeConfig.getBloomFilterNumEntries(), + writeConfig.getBloomFilterFPP(), + writeConfig.getDynamicBloomFilterMaxNumEntries(), + writeConfig.getBloomFilterType()); + HoodieRowDataParquetWriteSupport writeSupport = + new HoodieRowDataParquetWriteSupport(table.getHadoopConf(), rowType, filter, writeConfig.isDataSketchEnabled()); + return new HoodieRowDataParquetWriter( + path, new HoodieParquetConfig<>( + writeSupport, + writeConfig.getParquetCompressionCodec(), + writeConfig.getParquetBlockSize(), + writeConfig.getParquetPageSize(), + writeConfig.getParquetMaxFileSize(), + writeSupport.getHadoopConf(), + writeConfig.getParquetCompressionRatio(), + writeConfig.parquetDictionaryEnabled(), + writeConfig, + rowType.getFieldCount() + )); + } } } diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowDataParquetWriteSupport.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowDataParquetWriteSupport.java index 4a3109db60a33..558a643da1a86 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowDataParquetWriteSupport.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowDataParquetWriteSupport.java @@ -20,7 +20,9 @@ import org.apache.hudi.avro.HoodieBloomFilterWriteSupport; import org.apache.hudi.common.bloom.BloomFilter; +import org.apache.hudi.common.util.CollectionUtils; import org.apache.hudi.common.util.Option; +import org.apache.hudi.datasketch.DataSketchWriteSupport; import org.apache.flink.table.data.RowData; import org.apache.flink.table.types.logical.RowType; @@ -38,12 +40,14 @@ public class HoodieRowDataParquetWriteSupport extends RowDataParquetWriteSupport private final Configuration hadoopConf; private final Option> bloomFilterWriteSupportOpt; + private final Option dataSketchRowDataWriteSupportOpt; - public HoodieRowDataParquetWriteSupport(Configuration conf, RowType rowType, BloomFilter bloomFilter) { + public HoodieRowDataParquetWriteSupport(Configuration conf, RowType rowType, BloomFilter bloomFilter, boolean dataSketchEnabled) { super(rowType); this.hadoopConf = new Configuration(conf); this.bloomFilterWriteSupportOpt = Option.ofNullable(bloomFilter) .map(HoodieBloomFilterRowDataWriteSupport::new); + this.dataSketchRowDataWriteSupportOpt = dataSketchEnabled ? Option.of(new HoodieDataSketchRowDataWriteSupport()) : Option.empty(); } public Configuration getHadoopConf() { @@ -55,13 +59,16 @@ public WriteSupport.FinalizedWriteContext finalizeWrite() { Map extraMetadata = bloomFilterWriteSupportOpt.map(HoodieBloomFilterWriteSupport::finalizeMetadata) .orElse(Collections.emptyMap()); - + extraMetadata = CollectionUtils.combine(extraMetadata, + dataSketchRowDataWriteSupportOpt.map(DataSketchWriteSupport::finalizeMetadata) + .orElse(Collections.emptyMap())); return new WriteSupport.FinalizedWriteContext(extraMetadata); } public void add(String recordKey) { this.bloomFilterWriteSupportOpt.ifPresent(bloomFilterWriteSupport -> bloomFilterWriteSupport.addKey(recordKey)); + this.dataSketchRowDataWriteSupportOpt.ifPresent(sketch -> sketch.addKey(recordKey)); } private static class HoodieBloomFilterRowDataWriteSupport extends HoodieBloomFilterWriteSupport { @@ -74,4 +81,11 @@ protected byte[] getUTF8Bytes(String key) { return key.getBytes(StandardCharsets.UTF_8); } } + + private static class HoodieDataSketchRowDataWriteSupport extends DataSketchWriteSupport { + @Override + protected String getUTF8String(String key) { + return key; + } + } } diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowDataParquetWriter.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowDataParquetWriter.java index 17b3b6b37cf18..f51b8c15e5381 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowDataParquetWriter.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowDataParquetWriter.java @@ -18,46 +18,26 @@ package org.apache.hudi.io.storage.row; -import org.apache.hudi.common.fs.FSUtils; -import org.apache.hudi.common.fs.HoodieWrapperFileSystem; -import org.apache.hudi.io.storage.HoodieParquetConfig; - import org.apache.flink.table.data.RowData; import org.apache.hadoop.fs.Path; -import org.apache.parquet.hadoop.ParquetFileWriter; -import org.apache.parquet.hadoop.ParquetWriter; +import org.apache.hudi.io.storage.HoodieBaseParquetWriter; +import org.apache.hudi.io.storage.HoodieParquetConfig; import java.io.IOException; /** - * Parquet's impl of {@link HoodieRowDataFileWriter} to write {@link RowData}s. + * Parquet's impl of {@link HoodieRowDataFileWriter} to write fink {@link RowData}s. */ -public class HoodieRowDataParquetWriter extends ParquetWriter - implements HoodieRowDataFileWriter { +public class HoodieRowDataParquetWriter extends HoodieBaseParquetWriter + implements HoodieRowDataFileWriter { - private final Path file; - private final HoodieWrapperFileSystem fs; - private final long maxFileSize; private final HoodieRowDataParquetWriteSupport writeSupport; public HoodieRowDataParquetWriter(Path file, HoodieParquetConfig parquetConfig) - throws IOException { - super(HoodieWrapperFileSystem.convertToHoodiePath(file, parquetConfig.getHadoopConf()), - ParquetFileWriter.Mode.CREATE, parquetConfig.getWriteSupport(), parquetConfig.getCompressionCodecName(), - parquetConfig.getBlockSize(), parquetConfig.getPageSize(), parquetConfig.getPageSize(), - DEFAULT_IS_DICTIONARY_ENABLED, DEFAULT_IS_VALIDATING_ENABLED, - DEFAULT_WRITER_VERSION, FSUtils.registerFileSystem(file, parquetConfig.getHadoopConf())); - this.file = HoodieWrapperFileSystem.convertToHoodiePath(file, parquetConfig.getHadoopConf()); - this.fs = (HoodieWrapperFileSystem) this.file.getFileSystem(FSUtils.registerFileSystem(file, - parquetConfig.getHadoopConf())); - this.maxFileSize = parquetConfig.getMaxFileSize() - + Math.round(parquetConfig.getMaxFileSize() * parquetConfig.getCompressionRatio()); - this.writeSupport = parquetConfig.getWriteSupport(); - } + throws IOException { + super(file, parquetConfig); - @Override - public boolean canWrite() { - return fs.getBytesWritten(file) < maxFileSize; + this.writeSupport = parquetConfig.getWriteSupport(); } @Override @@ -70,9 +50,5 @@ public void writeRow(String key, RowData row) throws IOException { public void writeRow(RowData row) throws IOException { super.write(row); } - - @Override - public void close() throws IOException { - super.close(); - } } + diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/storage/row/LSMHoodieRowDataCreateHandle.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/storage/row/LSMHoodieRowDataCreateHandle.java new file mode 100644 index 0000000000000..88103ef33d135 --- /dev/null +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/storage/row/LSMHoodieRowDataCreateHandle.java @@ -0,0 +1,456 @@ +/* + * 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.io.storage.row; + +import org.apache.flink.table.types.logical.LogicalTypeRoot; +import org.apache.hudi.avro.AvroSchemaCache; +import org.apache.hudi.client.HoodieInternalWriteStatus; +import org.apache.hudi.client.model.HoodieFlinkRecord; +import org.apache.hudi.client.model.HoodieRowData; +import org.apache.hudi.client.model.HoodieRowDataCreation; +import org.apache.hudi.common.config.HoodieStorageConfig; +import org.apache.hudi.common.config.TypedProperties; +import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.model.HoodiePartitionMetadata; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecordMerger; +import org.apache.hudi.common.model.HoodieWriteStat; +import org.apache.hudi.common.model.IOType; +import org.apache.hudi.common.model.MultiplePartialUpdateUnit; +import org.apache.hudi.common.storage.HoodieStorageStrategy; +import org.apache.hudi.common.storage.HoodieStorageStrategyFactory; +import org.apache.hudi.common.table.HoodieTableConfig; +import org.apache.hudi.common.util.HoodieTimer; +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.HoodiePayloadConfig; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.exception.HoodieException; +import org.apache.hudi.exception.HoodieIOException; +import org.apache.hudi.exception.HoodieInsertException; +import org.apache.hudi.index.HoodieIndex; +import org.apache.hudi.index.bucket.BucketStrategist; +import org.apache.hudi.index.bucket.BucketStrategistFactory; +import org.apache.hudi.table.HoodieTable; +import org.apache.hudi.table.marker.WriteMarkers; +import org.apache.hudi.table.marker.WriteMarkersFactory; +import org.apache.hudi.util.AvroSchemaConverter; + +import org.apache.avro.Schema; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; + +import java.io.IOException; +import java.io.Serializable; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.atomic.AtomicLong; + +/** + * Copy From HoodieRowDataCreateHandle + * 区别在于独立控制了文件名称 getLSMFileName() + * bucketNumber_UUID_columnFamilyNumber_levelNumber_partitionId-stageId-attemptId.parquet + * + */ +public class LSMHoodieRowDataCreateHandle implements Serializable { + + private static final long serialVersionUID = 1L; + private static final Logger LOG = LogManager.getLogger(LSMHoodieRowDataCreateHandle.class); + private static final AtomicLong SEQGEN = new AtomicLong(1); + + private final String instantTime; + private final int taskPartitionId; + private final long taskId; + private final long taskEpochId; + private final HoodieTable table; + private final HoodieWriteConfig writeConfig; + protected final HoodieRowDataFileWriter fileWriter; + private final String partitionPath; + private final Path path; + private final String fileId; + private final boolean preserveHoodieMetadata; + private final FileSystem fs; + protected final HoodieInternalWriteStatus writeStatus; + private final HoodieTimer currTimer; + private final Map indexToFieldType; + private final RowType rowType; + private final int version; + private final int levelNumber; + private final HoodieTableConfig tableConfig; + private final HoodieRecordMerger recordMerger; + private final TypedProperties payloadProperties; + private final Schema writeSchema; + private final boolean isUpdateEventTime; + private final HoodieStorageStrategy hoodieStorageStrategy; + private String lastRecordKey = null; + private HoodieFlinkRecord lastFlinkRecord = null; + + public LSMHoodieRowDataCreateHandle(HoodieTable table, HoodieWriteConfig writeConfig, String partitionPath, String fileId, + String instantTime, int taskPartitionId, long taskId, long taskEpochId, + RowType rowType, boolean preserveHoodieMetadata, int levelNumber, int version) { + if (levelNumber == 1 && writeConfig.getParquetBlockSize() == 32 * 1024 * 1024) { + writeConfig.setValue(HoodieStorageConfig.PARQUET_BLOCK_SIZE.key(), HoodieStorageConfig.PARQUET_BLOCK_SIZE.defaultValue()); + } + this.version = version; + this.levelNumber = levelNumber; + this.hoodieStorageStrategy = HoodieStorageStrategyFactory.getInstant(table.getMetaClient()); + Option bucketStrategist; + if (writeConfig.getIndexType().equals(HoodieIndex.IndexType.BUCKET) && writeConfig.isBucketIndexAtPartitionLevel()) { + bucketStrategist = Option.of(BucketStrategistFactory.getInstant(writeConfig, table.getMetaClient().getFs())); + } else { + bucketStrategist = Option.empty(); + } + this.partitionPath = partitionPath; + this.table = table; + this.tableConfig = table.getMetaClient().getTableConfig(); + this.writeConfig = writeConfig; + this.instantTime = instantTime; + this.taskPartitionId = taskPartitionId; + this.taskId = taskId; + this.taskEpochId = taskEpochId; + this.fileId = fileId; + this.preserveHoodieMetadata = preserveHoodieMetadata; + this.currTimer = HoodieTimer.start(); + this.fs = table.getMetaClient().getFs(); + String fileName = getLSMFileName(); + this.path = makeNewPath(partitionPath, fileName); + this.rowType = preserveHoodieMetadata ? rowType : addMetadataFields(rowType, writeConfig.allowOperationMetadataField()); + this.writeSchema = preserveHoodieMetadata ? AvroSchemaCache.intern(AvroSchemaConverter.convertToSchema(rowType)) : + AvroSchemaCache.intern(AvroSchemaConverter.convertToSchema(addRecordKeyAndSeqIdMetaFields(rowType))); + this.indexToFieldType = new HashMap<>(); + parseEventTimeField(this.rowType, writeConfig); + this.isUpdateEventTime = supportUpdateEventTime(); + this.writeStatus = new HoodieInternalWriteStatus(!table.getIndex().isImplicitWithStorage(), + writeConfig.getWriteStatusFailureFraction()); + writeStatus.setPartitionPath(partitionPath); + writeStatus.setFileId(fileId); + writeStatus.setStat(new HoodieWriteStat()); + try { + HoodiePartitionMetadata partitionMetadata = + new HoodiePartitionMetadata( + fs, + instantTime, + new Path(writeConfig.getBasePath()), + hoodieStorageStrategy.storageLocation(partitionPath, instantTime), + table.getPartitionMetafileFormat(), + hoodieStorageStrategy, + bucketStrategist.map(strategist -> strategist.computeBucketNumber(partitionPath))); + partitionMetadata.trySave(taskPartitionId); + createMarkerFile(partitionPath, fileName); + this.fileWriter = createNewFileWriter(path, table, writeConfig, this.rowType); + this.recordMerger = writeConfig.getRecordMerger(); + this.payloadProperties = writeConfig.getPayloadConfig().getProps(); + } catch (IOException e) { + throw new HoodieInsertException("Failed to initialize file writer for path " + path, e); + } + LOG.info("New handle created for partition :" + partitionPath + " with fileId " + fileId); + } + + /** + * Writes an {@link RowData} to the underlying {@link HoodieRowDataFileWriter}. + * Before writing, value for meta columns are computed as required + * and wrapped in {@link HoodieRowData}. {@link HoodieRowData} is what gets written to HoodieRowDataFileWriter. + * + * @param recordKey The record key + * @param partitionPath The partition path + * @param record instance of {@link RowData} that needs to be written to the fileWriter. + * @throws IOException + */ + public void write(String recordKey, String partitionPath, RowData record, boolean needDedupe) throws IOException { + if (needDedupe) { + RowData currentRow = copyRow(record); + HoodieFlinkRecord currentFlinkRecord = new HoodieFlinkRecord(currentRow); + if (lastRecordKey != null && lastRecordKey.equals(recordKey)) { + // 相同key,合并记录 + Option> res = recordMerger.merge(lastFlinkRecord, writeSchema, + currentFlinkRecord, writeSchema, payloadProperties); + if (res.isPresent()) { + lastFlinkRecord = (HoodieFlinkRecord) res.get().getLeft(); + } else { + lastFlinkRecord = null; + } + } else { + // 不同key,先写出上一条合并后的记录(如果存在) + if (lastFlinkRecord != null) { + writeRow(lastRecordKey, partitionPath, lastFlinkRecord.getData()); + } + // 更新为当前记录 + lastRecordKey = recordKey; + lastFlinkRecord = currentFlinkRecord; + } + } else { + writeRow(recordKey, partitionPath, record); + } + } + + private void writeRow(String recordKey, String partitionPath, RowData record) { + try { + RowData rowData; + String commitInstant = preserveHoodieMetadata + ? record.getString(HoodieRecord.COMMIT_TIME_METADATA_FIELD_ORD).toString() + : instantTime; + if (preserveHoodieMetadata) { + String seqId = record.getString(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD_ORD).toString(); + rowData = HoodieRowDataCreation.create(commitInstant, seqId, recordKey, partitionPath, path.getName(), + record, writeConfig.allowOperationMetadataField(), preserveHoodieMetadata); + } else { + rowData = HoodieRowDataCreation.createLSMHoodieRowData(commitInstant, partitionPath, + path.getName(), record, writeConfig.allowOperationMetadataField()); + } + try { + fileWriter.writeRow(recordKey, rowData); + if (isUpdateEventTime) { + // set min/max + writeStatus.markSuccess(recordKey, String.valueOf(getValueFromRowData(rowData))); + } else { + writeStatus.markSuccess(recordKey); + } + } catch (Throwable t) { + LOG.error("Failed to write : key is " + recordKey + ", data is " + rowData, t); + writeStatus.markFailure(recordKey, t); + if (!writeConfig.getIgnoreWriteFailed()) { + throw new HoodieException(t.getMessage(), t); + } + } + } catch (Throwable ge) { + writeStatus.setGlobalError(ge); + throw ge; + } + } + + private RowData copyRow(RowData rowData) { + // no need to copy here + // return rowData.copy(); + return rowData; + } + + private void flush() { + if (lastFlinkRecord != null) { + writeRow(lastRecordKey, partitionPath, lastFlinkRecord.getData()); + lastFlinkRecord = null; + lastRecordKey = null; + } + } + + /** + * Parse the eventtime Field and store the mapping between field index and type. + */ + private void parseEventTimeField(RowType rowType, HoodieWriteConfig writeConfig) { + String eventTimeField = writeConfig.getPayloadConfig().getString(HoodiePayloadConfig.EVENT_TIME_FIELD.key()); + if (StringUtils.isNullOrEmpty(eventTimeField)) { + return; + } + + if (eventTimeField.contains("|") || eventTimeField.contains(":")) { + MultiplePartialUpdateUnit multiplePartialUpdateUnit = new MultiplePartialUpdateUnit(eventTimeField); + multiplePartialUpdateUnit.getAllOrderingFields().forEach(field -> + storeFieldIndexAndType(rowType, field) + ); + } else { + storeFieldIndexAndType(rowType, eventTimeField); + } + } + + private void storeFieldIndexAndType(RowType rowType, String fieldName) { + int fieldIndex = rowType.getFieldIndex(fieldName); + if (fieldIndex == -1) { + return; + } + LogicalType fieldType = rowType.getTypeAt(fieldIndex); + this.indexToFieldType.put(fieldIndex, fieldType); + } + + private Object getValueFromRowData(RowData rowData) { + String minEventTimeValue = null; + for (Map.Entry entry : this.indexToFieldType.entrySet()) { + int fieldIndex = entry.getKey(); + LogicalType fieldType = entry.getValue(); + + if (rowData.isNullAt(fieldIndex)) { + continue; + } + + LogicalTypeRoot typeRoot = fieldType.getTypeRoot(); + String currentValue = null; + if (typeRoot == LogicalTypeRoot.BIGINT) { + currentValue = String.valueOf(rowData.getLong(fieldIndex)); + } else if (typeRoot == LogicalTypeRoot.VARCHAR) { + currentValue = String.valueOf(rowData.getString(fieldIndex)); + } else { + continue; + } + + if (minEventTimeValue == null || currentValue.compareTo(minEventTimeValue) < 0) { + minEventTimeValue = currentValue; + } + } + return minEventTimeValue; + } + + /** + * Eventtime only supports bigint and string types. + */ + private boolean supportUpdateEventTime() { + for (Map.Entry entry : this.indexToFieldType.entrySet()) { + int fieldIndex = entry.getKey(); + LogicalTypeRoot typeRoot = entry.getValue().getTypeRoot(); + if (fieldIndex == -1) { + return false; + } + if (typeRoot != LogicalTypeRoot.BIGINT && typeRoot != LogicalTypeRoot.VARCHAR) { + return false; + } + } + return true; + } + + /** + * Returns {@code true} if this handle can take in more writes. else {@code false}. + */ + public boolean canWrite() { + return fileWriter.canWrite(); + } + + /** + * Closes the {@link LSMHoodieRowDataCreateHandle} and returns an instance of {@link HoodieInternalWriteStatus} containing the stats and + * status of the writes to this handle. + * + * @return the {@link HoodieInternalWriteStatus} containing the stats and status of the writes to this handle. + * @throws IOException + */ + public HoodieInternalWriteStatus close() throws IOException { + flush(); + fileWriter.close(); + HoodieWriteStat stat = writeStatus.getStat(); + stat.setPartitionPath(partitionPath); + stat.setNumWrites(writeStatus.getTotalRecords()); + stat.setNumDeletes(0); + stat.setNumInserts(writeStatus.getTotalRecords()); + stat.setPrevCommit(HoodieWriteStat.NULL_COMMIT); + stat.setFileId(fileId); + stat.setPath(hoodieStorageStrategy.getRelativePath(path)); + long fileSizeInBytes = FSUtils.getFileSize(table.getMetaClient().getFs(), path); + stat.setTotalWriteBytes(fileSizeInBytes); + stat.setFileSizeInBytes(fileSizeInBytes); + stat.setTotalWriteErrors(writeStatus.getFailedRowsSize()); + HoodieWriteStat.RuntimeStats runtimeStats = new HoodieWriteStat.RuntimeStats(); + runtimeStats.setTotalCreateTime(currTimer.endTimer()); + stat.setRuntimeStats(runtimeStats); + return writeStatus; + } + + public String getFileName() { + return path.getName(); + } + + private Path makeNewPath(String partitionPath, String fileName) { + Path path = hoodieStorageStrategy.storageLocation(partitionPath, instantTime); + try { + if (!fs.exists(path)) { + fs.mkdirs(path); // create a new partition as needed. + } + } catch (IOException e) { + throw new HoodieIOException("Failed to make dir " + path, e); + } + return new Path(path.toString(), fileName); + } + + /** + * Creates an empty marker file corresponding to storage writer path. + * + * @param partitionPath Partition path + */ + private void createMarkerFile(String partitionPath, String dataFileName) { + WriteMarkers writeMarkers = WriteMarkersFactory.get(writeConfig.getMarkersType(), table, instantTime); + writeMarkers.create(partitionPath, dataFileName, IOType.CREATE); + } + + private String getWriteToken() { + return taskPartitionId + "-" + taskId + "-" + taskEpochId; + } + + protected HoodieRowDataFileWriter createNewFileWriter( + Path path, HoodieTable hoodieTable, HoodieWriteConfig config, RowType rowType) + throws IOException { + return HoodieRowDataFileWriterFactory.getRowDataFileWriter( + path, hoodieTable, config, rowType); + } + + private String getLSMFileName() { + return FSUtils.makeLSMFileName(fileId, instantTime, String.valueOf(version), String.valueOf(levelNumber), + getWriteToken(), tableConfig.getBaseFileFormat().getFileExtension()); + } + + public static RowType addMetadataFields(RowType rowType, boolean withOperationField) { + List mergedFields = new ArrayList<>(); + + LogicalType metadataFieldType = DataTypes.STRING().getLogicalType(); + RowType.RowField commitTimeField = + new RowType.RowField(HoodieRecord.COMMIT_TIME_METADATA_FIELD, metadataFieldType, "commit time"); + RowType.RowField commitSeqnoField = + new RowType.RowField(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD, metadataFieldType, "commit seqno"); + RowType.RowField recordKeyField = + new RowType.RowField(HoodieRecord.RECORD_KEY_METADATA_FIELD, metadataFieldType, "record key"); + RowType.RowField partitionPathField = + new RowType.RowField(HoodieRecord.PARTITION_PATH_METADATA_FIELD, metadataFieldType, "partition path"); + RowType.RowField fileNameField = + new RowType.RowField(HoodieRecord.FILENAME_METADATA_FIELD, metadataFieldType, "field name"); + + mergedFields.add(commitTimeField); + mergedFields.add(commitSeqnoField); + mergedFields.add(recordKeyField); + mergedFields.add(partitionPathField); + mergedFields.add(fileNameField); + + if (withOperationField) { + RowType.RowField operationField = + new RowType.RowField(HoodieRecord.OPERATION_METADATA_FIELD, metadataFieldType, "operation"); + mergedFields.add(operationField); + } + + mergedFields.addAll(rowType.getFields()); + + return new RowType(false, mergedFields); + } + + public static RowType addRecordKeyAndSeqIdMetaFields(RowType rowType) { + List mergedFields = new ArrayList<>(); + + LogicalType metadataFieldType = DataTypes.STRING().getLogicalType(); + RowType.RowField recordKeyField = + new RowType.RowField(HoodieRecord.RECORD_KEY_METADATA_FIELD, metadataFieldType, "record key"); + RowType.RowField commitSeqnoField = + new RowType.RowField(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD, metadataFieldType, "commit seqno"); + + mergedFields.add(recordKeyField); + mergedFields.add(commitSeqnoField); + mergedFields.addAll(rowType.getFields()); + + return new RowType(false, mergedFields); + } +} diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/plan/strategy/SparkLsmBaseClusteringPlanStrategy.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/plan/strategy/SparkLsmBaseClusteringPlanStrategy.java new file mode 100644 index 0000000000000..cc7fab9069905 --- /dev/null +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/plan/strategy/SparkLsmBaseClusteringPlanStrategy.java @@ -0,0 +1,144 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.client.clustering.plan.strategy; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hudi.avro.model.HoodieClusteringGroup; +import org.apache.hudi.avro.model.HoodieClusteringPlan; +import org.apache.hudi.client.common.HoodieSparkEngineContext; +import org.apache.hudi.common.config.SerializableConfiguration; +import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.model.FileSlice; +import org.apache.hudi.common.model.HoodieLogFile; +import org.apache.hudi.common.model.lsm.HoodieLSMLogFile; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.StringUtils; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.table.HoodieTable; +import org.apache.hudi.table.action.cluster.strategy.LsmBaseClusteringPlanStrategy; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; +import scala.Tuple4; + +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +public class SparkLsmBaseClusteringPlanStrategy extends LsmBaseClusteringPlanStrategy { + private static final Logger LOG = LogManager.getLogger(SparkLsmBaseClusteringPlanStrategy.class); + + private transient HoodieSparkEngineContext hsc; + private final SerializableConfiguration serializableConfiguration; + + public SparkLsmBaseClusteringPlanStrategy(HoodieTable table, HoodieEngineContext engineContext, HoodieWriteConfig writeConfig) { + super(table, engineContext, writeConfig); + hsc = (HoodieSparkEngineContext) engineContext; + Configuration configuration = hsc.getHadoopConf().get(); + serializableConfiguration = new SerializableConfiguration(configuration); + } + + @Override + public Option generateClusteringPlan() { + if (plan != null && plan.isPresent()) { + return plan; + } + + if (!checkPrecondition()) { + return Option.empty(); + } + + HoodieTableMetaClient metaClient = getHoodieTable().getMetaClient(); + LOG.info("Scheduling clustering for " + metaClient.getBasePath()); + Pair, Pair, Set>> partitionsAndInstantsPair = getPartitionPathsToCluster(); + List partitionPaths = partitionsAndInstantsPair.getLeft(); + Set missingPartitions = new HashSet<>(); + + if (partitionPaths.isEmpty()) { + // In case no partitions could be picked, return no clustering plan + return Option.empty(); + } + + // + List>> partitionToLogFiles = hsc.getJavaSparkContext().parallelize(partitionPaths, partitionPaths.size()) + .flatMap(partitionPath -> { + Pair, Set> filesEligibleForClustering = getFilesEligibleForClustering(partitionPath); + boolean level1Exists = !filesEligibleForClustering.getRight().isEmpty(); + return filesEligibleForClustering.getLeft().stream().map(slice -> { + return new Tuple4<>(level1Exists, partitionPath, slice.getFileId(), slice.getLogFiles().collect(Collectors.toList())); + }).iterator(); + }).collect(); + + // + List>> partitionClusteringGroupPair = hsc.map( + partitionToLogFiles, + tuple -> { + // read footer when hoodie.clustering.lsm.readfooter.enabled is true + List lsmLogFiles = getLSMLogFiles(getWriteConfig().isLsmReadFooterEnabled(), tuple._4()); + Set level1InPendingClustering = getLevel1InPendingClustering(tuple._2()); + Pair> pair = buildClusteringGroups(lsmLogFiles, tuple._1(), level1InPendingClustering.contains(tuple._3()), tuple._2(), skipOP); + if (pair.getLeft()) { + return Pair.of(tuple._2(), pair.getRight()); + } + return Pair.of("", pair.getRight()); + }, Math.max(1, Math.min(partitionToLogFiles.size(), getWriteConfig().getLsmReadFooterParallelism()))); + + List clusteringGroups = partitionClusteringGroupPair.stream().flatMap(pair -> { + String missingPartition = pair.getLeft(); + if (!StringUtils.isNullOrEmpty(missingPartition)) { + // missingPartition value is not empty, which means it related candidate fileSlices all not all processed. + // so that we need to mark this kind of partition as missing partition. + missingPartitions.add(missingPartition); + } + return pair.getRight(); + }).collect(Collectors.toList()); + + if (clusteringGroups.isEmpty()) { + LOG.info("No data available to cluster"); + return Option.empty(); + } + + return buildClusteringPlan(metaClient, + clusteringGroups, + getWriteConfig(), + partitionsAndInstantsPair.getRight().getLeft(), + partitionsAndInstantsPair.getRight().getRight(), + missingPartitions, + getStrategyParams(), + getExtraMetadata(), + getPlanVersion()); + } + + private List getLSMLogFiles(boolean readFooterEnabled, List logFileList) { + if (readFooterEnabled) { + return logFileList.stream().map(logFile -> { + HoodieLSMLogFile lsmLogFile = (HoodieLSMLogFile) logFile; + String[] minMax = PARQUET_UTILS.readMinMaxRecordKeys(serializableConfiguration.get(), lsmLogFile.getPath()); + lsmLogFile.setMin(minMax[0]); + lsmLogFile.setMax(minMax[1]); + return lsmLogFile; + }).collect(Collectors.toList()); + } + return logFileList; + } + +} \ No newline at end of file diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/LsmSparkClusteringExecutionStrategy.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/LsmSparkClusteringExecutionStrategy.java new file mode 100644 index 0000000000000..b20448d213118 --- /dev/null +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/LsmSparkClusteringExecutionStrategy.java @@ -0,0 +1,196 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.client.clustering.run.strategy; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hudi.HoodieDatasetBulkInsertHelper; +import org.apache.hudi.HoodieSparkUtils; +import org.apache.hudi.SparkAdapterSupport$; +import org.apache.hudi.avro.AvroSchemaCache; +import org.apache.hudi.avro.HoodieAvroUtils; +import org.apache.hudi.avro.model.HoodieClusteringPlan; +import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.client.common.HoodieSparkEngineContext; +import org.apache.hudi.common.config.HoodieStorageConfig; +import org.apache.hudi.common.config.TypedProperties; +import org.apache.hudi.common.data.HoodieData; +import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.model.ClusteringGroupInfo; +import org.apache.hudi.common.model.ClusteringOperation; +import org.apache.hudi.common.model.HoodieFileGroupId; +import org.apache.hudi.common.model.HoodieKey; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecordMerger; +import org.apache.hudi.common.model.HoodieSparkRecord; +import org.apache.hudi.common.model.WriteOperationType; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.ValidationUtils; +import org.apache.hudi.config.HoodiePayloadConfig; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.data.HoodieJavaRDD; +import org.apache.hudi.exception.HoodieException; +import org.apache.hudi.io.lsm.MergeSorter; +import org.apache.hudi.io.lsm.RecordReader; +import org.apache.hudi.io.lsm.SparkRecordMergeWrapper; +import org.apache.hudi.io.lsm.SparkRecordReader; +import org.apache.hudi.table.HoodieTable; + +import org.apache.avro.Schema; +import org.apache.hudi.table.action.HoodieWriteMetadata; +import org.apache.hudi.table.action.cluster.strategy.ClusteringExecutionStrategy; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.sql.HoodieInternalRowUtils; +import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.execution.datasources.PartitionedFile; +import org.apache.spark.sql.hudi.SparkAdapter; +import org.apache.spark.sql.types.StructType; +import scala.Function1; + +import java.io.IOException; +import java.util.Comparator; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +import static org.apache.hudi.config.HoodieClusteringConfig.LSM_CLUSTERING_OUT_PUT_LEVEL; +import static org.apache.hudi.config.HoodieClusteringConfig.LSM_CLUSTERING_USING_STREAMING_COPY; + +public class LsmSparkClusteringExecutionStrategy extends + ClusteringExecutionStrategy>, HoodieData, HoodieData> { + private static final Logger LOG = LogManager.getLogger(LsmSparkClusteringExecutionStrategy.class); + + public LsmSparkClusteringExecutionStrategy(HoodieTable table, + HoodieEngineContext engineContext, + HoodieWriteConfig writeConfig) { + super(table, engineContext, writeConfig); + } + + @Override + public HoodieWriteMetadata> performClustering(final HoodieClusteringPlan clusteringPlan, final Schema schema, final String instantTime) { + ValidationUtils.checkArgument(getWriteConfig().getBooleanOrDefault("hoodie.datasource.write.row.writer.enable", false), "Please set hoodie.datasource.write.row.writer.enable true"); + + JavaSparkContext engineContext = HoodieSparkEngineContext.getSparkContext(getEngineContext()); + final List clusteringGroupInfos = clusteringPlan.getInputGroups().stream().map(ClusteringGroupInfo::create).collect(Collectors.toList()); + StructType structSchema = HoodieInternalRowUtils.getCachedSchema(schema); + Function1> parquetReader = getParquetReader(engineContext, structSchema); + + int parallelism = clusteringGroupInfos.size(); + JavaRDD groupInfosJavaRDD = engineContext.parallelize(clusteringGroupInfos, parallelism); + LOG.info("number of partitions for clustering " + groupInfosJavaRDD.getNumPartitions()); + List> writeStatuses = groupInfosJavaRDD + .map(groupInfo -> { + return runClusteringForGroupAsyncAsRow(groupInfo, parquetReader, clusteringPlan.getStrategy().getStrategyParams(), + Option.ofNullable(clusteringPlan.getPreserveHoodieMetadata()).orElse(true), instantTime, getWriteConfig()); + }).collect(); + + List statusList = writeStatuses.stream().flatMap(writeStatus -> writeStatus.stream()).collect(Collectors.toList()); + + HoodieWriteMetadata> writeMetadata = new HoodieWriteMetadata<>(); + writeMetadata.setWriteStatuses(HoodieJavaRDD.of(statusList, (HoodieSparkEngineContext) getEngineContext(), parallelism)); + return writeMetadata; + } + + protected List runClusteringForGroupAsyncAsRow(ClusteringGroupInfo clusteringGroupInfo, + Function1> parquetReader, + Map strategyParams, + boolean shouldPreserveHoodieMetadata, + String instantTime, + HoodieWriteConfig writeConfig) { + SparkAdapter sparkAdapter = SparkAdapterSupport$.MODULE$.sparkAdapter(); + List operations = clusteringGroupInfo.getOperations(); + final Schema schema = AvroSchemaCache.intern(HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(writeConfig.getSchema()))); + List> fileReaders = operations.stream().map(operation -> { + PartitionedFile partitionedFile = sparkAdapter.getSparkPartitionedFileUtils().createPartitionedFile( + InternalRow.empty(), new Path(operation.getDataFilePath()), 0, Long.MAX_VALUE); + try { + Iterator iterator = parquetReader.apply(partitionedFile); + return new SparkRecordReader(iterator, schema); + } catch (Exception e) { + throw new HoodieException(e.getMessage(), e); + } + }).collect(Collectors.toList()); + + // 判断是否是 major compaction + boolean isIgnoreDelete = "1".equals(clusteringGroupInfo.getExtraMeta().get(LSM_CLUSTERING_OUT_PUT_LEVEL)); + HoodieRecordMerger recordMerger = writeConfig.getRecordMerger(); + TypedProperties props = HoodiePayloadConfig.newBuilder().withPayloadOrderingField(writeConfig.getPreCombineField()).build().getProps(); + StructType structSchema = HoodieInternalRowUtils.getCachedSchema(schema); + RecordReader reader = new MergeSorter().mergeSort(fileReaders, new SparkRecordMergeWrapper(isIgnoreDelete, recordMerger, + schema, schema, props, structSchema), getHoodieSparkRecordComparator()); + + try { + Iterator iterator = reader.read(); + return performClusteringWithRecordsAsRow(iterator, 1, instantTime, strategyParams, structSchema, + null, shouldPreserveHoodieMetadata, clusteringGroupInfo.getExtraMeta()); + } catch (IOException e) { + throw new RuntimeException(e); + } finally { + try { + reader.close(); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + } + + protected Function1> getParquetReader(JavaSparkContext jsc, StructType schema) { + SQLContext sqlContext = new SQLContext(jsc.sc()); + Map params = new HashMap<>(); + params.put("path", getHoodieTable().getMetaClient().getTableConfig().getBasePath()); + return HoodieSparkUtils.getParquetReader(sqlContext.sparkSession(), schema, params, new Configuration(jsc.hadoopConfiguration())); + } + + public static Comparator getHoodieSparkRecordComparator() { + return new Comparator() { + @Override + public int compare(HoodieRecord o1, HoodieRecord o2) { + HoodieSparkRecord record1 = (HoodieSparkRecord) o1; + HoodieSparkRecord record2 = (HoodieSparkRecord) o2; + return record1.getUtf8RecordKey().compare(record2.getUtf8RecordKey()); + } + }; + } + + public List performClusteringWithRecordsAsRow(Iterator iterator, + int numOutputGroups, + String instantTime, Map strategyParams, + StructType schema, + List fileGroupIdList, + boolean shouldPreserveHoodieMetadata, + Map extraMetadata) { + TypedProperties props = getWriteConfig().getProps(); + props.put(LSM_CLUSTERING_OUT_PUT_LEVEL, extraMetadata.get(LSM_CLUSTERING_OUT_PUT_LEVEL)); + props.put(LSM_CLUSTERING_USING_STREAMING_COPY, extraMetadata.get(LSM_CLUSTERING_USING_STREAMING_COPY)); + HoodieWriteConfig newConfig = HoodieWriteConfig.newBuilder() + .withProps(props) + .withOperation(WriteOperationType.CLUSTER.name()) + .build(); + newConfig.setValue(HoodieStorageConfig.PARQUET_RECORDKEY_BLOOM_FILTER_ENABLED, String.valueOf(getWriteConfig().parquetRecordkeyClusteringBloomFilterEnabled())); + // inputRecords 已经有序且去重,直接写即可,不需要shuffle和merge + return HoodieDatasetBulkInsertHelper.bulkInsertWithLsmClustering(iterator, schema, instantTime, getHoodieTable(), newConfig, + true, shouldPreserveHoodieMetadata); + } +} diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/common/model/HoodieSparkRecord.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/common/model/HoodieSparkRecord.java index 7756d2502e897..5271f1abbf949 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/common/model/HoodieSparkRecord.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/common/model/HoodieSparkRecord.java @@ -82,6 +82,7 @@ public class HoodieSparkRecord extends HoodieRecord implements Kryo * (by Kryo) */ private final transient StructType schema; + private UTF8String utf8RecordKey; public HoodieSparkRecord(UnsafeRow data) { this(data, null); @@ -95,6 +96,15 @@ public HoodieSparkRecord(InternalRow data, StructType schema) { this.schema = schema; } + public HoodieSparkRecord(UTF8String recordKey, InternalRow data, StructType schema) { + super(null, data); + + validateRow(data, schema); + this.copy = false; + this.schema = schema; + this.utf8RecordKey = recordKey; + } + public HoodieSparkRecord(HoodieKey key, UnsafeRow data, boolean copy) { this(key, data, null, copy); } @@ -154,6 +164,10 @@ public String getRecordKey(Schema recordSchema, Option keyGene : data.getString(HoodieMetadataField.RECORD_KEY_METADATA_FIELD.ordinal()); } + public UTF8String getUtf8RecordKey() { + return this.utf8RecordKey; + } + @Override public String getRecordKey(Schema recordSchema, String keyFieldName) { if (key != null) { @@ -226,6 +240,11 @@ public boolean isDelete(Schema recordSchema, Properties props) throws IOExceptio if (null == data) { return true; } + + if (operation != null && HoodieOperation.isDelete(getOperation())) { + return true; + } + if (recordSchema.getField(HoodieRecord.HOODIE_IS_DELETED_FIELD) == null) { return false; } @@ -295,7 +314,7 @@ public HoodieSparkRecord copy() { } @Override - public Comparable getOrderingValue(Schema recordSchema, Properties props) { + public Comparable doGetOrderingValue(Schema recordSchema, Properties props) { StructType structType = HoodieInternalRowUtils.getCachedSchema(recordSchema); String orderingField = ConfigUtils.getOrderingField(props); scala.Option cachedNestedFieldPath = diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/BucketIndexBulkInsertPartitionerWithRows.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/BucketIndexBulkInsertPartitionerWithRows.java new file mode 100644 index 0000000000000..dc444a1ea999f --- /dev/null +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/BucketIndexBulkInsertPartitionerWithRows.java @@ -0,0 +1,50 @@ +/* + * 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.execution.bulkinsert; + +import org.apache.hudi.index.bucket.BucketStrategist; +import org.apache.hudi.table.BulkInsertPartitioner; + +import org.apache.spark.sql.BucketPartitionUtils$; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; + +/** + * Bulk_insert partitioner of Spark row using bucket index. + */ +public class BucketIndexBulkInsertPartitionerWithRows implements BulkInsertPartitioner> { + + private final String indexKeyFields; + protected final BucketStrategist bucketStrategist; + + public BucketIndexBulkInsertPartitionerWithRows(String indexKeyFields, BucketStrategist bucketStrategist) { + this.indexKeyFields = indexKeyFields; + this.bucketStrategist = bucketStrategist; + } + + @Override + public Dataset repartitionRecords(Dataset rows, int outputPartitions) { + return BucketPartitionUtils$.MODULE$.createDataFrame(rows, indexKeyFields, bucketStrategist, outputPartitions, true); + } + + @Override + public boolean arePartitionRecordsSorted() { + return true; + } +} diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/LSMBucketIndexBulkInsertPartitionerWithRows.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/LSMBucketIndexBulkInsertPartitionerWithRows.java new file mode 100644 index 0000000000000..1f07f4899f896 --- /dev/null +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/LSMBucketIndexBulkInsertPartitionerWithRows.java @@ -0,0 +1,56 @@ +/* + * 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.execution.bulkinsert; + +import org.apache.hudi.common.table.view.FileSystemViewStorageConfig; +import org.apache.hudi.config.HoodieWriteConfig; + +import org.apache.spark.Partitioner; +import org.apache.spark.sql.BucketPartitionUtils$; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; + +/** + * Bulk_insert partitioner of Spark row using bucket index. + */ +public class LSMBucketIndexBulkInsertPartitionerWithRows extends BucketIndexBulkInsertPartitionerWithRows { + private final HoodieWriteConfig writeConfig; + private boolean isUsingRemotePartitioner = false; + private FileSystemViewStorageConfig viewConf; + + public LSMBucketIndexBulkInsertPartitionerWithRows(HoodieWriteConfig writeConfig) { + super(writeConfig.getBucketIndexHashFieldWithDefault(), null); + this.writeConfig = writeConfig; + if (writeConfig.isEmbeddedTimelineServerEnabled() && writeConfig.isUsingRemotePartitioner()) { + this.isUsingRemotePartitioner = true; + this.viewConf = writeConfig.getViewStorageConfig(); + } + } + + @Override + public Dataset repartitionRecords(Dataset rows, int outputPartitions) { + String indexKeyFields = writeConfig.getBucketIndexHashFieldWithDefault(); + int numBuckets = writeConfig.getBucketIndexNumBuckets(); + int factor = writeConfig.getLsmShuffleFactor(); + Partitioner partitioner = isUsingRemotePartitioner ? BucketPartitionUtils$.MODULE$.getRemotePartitioner(numBuckets, outputPartitions, viewConf, factor) + : BucketPartitionUtils$.MODULE$.getDefaultLsmPartitioner(numBuckets, outputPartitions, factor); + + return BucketPartitionUtils$.MODULE$.createLSMDataFrame(rows, indexKeyFields, numBuckets, partitioner); + } +} diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/RDDBucketIndexPartitioner.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/RDDBucketIndexPartitioner.java index 6c490f5bebcec..f0c5b9f82ec3d 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/RDDBucketIndexPartitioner.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/RDDBucketIndexPartitioner.java @@ -18,15 +18,106 @@ package org.apache.hudi.execution.bulkinsert; +import org.apache.avro.Schema; +import org.apache.hudi.avro.HoodieAvroUtils; +import org.apache.hudi.common.config.SerializableSchema; +import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.table.BulkInsertPartitioner; +import org.apache.hudi.common.util.collection.FlatLists; +import org.apache.hudi.table.BucketIndexBulkInsertPartitioner; +import org.apache.hudi.table.HoodieTable; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; +import org.apache.spark.Partitioner; import org.apache.spark.api.java.JavaRDD; +import scala.Tuple2; + +import java.io.Serializable; +import java.util.Comparator; /** * Abstract of bucket index bulk_insert partitioner - * TODO implement partitioner for SIMPLE BUCKET INDEX */ -public abstract class RDDBucketIndexPartitioner - implements BulkInsertPartitioner>> { + +public abstract class RDDBucketIndexPartitioner extends BucketIndexBulkInsertPartitioner>> { + + public static final Logger LOG = LogManager.getLogger(RDDBucketIndexPartitioner.class); + + public RDDBucketIndexPartitioner(HoodieTable table, String sortString, boolean preserveHoodieMetadata) { + super(table, sortString, preserveHoodieMetadata); + } + + /** + * Execute partition using the given partitioner. + * If sorting is required, will do it within each data partition: + * - if sortColumnNames is specified, apply sort to the column (the behaviour is the same as `RDDCustomColumnsSortPartitioner`) + * - if table requires sort or BulkInsertSortMode is not None, then sort by record key within partition. + * By default, do partition only. + * + * @param records + * @param partitioner a default partition that accepts `HoodieKey` as the partition key + * @return + */ + + public JavaRDD> doPartition(JavaRDD> records, Partitioner partitioner) { + if (sortColumnNames != null && sortColumnNames.length > 0) { + return doPartitionAndCustomColumnSort(records, partitioner); + } else if (table.requireSortedRecords() || table.getConfig().getBulkInsertSortMode() != BulkInsertSortMode.NONE) { + return doPartitionAndSortByRecordKey(records, partitioner); + } else { + // By default, do partition only + return records.mapToPair(record -> new Tuple2<>(record.getKey(), record)) + .partitionBy(partitioner).map(Tuple2::_2); + } + } + + /** + * Sort by specified column value. The behaviour is the same as `RDDCustomColumnsSortPartitioner` + * + * @param records + * @param partitioner + * @return + */ + private JavaRDD> doPartitionAndCustomColumnSort(JavaRDD> records, Partitioner partitioner) { + final String[] sortColumns = sortColumnNames; + final SerializableSchema schema = new SerializableSchema(HoodieAvroUtils.addMetadataFields((new Schema.Parser().parse(table.getConfig().getSchema())))); + Comparator> comparator = (Comparator> & Serializable) (t1, t2) -> { + FlatLists.ComparableList obj1 = FlatLists.ofComparableArray(t1.getColumnValues(schema.get(), sortColumns, consistentLogicalTimestampEnabled)); + FlatLists.ComparableList obj2 = FlatLists.ofComparableArray(t2.getColumnValues(schema.get(), sortColumns, consistentLogicalTimestampEnabled)); + return obj1.compareTo(obj2); + }; + + return records.mapToPair(record -> new Tuple2<>(record, record)) + .repartitionAndSortWithinPartitions(new Partitioner() { + @Override + public int numPartitions() { + return partitioner.numPartitions(); + } + + @Override + public int getPartition(Object key) { + return partitioner.getPartition(((HoodieRecord) key).getKey()); + } + }, comparator).map(Tuple2::_2); + } + + /** + * Sort by record key within each partition. The behaviour is the same as BulkInsertSortMode.PARTITION_SORT. + * + * @param records + * @param partitioner + * @return + */ + private JavaRDD> doPartitionAndSortByRecordKey(JavaRDD> records, Partitioner partitioner) { + if (table.getConfig().getBulkInsertSortMode() == BulkInsertSortMode.GLOBAL_SORT) { + LOG.warn("Bucket index does not support global sort mode, the sort will only be done within each data partition"); + } + + Comparator comparator = (Comparator & Serializable) (t1, t2) -> t1.getRecordKey().compareTo(t2.getRecordKey()); + + return records.mapToPair(record -> new Tuple2<>(record.getKey(), record)) + .repartitionAndSortWithinPartitions(partitioner, comparator) + .map(Tuple2::_2); + } } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/RDDSimpleBucketBulkInsertPartitioner.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/RDDSimpleBucketBulkInsertPartitioner.java new file mode 100644 index 0000000000000..3fd5fd4f0a4e8 --- /dev/null +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/RDDSimpleBucketBulkInsertPartitioner.java @@ -0,0 +1,107 @@ +/* + * 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.execution.bulkinsert; + +import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.model.HoodieKey; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecordLocation; +import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.util.ValidationUtils; +import org.apache.hudi.index.bucket.BucketIdentifier; +import org.apache.hudi.index.bucket.HoodieSimpleBucketIndex; +import org.apache.hudi.table.HoodieTable; +import org.apache.spark.Partitioner; +import org.apache.spark.api.java.JavaRDD; + +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.stream.Collectors; + +public class RDDSimpleBucketBulkInsertPartitioner extends RDDBucketIndexPartitioner { + + public RDDSimpleBucketBulkInsertPartitioner(HoodieTable table) { + super(table, null, false); + ValidationUtils.checkArgument(table.getIndex() instanceof HoodieSimpleBucketIndex); + } + + @Override + public JavaRDD> repartitionRecords(JavaRDD> records, int outputPartitions) { + HoodieSimpleBucketIndex index = (HoodieSimpleBucketIndex) table.getIndex(); + Map fileIdPrefixToBucketIndex = new HashMap<>(); + + // Map > + Map> partitionMapper = getPartitionMapper(records, fileIdPrefixToBucketIndex); + + return doPartition(records, new Partitioner() { + @Override + public int numPartitions() { + return index.getNumBuckets() * partitionMapper.size(); + } + + @Override + public int getPartition(Object key) { + HoodieKey hoodieKey = (HoodieKey) key; + String partitionPath = hoodieKey.getPartitionPath(); + int bucketID = index.getBucketID(hoodieKey); + String fileID = partitionMapper.get(partitionPath).get(bucketID); + return fileIdPrefixToBucketIndex.get(fileID); + } + }); + } + + Map> getPartitionMapper(JavaRDD> records, + Map fileIdPrefixToBucketIndex) { + + HoodieSimpleBucketIndex index = (HoodieSimpleBucketIndex) table.getIndex(); + int numBuckets = index.getNumBuckets(); + return records + .map(HoodieRecord::getPartitionPath) + .distinct().collect().stream() + .collect(Collectors.toMap(p -> p, p -> { + Map locationMap = index.loadBucketIdToFileIdMappingForPartition(table, p); + Map bucketIdToFileIdPrefixMap = new HashMap<>(); + HashSet existsBucketID = new HashSet<>(); + + // Load an existing index + locationMap.forEach((k, v) -> { + String prefix = FSUtils.getFileIdPfxFromFileId(v.getFileId()); + bucketIdToFileIdPrefixMap.put(k, prefix); + fileIdPrefixToBucketIndex.put(prefix, fileIdPfxList.size()); + fileIdPfxList.add(prefix); + existsBucketID.add(BucketIdentifier.bucketIdFromFileId(prefix)); + doAppend.add(true); + }); + + // Generate a file that does not exist + for (int i = 0; i < numBuckets; i++) { + if (!existsBucketID.contains(i)) { + String fileIdPrefix = BucketIdentifier.newBucketFileIdPrefix(i); + fileIdPrefixToBucketIndex.put(fileIdPrefix, fileIdPfxList.size()); + fileIdPfxList.add(fileIdPrefix); + doAppend.add(false); + bucketIdToFileIdPrefixMap.put(i, fileIdPrefix); + } + } + return bucketIdToFileIdPrefixMap; + })); + } +} + diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/RowCustomColumnsSortPartitioner.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/RowCustomColumnsSortPartitioner.java index d1d9e0bbb4c2c..a61e6fd4101af 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/RowCustomColumnsSortPartitioner.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/RowCustomColumnsSortPartitioner.java @@ -28,7 +28,10 @@ import java.util.Arrays; /** - * A partitioner that does sorting based on specified column values for each spark partitions. + * A partitioner that globally sorts a {@link Dataset} based on partition path column and custom columns. + * + * @see GlobalSortPartitionerWithRows + * @see BulkInsertSortMode#GLOBAL_SORT */ public class RowCustomColumnsSortPartitioner implements BulkInsertPartitioner> { diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/lsm/SparkRecordMergeWrapper.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/lsm/SparkRecordMergeWrapper.java new file mode 100644 index 0000000000000..f2b4f3f8a825c --- /dev/null +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/lsm/SparkRecordMergeWrapper.java @@ -0,0 +1,135 @@ +/* + * 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.io.lsm; + +import org.apache.hudi.common.config.TypedProperties; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecordMerger; +import org.apache.hudi.common.model.HoodieSparkRecord; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.exception.HoodieIOException; + +import org.apache.avro.Schema; +import org.apache.spark.sql.HoodieInternalRowUtils; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.catalyst.expressions.UnsafeProjection; +import org.apache.spark.sql.types.StructType; + +import java.io.IOException; +import java.util.Iterator; +import java.util.List; +import java.util.Properties; + +public class SparkRecordMergeWrapper implements RecordMergeWrapper { + private final HoodieRecordMerger recordMerger; + private final Schema oldSchema; + private final Schema newSchema; + private final TypedProperties props; + private final StructType structTypeSchema; + + private final boolean isIgnoreDelete; + + private Option> nextRecord = Option.empty(); + + public SparkRecordMergeWrapper(boolean isIgnoreDelete, + HoodieRecordMerger recordMerger, + Schema oldSchema, + Schema newSchema, + TypedProperties props, + StructType structTypeSchema) { + this.isIgnoreDelete = isIgnoreDelete; + this.recordMerger = recordMerger; + this.oldSchema = oldSchema; + this.newSchema = newSchema; + this.props = props; + this.structTypeSchema = structTypeSchema; + } + + @Override + public Option merge(List recordGroup) { + + for (HoodieRecord hoodieRecord : recordGroup) { + HoodieSparkRecord sparkRecord = (HoodieSparkRecord) hoodieRecord; + if (!nextRecord.isPresent()) { + nextRecord = Option.of(Pair.of(sparkRecord, oldSchema)); + } else { + nextRecord = mergeInternal((HoodieSparkRecord)nextRecord.get().getKey(), sparkRecord); + } + } + + return nextRecord.map(pair -> { + StructType schema = HoodieInternalRowUtils.getCachedSchema(pair.getRight()); + UnsafeProjection projection = HoodieInternalRowUtils.getCachedUnsafeProjection(schema, structTypeSchema); + return projection.apply((InternalRow)pair.getLeft().getData()); + }); + } + + @Override + public Option merge(Iterator sameKeyIterator) { + while (sameKeyIterator.hasNext()) { + HoodieSparkRecord sparkRecord = (HoodieSparkRecord) sameKeyIterator.next(); + if (!nextRecord.isPresent()) { + nextRecord = Option.of(Pair.of(sparkRecord, oldSchema)); + } else { + nextRecord = mergeInternal((HoodieSparkRecord)nextRecord.get().getKey(), sparkRecord); + } + } + + return nextRecord.map(pair -> { + return (InternalRow)pair.getLeft().getData(); + }); + } + + @Override + public void merge(HoodieRecord record) { + HoodieSparkRecord sparkRecord = (HoodieSparkRecord) record; + if (!nextRecord.isPresent()) { + nextRecord = Option.of(Pair.of(sparkRecord, oldSchema)); + } else { + nextRecord = mergeInternal((HoodieSparkRecord) nextRecord.get().getKey(), sparkRecord); + } + } + + @Override + public Option getMergedResult() { + HoodieRecord left = nextRecord.get().getLeft(); + try { + if (isIgnoreDelete && left.isDelete(newSchema, new Properties())) { + return Option.empty(); + } + } catch (Exception e) { + throw new HoodieIOException(e.getMessage()); + } + return nextRecord.map(pair -> (InternalRow) pair.getLeft().getData()); + } + + @Override + public void reset() { + this.nextRecord = Option.empty(); + } + + private Option> mergeInternal(HoodieSparkRecord oldRecord, HoodieSparkRecord newRecord) { + try { + return recordMerger.merge(oldRecord, oldSchema, newRecord, newSchema, props); + } catch (IOException e) { + throw new HoodieIOException(e.getMessage(), e); + } + } +} diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/lsm/SparkRecordReader.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/lsm/SparkRecordReader.java new file mode 100644 index 0000000000000..f93efce941ad4 --- /dev/null +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/lsm/SparkRecordReader.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.io.lsm; + +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieSparkRecord; +import org.apache.hudi.common.util.collection.MappingIterator; + +import org.apache.avro.Schema; +import org.apache.spark.sql.HoodieInternalRowUtils; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.types.StructType; + +import java.io.IOException; +import java.util.Iterator; + +public class SparkRecordReader implements RecordReader { + private final StructType structType; + private final int recordKeyIndex; + private Iterator iterator; + + public SparkRecordReader(Iterator iterator, Schema readSchema) { + this.iterator = iterator; + this.structType = HoodieInternalRowUtils.getCachedSchema(readSchema); + this.recordKeyIndex = (int)structType.getFieldIndex(HoodieRecord.HoodieMetadataField.RECORD_KEY_METADATA_FIELD.getFieldName()).get(); + } + + @Override + public Iterator read() throws IOException { + return new MappingIterator<>(iterator, record -> { + return new HoodieSparkRecord(record.getUTF8String(recordKeyIndex), record, structType); + }); + } + + @Override + public void close() throws IOException { + // no op + } +} diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieInternalRowFileWriterFactory.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieInternalRowFileWriterFactory.java index 8a61c7c44d900..4949112786128 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieInternalRowFileWriterFactory.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieInternalRowFileWriterFactory.java @@ -21,6 +21,7 @@ import org.apache.hadoop.fs.Path; import org.apache.hudi.common.bloom.BloomFilter; import org.apache.hudi.common.bloom.BloomFilterFactory; +import org.apache.hudi.common.config.HoodieStorageConfig; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieWriteConfig; @@ -54,11 +55,37 @@ public static HoodieInternalRowFileWriter getInternalRowFileWriter(Path path, throws IOException { final String extension = FSUtils.getFileExtension(path.getName()); if (PARQUET.getFileExtension().equals(extension)) { - return newParquetInternalRowFileWriter(path, hoodieTable, writeConfig, schema, tryInstantiateBloomFilter(writeConfig)); + if (hoodieTable.getMetaClient().getTableConfig().isLSMBasedLogFormat()) { + return newLsmParquetInternalRowFileWriter(path, hoodieTable, writeConfig, schema); + } else { + return newParquetInternalRowFileWriter(path, hoodieTable, writeConfig, schema, tryInstantiateBloomFilter(writeConfig)); + } } throw new UnsupportedOperationException(extension + " format not supported yet."); } + private static HoodieInternalRowFileWriter newLsmParquetInternalRowFileWriter(Path path, + HoodieTable table, + HoodieWriteConfig writeConfig, + StructType structType) throws IOException { + HoodieLSMRowParquetWriteSupport writeSupport = + new HoodieLSMRowParquetWriteSupport(table.getHadoopConf(), structType, writeConfig.getStorageConfig()); + return new HoodieInternalRowParquetWriter( + path, + new HoodieParquetConfig<>( + writeSupport, + writeConfig.getParquetCompressionCodec(), + writeConfig.getParquetBlockSize(), + writeConfig.getParquetPageSize(), + writeConfig.getParquetMaxFileSize(), + writeSupport.getHadoopConf(), + writeConfig.getParquetCompressionRatio(), + writeConfig.parquetDictionaryEnabled(), + writeConfig, + structType.size() + )); + } + private static HoodieInternalRowFileWriter newParquetInternalRowFileWriter(Path path, HoodieTable table, HoodieWriteConfig writeConfig, @@ -66,8 +93,10 @@ private static HoodieInternalRowFileWriter newParquetInternalRowFileWriter(Path Option bloomFilterOpt ) throws IOException { + HoodieStorageConfig storageConfig = writeConfig.getStorageConfig(); HoodieRowParquetWriteSupport writeSupport = - new HoodieRowParquetWriteSupport(table.getHadoopConf(), structType, bloomFilterOpt, writeConfig.getStorageConfig()); + new HoodieRowParquetWriteSupport(table.getHadoopConf(), structType, bloomFilterOpt, storageConfig, + storageConfig.getBoolean(HoodieStorageConfig.DATASKETCH_ENABLED)); return new HoodieInternalRowParquetWriter( path, @@ -79,7 +108,9 @@ private static HoodieInternalRowFileWriter newParquetInternalRowFileWriter(Path writeConfig.getParquetMaxFileSize(), writeSupport.getHadoopConf(), writeConfig.getParquetCompressionRatio(), - writeConfig.parquetDictionaryEnabled() + writeConfig.parquetDictionaryEnabled(), + writeConfig, + structType.size() )); } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieLSMRowParquetWriteSupport.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieLSMRowParquetWriteSupport.java new file mode 100644 index 0000000000000..e4be42f0ea46b --- /dev/null +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieLSMRowParquetWriteSupport.java @@ -0,0 +1,72 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.io.storage.row; + +import org.apache.hudi.avro.HoodieMinMaxWriteSupport; +import org.apache.hudi.common.config.HoodieStorageConfig; +import org.apache.hudi.common.util.Option; + +import org.apache.hadoop.conf.Configuration; +import org.apache.spark.sql.types.StructType; +import org.apache.spark.unsafe.types.UTF8String; + +import java.util.Collections; +import java.util.Map; + +/** + * Hoodie Write Support for directly writing Row to Parquet. + */ +public class HoodieLSMRowParquetWriteSupport extends HoodieRowParquetWriteSupport { + private final Option minMaxWriteSupportOpt; + + public HoodieLSMRowParquetWriteSupport(Configuration conf, StructType structType, HoodieStorageConfig config) { + super(conf, structType, Option.empty(), config, false); + this.minMaxWriteSupportOpt = Option.of(new HoodieMinMaxRowWriteSupport()); + } + + @Override + public FinalizedWriteContext finalizeWrite() { + Map extraMetadata = + minMaxWriteSupportOpt.map(HoodieMinMaxWriteSupport::finalizeMetadata) + .orElse(Collections.emptyMap()); + + return new FinalizedWriteContext(extraMetadata); + } + + @Override + public void add(UTF8String recordKey) { + this.minMaxWriteSupportOpt.ifPresent(writeSupport -> + writeSupport.addKey(recordKey)); + } + + private static class HoodieMinMaxRowWriteSupport extends HoodieMinMaxWriteSupport { + public HoodieMinMaxRowWriteSupport() { + super(); + } + + @Override + protected UTF8String dereference(UTF8String key) { + // NOTE: [[clone]] is performed here (rather than [[copy]]) to only copy underlying buffer in + // cases when [[UTF8String]] is pointing into a buffer storing the whole containing record, + // and simply do a pass over when it holds a (immutable) buffer holding just the string + return key.clone(); + } + } + +} diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowCreateHandle.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowCreateHandle.java index 9da04f72600b7..0d3213845c214 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowCreateHandle.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowCreateHandle.java @@ -20,17 +20,26 @@ import org.apache.hudi.client.HoodieInternalWriteStatus; import org.apache.hudi.client.model.HoodieInternalRow; +import org.apache.hudi.common.config.HoodieStorageConfig; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodiePartitionMetadata; import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.model.HoodieWriteStat; import org.apache.hudi.common.model.IOType; +import org.apache.hudi.common.storage.HoodieStorageStrategy; +import org.apache.hudi.common.storage.HoodieStorageStrategyFactory; import org.apache.hudi.common.util.HoodieTimer; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.exception.HoodieInsertException; import org.apache.hudi.hadoop.CachingPath; +import org.apache.hudi.index.HoodieIndex; +import org.apache.hudi.index.bucket.BucketStrategist; +import org.apache.hudi.index.bucket.BucketStrategistFactory; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.marker.WriteMarkersFactory; @@ -47,6 +56,8 @@ import java.util.concurrent.atomic.AtomicLong; import java.util.function.Function; +import static org.apache.hudi.common.config.HoodieStorageConfig.DATASKETCH_ENABLED; + /** * Create handle with InternalRow for datasource implementation of bulk insert. */ @@ -55,28 +66,35 @@ public class HoodieRowCreateHandle implements Serializable { private static final long serialVersionUID = 1L; private static final Logger LOG = LogManager.getLogger(HoodieRowCreateHandle.class); - private static final AtomicLong GLOBAL_SEQ_NO = new AtomicLong(1); + protected static final AtomicLong GLOBAL_SEQ_NO = new AtomicLong(1); - private final HoodieTable table; - private final HoodieWriteConfig writeConfig; + protected final HoodieTable table; + protected final HoodieWriteConfig writeConfig; private final String partitionPath; + private final String version; + private final String levelNumber; private final Path path; - private final String fileId; + protected final String fileId; + + protected final boolean populateMetaFields; - private final boolean populateMetaFields; + protected final String fileName; - private final UTF8String fileName; - private final UTF8String commitTime; - private final Function seqIdGenerator; + protected final UTF8String commitTime; + protected final Function seqIdGenerator; - private final boolean shouldPreserveHoodieMetadata; + protected final boolean shouldPreserveHoodieMetadata; private final HoodieTimer currTimer; protected final HoodieInternalRowFileWriter fileWriter; protected final HoodieInternalWriteStatus writeStatus; + private final HoodieStorageStrategy hoodieStorageStrategy; + protected final StructType structType; + protected final UTF8String finalFilename; + public HoodieRowCreateHandle(HoodieTable table, HoodieWriteConfig writeConfig, String partitionPath, @@ -100,8 +118,41 @@ public HoodieRowCreateHandle(HoodieTable table, long taskEpochId, StructType structType, boolean shouldPreserveHoodieMetadata) { + this(table, writeConfig, partitionPath, fileId, instantTime, taskPartitionId, taskId, taskEpochId, + structType, shouldPreserveHoodieMetadata, "", ""); + } + + public HoodieRowCreateHandle(HoodieTable table, + HoodieWriteConfig writeConfig, + String partitionPath, + String fileId, + String instantTime, + int taskPartitionId, + long taskId, + long taskEpochId, + StructType structType, + boolean shouldPreserveHoodieMetadata, + String version, + String levelNumber) { + this.structType = structType; + this.hoodieStorageStrategy = HoodieStorageStrategyFactory.getInstant(table.getMetaClient()); + this.version = version; + this.levelNumber = levelNumber; + if (!StringUtils.isNullOrEmpty(levelNumber) && Integer.parseInt(levelNumber) == 1 && writeConfig.getParquetBlockSize() == 32 * 1024 * 1024) { + writeConfig.setValue(HoodieStorageConfig.PARQUET_BLOCK_SIZE.key(), HoodieStorageConfig.PARQUET_BLOCK_SIZE.defaultValue()); + } + Option bucketStrategist; + if (writeConfig.getIndexType().equals(HoodieIndex.IndexType.BUCKET) && writeConfig.isBucketIndexAtPartitionLevel()) { + bucketStrategist = Option.of(BucketStrategistFactory.getInstant(writeConfig, table.getMetaClient().getFs())); + } else { + bucketStrategist = Option.empty(); + } this.partitionPath = partitionPath; this.table = table; + if (HoodieTableType.COPY_ON_WRITE.equals(table.getMetaClient().getTableType())) { + writeConfig.getStorageConfig().setValue(DATASKETCH_ENABLED.key(), "false"); + writeConfig.setValue(DATASKETCH_ENABLED.key(), "false"); + } this.writeConfig = writeConfig; this.fileId = fileId; @@ -110,11 +161,12 @@ public HoodieRowCreateHandle(HoodieTable table, FileSystem fs = table.getMetaClient().getFs(); String writeToken = getWriteToken(taskPartitionId, taskId, taskEpochId); - String fileName = FSUtils.makeBaseFileName(instantTime, writeToken, this.fileId, table.getBaseFileExtension()); - this.path = makeNewPath(fs, partitionPath, fileName, writeConfig); + String fileName = makeFileName(instantTime, writeToken, version, levelNumber); + this.path = makeNewPath(fs, partitionPath, fileName, instantTime); this.populateMetaFields = writeConfig.populateMetaFields(); - this.fileName = UTF8String.fromString(path.getName()); + this.fileName = path.getName(); + this.finalFilename = getFileNameMetaValue(fileName); this.commitTime = UTF8String.fromString(instantTime); this.seqIdGenerator = (id) -> HoodieRecord.generateSequenceId(instantTime, taskPartitionId, id); @@ -131,8 +183,10 @@ public HoodieRowCreateHandle(HoodieTable table, fs, instantTime, new Path(writeConfig.getBasePath()), - FSUtils.getPartitionPath(writeConfig.getBasePath(), partitionPath), - table.getPartitionMetafileFormat()); + hoodieStorageStrategy.storageLocation(partitionPath, instantTime), + table.getPartitionMetafileFormat(), + hoodieStorageStrategy, + bucketStrategist.map(strategist -> strategist.computeBucketNumber(partitionPath))); partitionMetadata.trySave(taskPartitionId); createMarkerFile(partitionPath, fileName, instantTime, table, writeConfig); @@ -154,22 +208,21 @@ public HoodieRowCreateHandle(HoodieTable table, */ public void write(InternalRow row) throws IOException { if (populateMetaFields) { - writeRow(row); + writeRow(row, null); } else { writeRowNoMetaFields(row); } } - private void writeRow(InternalRow row) { + public String makeFileName(String instantTime, String writeToken, String version, String levelNumber) { + return FSUtils.makeBaseFileName(instantTime, writeToken, this.fileId, table.getBaseFileExtension()); + } + + protected void writeRow(InternalRow row, UTF8String recordKey) { try { - // NOTE: PLEASE READ THIS CAREFULLY BEFORE MODIFYING - // This code lays in the hot-path, and substantial caution should be - // exercised making changes to it to minimize amount of excessive: - // - Conversions b/w Spark internal types and JVM native ones (like [[UTF8String]] - // and [[String]]) - // - Repeated computations (for ex, converting file-path to [[UTF8String]] over and - // over again) - UTF8String recordKey = row.getUTF8String(HoodieRecord.RECORD_KEY_META_FIELD_ORD); + if (recordKey == null) { + recordKey = row.getUTF8String(HoodieRecord.RECORD_KEY_META_FIELD_ORD); + } UTF8String partitionPath = row.getUTF8String(HoodieRecord.PARTITION_PATH_META_FIELD_ORD); // This is the only meta-field that is generated dynamically, hence conversion b/w // [[String]] and [[UTF8String]] is unavoidable if preserveHoodieMetadata is false @@ -179,7 +232,7 @@ private void writeRow(InternalRow row) { : commitTime; InternalRow updatedRow = new HoodieInternalRow(writeCommitTime, seqId, recordKey, - partitionPath, fileName, row, true); + partitionPath, finalFilename, row, true); try { fileWriter.writeRow(recordKey, updatedRow); // NOTE: To avoid conversion on the hot-path we only convert [[UTF8String]] into [[String]] @@ -194,7 +247,11 @@ private void writeRow(InternalRow row) { } } - private void writeRowNoMetaFields(InternalRow row) { + protected UTF8String getFileNameMetaValue(String fileName) { + return UTF8String.fromString(fileName); + } + + protected void writeRowNoMetaFields(InternalRow row) { try { // TODO make sure writing w/ and w/o meta fields is consistent (currently writing w/o // meta-fields would fail if any record will, while when writing w/ meta-fields it won't) @@ -228,7 +285,7 @@ public HoodieInternalWriteStatus close() throws IOException { stat.setNumInserts(writeStatus.getTotalRecords()); stat.setPrevCommit(HoodieWriteStat.NULL_COMMIT); stat.setFileId(fileId); - stat.setPath(new Path(writeConfig.getBasePath()), path); + stat.setPath(hoodieStorageStrategy.getRelativePath(path)); long fileSizeInBytes = FSUtils.getFileSize(table.getMetaClient().getFs(), path); stat.setTotalWriteBytes(fileSizeInBytes); stat.setFileSizeInBytes(fileSizeInBytes); @@ -243,8 +300,8 @@ public String getFileName() { return path.getName(); } - private static Path makeNewPath(FileSystem fs, String partitionPath, String fileName, HoodieWriteConfig writeConfig) { - Path path = FSUtils.getPartitionPath(writeConfig.getBasePath(), partitionPath); + protected Path makeNewPath(FileSystem fs, String partitionPath, String fileName, String instantTime) { + Path path = hoodieStorageStrategy.storageLocation(partitionPath, instantTime); try { if (!fs.exists(path)) { fs.mkdirs(path); // create a new partition as needed. @@ -260,18 +317,33 @@ private static Path makeNewPath(FileSystem fs, String partitionPath, String file * * @param partitionPath Partition path */ - private static void createMarkerFile(String partitionPath, + protected void createMarkerFile(String partitionPath, String dataFileName, String instantTime, HoodieTable table, HoodieWriteConfig writeConfig) { + stopIfAborted(); WriteMarkersFactory.get(writeConfig.getMarkersType(), table, instantTime) .create(partitionPath, dataFileName, IOType.CREATE); } // TODO extract to utils - private static String getWriteToken(int taskPartitionId, long taskId, long taskEpochId) { + protected static String getWriteToken(int taskPartitionId, long taskId, long taskEpochId) { return taskPartitionId + "-" + taskId + "-" + taskEpochId; } + public boolean tryCleanWrittenFiles() { + try { + LOG.warn("Cleaning file " + path); + return table.getMetaClient().getFs().delete(path, false); + } catch (IOException e) { + return false; + } + } + + private void stopIfAborted() { + if (table.getTaskContextSupplier().isAborted()) { + throw new HoodieIOException("The task is already aborted, stop handling new records..."); + } + } } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowParquetWriteSupport.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowParquetWriteSupport.java index 3a1b6d000becc..8e0ba26d85d1d 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowParquetWriteSupport.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowParquetWriteSupport.java @@ -22,7 +22,10 @@ import org.apache.hudi.avro.HoodieBloomFilterWriteSupport; import org.apache.hudi.common.bloom.BloomFilter; import org.apache.hudi.common.config.HoodieStorageConfig; +import org.apache.hudi.common.util.CollectionUtils; import org.apache.hudi.common.util.Option; +import org.apache.hudi.datasketch.DataSketchWriteSupport; + import org.apache.parquet.hadoop.api.WriteSupport; import org.apache.spark.sql.execution.datasources.parquet.ParquetWriteSupport; import org.apache.spark.sql.types.StructType; @@ -40,8 +43,9 @@ public class HoodieRowParquetWriteSupport extends ParquetWriteSupport { private final Configuration hadoopConf; private final Option> bloomFilterWriteSupportOpt; + private final Option dataSketchRowWriteSupportOpt; - public HoodieRowParquetWriteSupport(Configuration conf, StructType structType, Option bloomFilterOpt, HoodieStorageConfig config) { + public HoodieRowParquetWriteSupport(Configuration conf, StructType structType, Option bloomFilterOpt, HoodieStorageConfig config, boolean dataSketchEnable) { Configuration hadoopConf = new Configuration(conf); hadoopConf.set("spark.sql.parquet.writeLegacyFormat", config.getString(HoodieStorageConfig.PARQUET_WRITE_LEGACY_FORMAT_ENABLED)); hadoopConf.set("spark.sql.parquet.outputTimestampType", config.getString(HoodieStorageConfig.PARQUET_OUTPUT_TIMESTAMP_TYPE)); @@ -50,6 +54,7 @@ public HoodieRowParquetWriteSupport(Configuration conf, StructType structType, O this.hadoopConf = hadoopConf; this.bloomFilterWriteSupportOpt = bloomFilterOpt.map(HoodieBloomFilterRowWriteSupport::new); + this.dataSketchRowWriteSupportOpt = dataSketchEnable ? Option.of(new HoodieDataSketchRowWriteSupport()) : Option.empty(); } public Configuration getHadoopConf() { @@ -62,12 +67,16 @@ public WriteSupport.FinalizedWriteContext finalizeWrite() { bloomFilterWriteSupportOpt.map(HoodieBloomFilterWriteSupport::finalizeMetadata) .orElse(Collections.emptyMap()); + extraMetadata = CollectionUtils.combine(extraMetadata, + dataSketchRowWriteSupportOpt.map(DataSketchWriteSupport::finalizeMetadata) + .orElse(Collections.emptyMap())); return new WriteSupport.FinalizedWriteContext(extraMetadata); } public void add(UTF8String recordKey) { this.bloomFilterWriteSupportOpt.ifPresent(bloomFilterWriteSupport -> bloomFilterWriteSupport.addKey(recordKey)); + this.dataSketchRowWriteSupportOpt.ifPresent(sketch -> sketch.addKey(recordKey)); } private static class HoodieBloomFilterRowWriteSupport extends HoodieBloomFilterWriteSupport { @@ -89,4 +98,19 @@ protected UTF8String dereference(UTF8String key) { } } + private static class HoodieDataSketchRowWriteSupport extends DataSketchWriteSupport { + @Override + protected String getUTF8String(UTF8String key) { + return key.toString(); + } + + @Override + protected UTF8String dereference(UTF8String key) { + // NOTE: [[clone]] is performed here (rather than [[copy]]) to only copy underlying buffer in + // cases when [[UTF8String]] is pointing into a buffer storing the whole containing record, + // and simply do a pass over when it holds a (immutable) buffer holding just the string + return key.clone(); + } + } + } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/LSMHoodieRowCreateHandle.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/LSMHoodieRowCreateHandle.java new file mode 100644 index 0000000000000..79a7791c940a5 --- /dev/null +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/LSMHoodieRowCreateHandle.java @@ -0,0 +1,200 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.io.storage.row; + +import org.apache.hudi.AvroConversionUtils; +import org.apache.hudi.avro.AvroSchemaCache; +import org.apache.hudi.client.HoodieInternalWriteStatus; +import org.apache.hudi.client.model.HoodieInternalRow; +import org.apache.hudi.common.config.TypedProperties; +import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecordMerger; +import org.apache.hudi.common.model.HoodieSparkRecord; +import org.apache.hudi.common.model.WriteOperationType; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.table.HoodieTable; + +import org.apache.avro.Schema; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.types.StructType; +import org.apache.spark.unsafe.types.UTF8String; + +import java.io.IOException; + +/** + * Create handle with InternalRow for datasource implementation of bulk insert. + */ +public class LSMHoodieRowCreateHandle extends HoodieRowCreateHandle { + + private static final long serialVersionUID = 1L; + + private static final Logger LOG = LogManager.getLogger(LSMHoodieRowCreateHandle.class); + private final HoodieRecordMerger recordMerger; + private final TypedProperties payloadProperties; + private final Schema writeSchema; + private UTF8String lastRecordKey = null; + private HoodieSparkRecord lastMergedSparkRecord = null; + + public LSMHoodieRowCreateHandle(HoodieTable table, + HoodieWriteConfig writeConfig, + String partitionPath, + String fileId, + String instantTime, + int taskPartitionId, + long taskId, + long taskEpochId, + StructType structType, + boolean shouldPreserveHoodieMetadata, + int columnFamilyNumber, + int levelNumber) { + super(table, writeConfig, partitionPath, fileId, instantTime, taskPartitionId, taskId, + taskEpochId, structType, shouldPreserveHoodieMetadata, String.valueOf(columnFamilyNumber), String.valueOf(levelNumber)); + Schema oriSchema = new Schema.Parser().parse(writeConfig.getWriteSchema()); + this.writeSchema = AvroSchemaCache.intern(AvroConversionUtils.convertStructTypeToAvroSchema(structType, + oriSchema.getName(), oriSchema.getNamespace())); + this.recordMerger = writeConfig.getRecordMerger(); + this.payloadProperties = writeConfig.getPayloadConfig().getProps(); + } + + @Override + public String makeFileName(String instantTime, String writeToken, String version, String levelNumber) { + return FSUtils.makeLSMFileNameWithSuffix(fileId, instantTime, version, levelNumber, + writeToken, table.getBaseFileExtension()); + } + + @Override + protected UTF8String getFileNameMetaValue(String fileName) { + return UTF8String.fromString(fileName.replace(FSUtils.LSM_TEMP_FILE_SUFFIX, "")); + } + + @Override + public boolean canWrite() { + return true; + } + + @Override + public void write(InternalRow row) throws IOException { + try { + if (writeConfig.getOperationType().equals(WriteOperationType.CLUSTER.name())) { + if (populateMetaFields) { + writeRow(row, row.getUTF8String(HoodieRecord.RECORD_KEY_META_FIELD_ORD)); + } else { + writeRowNoMetaFields(row); + } + return; + } + UTF8String recordKey = row.getUTF8String(HoodieRecord.RECORD_KEY_META_FIELD_ORD); + // 复制当前行,因为row可能是可重用的 + InternalRow currentRow = copyRow(row); + HoodieSparkRecord currentRecord = new HoodieSparkRecord(currentRow, structType); + + // 检查是否与上一条记录的key相同 + if (lastRecordKey != null && lastRecordKey.equals(recordKey)) { + // 相同key,合并记录 + Option> res = recordMerger.merge(lastMergedSparkRecord, + writeSchema, currentRecord, writeSchema, payloadProperties); + if (res.isPresent()) { + lastMergedSparkRecord = (HoodieSparkRecord)res.get().getLeft(); + } else { + lastMergedSparkRecord = null; + } + } else { + // 不同key,先写出上一条合并后的记录(如果存在) + if (lastMergedSparkRecord != null) { + if (populateMetaFields) { + writeRow(lastMergedSparkRecord.getData(), lastRecordKey); + } else { + writeRowNoMetaFields(lastMergedSparkRecord.getData()); + } + } + + // 更新为当前记录 + lastRecordKey = recordKey.clone(); // 克隆以确保引用安全 + lastMergedSparkRecord = currentRecord; + } + } catch (Exception e) { + writeStatus.setGlobalError(e); + throw e; + } + } + + // 辅助方法:复制行数据 + private InternalRow copyRow(InternalRow row) { + // 根据具体实现选择合适的复制方法 + // 例如,可以使用GenericInternalRow或UnsafeRow的复制方法 + return row.copy(); // 假设InternalRow有copy方法 + } + + /** + * when use spark write lsm, seqId just generate according to GLOBAL_SEQ_NO + * @param row + * @param recordKey + */ + @Override + public void writeRow(InternalRow row, UTF8String recordKey) { + try { + if (recordKey == null) { + recordKey = row.getUTF8String(HoodieRecord.RECORD_KEY_META_FIELD_ORD); + } + UTF8String partitionPath = row.getUTF8String(HoodieRecord.PARTITION_PATH_META_FIELD_ORD); + // This is the only meta-field that is generated dynamically, hence conversion b/w + // [[String]] and [[UTF8String]] is unavoidable if preserveHoodieMetadata is false + UTF8String seqId = shouldPreserveHoodieMetadata ? row.getUTF8String(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD_ORD) : + UTF8String.fromString(String.valueOf(GLOBAL_SEQ_NO.getAndIncrement())); + UTF8String writeCommitTime = shouldPreserveHoodieMetadata ? row.getUTF8String(HoodieRecord.COMMIT_TIME_METADATA_FIELD_ORD) + : commitTime; + InternalRow updatedRow = new HoodieInternalRow(writeCommitTime, seqId, recordKey, + partitionPath, finalFilename, row, true); + try { + fileWriter.writeRow(recordKey, updatedRow); + // NOTE: To avoid conversion on the hot-path we only convert [[UTF8String]] into [[String]] + // in cases when successful records' writes are being tracked + writeStatus.markSuccess(writeStatus.isTrackingSuccessfulWrites() ? recordKey.toString() : null); + } catch (Exception t) { + writeStatus.markFailure(recordKey.toString(), t); + } + } catch (Exception e) { + writeStatus.setGlobalError(e); + throw e; + } + } + + private void flush() { + if (lastMergedSparkRecord != null) { + if (populateMetaFields) { + writeRow(lastMergedSparkRecord.getData(), lastRecordKey); + } else { + writeRowNoMetaFields(lastMergedSparkRecord.getData()); + } + lastMergedSparkRecord = null; + lastRecordKey = null; + } + } + + @Override + public HoodieInternalWriteStatus close() throws IOException { + flush(); + return super.close(); + } +} diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/BaseSparkCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/BaseSparkCommitActionExecutor.java index ac5b8555b053f..3326af8d145f3 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/BaseSparkCommitActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/BaseSparkCommitActionExecutor.java @@ -131,7 +131,7 @@ private HoodieData> clusteringHandleUpdate(HoodieData pendingClusteringInstantsToRollback = getAllFileGroupsInPendingClusteringPlans(table.getMetaClient()).entrySet().stream() + Set pendingClusteringInstantsToRollback = getAllFileGroupsInPendingClusteringPlans(table.getMetaClient()).getLeft().entrySet().stream() .filter(e -> fileGroupsWithUpdatesAndPendingClustering.contains(e.getKey())) .map(Map.Entry::getValue) .collect(Collectors.toSet()); diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/BucketBulkInsertDataInternalWriterHelper.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/BucketBulkInsertDataInternalWriterHelper.java new file mode 100644 index 0000000000000..eb6990c94020a --- /dev/null +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/BucketBulkInsertDataInternalWriterHelper.java @@ -0,0 +1,155 @@ +/* + * 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.action.commit; + +import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.WriteOperationType; +import org.apache.hudi.config.HoodieIndexConfig; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.index.bucket.BucketIdentifier; +import org.apache.hudi.index.bucket.BucketStrategist; +import org.apache.hudi.index.bucket.BucketStrategistFactory; +import org.apache.hudi.io.storage.row.HoodieRowCreateHandle; +import org.apache.hudi.keygen.constant.KeyGeneratorOptions; +import org.apache.hudi.table.HoodieTable; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.types.StructType; +import org.apache.spark.unsafe.types.UTF8String; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; +import java.util.Objects; + +/** + * Helper class for native row writer for bulk_insert with bucket index. + */ +public class BucketBulkInsertDataInternalWriterHelper extends BulkInsertDataInternalWriterHelper { + + private static final Logger LOG = LoggerFactory.getLogger(BucketBulkInsertDataInternalWriterHelper.class); + + private int lastKnownBucketNum = -1; + // p -> (bucketNum -> handle) + protected final Map> bucketHandles; + protected final String indexKeyFields; + protected final BucketStrategist bucketStrategist; + + public BucketBulkInsertDataInternalWriterHelper(HoodieTable hoodieTable, HoodieWriteConfig writeConfig, + String instantTime, int taskPartitionId, long taskId, long taskEpochId, StructType structType, + boolean populateMetaFields, boolean arePartitionRecordsSorted) { + this(hoodieTable, writeConfig, instantTime, taskPartitionId, taskId, taskEpochId, structType, populateMetaFields, arePartitionRecordsSorted, false); + } + + public BucketBulkInsertDataInternalWriterHelper(HoodieTable hoodieTable, HoodieWriteConfig writeConfig, + String instantTime, int taskPartitionId, long taskId, long taskEpochId, StructType structType, + boolean populateMetaFields, boolean arePartitionRecordsSorted, boolean shouldPreserveHoodieMetadata) { + super(hoodieTable, writeConfig, instantTime, taskPartitionId, taskId, taskEpochId, structType, populateMetaFields, arePartitionRecordsSorted, shouldPreserveHoodieMetadata); + this.indexKeyFields = writeConfig.getStringOrDefault(HoodieIndexConfig.BUCKET_INDEX_HASH_FIELD, writeConfig.getString(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key())); + this.bucketStrategist = BucketStrategistFactory.getInstant(writeConfig, hoodieTable.getMetaClient().getFs()); + this.bucketHandles = new HashMap<>(); + } + + public void write(InternalRow row) throws IOException { + try { + UTF8String partitionPath = extractPartitionPath(row); + int bucketId; + if (hoodieTable.getMetaClient().getTableConfig().isLSMBasedLogFormat()) { + // for common write we get bucketID, for clustering we get file name, so that we need to get bucketID from filename + String fileName = row.getString(HoodieRecord.FILENAME_META_FIELD_ORD); + if (writeConfig.getOperationType().equalsIgnoreCase(WriteOperationType.CLUSTER.value())) { + bucketId = Integer.parseInt(FSUtils.getFileId(fileName)); + } else { + bucketId = Integer.parseInt(fileName); + } + } else { + UTF8String recordKey = extractRecordKey(row); + bucketId = BucketIdentifier.getBucketId(String.valueOf(recordKey), indexKeyFields, bucketStrategist.computeBucketNumber(partitionPath.toString())); + } + if (lastKnownPartitionPath == null || !Objects.equals(lastKnownPartitionPath, partitionPath) || !handle.canWrite() || bucketId != lastKnownBucketNum) { + handle = getBucketRowCreateHandle(String.valueOf(partitionPath), bucketId); + // NOTE: It's crucial to make a copy here, since [[UTF8String]] could be pointing into + // a mutable underlying buffer + lastKnownPartitionPath = partitionPath.clone(); + lastKnownBucketNum = bucketId; + } + + handle.write(row); + } catch (Throwable t) { + LOG.error("Global error thrown while trying to write records in HoodieRowCreateHandle ", t); + throw t; + } + } + + protected UTF8String extractRecordKey(InternalRow row) { + if (populateMetaFields) { + // In case meta-fields are materialized w/in the table itself, we can just simply extract + // partition path from there + // + // NOTE: Helper keeps track of [[lastKnownPartitionPath]] as [[UTF8String]] to avoid + // conversion from Catalyst internal representation into a [[String]] + return row.getUTF8String(HoodieRecord.RECORD_KEY_META_FIELD_ORD); + } else if (keyGeneratorOpt.isPresent()) { + return keyGeneratorOpt.get().getRecordKey(row, structType); + } else { + return UTF8String.EMPTY_UTF8; + } + } + + protected HoodieRowCreateHandle getBucketRowCreateHandle(String partitionPath, int bucketId) { + Map bucketHandleMap = bucketHandles.computeIfAbsent(partitionPath, p -> new HashMap<>()); + if (!bucketHandleMap.isEmpty() && bucketHandleMap.containsKey(bucketId)) { + return bucketHandleMap.get(bucketId); + } + LOG.info("Creating new file for partition path {} and bucket {}", partitionPath, bucketId); + HoodieRowCreateHandle rowCreateHandle = new HoodieRowCreateHandle(hoodieTable, writeConfig, partitionPath, getNextBucketFileId(bucketId), + instantTime, taskPartitionId, taskId, taskEpochId, structType, shouldPreserveHoodieMetadata); + bucketHandleMap.put(bucketId, rowCreateHandle); + return rowCreateHandle; + } + + @Override + public void close() throws IOException { + for (Map entry : bucketHandles.values()) { + for (HoodieRowCreateHandle rowCreateHandle : entry.values()) { + LOG.info("Closing bulk insert file " + rowCreateHandle.getFileName()); + writeStatusList.add(rowCreateHandle.close()); + } + entry.clear(); + } + bucketHandles.clear(); + handle = null; + } + + @Override + public void abort() { + for (Map entry : bucketHandles.values()) { + for (HoodieRowCreateHandle rowCreateHandle : entry.values()) { + rowCreateHandle.tryCleanWrittenFiles(); + } + entry.clear(); + } + } + + protected String getNextBucketFileId(int bucketInt) { + return BucketIdentifier.newBucketFileIdPrefix(getNextFileId(), bucketInt); + } +} diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/BucketLSMBulkInsertDataInternalWriterHelper.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/BucketLSMBulkInsertDataInternalWriterHelper.java new file mode 100644 index 0000000000000..4e1ec9a0e8d4c --- /dev/null +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/BucketLSMBulkInsertDataInternalWriterHelper.java @@ -0,0 +1,127 @@ +/* + * 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.action.commit; + +import org.apache.hudi.common.model.WriteOperationType; +import org.apache.hudi.common.util.FutureUtils; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.exception.HoodieIOException; +import org.apache.hudi.io.storage.row.HoodieRowCreateHandle; +import org.apache.hudi.io.storage.row.LSMHoodieRowCreateHandle; +import org.apache.hudi.table.HoodieTable; + +import org.apache.spark.sql.types.StructType; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; + +import static org.apache.hudi.config.HoodieClusteringConfig.LSM_CLUSTERING_OUT_PUT_LEVEL; + +/** + * Helper class for native row writer for bulk_insert with bucket index. + */ +public class BucketLSMBulkInsertDataInternalWriterHelper extends BucketBulkInsertDataInternalWriterHelper { + + private static final Logger LOG = LoggerFactory.getLogger(BucketLSMBulkInsertDataInternalWriterHelper.class); + private final int levelNumber; + + public BucketLSMBulkInsertDataInternalWriterHelper(HoodieTable hoodieTable, HoodieWriteConfig writeConfig, + String instantTime, int taskPartitionId, long taskId, long taskEpochId, StructType structType, + boolean populateMetaFields, boolean arePartitionRecordsSorted) { + this(hoodieTable, writeConfig, instantTime, taskPartitionId, taskId, taskEpochId, structType, populateMetaFields, arePartitionRecordsSorted, false); + } + + public BucketLSMBulkInsertDataInternalWriterHelper(HoodieTable hoodieTable, HoodieWriteConfig writeConfig, + String instantTime, int taskPartitionId, long taskId, long taskEpochId, StructType structType, + boolean populateMetaFields, boolean arePartitionRecordsSorted, boolean shouldPreserveHoodieMetadata) { + super(hoodieTable, writeConfig, instantTime, taskPartitionId, taskId, taskEpochId, structType, populateMetaFields, arePartitionRecordsSorted, shouldPreserveHoodieMetadata); + boolean isBulkInsert = WriteOperationType.BULK_INSERT == WriteOperationType.fromValue(writeConfig.getOperationType()); + if (((writeConfig.isForcingWriteToL1() && isBulkInsert)) || isOverWrite(writeConfig)) { + this.levelNumber = 1; + } else if (writeConfig.getOperationType().equalsIgnoreCase(WriteOperationType.CLUSTER.value())) { + this.levelNumber = Integer.parseInt(writeConfig.getProps().getString(LSM_CLUSTERING_OUT_PUT_LEVEL)); + } else { + this.levelNumber = 0; + } + } + + private boolean isOverWrite(HoodieWriteConfig writeConfig) { + if (writeConfig.getBulkInsertOverWriteOperationType() == null) { + return false; + } + switch (WriteOperationType.fromValue(writeConfig.getBulkInsertOverWriteOperationType())) { + case INSERT_OVERWRITE: + case INSERT_OVERWRITE_TABLE: + return true; + default: + return false; + } + } + + @Override + protected HoodieRowCreateHandle getBucketRowCreateHandle(String partitionPath, int bucketId) { + Map bucketHandleMap = bucketHandles.computeIfAbsent(partitionPath, p -> new HashMap<>()); + if (!bucketHandleMap.isEmpty() && bucketHandleMap.containsKey(bucketId)) { + return bucketHandleMap.get(bucketId); + } + LOG.info("Creating new LSM file for partition path {} and bucket {}", partitionPath, bucketId); + HoodieRowCreateHandle rowCreateHandle; + // use bucketId as fileID directly + rowCreateHandle = new LSMHoodieRowCreateHandle(hoodieTable, writeConfig, partitionPath, String.valueOf(bucketId), + instantTime, taskPartitionId, taskId, taskEpochId, structType, shouldPreserveHoodieMetadata, 0, levelNumber); + bucketHandleMap.put(bucketId, rowCreateHandle); + return rowCreateHandle; + } + + @Override + public void close() throws IOException { + ThreadPoolExecutor executor = new ThreadPoolExecutor( + writeConfig.getLsmFlushConcurrency(), writeConfig.getLsmFlushConcurrency(), + 0L, TimeUnit.MILLISECONDS, new LinkedBlockingQueue<>()); + + FutureUtils.allOf(bucketHandles.values().stream().flatMap(entry -> { + return entry.values().stream(); + }).map(handle -> CompletableFuture.supplyAsync(() -> { + try { + return handle.close(); + } catch (IOException e) { + throw new HoodieIOException(e.getMessage(), e); + } + }, executor)).collect(Collectors.toList())).whenComplete((result, throwable) -> { + writeStatusList.addAll(result); + }).join(); + + try { + executor.shutdown(); + executor.awaitTermination(24, TimeUnit.DAYS); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + bucketHandles.clear(); + handle = null; + } +} diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/BulkInsertDataInternalWriterHelper.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/BulkInsertDataInternalWriterHelper.java index 12e9dda81a5bc..b90af827f156e 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/BulkInsertDataInternalWriterHelper.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/BulkInsertDataInternalWriterHelper.java @@ -53,30 +53,30 @@ public class BulkInsertDataInternalWriterHelper { private static final Logger LOG = LogManager.getLogger(BulkInsertDataInternalWriterHelper.class); - private final String instantTime; - private final int taskPartitionId; - private final long taskId; - private final long taskEpochId; - private final HoodieTable hoodieTable; - private final HoodieWriteConfig writeConfig; - private final StructType structType; - private final Boolean arePartitionRecordsSorted; - private final List writeStatusList = new ArrayList<>(); - private final String fileIdPrefix; - private final Map handles = new HashMap<>(); - private final boolean populateMetaFields; - private final boolean shouldPreserveHoodieMetadata; - private final Option keyGeneratorOpt; - private final boolean simpleKeyGen; - private final int simplePartitionFieldIndex; - private final DataType simplePartitionFieldDataType; + protected final String instantTime; + protected final int taskPartitionId; + protected final long taskId; + protected final long taskEpochId; + protected final HoodieTable hoodieTable; + protected final HoodieWriteConfig writeConfig; + protected final StructType structType; + protected final Boolean arePartitionRecordsSorted; + protected final List writeStatusList = new ArrayList<>(); + protected final String fileIdPrefix; + protected final Map handles = new HashMap<>(); // bucketId to handler + protected final boolean populateMetaFields; + protected final boolean shouldPreserveHoodieMetadata; + protected final Option keyGeneratorOpt; + protected final boolean simpleKeyGen; + protected final int simplePartitionFieldIndex; + protected final DataType simplePartitionFieldDataType; /** * NOTE: This is stored as Catalyst's internal {@link UTF8String} to avoid * conversion (deserialization) b/w {@link UTF8String} and {@link String} */ - private UTF8String lastKnownPartitionPath = null; - private HoodieRowCreateHandle handle; - private int numFilesWritten = 0; + protected UTF8String lastKnownPartitionPath = null; + protected HoodieRowCreateHandle handle; + protected int numFilesWritten = 0; public BulkInsertDataInternalWriterHelper(HoodieTable hoodieTable, HoodieWriteConfig writeConfig, String instantTime, int taskPartitionId, long taskId, long taskEpochId, StructType structType, @@ -163,7 +163,11 @@ public List getWriteStatuses() throws IOException { return writeStatusList; } - public void abort() {} + public void abort() { + for (HoodieRowCreateHandle rowCreateHandle : handles.values()) { + rowCreateHandle.tryCleanWrittenFiles(); + } + } public void close() throws IOException { for (HoodieRowCreateHandle rowCreateHandle : handles.values()) { @@ -173,7 +177,7 @@ public void close() throws IOException { handle = null; } - private UTF8String extractPartitionPath(InternalRow row) { + protected UTF8String extractPartitionPath(InternalRow row) { if (populateMetaFields) { // In case meta-fields are materialized w/in the table itself, we can just simply extract // partition path from there @@ -211,7 +215,7 @@ private HoodieRowCreateHandle createHandle(String partitionPath) { instantTime, taskPartitionId, taskId, taskEpochId, structType, shouldPreserveHoodieMetadata); } - private String getNextFileId() { + protected String getNextFileId() { return String.format("%s-%d", fileIdPrefix, numFilesWritten++); } } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkBucketIndexPartitioner.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkBucketIndexPartitioner.java index a246a7150c932..7bcfdd23721c7 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkBucketIndexPartitioner.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkBucketIndexPartitioner.java @@ -18,31 +18,37 @@ package org.apache.hudi.table.action.commit; -import java.util.ArrayList; -import java.util.Collections; -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Map.Entry; -import java.util.Set; - -import org.apache.hudi.common.model.WriteOperationType; -import org.apache.hudi.index.bucket.BucketIdentifier; -import scala.Tuple2; - import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecordLocation; +import org.apache.hudi.common.model.WriteOperationType; 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.HoodieWriteConfig; import org.apache.hudi.exception.HoodieException; +import org.apache.hudi.index.bucket.BucketIdentifier; +import org.apache.hudi.index.bucket.BucketStrategist; +import org.apache.hudi.index.bucket.BucketStrategistFactory; import org.apache.hudi.index.bucket.HoodieBucketIndex; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.WorkloadProfile; import org.apache.hudi.table.WorkloadStat; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Set; + +import scala.Tuple2; + import static org.apache.hudi.common.model.WriteOperationType.INSERT_OVERWRITE; import static org.apache.hudi.common.model.WriteOperationType.INSERT_OVERWRITE_TABLE; @@ -52,9 +58,10 @@ public class SparkBucketIndexPartitioner extends SparkHoodiePartitioner { + private static final Logger LOG = LoggerFactory.getLogger(SparkBucketIndexPartitioner.class); + private final int numBuckets; private final String indexKeyField; - private final int totalPartitionPaths; private final List partitionPaths; /** * Helps get the RDD partition id, partition id is partition offset + bucket id. @@ -67,6 +74,17 @@ public class SparkBucketIndexPartitioner extends * Partition path and file groups in it pair. Decide the file group an incoming update should go to. */ private Map> updatePartitionPathFileIds; + private final BucketStrategist bucketStrategist; + private final boolean isPartitionBucketIndexEnabled; + private final int totalPartitions; + /** + * Direct mapping from partition number to partition path + */ + private String[] partitionNumberToPath; + /** + * Direct mapping from partition number to local bucket ID + */ + private Integer[] partitionNumberToLocalBucketId; public SparkBucketIndexPartitioner(WorkloadProfile profile, HoodieEngineContext context, @@ -78,19 +96,42 @@ public SparkBucketIndexPartitioner(WorkloadProfile profile, " Bucket index partitioner should only be used by BucketIndex other than " + table.getIndex().getClass().getSimpleName()); } + this.bucketStrategist = BucketStrategistFactory.getInstant(config, context.getHadoopConf().get()); + this.isPartitionBucketIndexEnabled = bucketStrategist.isPartitionLevel(); this.numBuckets = ((HoodieBucketIndex) table.getIndex()).getNumBuckets(); this.indexKeyField = config.getBucketIndexHashField(); - this.totalPartitionPaths = profile.getPartitionPaths().size(); partitionPaths = new ArrayList<>(profile.getPartitionPaths()); partitionPathOffset = new HashMap<>(); int i = 0; for (Object partitionPath : profile.getPartitionPaths()) { partitionPathOffset.put(partitionPath.toString(), i); - i += numBuckets; + if (isPartitionBucketIndexEnabled) { + i += bucketStrategist.getBucketNumber(partitionPath.toString()); + } else { + i += numBuckets; + } } + + this.totalPartitions = i; assignUpdates(profile); WriteOperationType operationType = profile.getOperationType(); this.isOverwrite = INSERT_OVERWRITE.equals(operationType) || INSERT_OVERWRITE_TABLE.equals(operationType); + + if (isPartitionBucketIndexEnabled) { + this.partitionNumberToPath = new String[totalPartitions]; + this.partitionNumberToLocalBucketId = new Integer[totalPartitions]; + + for (String partitionPath : partitionPaths) { + int offset = partitionPathOffset.get(partitionPath); + int numBuckets = bucketStrategist.getBucketNumber(partitionPath); + + for (int j = 0; j < numBuckets; j++) { + int partitionNumber = offset + j; + partitionNumberToPath[partitionNumber] = partitionPath; + partitionNumberToLocalBucketId[partitionNumber] = j; + } + } + } } private void assignUpdates(WorkloadProfile profile) { @@ -111,8 +152,9 @@ private void assignUpdates(WorkloadProfile profile) { @Override public BucketInfo getBucketInfo(int bucketNumber) { - String partitionPath = partitionPaths.get(bucketNumber / numBuckets); - String bucketId = BucketIdentifier.bucketIdStr(bucketNumber % numBuckets); + Pair res = computeBucketAndPartitionPath(bucketNumber); + String bucketId = res.getLeft(); + String partitionPath = res.getRight(); // Insert overwrite always generates new bucket file id if (isOverwrite) { return new BucketInfo(BucketType.INSERT, BucketIdentifier.newBucketFileIdPrefix(bucketId), partitionPath); @@ -128,9 +170,23 @@ public BucketInfo getBucketInfo(int bucketNumber) { } } + private Pair computeBucketAndPartitionPath(int bucketNumber) { + Integer bucket; + String partitionPath; + if (isPartitionBucketIndexEnabled) { + bucket = partitionNumberToLocalBucketId[bucketNumber]; + partitionPath = partitionNumberToPath[bucketNumber]; + ValidationUtils.checkArgument(bucket != null && partitionPath != null); + } else { + bucket = bucketNumber % numBuckets; + partitionPath = partitionPaths.get(bucketNumber / numBuckets); + } + return Pair.of(BucketIdentifier.bucketIdStr(bucket), partitionPath); + } + @Override public int numPartitions() { - return totalPartitionPaths * numBuckets; + return totalPartitions; } @Override @@ -140,7 +196,7 @@ public int getPartition(Object key) { Option location = keyLocation._2; int bucketId = location.isPresent() ? BucketIdentifier.bucketIdFromFileId(location.get().getFileId()) - : BucketIdentifier.getBucketId(keyLocation._1, indexKeyField, numBuckets); + : BucketIdentifier.getBucketId(keyLocation._1, indexKeyField, bucketStrategist.getBucketNumber(partitionPath)); return partitionPathOffset.get(partitionPath) + bucketId; } } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkBulkInsertHelper.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkBulkInsertHelper.java index fc4b8bf100624..b1f95eb852009 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkBulkInsertHelper.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkBulkInsertHelper.java @@ -132,4 +132,36 @@ public HoodieData bulkInsert(HoodieData> inputRecor return HoodieJavaRDD.of(writeStatusRDD); } + + @Override + public HoodieData bulkInsertForClustering(HoodieData> inputRecords, + String instantTime, + HoodieTable>, HoodieData, HoodieData> table, + HoodieWriteConfig config, + boolean performDedupe, + BulkInsertPartitioner partitioner, + boolean useWriterSchema, + int configuredParallelism, + WriteHandleFactory writeHandleFactory) { + + // De-dupe/merge if needed + HoodieData> dedupedRecords = inputRecords; + + int targetParallelism = deduceShuffleParallelism(inputRecords, configuredParallelism); + + if (performDedupe) { + dedupedRecords = (HoodieData>) HoodieWriteHelper.newInstance() + .combineOnCondition(config.shouldCombineBeforeInsert(), inputRecords, targetParallelism, table); + } + + // only JavaRDD is supported for Spark partitioner, but it is not enforced by BulkInsertPartitioner API. To improve this, TODO HUDI-3463 + final HoodieData> repartitionedRecords = + HoodieJavaRDD.of((JavaRDD>) partitioner.repartitionRecords(HoodieJavaRDD.getJavaRDD(dedupedRecords), targetParallelism)); + + List res = HoodieJavaRDD.getJavaRDD(repartitionedRecords) + .mapPartitionsWithIndex(new BulkInsertMapFunction<>(instantTime, + partitioner.arePartitionRecordsSorted(), config, table, useWriterSchema, partitioner, writeHandleFactory), true) + .flatMap(List::iterator).collect(); + return table.getContext().parallelize(res); + } } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertOverwriteCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertOverwriteCommitActionExecutor.java index b265b32da8edc..23ba95e692273 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertOverwriteCommitActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertOverwriteCommitActionExecutor.java @@ -34,6 +34,7 @@ import org.apache.spark.Partitioner; +import java.util.Collections; import java.util.Iterator; import java.util.List; import java.util.Map; @@ -78,6 +79,9 @@ protected String getCommitActionType() { @Override protected Map> getPartitionToReplacedFileIds(HoodieWriteMetadata> writeMetadata) { + if (!config.getReplacePartitionRecords()) { + return Collections.emptyMap(); + } return HoodieJavaPairRDD.getJavaPairRDD(writeMetadata.getWriteStatuses().map(status -> status.getStat().getPartitionPath()).distinct().mapToPair(partitionPath -> Pair.of(partitionPath, getAllExistingFileIds(partitionPath)))).collectAsMap(); } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertOverwriteTableCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertOverwriteTableCommitActionExecutor.java index 868c8e2b42a6f..bede1900e2fad 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertOverwriteTableCommitActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertOverwriteTableCommitActionExecutor.java @@ -45,6 +45,9 @@ public SparkInsertOverwriteTableCommitActionExecutor(HoodieEngineContext context @Override protected Map> getPartitionToReplacedFileIds(HoodieWriteMetadata> writeMetadata) { + if (!config.getReplacePartitionRecords()) { + return Collections.emptyMap(); + } List partitionPaths = FSUtils.getAllPartitionPaths(context, config.getMetadataConfig(), table.getMetaClient().getBasePath()); if (partitionPaths == null || partitionPaths.isEmpty()) { return Collections.emptyMap(); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/fs/FSUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/fs/FSUtils.java index a17ae25250b79..73cff710ea469 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/fs/FSUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/fs/FSUtils.java @@ -18,12 +18,14 @@ package org.apache.hudi.common.fs; +import org.apache.hudi.common.config.HoodieConfig; import org.apache.hudi.common.config.HoodieMetadataConfig; import org.apache.hudi.common.config.SerializableConfiguration; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieFileFormat; import org.apache.hudi.common.model.HoodieLogFile; import org.apache.hudi.common.model.HoodiePartitionMetadata; +import org.apache.hudi.common.storage.HoodieStorageStrategy; import org.apache.hudi.common.table.HoodieTableConfig; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.view.FileSystemViewStorageConfig; @@ -34,11 +36,13 @@ import org.apache.hudi.exception.HoodieException; import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.exception.HoodieValidationException; +import org.apache.hudi.exception.InvalidHoodieFileNameException; import org.apache.hudi.exception.InvalidHoodiePathException; import org.apache.hudi.hadoop.CachingPath; import org.apache.hudi.metadata.HoodieTableMetadata; - +import org.apache.hudi.shaded.com.jd.chubaofs.hadoop.ChubaoFSFileSystem; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.LocatedFileStatus; @@ -70,6 +74,12 @@ import java.util.stream.Collectors; import java.util.stream.Stream; +import static org.apache.hudi.common.table.HoodieTableConfig.HOODIE_BASE_PATH_KEY; +import static org.apache.hudi.common.table.HoodieTableConfig.HOODIE_STORAGE_CHUBAO_FS_LOG_DIR; +import static org.apache.hudi.common.table.HoodieTableConfig.HOODIE_STORAGE_CHUBAO_FS_LOG_LEVEL; +import static org.apache.hudi.common.table.HoodieTableConfig.HOODIE_STORAGE_CHUBAO_FS_OWNER; +import static org.apache.hudi.common.table.HoodieTableConfig.HOODIE_STORAGE_PATH; +import static org.apache.hudi.common.table.HoodieTableConfig.HOODIE_STORAGE_STRATEGY_CLASS_NAME; import static org.apache.hudi.hadoop.CachingPath.getPathWithoutSchemeAndAuthority; /** @@ -82,6 +92,11 @@ public class FSUtils { // Archive log files are of this pattern - .commits_.archive.1_1-0-1 public static final Pattern LOG_FILE_PATTERN = Pattern.compile("^\\.(.+)_(.*)\\.(log|archive)\\.(\\d+)(_((\\d+)-(\\d+)-(\\d+))(.cdc)?)?"); + + // 00000000_063f8be4-49d3-43fd-bc74-ee37b8329ceb_0_0_3-1-0_20250409161256974.parquet + // 8个group + public static final Pattern LSM_LOG_FILE_PATTERN = + Pattern.compile("^(\\d+)_([0-9a-fA-F]{8}-[0-9a-fA-F]{4}-[0-9a-fA-F]{4}-[0-9a-fA-F]{4}-[0-9a-fA-F]{12})_(\\d+)_(\\d+)_(\\d+)-(\\d+)-(\\d+)_(.*)\\.(parquet)$"); public static final Pattern PREFIX_BY_FILE_ID_PATTERN = Pattern.compile("^(.+)-(\\d+)"); private static final int MAX_ATTEMPTS_RECOVER_LEASE = 10; private static final long MIN_CLEAN_TO_KEEP = 10; @@ -89,6 +104,7 @@ public class FSUtils { private static final String HOODIE_ENV_PROPS_PREFIX = "HOODIE_ENV_"; private static final PathFilter ALLOW_ALL_FILTER = file -> true; + public static final String LSM_TEMP_FILE_SUFFIX = ".temp"; public static Configuration prepareHadoopConf(Configuration conf) { // look for all properties, prefixed to be picked up @@ -101,6 +117,10 @@ public static Configuration prepareHadoopConf(Configuration conf) { return conf; } + /** + * TODO zhangyue + * 待定 HDFSParquetImporterUtils + */ public static FileSystem getFs(String pathStr, Configuration conf) { return getFs(new Path(pathStr), conf); } @@ -123,6 +143,41 @@ public static FileSystem getFs(String pathStr, Configuration conf, boolean local return getFs(pathStr, conf); } + /** + * use this api carefully, because it will load table config from base path + * @param basePath + * @param conf + * @return + */ + public static HoodieWrapperFileSystem getHoodieWrapperFileSystem(String basePath, Configuration conf) { + FileSystem fileSystem = FSUtils.getFs(basePath, conf); + HoodieTableConfig tableConfig = HoodieTableConfig.fromBasePath(basePath, conf); + return new HoodieWrapperFileSystem(fileSystem, conf, tableConfig); + } + + public static HoodieWrapperFileSystem getHoodieWrapperFileSystem(String basePath, Configuration conf, HoodieTableConfig tableConfig) { + FileSystem fileSystem = FSUtils.getFs(basePath, conf); + if (tableConfig == null) { + return new HoodieWrapperFileSystem(fileSystem, conf, HoodieTableConfig.fromBasePath(basePath, conf)); + } else { + return new HoodieWrapperFileSystem(fileSystem, conf, tableConfig); + } + } + + public static void addChubaoFsConfig2HadoopConf(Configuration configuration, HoodieTableConfig tableConfig) { + if (StringUtils.nonEmpty(tableConfig.getChubaoFsOwner())) { + LOG.info("ChubaoFs configs are set, adding them to hadoop configuration"); + configuration.set("fs.chubaofs.impl", ChubaoFSFileSystem.class.getName()); + configuration.set("cfs.log.dir", tableConfig.getChubaoFsLogDir()); + configuration.set("cfs.log.level", tableConfig.getChubaoFsLogLevel()); + String chubaoFsOwner = tableConfig.getChubaoFsOwner(); + StringUtils.splitKeyValues(chubaoFsOwner, ",", ":") + .forEach((key, value) -> { + configuration.set("cfs.owner." + key, value); + }); + } + } + /** * Check if table already exists in the given path. * @param path base path of the table. @@ -173,6 +228,26 @@ public static String makeBaseFileName(String instantTime, String writeToken, Str return String.format("%s_%s_%s%s", fileId, writeToken, instantTime, fileExtension); } + // bucketNumber_UUID_version_levelNumber_partitionId-stageId-attemptId_writeInstant.parquet.temp + public static String makeLSMFileNameWithSuffix(String fileID, String writeInstant, String version, + String levelNumber, String writeToken, String fileExtension) { + String uuid = UUID.randomUUID().toString(); + return String.format("%s_%s_%s_%s_%s_%s%s%s", fileID, uuid, version, levelNumber, + writeToken, writeInstant, fileExtension, LSM_TEMP_FILE_SUFFIX); + } + + // bucketNumber_UUID_version_levelNumber_partitionId-stageId-attemptId_writeInstant.parquet + public static String makeLSMFileName(String fileID, String writeInstant, String version, + String levelNumber, String writeToken, String fileExtension) { + String uuid = UUID.randomUUID().toString(); + return String.format("%s_%s_%s_%s_%s_%s%s", fileID, uuid, version, levelNumber, writeToken, writeInstant, fileExtension); + } + + public static String makeLSMFileName(String fileID, String uuid, String writeInstant, String version, + String levelNumber, String writeToken, String fileExtension) { + return String.format("%s_%s_%s_%s_%s_%s%s", fileID, uuid, version, levelNumber, writeToken, writeInstant, fileExtension); + } + public static String makeBootstrapIndexFileName(String instantTime, String fileId, String ext) { return String.format("%s_%s_%s%s", fileId, "1-0-1", instantTime, ext); } @@ -188,15 +263,32 @@ public static String getCommitFromCommitFile(String commitFileName) { public static String getCommitTime(String fullFileName) { if (isLogFile(fullFileName)) { - return fullFileName.split("_")[1].split("\\.")[0]; + String[] splits = fullFileName.split("_"); + if (splits.length == 6) { + // LSM LogFile + return splits[5].split("\\.")[0]; + } else { + return splits[1].split("\\.")[0]; + } } return fullFileName.split("_")[2].split("\\.")[0]; } + public static String getCommitTimeWithFullPath(String path) { + String fullFileName; + if (path.contains("/")) { + fullFileName = path.substring(path.lastIndexOf("/") + 1); + } else { + fullFileName = path; + } + return getCommitTime(fullFileName); + } + public static long getFileSize(FileSystem fs, Path path) throws IOException { - return fs.getFileStatus(path).getLen(); + return fs.exists(path) ? fs.getFileStatus(path).getLen() : 0L; } + // Get FileID From BaseFileName public static String getFileId(String fullFileName) { return fullFileName.split("_")[0]; } @@ -219,6 +311,10 @@ public static List getAllPartitionFoldersThreeLevelsDown(FileSystem fs, /** * Given a base partition and a partition path, return relative path of partition path to the base path. + * TODO zhangyue: + * HoodieBackedTableMetadataWriter + * BootstrapUtils + * BaseTableMetadata */ public static String getRelativePartitionPath(Path basePath, Path fullPartitionPath) { basePath = getPathWithoutSchemeAndAuthority(basePath); @@ -248,7 +344,8 @@ public static String getRelativePartitionPath(Path basePath, Path fullPartitionP * @param fs FileSystem instance * @param basePathStr base directory */ - public static List getAllFoldersWithPartitionMetaFile(FileSystem fs, String basePathStr) throws IOException { + public static List getAllFoldersWithPartitionMetaFile(FileSystem fs, String basePathStr, + HoodieStorageStrategy hoodieStorageStrategy) throws IOException { // If the basePathStr is a folder within the .hoodie directory then we are listing partitions within an // internal table. final boolean isMetadataTable = HoodieTableMetadata.isMetadataTable(basePathStr); @@ -257,7 +354,8 @@ public static List getAllFoldersWithPartitionMetaFile(FileSystem fs, Str processFiles(fs, basePathStr, (locatedFileStatus) -> { Path filePath = locatedFileStatus.getPath(); if (filePath.getName().startsWith(HoodiePartitionMetadata.HOODIE_PARTITION_METAFILE_PREFIX)) { - partitions.add(getRelativePartitionPath(basePath, filePath.getParent())); + partitions.add( + hoodieStorageStrategy.getRelativePath(filePath.getParent())); } return true; }, !isMetadataTable); @@ -335,8 +433,14 @@ public static Map getFilesInPartitions(HoodieEngineContext } public static String getFileExtension(String fullName) { - Objects.requireNonNull(fullName); - String fileName = new File(fullName).getName(); + String actualName; + if (fullName.endsWith(LSM_TEMP_FILE_SUFFIX)) { + actualName = fullName.replace(LSM_TEMP_FILE_SUFFIX, ""); + } else { + actualName = fullName; + } + Objects.requireNonNull(actualName); + String fileName = new File(actualName).getName(); int dotIndex = fileName.lastIndexOf('.'); return dotIndex == -1 ? "" : fileName.substring(dotIndex); } @@ -369,14 +473,42 @@ public static String createNewFileId(String idPfx, int id) { } /** - * Get the file extension from the log file. + * Get the file extension from the log file. 前面没有'.' */ public static String getFileExtensionFromLog(Path logPath) { - Matcher matcher = LOG_FILE_PATTERN.matcher(logPath.getName()); - if (!matcher.find()) { - throw new InvalidHoodiePathException(logPath, "LogFile"); + String actualFilename = logPath.getName().endsWith(LSM_TEMP_FILE_SUFFIX) ? logPath.getName().replace(LSM_TEMP_FILE_SUFFIX, "") : logPath.getName(); + Option matchLSMLogFile = matchLSMLogFile(actualFilename); + if (matchLSMLogFile.isPresent()) { + return matchLSMLogFile.get().group(9); + } else { + Matcher matcher = LOG_FILE_PATTERN.matcher(actualFilename); + if (!matcher.find()) { + throw new InvalidHoodiePathException(logPath, "LogFile"); + } + return matcher.group(3); + } + } + + /** + * Get the fileId from a fileName. + */ + public static String getFileIdFromFileName(String fileName) { + if (FSUtils.matchCommonLogFile(fileName).isPresent()) { + Matcher matcher = LOG_FILE_PATTERN.matcher(fileName); + if (!matcher.find()) { + throw new InvalidHoodieFileNameException(fileName, "LogFile"); + } + return matcher.group(1); } - return matcher.group(3); + if (FSUtils.matchLSMLogFile(fileName).isPresent()) { + Matcher matcher = FSUtils.LSM_LOG_FILE_PATTERN.matcher(fileName); + if (!matcher.find()) { + throw new InvalidHoodieFileNameException(fileName, "LogFile"); + } + return matcher.group(1); + } + + return FSUtils.getFileId(fileName); } /** @@ -384,21 +516,33 @@ public static String getFileExtensionFromLog(Path logPath) { * the file name. */ public static String getFileIdFromLogPath(Path path) { - Matcher matcher = LOG_FILE_PATTERN.matcher(path.getName()); - if (!matcher.find()) { - throw new InvalidHoodiePathException(path, "LogFile"); + Option matchLSMLogFile = matchLSMLogFile(path.getName()); + if (matchLSMLogFile.isPresent()) { + return matchLSMLogFile.get().group(1); + } else { + Matcher matcher = LOG_FILE_PATTERN.matcher(path.getName()); + if (!matcher.find()) { + throw new InvalidHoodiePathException(path, "LogFile"); + } + return matcher.group(1); } - return matcher.group(1); } /** * Check if the file is a base file of a log file. Then get the fileId appropriately. */ public static String getFileIdFromFilePath(Path filePath) { - if (FSUtils.isLogFile(filePath)) { - return FSUtils.getFileIdFromLogPath(filePath); + Option matchCommonLogFile = matchCommonLogFile(filePath.getName()); + if (matchCommonLogFile.isPresent()) { + return matchCommonLogFile.get().group(1); + } else { + Option matchLSMLogFile = matchLSMLogFile(filePath.getName()); + if (matchLSMLogFile.isPresent()) { + return matchLSMLogFile.get().group(1); + } else { + return FSUtils.getFileId(filePath.getName()); + } } - return FSUtils.getFileId(filePath.getName()); } /** @@ -406,58 +550,86 @@ public static String getFileIdFromFilePath(Path filePath) { * the file name. */ public static String getBaseCommitTimeFromLogPath(Path path) { - Matcher matcher = LOG_FILE_PATTERN.matcher(path.getName()); - if (!matcher.find()) { - throw new InvalidHoodiePathException(path, "LogFile"); + Option matchLSMLogFile = matchLSMLogFile(path.getName()); + if (matchLSMLogFile.isPresent()) { + return matchLSMLogFile.get().group(8); + } else { + Matcher matcher = LOG_FILE_PATTERN.matcher(path.getName()); + if (!matcher.find()) { + throw new InvalidHoodiePathException(path, "LogFile"); + } + return matcher.group(2); } - return matcher.group(2); } /** * Get TaskPartitionId used in log-path. */ public static Integer getTaskPartitionIdFromLogPath(Path path) { - Matcher matcher = LOG_FILE_PATTERN.matcher(path.getName()); - if (!matcher.find()) { - throw new InvalidHoodiePathException(path, "LogFile"); + Option matchLSMLogFile = matchLSMLogFile(path.getName()); + if (matchLSMLogFile.isPresent()) { + String val = matchLSMLogFile.get().group(5); + return val == null ? null : Integer.parseInt(val); + } else { + Matcher matcher = LOG_FILE_PATTERN.matcher(path.getName()); + if (!matcher.find()) { + throw new InvalidHoodiePathException(path, "LogFile"); + } + String val = matcher.group(7); + return val == null ? null : Integer.parseInt(val); } - String val = matcher.group(7); - return val == null ? null : Integer.parseInt(val); } /** * Get Write-Token used in log-path. */ public static String getWriteTokenFromLogPath(Path path) { - Matcher matcher = LOG_FILE_PATTERN.matcher(path.getName()); - if (!matcher.find()) { - throw new InvalidHoodiePathException(path, "LogFile"); + Option matchLSMLogFile = matchLSMLogFile(path.getName()); + if (matchLSMLogFile.isPresent()) { + return matchLSMLogFile.get().group(5) + "-" + matchLSMLogFile.get().group(6) + "-" + matchLSMLogFile.get().group(7); + } else { + Matcher matcher = LOG_FILE_PATTERN.matcher(path.getName()); + if (!matcher.find()) { + throw new InvalidHoodiePathException(path, "LogFile"); + } + return matcher.group(6); } - return matcher.group(6); } /** * Get StageId used in log-path. */ public static Integer getStageIdFromLogPath(Path path) { - Matcher matcher = LOG_FILE_PATTERN.matcher(path.getName()); - if (!matcher.find()) { - throw new InvalidHoodiePathException(path, "LogFile"); + Option matchLSMLogFile = matchLSMLogFile(path.getName()); + if (matchLSMLogFile.isPresent()) { + String val = matchLSMLogFile.get().group(6); + return val == null ? null : Integer.parseInt(val); + } else { + Matcher matcher = LOG_FILE_PATTERN.matcher(path.getName()); + if (!matcher.find()) { + throw new InvalidHoodiePathException(path, "LogFile"); + } + String val = matcher.group(8); + return val == null ? null : Integer.parseInt(val); } - String val = matcher.group(8); - return val == null ? null : Integer.parseInt(val); } /** * Get Task Attempt Id used in log-path. */ public static Integer getTaskAttemptIdFromLogPath(Path path) { - Matcher matcher = LOG_FILE_PATTERN.matcher(path.getName()); - if (!matcher.find()) { - throw new InvalidHoodiePathException(path, "LogFile"); + Option matchLSMLogFile = matchLSMLogFile(path.getName()); + if (matchLSMLogFile.isPresent()) { + String val = matchLSMLogFile.get().group(7); + return val == null ? null : Integer.parseInt(val); + } else { + Matcher matcher = LOG_FILE_PATTERN.matcher(path.getName()); + if (!matcher.find()) { + throw new InvalidHoodiePathException(path, "LogFile"); + } + String val = matcher.group(9); + return val == null ? null : Integer.parseInt(val); } - String val = matcher.group(9); - return val == null ? null : Integer.parseInt(val); } /** @@ -467,14 +639,46 @@ public static int getFileVersionFromLog(Path logPath) { return getFileVersionFromLog(logPath.getName()); } - public static int getFileVersionFromLog(String logFileName) { - Matcher matcher = LOG_FILE_PATTERN.matcher(logFileName); + public static int getLevelNumFromLog(Path logPath) { + String logFileName = logPath.getName(); + Matcher matcher = LSM_LOG_FILE_PATTERN.matcher(logFileName); if (!matcher.find()) { throw new HoodieIOException("Invalid log file name: " + logFileName); } return Integer.parseInt(matcher.group(4)); } + public static String getLSMFilePrefix(Path path) { + String lsmFileName = path.getName(); + Matcher matcher = LSM_LOG_FILE_PATTERN.matcher(lsmFileName); + if (!matcher.find()) { + throw new HoodieIOException("Invalid log file name: " + lsmFileName); + } + return matcher.group(1) + "_" + matcher.group(2); + } + + public static String getUUIDFromLog(Path logPath) { + String logFileName = logPath.getName(); + Matcher matcher = LSM_LOG_FILE_PATTERN.matcher(logFileName); + if (!matcher.find()) { + throw new HoodieIOException("Invalid log file name: " + logFileName); + } + return matcher.group(2); + } + + public static int getFileVersionFromLog(String logFileName) { + Option matchLSMLogFile = matchLSMLogFile(logFileName); + if (matchLSMLogFile.isPresent()) { + return Integer.parseInt(matchLSMLogFile.get().group(3)); + } else { + Matcher matcher = LOG_FILE_PATTERN.matcher(logFileName); + if (!matcher.find()) { + throw new HoodieIOException("Invalid log file name: " + logFileName); + } + return Integer.parseInt(matcher.group(4)); + } + } + public static String getSuffixFromLogPath(Path path) { Matcher matcher = LOG_FILE_PATTERN.matcher(path.getName()); if (!matcher.find()) { @@ -494,7 +698,8 @@ public static String makeLogFileName(String fileId, String logFileExtension, Str public static boolean isBaseFile(Path path) { String extension = getFileExtension(path.getName()); - return HoodieFileFormat.BASE_FILE_EXTENSIONS.contains(extension); + return HoodieFileFormat.BASE_FILE_EXTENSIONS.contains(extension) && !matchLSMLogFile(path.getName()).isPresent() + && !path.getName().contains(LSM_TEMP_FILE_SUFFIX); } public static boolean isLogFile(Path logPath) { @@ -502,8 +707,37 @@ public static boolean isLogFile(Path logPath) { } public static boolean isLogFile(String fileName) { + return matchCommonLogFile(fileName).isPresent() || matchLSMLogFile(fileName).isPresent(); + } + + public static Option matchCommonLogFile(String fileName) { + if (StringUtils.isNullOrEmpty(fileName)) { + return Option.empty(); + } + if (fileName.contains("/")) { + fileName = fileName.substring(fileName.lastIndexOf("/") + 1); + } Matcher matcher = LOG_FILE_PATTERN.matcher(fileName); - return matcher.find() && fileName.contains(".log"); + if (matcher.find() && fileName.contains(".log")) { + return Option.of(matcher); + } else { + return Option.empty(); + } + } + + public static Option matchLSMLogFile(String fileName) { + if (StringUtils.isNullOrEmpty(fileName)) { + return Option.empty(); + } + if (fileName.contains("/")) { + fileName = fileName.substring(fileName.lastIndexOf("/") + 1); + } + Matcher matcher = LSM_LOG_FILE_PATTERN.matcher(fileName); + if (matcher.find()) { + return Option.of(matcher); + } else { + return Option.empty(); + } } /** @@ -626,6 +860,18 @@ public static Long getSizeInMB(long sizeInBytes) { return sizeInBytes / (1024 * 1024); } + /** + * TODO zhangyue need change + * AWSGlueCatalogSyncClient + * RepairsCommand + * HoodieBackedTableMetadataWriter + * CleanPlanV2MigrationHandler + * RepairMigratePartitionMetaProcedure + * HoodieAdbJdbcClient + * HoodieMetadataTableValidator + * HoodieSnapshotCopier + * HoodieSnapshotExporter + */ public static Path getPartitionPath(String basePath, String partitionPath) { if (StringUtils.isNullOrEmpty(partitionPath)) { return new Path(basePath); @@ -639,6 +885,16 @@ public static Path getPartitionPath(String basePath, String partitionPath) { return getPartitionPath(new CachingPath(basePath), properPartitionPath); } + /** + * TODO zhangyue need change + * RepairsCommand + * HoodieSparkConsistentBucketIndex + * HoodieCDCExtractor -- CDC 用不了 + * CleanMetadataV1MigrationHandler + * CompactionV1MigrationHandler + * RepairAddpartitionmetaProcedure + * HoodieMetadataTableValidator + */ public static Path getPartitionPath(Path basePath, String partitionPath) { // For non-partitioned table, return only base-path return StringUtils.isNullOrEmpty(partitionPath) ? basePath : new CachingPath(basePath, partitionPath); @@ -687,11 +943,27 @@ public static boolean isCHDFileSystem(FileSystem fs) { return StorageSchemes.CHDFS.getScheme().equals(fs.getScheme()); } - public static Configuration registerFileSystem(Path file, Configuration conf) { + public static Configuration registerFileSystem(Path file, Configuration conf, HoodieConfig hoodieConfig) { Configuration returnConf = new Configuration(conf); String scheme = FSUtils.getFs(file.toString(), conf).getScheme(); returnConf.set("fs." + HoodieWrapperFileSystem.getHoodieScheme(scheme) + ".impl", HoodieWrapperFileSystem.class.getName()); + if (hoodieConfig.contains(HOODIE_BASE_PATH_KEY)) { + returnConf.set(HOODIE_BASE_PATH_KEY, hoodieConfig.getString(HOODIE_BASE_PATH_KEY)); + } + + if (hoodieConfig.contains(HOODIE_STORAGE_STRATEGY_CLASS_NAME) && hoodieConfig.contains(HOODIE_STORAGE_PATH) + && !StringUtils.isNullOrEmpty(hoodieConfig.getString(HOODIE_STORAGE_STRATEGY_CLASS_NAME)) + && !StringUtils.isNullOrEmpty(hoodieConfig.getString(HOODIE_STORAGE_PATH))) { + returnConf.set(HoodieTableConfig.HOODIE_STORAGE_STRATEGY_CLASS_NAME.key(), hoodieConfig.getString(HOODIE_STORAGE_STRATEGY_CLASS_NAME.key())); + returnConf.set(HoodieTableConfig.HOODIE_STORAGE_PATH.key(), hoodieConfig.getString(HOODIE_STORAGE_PATH.key())); + + if ((hoodieConfig.contains(HOODIE_STORAGE_CHUBAO_FS_OWNER) && !StringUtils.isNullOrEmpty(hoodieConfig.getString(HOODIE_STORAGE_CHUBAO_FS_OWNER)))) { + returnConf.set(HoodieTableConfig.HOODIE_STORAGE_CHUBAO_FS_LOG_DIR.key(), hoodieConfig.getStringOrDefault(HOODIE_STORAGE_CHUBAO_FS_LOG_DIR)); + returnConf.set(HoodieTableConfig.HOODIE_STORAGE_CHUBAO_FS_LOG_LEVEL.key(), hoodieConfig.getStringOrDefault(HOODIE_STORAGE_CHUBAO_FS_LOG_LEVEL)); + returnConf.set(HoodieTableConfig.HOODIE_STORAGE_CHUBAO_FS_OWNER.key(), hoodieConfig.getString(HOODIE_STORAGE_CHUBAO_FS_OWNER)); + } + } return returnConf; } @@ -820,6 +1092,41 @@ public static boolean deleteSubPath(String subPathStr, SerializableConfiguration } } + public static List getAllDataFileStatus(FileSystem fs, Path path) throws IOException { + List statuses = new ArrayList<>(); + for (FileStatus status : fs.listStatus(path)) { + if (!status.getPath().toString().contains(HoodieTableMetaClient.METAFOLDER_NAME)) { + if (status.isDirectory()) { + statuses.addAll(getAllDataFileStatus(fs, status.getPath())); + } else { + statuses.add(status); + } + } + } + return statuses; + } + + /** + * Deletes a sub-path. + * + * @param fs The file system implementation for this table. + * @param path the base path of hudi table. + * @return A list of file status of files under the path. + */ + public static List getAllDataFileStatusExcludingMetaFolder(FileSystem fs, Path path) throws IOException { + List statuses = new ArrayList<>(); + for (FileStatus status : fs.listStatus(path)) { + if (!status.getPath().toString().contains(HoodieTableMetaClient.METAFOLDER_NAME)) { + if (status.isDirectory()) { + statuses.addAll(getAllDataFileStatusExcludingMetaFolder(fs, status.getPath())); + } else { + statuses.add(status); + } + } + } + return statuses; + } + /** * Lists file status at a certain level in the directory hierarchy. *

    @@ -879,4 +1186,118 @@ public interface SerializableFunction extends Function, Serializable private static Option getLatestLogFile(Stream logFiles) { return Option.fromJavaOptional(logFiles.min(HoodieLogFile.getReverseLogFileComparator())); } -} + + /** + * Copy from HoodieWrapperFileSystem.createImmutableFileInPath(xx) + */ + public static void createImmutableFileInPath(FileSystem fs, Path fullPath, Option content, boolean needTempFile) + throws HoodieIOException { + FSDataOutputStream fsout = null; + Path tmpPath = null; + boolean isWrote = false; + try { + if (!content.isPresent()) { + fsout = fs.create(fullPath, false); + } + + if (content.isPresent() && needTempFile) { + Path parent = fullPath.getParent(); + tmpPath = new Path(parent, fullPath.getName() + "." + UUID.randomUUID()); + fsout = fs.create(tmpPath, false); + fsout.write(content.get()); + isWrote = true; + } + + if (content.isPresent() && !needTempFile) { + fsout = fs.create(fullPath, false); + fsout.write(content.get()); + } + } catch (IOException e) { + String errorMsg = "Failed to create file" + (tmpPath != null ? tmpPath : fullPath); + throw new HoodieIOException(errorMsg, e); + } finally { + boolean isClosed = false; + boolean renameSuccess = false; + try { + if (null != fsout) { + fsout.close(); + isClosed = true; + } + if (null != tmpPath && isClosed && isWrote) { + renameSuccess = fs.rename(tmpPath, fullPath); + } + } catch (IOException e) { + throw new HoodieIOException("HoodieIOE occurs, rename " + tmpPath + " to the target " + + fullPath + ": " + renameSuccess + ", " + " closing : " + isClosed, e); + } finally { + if (!renameSuccess && null != tmpPath) { + try { + fs.delete(tmpPath); + LOG.warn("Fail to rename " + tmpPath + " to " + fullPath + + ", target file exists: " + fs.exists(fullPath)); + } catch (IOException e) { + throw new HoodieIOException("Failed to delete tmp file " + tmpPath, e); + } + } + } + } + } + + /** + * Delete inflight and requested rollback instant file after temp file created. + */ + public static void createImmutableRollbackFileInPath(FileSystem fs, Path fullPath, Option content, boolean needTempFile, Runnable deleteInflightAndRequestedInstantFunc) + throws HoodieIOException { + FSDataOutputStream fsout = null; + Path tmpPath = null; + boolean isWrote = false; + try { + if (content.isPresent() && needTempFile) { + Path parent = fullPath.getParent(); + tmpPath = new Path(parent, fullPath.getName() + "." + UUID.randomUUID()); + fsout = fs.create(tmpPath, false); + fsout.write(content.get()); + isWrote = true; + } + + deleteInflightAndRequestedInstantFunc.run(); + + if (!content.isPresent()) { + fsout = fs.create(fullPath, false); + } + + if (content.isPresent() && !needTempFile) { + fsout = fs.create(fullPath, false); + fsout.write(content.get()); + } + } catch (IOException e) { + String errorMsg = "Failed to create file" + (tmpPath != null ? tmpPath : fullPath); + throw new HoodieIOException(errorMsg, e); + } finally { + boolean isClosed = false; + boolean renameSuccess = false; + try { + if (null != fsout) { + fsout.close(); + isClosed = true; + } + if (null != tmpPath && isClosed && isWrote) { + renameSuccess = fs.rename(tmpPath, fullPath); + } + } catch (IOException e) { + throw new HoodieIOException("HoodieIOE occurs, rename " + tmpPath + " to the target " + + fullPath + ": " + renameSuccess + ", " + " closing : " + isClosed, e); + } finally { + if (!renameSuccess && null != tmpPath) { + try { + // Don't delete temp rollback file if rename failed + LOG.warn("Fail to rename " + tmpPath + " to " + fullPath + + ", target file exists: " + fs.exists(fullPath)); + } catch (IOException e) { + throw new HoodieIOException("Failed to check target file exists or not: " + fullPath, e); + } + } + } + } + } +} \ No newline at end of file diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/ClusteringGroupInfo.java b/hudi-common/src/main/java/org/apache/hudi/common/model/ClusteringGroupInfo.java index 24a666a532ffe..b27af5f0af89b 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/model/ClusteringGroupInfo.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/ClusteringGroupInfo.java @@ -22,6 +22,7 @@ import java.io.Serializable; import java.util.List; +import java.util.Map; import java.util.Objects; import java.util.stream.Collectors; @@ -31,6 +32,7 @@ */ public class ClusteringGroupInfo implements Serializable { + private Map extraMeta; private List operations; private int numOutputGroups; @@ -38,18 +40,26 @@ public static ClusteringGroupInfo create(HoodieClusteringGroup clusteringGroup) List operations = clusteringGroup.getSlices().stream() .map(ClusteringOperation::create).collect(Collectors.toList()); - return new ClusteringGroupInfo(operations, clusteringGroup.getNumOutputFileGroups()); + return new ClusteringGroupInfo(operations, clusteringGroup.getNumOutputFileGroups(), clusteringGroup.getExtraMetadata()); } // Only for serialization/de-serialization @Deprecated - public ClusteringGroupInfo() {} + public ClusteringGroupInfo() { + + } private ClusteringGroupInfo(final List operations, final int numOutputGroups) { this.operations = operations; this.numOutputGroups = numOutputGroups; } + private ClusteringGroupInfo(final List operations, final int numOutputGroups, Map extraMeta) { + this.operations = operations; + this.numOutputGroups = numOutputGroups; + this.extraMeta = extraMeta; + } + public List getOperations() { return this.operations; } @@ -66,6 +76,10 @@ public void setNumOutputGroups(final int numOutputGroups) { this.numOutputGroups = numOutputGroups; } + public Map getExtraMeta() { + return extraMeta; + } + @Override public boolean equals(final Object o) { if (this == o) { diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieFileGroup.java b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieFileGroup.java index 9b5e8c1dd6f02..83cea91942615 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieFileGroup.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieFileGroup.java @@ -22,6 +22,9 @@ import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.util.Option; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; + import java.io.Serializable; import java.util.Comparator; import java.util.List; @@ -37,6 +40,8 @@ */ public class HoodieFileGroup implements Serializable { + private static final Logger LOG = LogManager.getLogger(HoodieFileGroup.class); + public static Comparator getReverseCommitTimeComparator() { return Comparator.reverseOrder(); } @@ -49,7 +54,7 @@ public static Comparator getReverseCommitTimeComparator() { /** * Slices of files in this group, sorted with greater commit first. */ - private final TreeMap fileSlices; + private TreeMap fileSlices; /** * Timeline, based on which all getter work. @@ -61,15 +66,19 @@ public static Comparator getReverseCommitTimeComparator() { */ private final Option lastInstant; + // todo zhangyue143 isLSMBasedLogFormat 能否去掉 + private final boolean isLSMBasedLogFormat; + public HoodieFileGroup(HoodieFileGroup fileGroup) { this.timeline = fileGroup.timeline; this.fileGroupId = fileGroup.fileGroupId; this.fileSlices = new TreeMap<>(fileGroup.fileSlices); this.lastInstant = fileGroup.lastInstant; + this.isLSMBasedLogFormat = false; } public HoodieFileGroup(String partitionPath, String id, HoodieTimeline timeline) { - this(new HoodieFileGroupId(partitionPath, id), timeline); + this(new HoodieFileGroupId(partitionPath, id), timeline, false); } public HoodieFileGroup(HoodieFileGroupId fileGroupId, HoodieTimeline timeline) { @@ -77,6 +86,15 @@ public HoodieFileGroup(HoodieFileGroupId fileGroupId, HoodieTimeline timeline) { this.fileSlices = new TreeMap<>(HoodieFileGroup.getReverseCommitTimeComparator()); this.timeline = timeline; this.lastInstant = timeline.lastInstant(); + this.isLSMBasedLogFormat = false; + } + + public HoodieFileGroup(HoodieFileGroupId fileGroupId, HoodieTimeline timeline, boolean isLSMBasedLogFormat) { + this.fileGroupId = fileGroupId; + this.fileSlices = new TreeMap<>(HoodieFileGroup.getReverseCommitTimeComparator()); + this.timeline = timeline; + this.lastInstant = timeline.lastInstant(); + this.isLSMBasedLogFormat = isLSMBasedLogFormat; } /** @@ -96,7 +114,29 @@ public void addBaseFile(HoodieBaseFile dataFile) { if (!fileSlices.containsKey(dataFile.getCommitTime())) { fileSlices.put(dataFile.getCommitTime(), new FileSlice(fileGroupId, dataFile.getCommitTime())); } - fileSlices.get(dataFile.getCommitTime()).setBaseFile(dataFile); + FileSlice fileSlice = fileSlices.get(dataFile.getCommitTime()); + Option oldBaseFileOpt = fileSlice.getBaseFile(); + if (!oldBaseFileOpt.isPresent()) { + fileSlice.setBaseFile(dataFile); + } else { + if (oldBaseFileOpt.get().getModifyTime() > 0 && dataFile.getModifyTime() > 0) { + // 若base文件发生冲突,优先选择modify time小的那个文件 + if (dataFile.getModifyTime() < oldBaseFileOpt.get().getModifyTime()) { + LOG.warn("Modify Time Compare ==> Replace base file of file slice " + fileSlice + " from " + oldBaseFileOpt.get() + " to " + dataFile); + fileSlice.setBaseFile(dataFile); + } else { + LOG.warn("Modify Time Compare ==> Skip tp replace base file of file slice " + fileSlice + " from " + oldBaseFileOpt.get() + " to " + dataFile); + } + } else { + // 若base文件发生冲突,优先选择length大的那个文件的那个文件 + if (oldBaseFileOpt.get().getFileLen() <= dataFile.getFileLen()) { + LOG.warn("File Length Compare ==> Replace base file of file slice " + fileSlice + " from " + oldBaseFileOpt.get() + " to " + dataFile); + fileSlice.setBaseFile(dataFile); + } else { + LOG.warn("File Length Compare ==> Skip tp replace base file of file slice " + fileSlice + " from " + oldBaseFileOpt.get() + " to " + dataFile); + } + } + } } /** @@ -154,7 +194,8 @@ public Stream getAllFileSlices() { } public Stream getAllFileSlicesBeforeOn(String maxInstantTime) { - return fileSlices.values().stream().filter(slice -> compareTimestamps(slice.getBaseInstantTime(), LESSER_THAN_OR_EQUALS, maxInstantTime)); + return isLSMBasedLogFormat ? getAllFileSlices().filter(slice -> compareTimestamps(slice.getBaseInstantTime(), LESSER_THAN_OR_EQUALS, maxInstantTime)) : + fileSlices.values().stream().filter(slice -> compareTimestamps(slice.getBaseInstantTime(), LESSER_THAN_OR_EQUALS, maxInstantTime)); } /** @@ -219,6 +260,10 @@ public void addFileSlice(FileSlice slice) { fileSlices.put(slice.getBaseInstantTime(), slice); } + public void initFileSlices(TreeMap fileSlices) { + this.fileSlices = fileSlices; + } + public Stream getAllRawFileSlices() { return fileSlices.values().stream(); } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieLogFile.java b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieLogFile.java index 988194964f71d..374e905679d25 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieLogFile.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieLogFile.java @@ -19,6 +19,7 @@ package org.apache.hudi.common.model; import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.model.lsm.HoodieLSMLogFile; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; @@ -42,11 +43,13 @@ public class HoodieLogFile implements Serializable { public static final String LOG_FILE_PREFIX = "."; public static final Integer LOGFILE_BASE_VERSION = 1; + // 从小到大排序,小的commit以及小的version在前面 private static final Comparator LOG_FILE_COMPARATOR = new LogFileComparator(); + // 从大到小排序,大的commit以及大的version在前面 private static final Comparator LOG_FILE_COMPARATOR_REVERSED = new LogFileComparator().reversed(); private transient FileStatus fileStatus; - private final String pathStr; + protected final String pathStr; private long fileLen; public HoodieLogFile(HoodieLogFile logFile) { @@ -137,6 +140,16 @@ public HoodieLogFile rollOver(FileSystem fs, String logWriteToken) throws IOExce FSUtils.makeLogFileName(fileId, extension, baseCommitTime, newVersion, logWriteToken))); } + public HoodieLogFile rollOverDirectly(String logWriteToken) { + String fileId = getFileId(); + String baseCommitTime = getBaseCommitTime(); + Path path = getPath(); + String extension = "." + FSUtils.getFileExtensionFromLog(path); + int newVersion = getLogVersion() + 1; + return new HoodieLogFile(new Path(path.getParent(), + FSUtils.makeLogFileName(fileId, extension, baseCommitTime, newVersion, logWriteToken))); + } + public static Comparator getLogFileComparator() { return LOG_FILE_COMPARATOR; } @@ -163,6 +176,14 @@ private Comparator getWriteTokenComparator() { @Override public int compare(HoodieLogFile o1, HoodieLogFile o2) { + if (o1 instanceof HoodieLSMLogFile) { + return compareLSMLogFile(o1, o2); + } else { + return compareLogFile(o1, o2); + } + } + + private int compareLogFile(HoodieLogFile o1, HoodieLogFile o2) { String baseInstantTime1 = o1.getBaseCommitTime(); String baseInstantTime2 = o2.getBaseCommitTime(); @@ -188,6 +209,47 @@ public int compare(HoodieLogFile o1, HoodieLogFile o2) { // compare by base-commits return baseInstantTime1.compareTo(baseInstantTime2); } + + /** + * LSM Log 的排序规则: + * 1. 先按CommitTime排序 + * 2. CommitTime相同按照Version排序 + * 3. CommitTime相同 且 Version相同 按照 UUID 排序 + */ + private int compareLSMLogFile(HoodieLogFile o1, HoodieLogFile o2) { + HoodieLSMLogFile lsmO1 = (HoodieLSMLogFile) o1; + HoodieLSMLogFile lsmO2 = (HoodieLSMLogFile) o2; + + String baseCommitTime1 = lsmO1.getBaseCommitTime(); + String baseCommitTime2 = lsmO2.getBaseCommitTime(); + if (baseCommitTime1.equals(baseCommitTime2)) { + int logVersion1 = lsmO1.getLogVersion(); + int logVersion2 = lsmO2.getLogVersion(); + if (logVersion1 == logVersion2) { + String uuid1 = lsmO1.getUUID(); + String uuid2 = lsmO2.getUUID(); + return uuid1.compareToIgnoreCase(uuid2); + } + return Integer.compare(logVersion1, logVersion2); + } + return baseCommitTime1.compareToIgnoreCase(baseCommitTime2); + } + } + + public static class LSMLogFileComparatorWithLevel implements Comparator, Serializable { + private final LogFileComparator secondary = new LogFileComparator(); + + @Override + public int compare(HoodieLogFile o1, HoodieLogFile o2) { + HoodieLSMLogFile lsmO1 = (HoodieLSMLogFile) o1; + HoodieLSMLogFile lsmO2 = (HoodieLSMLogFile) o2; + int levelNumber1 = lsmO1.getLevelNumber(); + int levelNumber2 = lsmO2.getLevelNumber(); + if (levelNumber1 == levelNumber2) { + return secondary.compareLSMLogFile(o1, o2); + } + return Integer.compare(levelNumber1, levelNumber2); + } } @Override diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieRecord.java b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieRecord.java index d78241aaeb455..a276392638d7f 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieRecord.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieRecord.java @@ -145,6 +145,7 @@ public String getFieldName() { * The metaData of the record. */ protected Option> metaData; + protected transient Comparable orderingValue; public HoodieRecord(HoodieKey key, T data) { this(key, data, null, Option.empty()); @@ -180,7 +181,9 @@ public HoodieRecord(HoodieRecord record) { this.sealed = record.sealed; } - public HoodieRecord() {} + public HoodieRecord() { + + } public abstract HoodieRecord newInstance(); @@ -192,11 +195,22 @@ public HoodieKey getKey() { return key; } + public void setKey(HoodieKey key) { + this.key = key; + } + public HoodieOperation getOperation() { return operation; } - public abstract Comparable getOrderingValue(Schema recordSchema, Properties props); + public Comparable getOrderingValue(Schema recordSchema, Properties props) { + if (orderingValue == null) { + orderingValue = doGetOrderingValue(recordSchema, props); + } + return orderingValue; + } + + public abstract Comparable doGetOrderingValue(Schema recordSchema, Properties props); public T getData() { if (data == null) { @@ -289,6 +303,10 @@ public String getRecordKey() { public abstract String getRecordKey(Schema recordSchema, String keyFieldName); + public String getPartitionPath(Schema recordSchema, Option keyGeneratorOpt) { + return getPartitionPath(); + } + public void seal() { this.sealed = true; } @@ -408,10 +426,14 @@ protected static boolean hasMetaFields(Schema schema) { * org.apache.spark.sql.hudi.command.payload.ExpressionPayload */ private static class EmptyRecord implements GenericRecord { - private EmptyRecord() {} + private EmptyRecord() { + + } @Override - public void put(int i, Object v) {} + public void put(int i, Object v) { + + } @Override public Object get(int i) { @@ -424,7 +446,9 @@ public Schema getSchema() { } @Override - public void put(String key, Object v) {} + public void put(String key, Object v) { + + } @Override public Object get(String key) { @@ -433,6 +457,6 @@ public Object get(String key) { } public enum HoodieRecordType { - AVRO, SPARK + AVRO, SPARK, FLINK } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieRecordMerger.java b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieRecordMerger.java index da413592abc6e..2aedb7efccaa8 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieRecordMerger.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieRecordMerger.java @@ -38,6 +38,26 @@ public interface HoodieRecordMerger extends Serializable { String DEFAULT_MERGER_STRATEGY_UUID = "eeb8d96f-b1e4-49fd-bbf8-28ac514178e5"; + // Uses event time ordering to determine which record is chosen + String EVENT_TIME_BASED_MERGE_STRATEGY_UUID = "53ce6957-2a07-4c95-bf1e-548663905b70"; + + // Always chooses the most recently written record + String COMMIT_TIME_BASED_MERGE_STRATEGY_UUID = "ce9acb64-bde0-424c-9b91-f6ebba25356d"; + + // Use overwrite with latest record + // String OVERWRITE_LATEST_MERGE_STRATEGY_UUID = "9b450e53-5324-424b-a486-af177c427e49"; + + // Uses customized merge strategy to merge records + String CUSTOM_MERGE_STRATEGY_UUID = "1897ef5f-18bc-4557-939c-9d6a8afd1519"; + + // partial update + String PARTIAL_UPDATE_MERGE_STRATEGY_UUID = "759aa20e-3966-4244-ba6f-c1c2300e8784"; + + // partial update support multi ts + String PARTIAL_UPDATE_MULTI_TS_MERGE_STRATEGY_UUID = "f80795fd-d331-4a78-97c2-6f19e750ac29"; + + // Use avro payload to merge records + String PAYLOAD_BASED_MERGE_STRATEGY_UUID = "00000000-0000-0000-0000-000000000000"; /** * This method converges combineAndGetUpdateValue and precombine from HoodiePayload. diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/ModelUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/model/ModelUtils.java new file mode 100644 index 0000000000000..871b4f5e01cef --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/ModelUtils.java @@ -0,0 +1,32 @@ +/* + * 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.common.model; + +public class ModelUtils { + + public static String getSupportedRecordMerger() { + return "org.apache.hudi.common.model.HoodieAvroRecordMerger," + + "org.apache.hudi.HoodieSparkRecordMerger," + + "org.apache.hudi.CommitTimeSparkRecordMerger," + + "org.apache.spark.sql.hudi.command.HoodieSparkValidateDuplicateKeyRecordMerger," + + "org.apache.hudi.EventTimeHoodieSparkRecordMerger," + + "org.apache.hudi.PartialUpdateSparkRecordMerger," + + "org.apache.hudi.PartialUpdateSupportMultiTsSparkRecordMerger"; + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/lsm/HoodieLSMLogFile.java b/hudi-common/src/main/java/org/apache/hudi/common/model/lsm/HoodieLSMLogFile.java new file mode 100644 index 0000000000000..6c210a35fb1bf --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/lsm/HoodieLSMLogFile.java @@ -0,0 +1,107 @@ +/* + * 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.common.model.lsm; + +import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.model.HoodieLogFile; + +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.Path; + +import java.util.Objects; + +/** + * Abstracts a single log file. Contains methods to extract metadata like the fileId, version and extension from the log + * file path. + *

    + * Also contains logic to roll-over the log file + */ +public class HoodieLSMLogFile extends HoodieLogFile { + + private static final long serialVersionUID = 1L; + private String min; + private String max; + + public HoodieLSMLogFile(HoodieLSMLogFile logFile) { + super(logFile); + } + + public HoodieLSMLogFile(FileStatus fileStatus) { + super(fileStatus); + } + + public HoodieLSMLogFile(Path logPath) { + super(logPath); + } + + public HoodieLSMLogFile(Path logPath, Long fileLen) { + super(logPath, fileLen); + } + + public HoodieLSMLogFile(String logPathStr) { + super(logPathStr); + } + + // used for test only + public HoodieLSMLogFile(String logPathStr, long fileLength, String min, String max) { + super(new Path(logPathStr), fileLength); + this.min = min; + this.max = max; + } + + public String getUUID() { + return FSUtils.getUUIDFromLog(getPath()); + } + + public int getLevelNumber() { + return FSUtils.getLevelNumFromLog(getPath()); + } + + public String getFilePrefix() { + return FSUtils.getLSMFilePrefix(getPath()); + } + + public String getMin() { + return min; + } + + public void setMin(String min) { + this.min = min; + } + + public String getMax() { + return max; + } + + public void setMax(String max) { + this.max = max; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + HoodieLSMLogFile that = (HoodieLSMLogFile) o; + return Objects.equals(pathStr, that.pathStr); + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/dto/DTOUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/dto/DTOUtils.java index ef5a886948765..f180e72a49602 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/dto/DTOUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/dto/DTOUtils.java @@ -57,6 +57,10 @@ public static Stream fileGroupDTOsToFileGroups(List FileGroupDTO.toFileGroup(dto, timeline)); + if (metaClient.getTableConfig().isLSMBasedLogFormat()) { + return dtos.stream().map(dto -> FileGroupDTO.toLSMFileGroup(dto, timeline)); + } else { + return dtos.stream().map(dto -> FileGroupDTO.toFileGroup(dto, timeline)); + } } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/dto/FileGroupDTO.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/dto/FileGroupDTO.java index bc5cbdb8022a0..23200c4110689 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/dto/FileGroupDTO.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/dto/FileGroupDTO.java @@ -19,6 +19,7 @@ package org.apache.hudi.common.table.timeline.dto; import org.apache.hudi.common.model.HoodieFileGroup; +import org.apache.hudi.common.model.HoodieFileGroupId; import org.apache.hudi.common.table.timeline.HoodieTimeline; import com.fasterxml.jackson.annotation.JsonIgnoreProperties; @@ -61,4 +62,10 @@ public static HoodieFileGroup toFileGroup(FileGroupDTO dto, HoodieTimeline fgTim dto.slices.stream().map(FileSliceDTO::toFileSlice).forEach(fileGroup::addFileSlice); return fileGroup; } + + public static HoodieFileGroup toLSMFileGroup(FileGroupDTO dto, HoodieTimeline fgTimeline) { + HoodieFileGroup fileGroup = new HoodieFileGroup(new HoodieFileGroupId(dto.partition, dto.id), fgTimeline, true); + dto.slices.stream().map(FileSliceDTO::toLSMFileSlice).forEach(fileGroup::addFileSlice); + return fileGroup; + } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/dto/FileSliceDTO.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/dto/FileSliceDTO.java index b3f7d24b7a52d..9e2cd0771d856 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/dto/FileSliceDTO.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/dto/FileSliceDTO.java @@ -19,11 +19,13 @@ package org.apache.hudi.common.table.timeline.dto; import org.apache.hudi.common.model.FileSlice; +import org.apache.hudi.common.model.HoodieLogFile; import com.fasterxml.jackson.annotation.JsonIgnoreProperties; import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; +import java.util.TreeSet; import java.util.stream.Collectors; /** @@ -56,7 +58,18 @@ public static FileSliceDTO fromFileSlice(FileSlice slice) { public static FileSlice toFileSlice(FileSliceDTO dto) { FileSlice slice = new FileSlice(dto.partitionPath, dto.baseInstantTime, dto.fileId); slice.setBaseFile(BaseFileDTO.toHoodieBaseFile(dto.baseFile)); - dto.logFiles.stream().forEach(lf -> slice.addLogFile(LogFileDTO.toHoodieLogFile(lf))); + dto.logFiles.forEach(lf -> slice.addLogFile(LogFileDTO.toHoodieLogFile(lf))); + return slice; + } + + public static FileSlice toLSMFileSlice(FileSliceDTO dto) { + FileSlice slice = new FileSlice(dto.partitionPath, dto.baseInstantTime, dto.fileId); + TreeSet logfiles = new TreeSet<>(HoodieLogFile.getReverseLogFileComparator()); + dto.logFiles.forEach(lf -> { + HoodieLogFile lsmLog = LogFileDTO.toLSMHoodieLogFile(lf); + logfiles.add(lsmLog); + }); + slice.resetLogFiles(logfiles); return slice; } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/dto/LogFileDTO.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/dto/LogFileDTO.java index 5f083d02e327c..34e5021e474eb 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/dto/LogFileDTO.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/dto/LogFileDTO.java @@ -19,6 +19,7 @@ package org.apache.hudi.common.table.timeline.dto; import org.apache.hudi.common.model.HoodieLogFile; +import org.apache.hudi.common.model.lsm.HoodieLSMLogFile; import com.fasterxml.jackson.annotation.JsonIgnoreProperties; import com.fasterxml.jackson.annotation.JsonProperty; @@ -51,4 +52,11 @@ public static LogFileDTO fromHoodieLogFile(HoodieLogFile dataFile) { logFile.fileStatus = FileStatusDTO.fromFileStatus(dataFile.getFileStatus()); return logFile; } + + public static HoodieLogFile toLSMHoodieLogFile(LogFileDTO dto) { + FileStatus status = FileStatusDTO.toFileStatus(dto.fileStatus); + HoodieLogFile logFile = (status == null) ? new HoodieLSMLogFile(dto.pathStr) : new HoodieLSMLogFile(status); + logFile.setFileLen(dto.fileLen); + return logFile; + } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/view/AbstractTableFileSystemView.java b/hudi-common/src/main/java/org/apache/hudi/common/table/view/AbstractTableFileSystemView.java index 37c43e56a6d85..6653c35729511 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/view/AbstractTableFileSystemView.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/view/AbstractTableFileSystemView.java @@ -27,8 +27,12 @@ import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodieFileGroup; import org.apache.hudi.common.model.HoodieFileGroupId; +import org.apache.hudi.common.model.lsm.HoodieLSMLogFile; import org.apache.hudi.common.model.HoodieLogFile; import org.apache.hudi.common.model.HoodieReplaceCommitMetadata; +import org.apache.hudi.common.storage.HoodieStorageStrategy; +import org.apache.hudi.common.storage.HoodieStorageStrategyFactory; +import org.apache.hudi.common.table.HoodieTableConfig; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; @@ -36,10 +40,12 @@ import org.apache.hudi.common.util.CompactionUtils; import org.apache.hudi.common.util.HoodieTimer; import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.common.util.ValidationUtils; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.exception.HoodieIOException; +import org.apache.hudi.util.Lazy; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; @@ -57,7 +63,10 @@ import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.Set; +import java.util.TreeMap; +import java.util.TreeSet; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.locks.ReentrantReadWriteLock; import java.util.concurrent.locks.ReentrantReadWriteLock.ReadLock; @@ -103,19 +112,29 @@ public abstract class AbstractTableFileSystemView implements SyncableFileSystemV protected final WriteLock writeLock = globalLock.writeLock(); private BootstrapIndex bootstrapIndex; + protected HoodieStorageStrategy hoodieStorageStrategy; + private HoodieTableConfig tableConfig; + private boolean isLSMBasedLogFormat; private String getPartitionPathFor(HoodieBaseFile baseFile) { - return FSUtils.getRelativePartitionPath(metaClient.getBasePathV2(), baseFile.getHadoopPath().getParent()); + return hoodieStorageStrategy.getRelativePath(baseFile.getHadoopPath().getParent()); + } + + protected void init(HoodieTableMetaClient metaClient, HoodieTimeline visibleActiveTimeline) { + init(metaClient, visibleActiveTimeline, null); } /** * Initialize the view. */ - protected void init(HoodieTableMetaClient metaClient, HoodieTimeline visibleActiveTimeline) { + protected void init(HoodieTableMetaClient metaClient, HoodieTimeline visibleActiveTimeline, HoodieStorageStrategy storageStrategy) { this.metaClient = metaClient; refreshTimeline(visibleActiveTimeline); resetFileGroupsReplaced(visibleCommitsAndCompactionTimeline); this.bootstrapIndex = BootstrapIndex.getBootstrapIndex(metaClient); + this.tableConfig = metaClient.getTableConfig(); + this.isLSMBasedLogFormat = tableConfig.isLSMBasedLogFormat(); + this.hoodieStorageStrategy = storageStrategy == null ? HoodieStorageStrategyFactory.getInstant(metaClient) : storageStrategy; // Load Pending Compaction Operations resetPendingCompactionOperations(CompactionUtils.getAllPendingCompactionOperations(metaClient).values().stream() .map(e -> Pair.of(e.getKey(), CompactionOperation.convertFromAvroRecordInstance(e.getValue())))); @@ -124,7 +143,11 @@ protected void init(HoodieTableMetaClient metaClient, HoodieTimeline visibleActi .map(e -> Pair.of(e.getKey(), CompactionOperation.convertFromAvroRecordInstance(e.getValue())))); resetBootstrapBaseFileMapping(Stream.empty()); - resetFileGroupsInPendingClustering(ClusteringUtils.getAllFileGroupsInPendingClusteringPlans(metaClient)); + Pair, Map>> allFileGroupsInPendingClusteringPlans = ClusteringUtils.getAllFileGroupsInPendingClusteringPlans(metaClient); + resetFileGroupsInPendingClustering(allFileGroupsInPendingClusteringPlans.getLeft()); + if (metaClient.getTableConfig().isLSMBasedLogFormat()) { + resetLevel1FileIdInPendingClustering(allFileGroupsInPendingClusteringPlans.getRight()); + } } /** @@ -139,9 +162,11 @@ protected void refreshTimeline(HoodieTimeline visibleActiveTimeline) { /** * Adds the provided statuses into the file system view, and also caches it inside this object. */ - public List addFilesToView(FileStatus[] statuses) { + public List addFilesToView(List statuses) { HoodieTimer timer = HoodieTimer.start(); - List fileGroups = buildFileGroups(statuses, visibleCommitsAndCompactionTimeline, true); + List fileGroups = tableConfig.isLSMBasedLogFormat() + ? buildLSMFileGroups(statuses, visibleCommitsAndCompactionTimeline) : + buildFileGroups(statuses, visibleCommitsAndCompactionTimeline, true); long fgBuildTimeTakenMs = timer.endTimer(); timer.startTimer(); // Group by partition for efficient updates for both InMemory and DiskBased structures. @@ -161,7 +186,7 @@ public List addFilesToView(FileStatus[] statuses) { } }); long storePartitionsTs = timer.endTimer(); - LOG.info("addFilesToView: NumFiles=" + statuses.length + ", NumFileGroups=" + fileGroups.size() + LOG.info("addFilesToView: NumFiles=" + statuses.size() + ", NumFileGroups=" + fileGroups.size() + ", FileGroupsCreationTime=" + fgBuildTimeTakenMs + ", StoreTimeTaken=" + storePartitionsTs); return fileGroups; @@ -170,7 +195,7 @@ public List addFilesToView(FileStatus[] statuses) { /** * Build FileGroups from passed in file-status. */ - protected List buildFileGroups(FileStatus[] statuses, HoodieTimeline timeline, + protected List buildFileGroups(List statuses, HoodieTimeline timeline, boolean addPendingCompactionFileSlice) { return buildFileGroups(convertFileStatusesToBaseFiles(statuses), convertFileStatusesToLogFiles(statuses), timeline, addPendingCompactionFileSlice); @@ -185,8 +210,7 @@ protected List buildFileGroups(Stream baseFileS })); Map, List> logFiles = logFileStream.collect(Collectors.groupingBy((logFile) -> { - String partitionPathStr = - FSUtils.getRelativePartitionPath(metaClient.getBasePathV2(), logFile.getPath().getParent()); + String partitionPathStr = hoodieStorageStrategy.getRelativePath(logFile.getPath().getParent()); return Pair.of(partitionPathStr, logFile.getFileId()); })); @@ -220,6 +244,71 @@ protected List buildFileGroups(Stream baseFileS return fileGroups; } + protected List buildLSMFileGroups(List statuses, HoodieTimeline timeline) { + HoodieTimeline committedTimeline = timeline.filterCompletedInstants(); + // 获取第一个completed commit之前的pending commit + HoodieTimeline pendingBeforeFirstCommitted; + Option firstCommitted = committedTimeline.firstInstant(); + if (firstCommitted.isPresent()) { + pendingBeforeFirstCommitted = metaClient.getActiveTimeline().getCommitsTimeline() + .findInstantsBefore(firstCommitted.get().getTimestamp()) + .filterInflightsAndRequested(); + } else { + pendingBeforeFirstCommitted = null; + } + // 预编译逆序比较器,避免重复实例化 + Comparator reverseLogComparator = HoodieLogFile.getReverseLogFileComparator(); + + // 步骤1: 构建 logFilesMap (分组并排序) + Map, TreeSet> logFilesMap = convertFileStatusesToLSMLogFiles(statuses) + .filter(lsmLog -> committedTimeline.containsOrBeforeTimelineStarts(lsmLog.getBaseCommitTime()) + && (pendingBeforeFirstCommitted == null || !pendingBeforeFirstCommitted.containsInstant(lsmLog.getBaseCommitTime()))) + .collect(Collectors.groupingBy( + logFile -> Pair.of( + hoodieStorageStrategy.getRelativePath(logFile.getPath().getParent()), + logFile.getFileId() + ), + Collectors.toCollection(() -> new TreeSet<>(reverseLogComparator)) + )); + + return logFilesMap.entrySet().stream() + .map(entry -> { + Pair pair = entry.getKey(); + TreeSet hoodieLogFiles = entry.getValue(); + HoodieFileGroup group = new HoodieFileGroup(new HoodieFileGroupId(pair.getKey(), pair.getValue()), timeline, true); + + // 步骤3: 预生成 baseInstants 避免重复计算 + Set baseInstants = hoodieLogFiles.stream() + .map(HoodieLogFile::getBaseCommitTime) + .collect(Collectors.toCollection(TreeSet::new)); // 利用 TreeSet 去重且有序 + + // 步骤4: 构建 FileSlices 并优化过滤逻辑 + TreeMap fileSlices = baseInstants.stream() + .map(baseInstant -> { + // String.valueOf(baseInstant) to do deep copy + FileSlice fileSlice = new FileSlice(group.getFileGroupId(), String.valueOf(baseInstant)); + // 构建logfiles,这里使用Lazy的方式,避免内存膨胀 + Lazy> lazyLogFiles = Lazy.lazily(() -> { + return hoodieLogFiles.stream().filter(logFile -> { + return logFile.getBaseCommitTime().compareToIgnoreCase(String.valueOf(baseInstant)) <= 0; + }).collect(Collectors.toCollection( + () -> new TreeSet<>(HoodieLogFile.getReverseLogFileComparator()) + )); + }); + fileSlice.initLazyLogFiles(lazyLogFiles); + return Pair.of(baseInstant, fileSlice); + }).collect(Collectors.toMap( + Pair::getKey, + Pair::getValue, + (existing, replacement) -> existing, + () -> new TreeMap<>(HoodieFileGroup.getReverseCommitTimeComparator()) + )); + + group.initFileSlices(fileSlices); + return group; + }).collect(Collectors.toList()); + } + /** * Get replaced instant for each file group by looking at all commit instants. */ @@ -227,7 +316,12 @@ private void resetFileGroupsReplaced(HoodieTimeline timeline) { HoodieTimer hoodieTimer = HoodieTimer.start(); // for each REPLACE instant, get map of (partitionPath -> deleteFileGroup) HoodieTimeline replacedTimeline = timeline.getCompletedReplaceTimeline(); + List backtrackInstances = metaClient.getBacktrackInstances(); Stream> resultStream = replacedTimeline.getInstantsAsStream().flatMap(instant -> { + if (backtrackInstances.contains(instant.getTimestamp())) { + // It is an instance of backtracking, skip it. + return Stream.empty(); + } try { HoodieReplaceCommitMetadata replaceMetadata = HoodieReplaceCommitMetadata.fromBytes(metaClient.getActiveTimeline().getInstantDetails(instant).get(), HoodieReplaceCommitMetadata.class); @@ -354,7 +448,7 @@ private void ensurePartitionsLoadedCorrectly(List partitionList) { LOG.debug("Time taken to list partitions " + partitionSet + " =" + (endLsTs - beginLsTs)); statusesMap.forEach((partitionPair, statuses) -> { String relativePartitionStr = partitionPair.getLeft(); - List groups = addFilesToView(statuses); + List groups = addFilesToView(Arrays.stream(statuses).collect(Collectors.toList())); if (groups.isEmpty()) { storePartitionView(relativePartitionStr, new ArrayList<>()); } @@ -396,9 +490,7 @@ protected Map, FileStatus[]> listPartitions( try { fileStatusMap.put(partitionPair, metaClient.getFs().listStatus(absolutePartitionPath)); } catch (IOException e) { - // Create the path if it does not exist already if (!metaClient.getFs().exists(absolutePartitionPath)) { - metaClient.getFs().mkdirs(absolutePartitionPath); fileStatusMap.put(partitionPair, new FileStatus[0]); } else { // in case the partition path was created by another caller @@ -427,13 +519,18 @@ private void ensurePartitionLoadedCorrectly(String partition) { try { LOG.info("Building file system view for partition (" + partitionPathStr + ")"); - Path partitionPath = FSUtils.getPartitionPath(metaClient.getBasePathV2(), partitionPathStr); long beginLsTs = System.currentTimeMillis(); - FileStatus[] statuses = listPartition(partitionPath); + List files = new ArrayList<>(); + Set allLocations = hoodieStorageStrategy.getAllLocations(partitionPathStr, false); + for (Path fullPath : allLocations) { + FileStatus[] res = listPartition(fullPath); + files.addAll(Arrays.asList(res)); + } + long endLsTs = System.currentTimeMillis(); - LOG.debug("#files found in partition (" + partitionPathStr + ") =" + statuses.length + ", Time taken =" + LOG.info("#files found in partition (" + partitionPathStr + ") =" + files.size() + ", Time taken =" + (endLsTs - beginLsTs)); - List groups = addFilesToView(statuses); + List groups = addFilesToView(files); if (groups.isEmpty()) { storePartitionView(partitionPathStr, new ArrayList<>()); @@ -442,10 +539,10 @@ private void ensurePartitionLoadedCorrectly(String partition) { throw new HoodieIOException("Failed to list base files in partition " + partitionPathStr, e); } } else { - LOG.debug("View already built for Partition :" + partitionPathStr + ", FOUND is "); + LOG.info("View already built for Partition :" + partitionPathStr + ", FOUND is "); } long endTs = System.currentTimeMillis(); - LOG.debug("Time to load partition (" + partitionPathStr + ") =" + (endTs - beginTs)); + LOG.info("Time to load partition (" + partitionPathStr + ") =" + (endTs - beginTs)); return true; }); } @@ -460,9 +557,7 @@ protected FileStatus[] listPartition(Path partitionPath) throws IOException { try { return metaClient.getFs().listStatus(partitionPath); } catch (IOException e) { - // Create the path if it does not exist already if (!metaClient.getFs().exists(partitionPath)) { - metaClient.getFs().mkdirs(partitionPath); return new FileStatus[0]; } else { // in case the partition path was created by another caller @@ -476,10 +571,10 @@ protected FileStatus[] listPartition(Path partitionPath) throws IOException { * * @param statuses List of File-Status */ - private Stream convertFileStatusesToBaseFiles(FileStatus[] statuses) { + private Stream convertFileStatusesToBaseFiles(List statuses) { Predicate roFilePredicate = fileStatus -> fileStatus.getPath().getName() .contains(metaClient.getTableConfig().getBaseFileFormat().getFileExtension()); - return Arrays.stream(statuses).filter(roFilePredicate).map(HoodieBaseFile::new); + return statuses.stream().filter(roFilePredicate).map(HoodieBaseFile::new); } /** @@ -487,13 +582,22 @@ private Stream convertFileStatusesToBaseFiles(FileStatus[] statu * * @param statuses List of FIle-Status */ - private Stream convertFileStatusesToLogFiles(FileStatus[] statuses) { + private Stream convertFileStatusesToLogFiles(List statuses) { Predicate rtFilePredicate = fileStatus -> { String fileName = fileStatus.getPath().getName(); Matcher matcher = FSUtils.LOG_FILE_PATTERN.matcher(fileName); return matcher.find() && fileName.contains(metaClient.getTableConfig().getLogFileFormat().getFileExtension()); }; - return Arrays.stream(statuses).filter(rtFilePredicate).map(HoodieLogFile::new); + return statuses.stream().filter(rtFilePredicate).map(HoodieLogFile::new); + } + + private Stream convertFileStatusesToLSMLogFiles(List statuses) { + Predicate rtFilePredicate = fileStatus -> { + String fileName = fileStatus.getPath().getName(); + Matcher matcher = FSUtils.LSM_LOG_FILE_PATTERN.matcher(fileName); + return matcher.find(); + }; + return statuses.stream().filter(rtFilePredicate).map(HoodieLSMLogFile::new); } /** @@ -766,10 +870,20 @@ public final Stream getLatestFileSlices(String partitionStr) { readLock.lock(); String partitionPath = formatPartitionKey(partitionStr); ensurePartitionLoadedCorrectly(partitionPath); - return fetchLatestFileSlices(partitionPath) - .filter(slice -> !isFileGroupReplaced(slice.getFileGroupId())) - .flatMap(slice -> this.filterBaseFileAfterPendingCompaction(slice, true)) - .map(this::addBootstrapBaseFileIfPresent); + if (isLSMBasedLogFormat) { + return fetchLatestFileSlices(partitionPath) + .filter(slice -> !isFileGroupReplaced(slice.getFileGroupId())) + .map(slice -> { + return processReplacedLsmLog(slice, null); + }).filter(Objects::nonNull) + .flatMap(slice -> this.filterBaseFileAfterPendingCompaction(slice, true)) + .map(this::addBootstrapBaseFileIfPresent); + } else { + return fetchLatestFileSlices(partitionPath) + .filter(slice -> !isFileGroupReplaced(slice.getFileGroupId())) + .flatMap(slice -> this.filterBaseFileAfterPendingCompaction(slice, true)) + .map(this::addBootstrapBaseFileIfPresent); + } } finally { readLock.unlock(); } @@ -788,6 +902,13 @@ public final Option getLatestFileSlice(String partitionStr, String fi return Option.empty(); } else { Option fs = fetchLatestFileSlice(partitionPath, fileId); + + if (isLSMBasedLogFormat) { + fs = fs.map(slice -> { + return processReplacedLsmLog(slice, null); + }); + } + if (!fs.isPresent()) { return Option.empty(); } @@ -800,6 +921,10 @@ public final Option getLatestFileSlice(String partitionStr, String fi @Override public final Stream getLatestUnCompactedFileSlices(String partitionStr) { + if (isLSMBasedLogFormat) { + throw new HoodieException("Not support for lsm"); + } + try { readLock.lock(); String partitionPath = formatPartitionKey(partitionStr); @@ -831,7 +956,15 @@ public final Stream getLatestFileSlicesBeforeOrOn(String partitionStr ensurePartitionLoadedCorrectly(partitionPath); Stream> allFileSliceStream = fetchAllStoredFileGroups(partitionPath) .filter(slice -> !isFileGroupReplacedBeforeOrOn(slice.getFileGroupId(), maxCommitTime)) - .map(fg -> fg.getAllFileSlicesBeforeOn(maxCommitTime)); + .map(fg -> { + if (isLSMBasedLogFormat) { + return fg.getAllFileSlicesBeforeOn(maxCommitTime).map(slice -> { + return processReplacedLsmLog(slice, maxCommitTime); + }).filter(Objects::nonNull); + } else { + return fg.getAllFileSlicesBeforeOn(maxCommitTime); + } + }); if (includeFileSlicesInPendingCompaction) { return allFileSliceStream.map(sliceStream -> sliceStream.flatMap(slice -> this.filterBaseFileAfterPendingCompaction(slice, false))) .map(sliceStream -> Option.fromJavaOptional(sliceStream.findFirst())).filter(Option::isPresent).map(Option::get) @@ -850,6 +983,30 @@ public final Stream getLatestFileSlicesBeforeOrOn(String partitionStr } } + // TODO zhangyue143 lsm support savepoint + @Override + public final Map> getAllLatestFileSlicesBeforeOrOn(String maxCommitTime) { + if (isLSMBasedLogFormat) { + throw new HoodieException("Not support for lsm"); + } + + try { + readLock.lock(); + List formattedPartitionList = ensureAllPartitionsLoadedCorrectly(); + return formattedPartitionList.stream().collect(Collectors.toMap( + Function.identity(), + partitionPath -> fetchAllStoredFileGroups(partitionPath) + .filter(slice -> !isFileGroupReplacedBeforeOrOn(slice.getFileGroupId(), maxCommitTime)) + .map(fg -> fg.getAllFileSlicesBeforeOn(maxCommitTime)) + .map(sliceStream -> sliceStream.flatMap(slice -> this.filterBaseFileAfterPendingCompaction(slice, false))) + .map(sliceStream -> Option.fromJavaOptional(sliceStream.findFirst())).filter(Option::isPresent).map(Option::get) + .map(this::addBootstrapBaseFileIfPresent) + )); + } finally { + readLock.unlock(); + } + } + @Override public final Stream getLatestMergedFileSlicesBeforeOrOn(String partitionStr, String maxInstantTime) { try { @@ -859,7 +1016,15 @@ public final Stream getLatestMergedFileSlicesBeforeOrOn(String partit return fetchAllStoredFileGroups(partition) .filter(fg -> !isFileGroupReplacedBeforeOrOn(fg.getFileGroupId(), maxInstantTime)) .map(fileGroup -> { - Option fileSlice = fileGroup.getLatestFileSliceBeforeOrOn(maxInstantTime); + Option fileSlice; + if (isLSMBasedLogFormat) { + Option fileSliceTmp = fileGroup.getLatestFileSliceBeforeOrOn(maxInstantTime); + fileSlice = fileSliceTmp.map(slice -> { + return processReplacedLsmLog(slice, maxInstantTime); + }); + } else { + fileSlice = fileGroup.getLatestFileSliceBeforeOrOn(maxInstantTime); + } // if the file-group is under construction, pick the latest before compaction instant time. if (fileSlice.isPresent()) { fileSlice = Option.of(fetchMergedFileSlice(fileGroup, fileSlice.get())); @@ -889,10 +1054,21 @@ public final Stream getAllLogsMergedFileSliceBeforeOrOn(String partit readLock.lock(); String partition = formatPartitionKey(partitionStr); ensurePartitionLoadedCorrectly(partition); - return fetchAllStoredFileGroups(partition) - .filter(fg -> !isFileGroupReplacedBeforeOrOn(fg.getFileGroupId(), maxInstantTime)) - .map(fileGroup -> fetchAllLogsMergedFileSlice(fileGroup, maxInstantTime)) - .filter(Option::isPresent).map(Option::get).map(this::addBootstrapBaseFileIfPresent); + if (isLSMBasedLogFormat) { + return fetchAllStoredFileGroups(partition) + .filter(fg -> !isFileGroupReplacedBeforeOrOn(fg.getFileGroupId(), maxInstantTime)) + .map(fileGroup -> fetchAllLogsMergedFileSlice(fileGroup, maxInstantTime)) + .filter(Option::isPresent).map(Option::get) + .map(slice -> { + return processReplacedLsmLog(slice, maxInstantTime); + }).filter(Objects::nonNull) + .map(this::addBootstrapBaseFileIfPresent); + } else { + return fetchAllStoredFileGroups(partition) + .filter(fg -> !isFileGroupReplacedBeforeOrOn(fg.getFileGroupId(), maxInstantTime)) + .map(fileGroup -> fetchAllLogsMergedFileSlice(fileGroup, maxInstantTime)) + .filter(Option::isPresent).map(Option::get).map(this::addBootstrapBaseFileIfPresent); + } } finally { readLock.unlock(); } @@ -902,9 +1078,19 @@ public final Stream getAllLogsMergedFileSliceBeforeOrOn(String partit public final Stream getLatestFileSliceInRange(List commitsToReturn) { try { readLock.lock(); - return fetchLatestFileSliceInRange(commitsToReturn) - .filter(slice -> !isFileGroupReplacedBeforeAny(slice.getFileGroupId(), commitsToReturn)) - .map(this::addBootstrapBaseFileIfPresent); + if (isLSMBasedLogFormat) { + String maxInstant = commitsToReturn.stream().max(Comparator.naturalOrder()).get(); + return fetchLatestFileSliceInRange(commitsToReturn) + .filter(slice -> !isFileGroupReplacedBeforeAny(slice.getFileGroupId(), commitsToReturn)) + .map(slice -> { + return processReplacedLsmLog(slice, maxInstant); + }).filter(Objects::nonNull) + .map(this::addBootstrapBaseFileIfPresent); + } else { + return fetchLatestFileSliceInRange(commitsToReturn) + .filter(slice -> !isFileGroupReplacedBeforeAny(slice.getFileGroupId(), commitsToReturn)) + .map(this::addBootstrapBaseFileIfPresent); + } } finally { readLock.unlock(); } @@ -916,7 +1102,16 @@ public final Stream getAllFileSlices(String partitionStr) { readLock.lock(); String partition = formatPartitionKey(partitionStr); ensurePartitionLoadedCorrectly(partition); - return fetchAllFileSlices(partition).filter(slice -> !isFileGroupReplaced(slice.getFileGroupId())).map(this::addBootstrapBaseFileIfPresent); + if (isLSMBasedLogFormat) { + return fetchAllFileSlices(partition) + .filter(slice -> !isFileGroupReplaced(slice.getFileGroupId())) + .map(slice -> { + return processReplacedLsmLog(slice, null); + }).filter(Objects::nonNull) + .map(this::addBootstrapBaseFileIfPresent); + } else { + return fetchAllFileSlices(partition).filter(slice -> !isFileGroupReplaced(slice.getFileGroupId())).map(this::addBootstrapBaseFileIfPresent); + } } finally { readLock.unlock(); } @@ -973,6 +1168,16 @@ public final Stream> getFileGroupsInPendi } } + @Override + public Set getLevel1FileIdInPendingClustering(String partitionPath) { + try { + readLock.lock(); + return fetchLevel1FileIdInPendingClustering(partitionPath); + } finally { + readLock.unlock(); + } + } + // Fetch APIs to be implemented by concrete sub-classes /** @@ -1052,11 +1257,21 @@ public final Stream> getFileGroupsInPendi */ protected abstract Stream> fetchFileGroupsInPendingClustering(); + /** + * Fetch all lsm level1 fileId in pending clustering + */ + protected abstract Set fetchLevel1FileIdInPendingClustering(String partitionPath); + /** * resets the pending clustering operation and overwrite with the new list. */ abstract void resetFileGroupsInPendingClustering(Map fgIdToInstantMap); + /** + * resets lsm level1 fileId in pending clustering and overwrite with the new list. + */ + abstract void resetLevel1FileIdInPendingClustering(Map> level1InPendingClustering); + /** * Add metadata for file groups in pending clustering operations to the view. */ @@ -1384,6 +1599,23 @@ private boolean isFileGroupReplacedBeforeOrOn(HoodieFileGroupId fileGroupId, Str return HoodieTimeline.compareTimestamps(instant, GREATER_THAN_OR_EQUALS, hoodieInstantOption.get().getTimestamp()); } + public boolean isLSMFileGroupReplaced(String partitionPath, HoodieLogFile logfile, String maxInstantTime) { + String fileName = logfile.getPath().getName(); + if (StringUtils.isNullOrEmpty(maxInstantTime)) { + return isFileGroupReplaced(new HoodieFileGroupId(partitionPath, fileName)); + } + return isFileGroupReplacedBeforeOrOn(new HoodieFileGroupId(partitionPath, fileName), maxInstantTime); + } + + // could be null + private FileSlice processReplacedLsmLog(FileSlice slice, String maxInstant) { + TreeSet newLogs = slice.getLogFiles().filter(logfile -> { + return !isLSMFileGroupReplaced(slice.getPartitionPath(), logfile, maxInstant); + }).collect(Collectors.toCollection(() -> new TreeSet<>(HoodieLogFile.getReverseLogFileComparator()))); + slice.resetLogFiles(newLogs); + return newLogs.isEmpty() ? null : slice; + } + @Override public Option getLastInstant() { return getTimeline().lastInstant(); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/CachedIterator.java b/hudi-common/src/main/java/org/apache/hudi/common/util/CachedIterator.java new file mode 100644 index 0000000000000..a930810927730 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/CachedIterator.java @@ -0,0 +1,57 @@ +/* + * 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.common.util; + +import java.util.Iterator; + +/** + * An iterator that give a chance to release resources. + * + * @param The return type + */ +public class CachedIterator implements Iterator { + private final Iterator iterator; + private R currentRow = null; + private boolean advanced = false; + + public CachedIterator(Iterator iterator) { + this.iterator = iterator; + } + + @Override + public boolean hasNext() { + advanceIfNeeded(); + return currentRow != null; + } + + @Override + public R next() { + return null; + } + + private void advanceIfNeeded() { + if (!advanced) { + advanced = true; + if (iterator.hasNext()) { + currentRow = iterator.next(); + } + } + } + +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/ClusteringUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/util/ClusteringUtils.java index b8ce42dc5a53f..097549e946e6c 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/ClusteringUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/ClusteringUtils.java @@ -42,12 +42,16 @@ import org.apache.log4j.LogManager; import org.apache.log4j.Logger; +import java.io.FileNotFoundException; import java.io.IOException; import java.util.AbstractMap; import java.util.Arrays; +import java.util.Collections; import java.util.HashMap; +import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Set; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -61,6 +65,8 @@ public class ClusteringUtils { public static final String TOTAL_IO_READ_MB = "TOTAL_IO_READ_MB"; public static final String TOTAL_LOG_FILE_SIZE = "TOTAL_LOG_FILES_SIZE"; public static final String TOTAL_LOG_FILES = "TOTAL_LOG_FILES"; + public static final String LSM_CLUSTERING_OUT_PUT_LEVEL = "hoodie.clustering.lsm.output.level"; + public static final String LSM_CLUSTERING_OUT_PUT_LEVEL_1 = "1"; /** * Get all pending clustering plans along with their instants. @@ -111,19 +117,53 @@ public static Option> getClusteringPla return Option.of(Pair.of(pendingReplaceInstant, requestedReplaceMetadata.get().getClusteringPlan())); } return Option.empty(); + } catch (HoodieIOException hie) { + if (hie.getCause() instanceof FileNotFoundException) { + LOG.warn(hie); + return Option.empty(); + } + throw new HoodieException("Error reading clustering plan " + pendingReplaceInstant.getTimestamp(), hie); + } catch (IOException e) { + throw new HoodieIOException("Error reading clustering plan " + pendingReplaceInstant.getTimestamp(), e); + } + } + + /** + * Get Clustering plan from timeline. + * + * @param metaClient + * @param pendingReplaceInstant + * @return + */ + public static Option> getClusteringPlan(HoodieTableMetaClient metaClient, + HoodieInstant pendingReplaceInstant, + boolean isCallProcedure) { + try { + Option requestedReplaceMetadata = getRequestedReplaceMetadata(metaClient, pendingReplaceInstant); + boolean isClustering = WriteOperationType.CLUSTER.name().equals(requestedReplaceMetadata.get().getOperationType()); + if (isCallProcedure) { + isClustering = true; + } + if (requestedReplaceMetadata.isPresent() && isClustering) { + return Option.of(Pair.of(pendingReplaceInstant, requestedReplaceMetadata.get().getClusteringPlan())); + } + return Option.empty(); } catch (IOException e) { throw new HoodieIOException("Error reading clustering plan " + pendingReplaceInstant.getTimestamp(), e); } } + /** * Get filegroups to pending clustering instant mapping for all pending clustering plans. * This includes all clustering operations in 'requested' and 'inflight' states. + * + * @return Pair of Map(HoodieFileGroupId, HoodieInstant) and Map(PartitionPath, LSM FileId which contains level 1 file) */ - public static Map getAllFileGroupsInPendingClusteringPlans( + public static Pair, Map>> getAllFileGroupsInPendingClusteringPlans( HoodieTableMetaClient metaClient) { - Stream> pendingClusteringPlans = getAllPendingClusteringPlans(metaClient); - Stream> resultStream = pendingClusteringPlans.flatMap(clusteringPlan -> + List> pendingClusteringPlans = getAllPendingClusteringPlans(metaClient).collect(Collectors.toList()); + Stream> resultStream = pendingClusteringPlans.stream().flatMap(clusteringPlan -> // get all filegroups in the plan getFileGroupEntriesInClusteringPlan(clusteringPlan.getLeft(), clusteringPlan.getRight())); @@ -138,7 +178,33 @@ public static Map getAllFileGroupsInPendingClu throw new HoodieException("Error getting all file groups in pending clustering", e); } LOG.info("Found " + resultMap.size() + " files in pending clustering operations"); - return resultMap; + + if (metaClient.getTableConfig().isLSMBasedLogFormat()) { + // fileId> + Map> partitionPathToFileId = new HashMap<>(); + pendingClusteringPlans.forEach(clusteringPlan -> getLevel1ExistsInClusteringPlan(partitionPathToFileId, clusteringPlan.getRight())); + LOG.info("Found " + partitionPathToFileId.size() + " partitions exists level 1 file in pending clustering"); + return Pair.of(resultMap, partitionPathToFileId); + } + + return Pair.of(resultMap, Collections.emptyMap()); + } + + private static void getLevel1ExistsInClusteringPlan(Map> partitionPathToFileId, HoodieClusteringPlan clusteringPlan) { + clusteringPlan.getInputGroups().stream().filter(inputGroup -> { + if (inputGroup.getExtraMetadata() != null) { + return LSM_CLUSTERING_OUT_PUT_LEVEL_1.equals(inputGroup.getExtraMetadata().get(LSM_CLUSTERING_OUT_PUT_LEVEL)); + } + return false; + }).forEach(inputGroup -> { + List slices = inputGroup.getSlices(); + if (slices != null && slices.size() > 0) { + // files in one slice have the same partition and fileId + HoodieSliceInfo slice0Info = slices.get(0); + String partitionPath = slice0Info.getPartitionPath(); + partitionPathToFileId.computeIfAbsent(partitionPath, p -> new HashSet<>()).add(FSUtils.getFileId(slice0Info.getFileId())); + } + }); } public static Stream> getFileGroupsInPendingClusteringInstant( @@ -182,6 +248,7 @@ public static HoodieClusteringPlan createClusteringPlan(String strategyClassName .setInputGroups(clusteringGroups) .setExtraMetadata(extraMetadata) .setStrategy(strategy) + .setPreserveHoodieMetadata(true) .build(); } @@ -201,7 +268,7 @@ private static Map buildMetrics(List fileSlices) { long totalIORead = 0; for (FileSlice slice : fileSlices) { - numLogFiles += slice.getLogFiles().count(); + numLogFiles += slice.getLogFiles().count(); // Total size of all the log files totalLogFileSize += slice.getLogFiles().map(HoodieLogFile::getFileSize).filter(size -> size >= 0) .reduce(Long::sum).orElse(0L); @@ -269,7 +336,7 @@ public static Option getOldestInstantToRetainForClustering( oldestInstantToRetain = replaceTimeline.firstInstant(); } - Option pendingInstantOpt = replaceTimeline.filterInflights().firstInstant(); + Option pendingInstantOpt = replaceTimeline.filterInflightsAndRequested().firstInstant(); if (pendingInstantOpt.isPresent()) { // Get the previous commit before the first inflight clustering instant. Option beforePendingInstant = activeTimeline.getCommitsTimeline() diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/Functions.java b/hudi-common/src/main/java/org/apache/hudi/common/util/Functions.java index 728ac717e4cd5..0ff5a24252a82 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/Functions.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/Functions.java @@ -37,6 +37,10 @@ interface Function0 extends Serializable { R apply(); } + interface VoidFunction0 extends Serializable { + void apply(); + } + /** * A function which contains only one parameter. */ @@ -57,4 +61,8 @@ interface Function2 extends Serializable { interface Function3 extends Serializable { R apply(T1 val1, T2 val2, T3 val3); } + + interface VoidFunction3 extends Serializable { + void apply(T1 val1, T2 val2, T3 val3); + } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/LSMRemotePartitionerHelper.java b/hudi-common/src/main/java/org/apache/hudi/common/util/LSMRemotePartitionerHelper.java new file mode 100644 index 0000000000000..dd6a600222e9b --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/LSMRemotePartitionerHelper.java @@ -0,0 +1,94 @@ +/* + * 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.common.util; + +import org.apache.hudi.common.table.view.FileSystemViewStorageConfig; + +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.http.Consts; +import org.apache.http.client.fluent.Request; +import org.apache.http.client.fluent.Response; +import org.apache.http.client.utils.URIBuilder; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; + +import java.io.Serializable; +import java.util.HashMap; + +public class LSMRemotePartitionerHelper implements Serializable { + private static final Logger LOG = LogManager.getLogger(LSMRemotePartitionerHelper.class); + + public static final String URL = "/v1/hoodie/partitioner/getpartitionindex"; + public static final String NUM_BUCKETS_PARAM = "numbuckets"; + public static final String PARTITION_PATH_PARAM = "partitionpath"; + public static final String CUR_BUCKET_PARAM = "curbucket"; + public static final String PARTITION_NUM_PARAM = "partitionnum"; + public static final String FACTOR_PARAM = "factor"; + private final RetryHelper retryHelper; + private final String serverHost; + private final Integer serverPort; + private final ObjectMapper mapper; + private final int timeoutMs; + private final HashMap cache; // dataPartition -> sparkPartitionIndex + + public LSMRemotePartitionerHelper(FileSystemViewStorageConfig viewConf) { + this.retryHelper = new RetryHelper( + viewConf.getRemoteTimelineClientMaxRetryIntervalMs(), + viewConf.getRemoteTimelineClientMaxRetryNumbers(), + viewConf.getRemoteTimelineInitialRetryIntervalMs(), + viewConf.getRemoteTimelineClientRetryExceptions(), + "Sending request"); + this.serverHost = viewConf.getRemoteViewServerHost(); + this.serverPort = viewConf.getRemoteViewServerPort(); + this.timeoutMs = viewConf.getRemoteTimelineClientTimeoutSecs() * 1000; + this.mapper = new ObjectMapper(); + this.cache = new HashMap<>(); + } + + // todo zhangyue143 限速 + Factor能力 + public int getPartition(int numBuckets, String partitionPath, int curBucket, int partitionNum, int factor) throws Exception { + if (cache.containsKey(partitionPath)) { + return computeActualPartition(cache.get(partitionPath), curBucket, partitionNum); + } + URIBuilder builder = + new URIBuilder().setHost(serverHost).setPort(serverPort).setPath(URL).setScheme("http"); + + builder.addParameter(NUM_BUCKETS_PARAM, String.valueOf(numBuckets)); + builder.addParameter(PARTITION_PATH_PARAM, partitionPath); + builder.addParameter(CUR_BUCKET_PARAM, String.valueOf(curBucket)); + builder.addParameter(PARTITION_NUM_PARAM, String.valueOf(partitionNum)); + builder.addParameter(FACTOR_PARAM, String.valueOf(factor)); + + String url = builder.toString(); + LOG.info("Sending request : (" + url + ")"); + Response response = (Response)(retryHelper != null ? retryHelper.start(() -> Request.Get(url).connectTimeout(timeoutMs).socketTimeout(timeoutMs).execute()) + : Request.Get(url).connectTimeout(timeoutMs).socketTimeout(timeoutMs).execute()); + String content = response.returnContent().asString(Consts.UTF_8); + int partitionIndex = Integer.parseInt(mapper.readValue(content, new TypeReference() {})); + cache.put(partitionPath, partitionIndex); + return computeActualPartition(partitionIndex, curBucket, partitionNum); + } + + private int computeActualPartition(int startOffset, int curBucket, int partitionNum) { + int res = startOffset + curBucket; + return res >= partitionNum ? res % partitionNum : res; + } + +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/collection/FilterIterator.java b/hudi-common/src/main/java/org/apache/hudi/common/util/collection/FilterIterator.java new file mode 100644 index 0000000000000..2fae2598cd221 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/collection/FilterIterator.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.common.util.collection; + +import java.util.Iterator; +import java.util.NoSuchElementException; +import java.util.function.Predicate; + +/** + * An Iterator that filters another Iterator. + * + * @param the type of objects produced by this Iterator + */ +public class FilterIterator implements Iterator { + public static FilterIterator filter(Iterator items, Predicate shouldKeep) { + return new FilterIterator(items, shouldKeep); + } + + private final Iterator items; + private final Predicate shouldKeep; + private boolean nextReady; + private T next; + + private FilterIterator(Iterator items, Predicate shouldKeep) { + this.items = items; + this.shouldKeep = shouldKeep; + this.next = null; + this.nextReady = false; + } + + @Override + public boolean hasNext() { + return nextReady || advance(); + } + + @Override + public T next() { + if (!hasNext()) { + throw new NoSuchElementException(); + } + this.nextReady = false; + return next; + } + + private boolean advance() { + while (items.hasNext()) { + this.next = items.next(); + if (shouldKeep.test(next)) { + this.nextReady = true; + return true; + } + } + + this.nextReady = false; + return false; + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/collection/MappingIterator.java b/hudi-common/src/main/java/org/apache/hudi/common/util/collection/MappingIterator.java index 24b0961470bde..e87fa58af0860 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/collection/MappingIterator.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/collection/MappingIterator.java @@ -23,9 +23,14 @@ /** * Iterator mapping elements of the provided source {@link Iterator} from {@code I} to {@code O} + * TODO zhangyue143 这里能够基于Disruptor 异步化来提升IO性能 */ public class MappingIterator implements Iterator { + public static MappingIterator transform(Iterator source, Function mapper) { + return new MappingIterator(source, mapper); + } + protected final Iterator source; private final Function mapper; diff --git a/hudi-common/src/main/java/org/apache/hudi/disruptor/DisruptorData.java b/hudi-common/src/main/java/org/apache/hudi/disruptor/DisruptorData.java new file mode 100644 index 0000000000000..52956b2a2d29f --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/disruptor/DisruptorData.java @@ -0,0 +1,24 @@ +/* + * 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.disruptor; + +public interface DisruptorData { + + T getData(); +} diff --git a/hudi-common/src/main/java/org/apache/hudi/disruptor/DisruptorEvent.java b/hudi-common/src/main/java/org/apache/hudi/disruptor/DisruptorEvent.java new file mode 100644 index 0000000000000..7ca144d74b401 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/disruptor/DisruptorEvent.java @@ -0,0 +1,34 @@ +/* + * 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.disruptor; + +public class DisruptorEvent { + private T data; + public T getData() { + return data; + } + + public void setData(T data) { + this.data = data; + } + + public void clear() { + this.data = null; + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/disruptor/DisruptorEventFactory.java b/hudi-common/src/main/java/org/apache/hudi/disruptor/DisruptorEventFactory.java new file mode 100644 index 0000000000000..7f007439b15b6 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/disruptor/DisruptorEventFactory.java @@ -0,0 +1,29 @@ +/* + * 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.disruptor; + +import com.lmax.disruptor.EventFactory; + +public class DisruptorEventFactory implements EventFactory> { + + @Override + public DisruptorEvent newInstance() { + return new DisruptorEvent<>(); + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/disruptor/DisruptorEventHandler.java b/hudi-common/src/main/java/org/apache/hudi/disruptor/DisruptorEventHandler.java new file mode 100644 index 0000000000000..985483efe3811 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/disruptor/DisruptorEventHandler.java @@ -0,0 +1,26 @@ +/* + * 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.disruptor; + +import com.lmax.disruptor.EventHandler; + +public interface DisruptorEventHandler extends EventHandler> { + + boolean isStoped(); +} diff --git a/hudi-common/src/main/java/org/apache/hudi/disruptor/DisruptorQueue.java b/hudi-common/src/main/java/org/apache/hudi/disruptor/DisruptorQueue.java new file mode 100644 index 0000000000000..e79ed36775cc5 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/disruptor/DisruptorQueue.java @@ -0,0 +1,88 @@ +/* + * 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.disruptor; + +import org.apache.hudi.common.util.CustomizedThreadFactory; +import org.apache.hudi.common.util.queue.WaitStrategyFactory; +import org.apache.hudi.exception.HoodieException; + +import com.lmax.disruptor.RingBuffer; +import com.lmax.disruptor.TimeoutException; +import com.lmax.disruptor.WaitStrategy; +import com.lmax.disruptor.dsl.Disruptor; +import com.lmax.disruptor.dsl.ProducerType; + +import java.io.Closeable; +import java.util.concurrent.TimeUnit; + +public class DisruptorQueue implements Closeable { + + private final Disruptor> disruptor; + private final RingBuffer> ringBuffer; + private DisruptorEventHandler handler; + + public DisruptorQueue(int bufferSize, String waitStrategyId) { + WaitStrategy waitStrategy = WaitStrategyFactory.build(waitStrategyId); + disruptor = new Disruptor<>(new DisruptorEventFactory<>(), bufferSize, new CustomizedThreadFactory("disruptor"), ProducerType.SINGLE, waitStrategy); + ringBuffer = disruptor.getRingBuffer(); + } + + public void start() { + disruptor.start(); + } + + public void produce(T data) { + long sequence = ringBuffer.next(); + try { + DisruptorEvent event = ringBuffer.get(sequence); + event.setData(data); + } finally { + ringBuffer.publish(sequence); + } + } + + public void addConsumer(DisruptorEventHandler handler) { + this.handler = handler; + disruptor.handleEventsWith(handler); + } + + public boolean waitFor() { + while (!handler.isStoped()) { + try { + Thread.sleep(1000); + } catch (InterruptedException e) { + throw new HoodieException(e.getMessage(), e); + } + } + return true; + } + + @Override + public void close() { + disruptor.shutdown(); + } + + public void closeNow() { + try { + disruptor.shutdown(1, TimeUnit.SECONDS); + } catch (TimeoutException e) { + throw new HoodieException(e.getMessage(), e); + } + } +} diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/configuration/OptionsInference.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/configuration/OptionsInference.java index 3e02d23732703..7cf0cc32a575b 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/configuration/OptionsInference.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/configuration/OptionsInference.java @@ -61,5 +61,15 @@ public static void setupSinkTasks(Configuration conf, int envTasks) { if (!conf.contains(FlinkOptions.CLUSTERING_TASKS)) { conf.setInteger(FlinkOptions.CLUSTERING_TASKS, writeTasks); } + // lsm clustering tasks, default same as write tasks + if (!conf.contains(FlinkOptions.LSM_CLUSTERING_TASKS)) { + conf.setInteger(FlinkOptions.LSM_CLUSTERING_TASKS, writeTasks); + } + if (!conf.contains(FlinkOptions.CLUSTERING_READ_FOOTER_TASKS)) { + conf.setInteger(FlinkOptions.CLUSTERING_READ_FOOTER_TASKS, writeTasks); + } + if (!conf.contains(FlinkOptions.LSM_CLUSTERING_LIST_TASKS)) { + conf.setInteger(FlinkOptions.LSM_CLUSTERING_LIST_TASKS, writeTasks); + } } } diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/configuration/OptionsResolver.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/configuration/OptionsResolver.java index 8969d055dee5f..30051c40f328a 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/configuration/OptionsResolver.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/configuration/OptionsResolver.java @@ -127,6 +127,17 @@ public static boolean isBucketIndexType(Configuration conf) { return conf.getString(FlinkOptions.INDEX_TYPE).equalsIgnoreCase(HoodieIndex.IndexType.BUCKET.name()); } + /** + * Returns whether to read the instants using completion time. + * + *

    A Hudi instant contains both the txn start time and completion time, for incremental subscription + * of the source reader, using completion time to filter the candidate instants can avoid data loss + * in scenarios like multiple writers. + */ + public static boolean isReadByTxnCompletionTime(Configuration conf) { + return conf.getBoolean(HoodieCommonConfig.READ_BY_STATE_TRANSITION_TIME.key(), HoodieCommonConfig.READ_BY_STATE_TRANSITION_TIME.defaultValue()); + } + /** * Returns whether the source should emit changelog. * diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/bucket/BucketBulkInsertWriterHelper.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/bucket/BucketBulkInsertWriterHelper.java index 7d1400cb5c1d2..a72a314a49e46 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/bucket/BucketBulkInsertWriterHelper.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/bucket/BucketBulkInsertWriterHelper.java @@ -20,6 +20,7 @@ import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.index.bucket.BucketIdentifier; +import org.apache.hudi.index.bucket.BucketStrategist; import org.apache.hudi.io.storage.row.HoodieRowDataCreateHandle; import org.apache.hudi.sink.bulk.BulkInsertWriterHelper; import org.apache.hudi.sink.bulk.RowDataKeyGen; @@ -92,16 +93,16 @@ public static SortOperatorGen getFileIdSorterGen(RowType rowType) { return new SortOperatorGen(rowType, new String[] {FILE_GROUP_META_FIELD}); } - private static String getFileId(Map bucketIdToFileId, RowDataKeyGen keyGen, RowData record, String indexKeys, int numBuckets) { + private static String getFileId(Map bucketIdToFileId, RowDataKeyGen keyGen, RowData record, String indexKeys, BucketStrategist bucketStrategist) { String recordKey = keyGen.getRecordKey(record); String partition = keyGen.getPartitionPath(record); - final int bucketNum = BucketIdentifier.getBucketId(recordKey, indexKeys, numBuckets); + final int bucketNum = BucketIdentifier.getBucketId(recordKey, indexKeys, bucketStrategist.computeBucketNumber(partition)); String bucketId = partition + bucketNum; return bucketIdToFileId.computeIfAbsent(bucketId, k -> BucketIdentifier.newBucketFileIdPrefix(bucketNum)); } - public static RowData rowWithFileId(Map bucketIdToFileId, RowDataKeyGen keyGen, RowData record, String indexKeys, int numBuckets) { - final String fileId = getFileId(bucketIdToFileId, keyGen, record, indexKeys, numBuckets); + public static RowData rowWithFileId(Map bucketIdToFileId, RowDataKeyGen keyGen, RowData record, String indexKeys, BucketStrategist bucketStrategist) { + final String fileId = getFileId(bucketIdToFileId, keyGen, record, indexKeys, bucketStrategist); return GenericRowData.of(StringData.fromString(fileId), record); } diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/bucket/BucketStreamWriteFunction.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/bucket/BucketStreamWriteFunction.java index afd55639c3c15..8706c95988829 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/bucket/BucketStreamWriteFunction.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/bucket/BucketStreamWriteFunction.java @@ -21,10 +21,18 @@ import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordLocation; +import org.apache.hudi.common.model.HoodieReplaceCommitMetadata; +import org.apache.hudi.common.model.WriteOperationType; +import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.util.LSMRemotePartitionerHelper; import org.apache.hudi.configuration.FlinkOptions; import org.apache.hudi.configuration.OptionsResolver; import org.apache.hudi.index.bucket.BucketIdentifier; +import org.apache.hudi.index.bucket.BucketStrategist; +import org.apache.hudi.index.bucket.BucketStrategistFactory; import org.apache.hudi.sink.StreamWriteFunction; +import org.apache.hudi.util.FlinkWriteClients; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.state.FunctionInitializationContext; @@ -39,6 +47,8 @@ import java.util.Map; import java.util.Set; +import static org.apache.hudi.config.HoodieWriteConfig.LSM_SHUFFLE_FACTOR_VALUE; + /** * A stream write function with bucket hash index. * @@ -51,25 +61,35 @@ public class BucketStreamWriteFunction extends StreamWriteFunction { private static final Logger LOG = LoggerFactory.getLogger(BucketStreamWriteFunction.class); + protected int parallelism; - private int parallelism; - - private int bucketNum; - - private String indexKeyFields; + protected String indexKeyFields; /** * BucketID to file group mapping in each partition. * Map(partition -> Map(bucketId, fileID)). */ - private Map> bucketIndex; + protected Map> bucketIndex; /** * Incremental bucket index of the current checkpoint interval, * it is needed because the bucket type('I' or 'U') should be decided based on the committed files view, * all the records in one bucket should have the same bucket type. */ - private Set incBucketIndex; + protected Set incBucketIndex; + protected BucketStrategist bucketStrategist; + + /** + * Used to record the last replacement commit time + */ + private transient String lastRefreshInstant; + + /** + * Adapt remote partitioner when call index bootstrap. + */ + private transient Boolean remotePartitionerEnable; + private transient LSMRemotePartitionerHelper remotePartitionerHelper; + private Integer factor; /** * Constructs a BucketStreamWriteFunction. @@ -83,25 +103,56 @@ public BucketStreamWriteFunction(Configuration config) { @Override public void open(Configuration parameters) throws IOException { super.open(parameters); - this.bucketNum = config.getInteger(FlinkOptions.BUCKET_INDEX_NUM_BUCKETS); this.indexKeyFields = config.getString(FlinkOptions.INDEX_KEY_FIELD); this.taskID = getRuntimeContext().getIndexOfThisSubtask(); this.parallelism = getRuntimeContext().getNumberOfParallelSubtasks(); this.bucketIndex = new HashMap<>(); this.incBucketIndex = new HashSet<>(); + this.lastRefreshInstant = HoodieTimeline.INIT_INSTANT_TS; + if (OptionsResolver.enableRemotePartitioner(config)) { + LOG.info("BucketStreamWriteFunction enable remote partitioner."); + this.remotePartitionerEnable = true; + this.remotePartitionerHelper = new LSMRemotePartitionerHelper(writeClient.getConfig().getViewStorageConfig()); + this.factor = config.getInteger(LSM_SHUFFLE_FACTOR_VALUE.key(), Integer.valueOf(LSM_SHUFFLE_FACTOR_VALUE.defaultValue())); + } else { + this.remotePartitionerEnable = false; + } } @Override public void initializeState(FunctionInitializationContext context) throws Exception { super.initializeState(context); + this.bucketStrategist = BucketStrategistFactory.getInstant(FlinkWriteClients.getHoodieClientConfig(config), metaClient.getFs()); } @Override - public void snapshotState() { + public void snapshotState() throws IOException { super.snapshotState(); + // only partition level bucket index need remove cache + if (bucketStrategist.isPartitionLevel()) { + removePartitionCache(); + } this.incBucketIndex.clear(); } + private void removePartitionCache() throws IOException { + HoodieTimeline timeline = writeClient.getHoodieTable().getActiveTimeline() + .getCompletedReplaceTimeline().findInstantsAfter(lastRefreshInstant); + if (!timeline.empty()) { + for (HoodieInstant instant : timeline.getInstants()) { + HoodieReplaceCommitMetadata commitMetadata = HoodieReplaceCommitMetadata.fromBytes( + timeline.getInstantDetails(instant).get(), HoodieReplaceCommitMetadata.class); + // only take care of insert operation here. + if (commitMetadata.getOperationType().equals(WriteOperationType.INSERT_OVERWRITE)) { + Set affectedPartitions = commitMetadata.getPartitionToReplaceFileIds().keySet(); + LOG.info("Clear up cached hashing metadata because find a new insert overwrite commit.\n Instant: {}.\n Effected Partitions: {}.", lastRefreshInstant, affectedPartitions); + affectedPartitions.forEach(bucketStrategist::updateBucketInfo); + } + } + this.lastRefreshInstant = timeline.lastInstant().get().getTimestamp(); + } + } + @Override public void processElement(I i, ProcessFunction.Context context, Collector collector) throws Exception { HoodieRecord record = (HoodieRecord) i; @@ -111,9 +162,11 @@ public void processElement(I i, ProcessFunction.Context context, Coll bootstrapIndexIfNeed(partition); Map bucketToFileId = bucketIndex.computeIfAbsent(partition, p -> new HashMap<>()); - final int bucketNum = BucketIdentifier.getBucketId(hoodieKey, indexKeyFields, this.bucketNum); + final int bucketNum = BucketIdentifier.getBucketId(hoodieKey, indexKeyFields, bucketStrategist.getBucketNumber(partition)); final String bucketId = partition + "/" + bucketNum; + // 新增的桶的数据都是 I + // 老桶的数据都是 U if (incBucketIndex.contains(bucketId)) { location = new HoodieRecordLocation("I", bucketToFileId.get(bucketNum)); } else if (bucketToFileId.containsKey(bucketNum)) { @@ -135,9 +188,24 @@ public void processElement(I i, ProcessFunction.Context context, Coll * (partition + curBucket) % numPartitions == this taskID belongs to this task. */ public boolean isBucketToLoad(int bucketNumber, String partition) { - final int partitionIndex = (partition.hashCode() & Integer.MAX_VALUE) % parallelism; - int globalIndex = partitionIndex + bucketNumber; - return BucketIdentifier.mod(globalIndex, parallelism) == taskID; + if (remotePartitionerEnable) { + try { + int subtaskId = remotePartitionerHelper.getPartition( + bucketStrategist.computeBucketNumber(partition), + partition, + bucketNumber, + parallelism, + factor); + return subtaskId == taskID; + } catch (Exception e) { + throw new RuntimeException("Get Remote Partition Failed.", e); + } + + } else { + final int partitionIndex = (partition.hashCode() & Integer.MAX_VALUE) % parallelism; + int globalIndex = partitionIndex + bucketNumber; + return BucketIdentifier.mod(globalIndex, parallelism) == taskID; + } } /** diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/bucket/BucketStreamWriteOperator.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/bucket/BucketStreamWriteOperator.java index a48ea44ddc44a..cd935d28e594b 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/bucket/BucketStreamWriteOperator.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/bucket/BucketStreamWriteOperator.java @@ -22,6 +22,7 @@ import org.apache.hudi.sink.common.WriteOperatorFactory; import org.apache.flink.configuration.Configuration; +import org.apache.flink.table.types.logical.RowType; /** * Operator for {@link BucketStreamWriteFunction}. @@ -34,7 +35,15 @@ public BucketStreamWriteOperator(Configuration conf) { super(new BucketStreamWriteFunction<>(conf)); } + public BucketStreamWriteOperator(Configuration conf, RowType rowType) { + super(new LSMBucketStreamWriteFunction<>(conf, rowType)); + } + public static WriteOperatorFactory getFactory(Configuration conf) { return WriteOperatorFactory.instance(conf, new BucketStreamWriteOperator<>(conf)); } + + public static WriteOperatorFactory getFactory(Configuration conf, RowType rowType) { + return WriteOperatorFactory.instance(conf, new BucketStreamWriteOperator<>(conf, rowType)); + } } diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/bucket/BufferSizeDetector.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/bucket/BufferSizeDetector.java new file mode 100644 index 0000000000000..5b865687dafd1 --- /dev/null +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/bucket/BufferSizeDetector.java @@ -0,0 +1,62 @@ +/* + * 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.bucket; + +import org.apache.hudi.common.util.ObjectSizeCalculator; +import org.apache.flink.table.data.binary.BinaryRowData; +import java.util.Random; + +public class BufferSizeDetector { + private final Random random = new Random(47); + private static final int DENOMINATOR = 100; + private final double batchSizeBytes; + public long lastRecordSize = -1L; + public long totalSize = 0L; + public BufferSizeDetector(double batchSizeMb) { + this.batchSizeBytes = batchSizeMb * 1024 * 1024; + } + + public boolean detect(Object record) { + if (record instanceof BinaryRowData) { + lastRecordSize = ((BinaryRowData) record).getSizeInBytes(); + } else if (lastRecordSize == -1 || sampling()) { + lastRecordSize = ObjectSizeCalculator.getObjectSize(record); + } + totalSize += lastRecordSize; + return totalSize > this.batchSizeBytes; + } + + public boolean isFull() { + return totalSize > batchSizeBytes; + } + + public long getLastRecordSize() { + return lastRecordSize; + } + + public boolean sampling() { + // 0.01 sampling percentage + return random.nextInt(DENOMINATOR) == 1; + } + + public void reset() { + this.lastRecordSize = -1L; + this.totalSize = 0L; + } +} \ No newline at end of file diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/bucket/DataBucket.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/bucket/DataBucket.java new file mode 100644 index 0000000000000..736bda13ce89a --- /dev/null +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/bucket/DataBucket.java @@ -0,0 +1,98 @@ +/* + * 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.bucket; + +import org.apache.hudi.sink.utils.BufferUtils; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.binary.BinaryRowData; +import org.apache.flink.table.runtime.operators.sort.BinaryInMemorySortBuffer; +import org.apache.flink.util.MutableObjectIterator; +import java.io.IOException; + +/** + * Data bucket. + */ +public class DataBucket { + private final BufferSizeDetector detector; + private final String partitionPath; + private final String bucketID; + private final BinaryInMemorySortBuffer dataBuffer; + private final String fileID; + public DataBucket(Double batchSize, String partitionPath, String bucketID, BinaryInMemorySortBuffer dataBuffer, String fileID) { + this.detector = new BufferSizeDetector(batchSize); + this.partitionPath = partitionPath; + this.bucketID = bucketID; + this.dataBuffer = dataBuffer; + this.fileID = fileID; + } + + public boolean writeRow(RowData rowData) throws IOException { + boolean success = dataBuffer.write(rowData); + if (success) { + detector.detect(rowData); + } + return success; + } + + public long getBufferSize() { + return detector.totalSize; + } + + public String getPartitionPath() { + return partitionPath; + } + + public void reset() { + this.detector.reset(); + } + + public MutableObjectIterator getDataIterator() { + return dataBuffer.getIterator(); + } + + public MutableObjectIterator getSortedDataIterator() { + BufferUtils.sort(dataBuffer); + return dataBuffer.getIterator(); + } + + public String getBucketId() { + return bucketID; + } + + public boolean isEmpty() { + return dataBuffer.isEmpty(); + } + + public boolean isFull() { + return detector.isFull(); + } + + public long getLastRecordSize() { + return detector.getLastRecordSize(); + } + + public void dispose() { + dataBuffer.dispose(); + detector.reset(); + } + + public String getFileID() { + return fileID; + } +} \ No newline at end of file diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/bucket/LSMBucketStreamWriteFunction.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/bucket/LSMBucketStreamWriteFunction.java new file mode 100644 index 0000000000000..13c1ebd8d5aec --- /dev/null +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/bucket/LSMBucketStreamWriteFunction.java @@ -0,0 +1,401 @@ +/* + * 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.bucket; + +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.client.model.HoodieRowDataCreation; +import org.apache.hudi.client.model.LSMHoodieInternalRowData; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.util.StringUtils; +import org.apache.hudi.common.util.ValidationUtils; +import org.apache.hudi.configuration.FlinkOptions; +import org.apache.hudi.disruptor.DisruptorData; +import org.apache.hudi.disruptor.DisruptorEventHandler; +import org.apache.hudi.disruptor.DisruptorQueue; +import org.apache.hudi.exception.HoodieException; +import org.apache.hudi.exception.HoodieIOException; +import org.apache.hudi.index.bucket.BucketIdentifier; +import org.apache.hudi.io.storage.row.LSMHoodieRowDataCreateHandle; +import org.apache.hudi.sink.bucket.disruptor.FlinkDisruptorWriteAntidote; +import org.apache.hudi.sink.bucket.disruptor.FlinkDisruptorWriteData; +import org.apache.hudi.sink.bucket.disruptor.FlinkDisruptorWritePoison; +import org.apache.hudi.sink.bucket.disruptor.FlinkRowDataEventHandler; +import org.apache.hudi.sink.buffer.MemorySegmentPoolFactory; +import org.apache.hudi.sink.buffer.TotalSizeTracer; +import org.apache.hudi.sink.bulk.RowDataKeyGen; +import org.apache.hudi.sink.event.WriteMetadataEvent; +import org.apache.hudi.sink.event.WriteResultEvent; +import org.apache.hudi.sink.utils.BufferUtils; +import org.apache.hudi.util.MutableIteratorWrapperIterator; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.state.FunctionInitializationContext; +import org.apache.flink.streaming.api.functions.ProcessFunction; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.binary.BinaryRowData; +import org.apache.flink.table.runtime.util.MemorySegmentPool; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.util.Collector; +import org.apache.hudi.util.RowDataProjection; +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.Comparator; +import java.util.HashMap; +import java.util.Iterator; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.NoSuchElementException; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicLong; +import java.util.stream.Collectors; + +import static org.apache.hudi.common.util.FutureUtils.allOf; + +/** + * Used for LSM based bucket stream write function + * @param + */ +public class LSMBucketStreamWriteFunction extends BucketStreamWriteFunction { + + private static final Logger LOG = LoggerFactory.getLogger(LSMBucketStreamWriteFunction.class); + private final RowType rowType; + private final RowType rowTypeWithRecordKeyAndSeqId; + private final RowDataKeyGen keyGen; + private final Integer flushConcurrency; + private final Integer bufferSize; + private final String waitStrategy; + private RowDataProjection indexKeyFieldsProjection; + private final HashMap buckets; + protected transient MemorySegmentPool memorySegmentPool; + private TotalSizeTracer tracer; + private ArrayList hashingFields; + private int version = 0; + private DisruptorQueue> queue; + private String[] sortColumns; + private int taskPartitionId; + private static final AtomicLong SEQGEN = new AtomicLong(1); + + /** + * Constructs a BucketStreamWriteFunction. + * + * @param conf The config options + */ + public LSMBucketStreamWriteFunction(Configuration conf, RowType rowType) { + super(conf); + this.rowType = rowType; + this.rowTypeWithRecordKeyAndSeqId = addRecordKeyAndSeqIdMetaFields(rowType); + this.keyGen = RowDataKeyGen.instance(conf, rowType); + this.buckets = new LinkedHashMap<>(); + this.flushConcurrency = conf.get(FlinkOptions.WRITE_BULK_INSERT_FLUSH_CONCURRENCY); + this.bufferSize = conf.get(FlinkOptions.WRITE_LSM_ASYNC_BUFFER_SIZE); + this.waitStrategy = conf.get(FlinkOptions.WRITE_LSM_ASYNC_WAITING_STRATEGY); + } + + @Override + public void open(Configuration parameters) throws IOException { + super.open(parameters); + this.taskPartitionId = getRuntimeContext().getNumberOfParallelSubtasks(); + this.memorySegmentPool = MemorySegmentPoolFactory.createMemorySegmentPool(config); + this.tracer = new TotalSizeTracer(this.config); + this.sortColumns = new String[]{HoodieRecord.RECORD_KEY_METADATA_FIELD}; + List fieldNames = rowType.getFieldNames(); + List fieldTypes = rowType.getChildren(); + this.indexKeyFieldsProjection = getProjection(this.indexKeyFields.split(","), fieldNames, fieldTypes); + this.hashingFields = new ArrayList<>(this.indexKeyFields.split(",").length); + initialDisruptorQueue(); + } + + private void initialDisruptorQueue() { + this.queue = new DisruptorQueue<>(bufferSize, waitStrategy); + DisruptorEventHandler> handler = new FlinkRowDataEventHandler((value, partitionPath, fileID) -> { + try { + bufferRecord(value, partitionPath, fileID); + } catch (IOException e) { + throw new HoodieIOException(e.getMessage(), e); + } + }); + queue.addConsumer(handler); + queue.start(); + } + + @Override + public void initializeState(FunctionInitializationContext context) throws Exception { + super.initializeState(context); + } + + @Override + public void processElement(I i, ProcessFunction.Context context, Collector collector) throws Exception { + RowData record = (RowData) i; + final String partition = keyGen.getPartitionPath(record); + + Object[] keyValues = this.indexKeyFieldsProjection.projectAsValues(record); + for (Object value: keyValues) { + hashingFields.add(StringUtils.objToString(value)); + } + // here use bucketID as FileID directly + final int id = BucketIdentifier.getBucketId(hashingFields, bucketStrategist.getBucketNumber(partition)); + String fileID = String.valueOf(id); + queue.produce(new FlinkDisruptorWriteData(record, partition, fileID)); + hashingFields.clear(); + } + + @Override + public void snapshotState() throws IOException { + poison(); + super.snapshotState(); + queue.produce(new FlinkDisruptorWriteAntidote()); + } + + @Override + public void endInput() { + poison(); + super.endInput(); + queue.close(); + } + + private void poison() { + // produce poison + queue.produce(new FlinkDisruptorWritePoison()); + queue.waitFor(); + } + + /** + * Copy from RowDataKeyGen + * Returns the row data projection for the given field names and table schema. + */ + private static RowDataProjection getProjection(String[] fields, List schemaFields, List schemaTypes) { + int[] positions = getFieldPositions(fields, schemaFields); + LogicalType[] types = Arrays.stream(positions).mapToObj(schemaTypes::get).toArray(LogicalType[]::new); + return RowDataProjection.instance(types, positions); + } + + /** + * Copy from RowDataKeyGen + * Returns the field positions of the given fields {@code fields} among all the fields {@code allFields}. + */ + private static int[] getFieldPositions(String[] fields, List allFields) { + return Arrays.stream(fields).mapToInt(allFields::indexOf).toArray(); + } + + protected void bufferRecord(RowData value, String partitionPath, String fileID) throws IOException { + String bucketID = partitionPath + "_" + fileID; + String recordKey = keyGen.getRecordKey(value); + RowData internalHoodieRowData = HoodieRowDataCreation.createLSMHoodieInternalRowData(recordKey, String.valueOf(SEQGEN.getAndIncrement()), value); + boolean success = doBufferRecord(bucketID, internalHoodieRowData, partitionPath, fileID); + + // 1. flushing bucket for memory pool is full. + if (!success) { + // flush the biggest data bucket + DataBucket bucketToFlush = this.buckets.values().stream() + .max(Comparator.comparingLong(DataBucket::getBufferSize)) + .orElseThrow(NoSuchElementException::new); + if (flushBucket(bucketToFlush)) { + this.tracer.countDown(bucketToFlush.getBufferSize()); + disposeBucket(bucketToFlush); + } else { + LOG.warn("The buffer size hits the threshold {}, but still flush the max size data bucket failed!", this.tracer.maxBufferSize); + } + + // flush current data bucket to avoid conflict + if (!bucketToFlush.getBucketId().equals(bucketID)) { + DataBucket curBucket = this.buckets.get(bucketID); + if (flushBucket(curBucket)) { + this.tracer.countDown(curBucket.getBufferSize()); + disposeBucket(curBucket); + } else { + LOG.warn("The buffer size hits the threshold {}, flush the max size data bucket success but flush cur bucket failed!", this.tracer.maxBufferSize); + } + } + + // try write row again + success = doBufferRecord(bucketID, internalHoodieRowData, partitionPath, fileID); + if (!success) { + throw new RuntimeException("Buffer is too small to hold a single record."); + } + } + + DataBucket bucket = this.buckets.get(bucketID); + this.tracer.trace(bucket.getLastRecordSize()); + // 2. flushing bucket for bucket is full. + if (bucket.isFull()) { + if (flushBucket(bucket)) { + this.tracer.countDown(bucket.getBufferSize()); + disposeBucket(bucket); + } + } + } + + @Override + protected void flushRemaining(boolean endInput) { + this.currentInstant = instantToWrite(hasData()); + if (this.currentInstant == null) { + // in case there are empty checkpoints that has no input data + throw new HoodieException("No inflight instant when flushing data!"); + } + final List writeStatus; + if (!buckets.isEmpty()) { + writeStatus = new ArrayList<>(); + ThreadPoolExecutor executor = new ThreadPoolExecutor( + flushConcurrency, flushConcurrency, 0L, + TimeUnit.MILLISECONDS, new LinkedBlockingQueue<>()); + allOf(buckets.values().stream().filter(bucket -> !bucket.isEmpty()) + .map(bucket -> CompletableFuture.supplyAsync(() -> { + WriteStatus innerStatus = writeRecords(currentInstant, bucket); + bucket.dispose(); + return innerStatus; + }, executor)).collect(Collectors.toList())) + .whenComplete((result, throwable) -> { + writeStatus.addAll(result); + }).join(); + + try { + executor.shutdown(); + executor.awaitTermination(24, TimeUnit.DAYS); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + } else { + LOG.info("No data to write in subtask [{}] for instant [{}]", taskID, currentInstant); + writeStatus = Collections.emptyList(); + } + final WriteMetadataEvent event = WriteMetadataEvent.builder() + .taskID(taskID) + .instantTime(currentInstant) + .writeStatus(writeStatus) + .lastBatch(true) + .endInput(endInput) + .build(); + + this.eventGateway.sendEventToCoordinator(new WriteResultEvent(event, currentInstant)); + this.buckets.clear(); + this.tracer.reset(); + this.writeClient.cleanHandles(); + this.writeStatuses.addAll(writeStatus); + this.version = 0; + // blocks flushing until the coordinator starts a new instant + this.confirming = true; + } + + @Override + public void close() { + super.close(); + if (queue != null) { + queue.closeNow(); + } + } + + private boolean doBufferRecord(String bucketID, RowData value, String partitionPath, String fileID) throws IOException { + DataBucket bucket = this.buckets.computeIfAbsent(bucketID, + k -> new DataBucket( + this.config.getDouble(FlinkOptions.WRITE_BATCH_SIZE), + partitionPath, + bucketID, + BufferUtils.createBuffer(rowTypeWithRecordKeyAndSeqId, + memorySegmentPool, + getRuntimeContext().getUserCodeClassLoader(), + sortColumns), + fileID)); + return bucket.writeRow(value); + } + + private void disposeBucket(DataBucket dataBucket) { + dataBucket.dispose(); + this.buckets.remove(dataBucket.getBucketId()); + } + + private boolean flushBucket(DataBucket bucket) throws IOException { + String instant = instantToWrite(true); + + if (instant == null) { + // in case there are empty checkpoints that has no input data + LOG.info("No inflight instant when flushing data, skip."); + return false; + } + + ValidationUtils.checkState(!bucket.isEmpty(), "Data bucket to flush has no buffering records"); + // TODO zhangyue143 support pre_combine + // if (config.getBoolean(FlinkOptions.PRE_COMBINE)) { + // records = (List) FlinkWriteHelper.newInstance() + // .deduplicateRecords(records, null, -1, this.writeClient.getConfig().getSchema(), this.writeClient.getConfig().getProps(), recordMerger); + // } + + final WriteStatus writeStatus = writeRecords(instant, bucket); + final WriteMetadataEvent event = WriteMetadataEvent.builder() + .taskID(taskID) + .instantTime(instant) // the write instant may shift but the event still use the currentInstant. + .writeStatus(Collections.singletonList(writeStatus)) + .lastBatch(false) + .endInput(false) + .build(); + + this.eventGateway.sendEventToCoordinator(new WriteResultEvent(event, currentInstant)); + writeStatuses.add(writeStatus); + return true; + } + + private WriteStatus writeRecords(String instant, DataBucket bucket) { + Iterator rowItr = + new MutableIteratorWrapperIterator<>( + bucket.getSortedDataIterator(), () -> new BinaryRowData(rowTypeWithRecordKeyAndSeqId.getFieldCount())); + + LSMHoodieRowDataCreateHandle rowCreateHandle = new LSMHoodieRowDataCreateHandle(this.writeClient.getHoodieTable(), + this.writeClient.getConfig(), bucket.getPartitionPath(), bucket.getFileID(), instant, + getRuntimeContext().getNumberOfParallelSubtasks(), taskID, getRuntimeContext().getAttemptNumber(), + rowType, false, 0, version++); + rowItr.forEachRemaining(record -> { + try { + rowCreateHandle.write(record.getString(LSMHoodieInternalRowData.LSM_INTERNAL_RECORD_KEY_META_FIELD_ORD).toString(), bucket.getPartitionPath(), record, true); + } catch (IOException e) { + throw new HoodieIOException(e.getMessage(), e); + } + }); + + try { + return rowCreateHandle.close().toWriteStatus(); + } catch (IOException e) { + throw new HoodieIOException(e.getMessage(), e); + } + } + + public static RowType addRecordKeyAndSeqIdMetaFields(RowType rowType) { + List mergedFields = new ArrayList<>(); + + LogicalType metadataFieldType = DataTypes.STRING().getLogicalType(); + RowType.RowField recordKeyField = + new RowType.RowField(HoodieRecord.RECORD_KEY_METADATA_FIELD, metadataFieldType, "record key"); + RowType.RowField commitSeqnoField = + new RowType.RowField(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD, metadataFieldType, "commit seqno"); + + mergedFields.add(recordKeyField); + mergedFields.add(commitSeqnoField); + mergedFields.addAll(rowType.getFields()); + + return new RowType(false, mergedFields); + } +} diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/bucket/disruptor/FlinkDisruptorWriteAntidote.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/bucket/disruptor/FlinkDisruptorWriteAntidote.java new file mode 100644 index 0000000000000..cf53afe1b1ed2 --- /dev/null +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/bucket/disruptor/FlinkDisruptorWriteAntidote.java @@ -0,0 +1,26 @@ +/* + * 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.bucket.disruptor; + +public class FlinkDisruptorWriteAntidote extends FlinkDisruptorWriteData { + + public FlinkDisruptorWriteAntidote() { + super(null, "-1", "-1"); + } +} diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/bucket/disruptor/FlinkDisruptorWriteData.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/bucket/disruptor/FlinkDisruptorWriteData.java new file mode 100644 index 0000000000000..1cd793ad3960a --- /dev/null +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/bucket/disruptor/FlinkDisruptorWriteData.java @@ -0,0 +1,49 @@ +/* + * 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.bucket.disruptor; + +import org.apache.hudi.disruptor.DisruptorData; + +import org.apache.flink.table.data.RowData; + +public class FlinkDisruptorWriteData implements DisruptorData { + + private final RowData data; + private final String partitionPath; + private final String fileID; + + public FlinkDisruptorWriteData(RowData data, String partitionPath, String fileID) { + this.data = data; + this.partitionPath = partitionPath; + this.fileID = fileID; + } + + @Override + public RowData getData() { + return data; + } + + public String getPartitionPath() { + return partitionPath; + } + + public String getFileID() { + return fileID; + } +} diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/bucket/disruptor/FlinkDisruptorWritePoison.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/bucket/disruptor/FlinkDisruptorWritePoison.java new file mode 100644 index 0000000000000..970d138fda0b0 --- /dev/null +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/bucket/disruptor/FlinkDisruptorWritePoison.java @@ -0,0 +1,26 @@ +/* + * 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.bucket.disruptor; + +public class FlinkDisruptorWritePoison extends FlinkDisruptorWriteData { + + public FlinkDisruptorWritePoison() { + super(null, "-1", "-1"); + } +} diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/bucket/disruptor/FlinkRowDataEventHandler.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/bucket/disruptor/FlinkRowDataEventHandler.java new file mode 100644 index 0000000000000..8762a6b77d810 --- /dev/null +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/bucket/disruptor/FlinkRowDataEventHandler.java @@ -0,0 +1,66 @@ +/* + * 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.bucket.disruptor; + +import org.apache.hudi.common.util.Functions; +import org.apache.hudi.disruptor.DisruptorData; +import org.apache.hudi.disruptor.DisruptorEvent; +import org.apache.hudi.disruptor.DisruptorEventHandler; + +import org.apache.flink.table.data.RowData; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class FlinkRowDataEventHandler implements DisruptorEventHandler> { + + private static final Logger LOG = LoggerFactory.getLogger(FlinkRowDataEventHandler.class); + private final Functions.VoidFunction3 bufferRecordFunc; + private boolean suicide = false; + + public FlinkRowDataEventHandler(Functions.VoidFunction3 bufferRecordFunc) { + this.bufferRecordFunc = bufferRecordFunc; + } + + @Override + public void onEvent(DisruptorEvent> event, long sequence, boolean endOfBatch) throws Exception { + if (event.getData() instanceof FlinkDisruptorWritePoison) { + LOG.info("Eat Poison, suicide now!"); + this.suicide = true; + return; + } + + if (suicide && event.getData() instanceof FlinkDisruptorWriteAntidote) { + LOG.info("Find antidote!"); + this.suicide = false; + return; + } + + FlinkDisruptorWriteData data = (FlinkDisruptorWriteData) event.getData(); + String fileID = data.getFileID(); + RowData record = data.getData(); + String partitionPath = data.getPartitionPath(); + bufferRecordFunc.apply(record, partitionPath, fileID); + event.clear(); + } + + @Override + public boolean isStoped() { + return suicide; + } +} diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/buffer/HeapMemorySegmentPool.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/buffer/HeapMemorySegmentPool.java new file mode 100644 index 0000000000000..af7990b46edb9 --- /dev/null +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/buffer/HeapMemorySegmentPool.java @@ -0,0 +1,73 @@ +/* + * 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.buffer; + +import org.apache.flink.core.memory.MemorySegment; +import org.apache.flink.core.memory.MemorySegmentFactory; +import org.apache.flink.table.runtime.util.MemorySegmentPool; + +import java.util.LinkedList; +import java.util.List; + +/** + * A heap memory based ${code MemorySegmentPool}. + */ +public class HeapMemorySegmentPool implements MemorySegmentPool { + + private final LinkedList cachePages; + private final int maxPages; + private final int pageSize; + private int allocateNum; + + public HeapMemorySegmentPool(int pageSize, long maxSizeInBytes) { + this.cachePages = new LinkedList<>(); + this.maxPages = (int) (maxSizeInBytes / pageSize); + this.pageSize = pageSize; + this.allocateNum = 0; + } + + @Override + public int pageSize() { + return this.pageSize; + } + + @Override + public void returnAll(List list) { + synchronized (cachePages) { + cachePages.addAll(list); + } + } + + @Override + public int freePages() { + return maxPages - allocateNum + cachePages.size(); + } + + @Override + public MemorySegment nextSegment() { + if (!cachePages.isEmpty()) { + return cachePages.poll(); + } + if (freePages() > 0) { + allocateNum += 1; + return MemorySegmentFactory.wrap(new byte[pageSize]); + } + return null; + } +} diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/buffer/MemorySegmentPoolFactory.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/buffer/MemorySegmentPoolFactory.java new file mode 100644 index 0000000000000..bb36a7ae213a4 --- /dev/null +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/buffer/MemorySegmentPoolFactory.java @@ -0,0 +1,44 @@ +/* + * 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.buffer; + +import org.apache.hudi.common.util.ValidationUtils; +import org.apache.hudi.configuration.FlinkOptions; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.table.runtime.util.MemorySegmentPool; + +/** + * Factory to create {@code MemorySegmentPool}, currently only heap based memory pool {@code HeapMemorySegmentPool} + * is supported. + * + *

    todo support memory segment pool based on flink managed memory, currently support heap pool only, see HUDI-9189. + */ +public class MemorySegmentPoolFactory { + public static MemorySegmentPool createMemorySegmentPool(Configuration conf) { + long mergeReaderMem = 100; // constant 100MB + long mergeMapMaxMem = conf.get(FlinkOptions.WRITE_MERGE_MAX_MEMORY); + long maxBufferSize = (long) ((conf.get(FlinkOptions.WRITE_TASK_MAX_SIZE) - mergeReaderMem - mergeMapMaxMem) * 1024 * 1024); + final String errMsg = String.format("'%s' should be at least greater than '%s' plus merge reader memory(constant 100MB now)", + FlinkOptions.WRITE_TASK_MAX_SIZE.key(), FlinkOptions.WRITE_MERGE_MAX_MEMORY.key()); + ValidationUtils.checkState(maxBufferSize > 0, errMsg); + + return new HeapMemorySegmentPool(conf.get(FlinkOptions.WRITE_MEMORY_SEGMENT_PAGE_SIZE), maxBufferSize); + } +} diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/buffer/TotalSizeTracer.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/buffer/TotalSizeTracer.java new file mode 100644 index 0000000000000..bd512340c56fa --- /dev/null +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/buffer/TotalSizeTracer.java @@ -0,0 +1,62 @@ +/* + * 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.buffer; + +import org.apache.hudi.common.util.ValidationUtils; +import org.apache.hudi.configuration.FlinkOptions; + +import org.apache.flink.configuration.Configuration; + +/** + * Tool to trace the total buffer size. It computes the maximum buffer size, + * if current buffer size is greater than the maximum buffer size, the data bucket + * flush triggers. + */ +public class TotalSizeTracer { + public long bufferSize = 0L; + public final double maxBufferSize; + + public TotalSizeTracer(Configuration conf) { + long mergeReaderMem = 100; // constant 100MB + long mergeMapMaxMem = conf.getInteger(FlinkOptions.WRITE_MERGE_MAX_MEMORY); + this.maxBufferSize = (conf.getDouble(FlinkOptions.WRITE_TASK_MAX_SIZE) - mergeReaderMem - mergeMapMaxMem) * 1024 * 1024; + final String errMsg = String.format("'%s' should be at least greater than '%s' plus merge reader memory(constant 100MB now)", + FlinkOptions.WRITE_TASK_MAX_SIZE.key(), FlinkOptions.WRITE_MERGE_MAX_MEMORY.key()); + ValidationUtils.checkState(this.maxBufferSize > 0, errMsg); + } + + /** + * Trace the given record size {@code recordSize}. + * + * @param recordSize The record size + * @return true if the buffer size exceeds the maximum buffer size + */ + public boolean trace(long recordSize) { + this.bufferSize += recordSize; + return this.bufferSize > this.maxBufferSize; + } + + public void countDown(long size) { + this.bufferSize -= size; + } + + public void reset() { + this.bufferSize = 0; + } +} diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/bulk/BulkInsertWriterHelper.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/bulk/BulkInsertWriterHelper.java index 0bcaa58dac759..2192605eb4c2d 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/bulk/BulkInsertWriterHelper.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/bulk/BulkInsertWriterHelper.java @@ -24,6 +24,7 @@ import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.configuration.FlinkOptions; import org.apache.hudi.exception.HoodieException; +import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.io.storage.row.HoodieRowDataCreateHandle; import org.apache.hudi.table.HoodieTable; @@ -43,8 +44,14 @@ import java.util.List; import java.util.Map; import java.util.UUID; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; +import static org.apache.hudi.common.util.FutureUtils.allOf; + /** * Helper class for bulk insert used by Flink. */ @@ -67,7 +74,9 @@ public class BulkInsertWriterHelper { private final String fileIdPrefix; private int numFilesWritten = 0; protected final Map handles = new HashMap<>(); - @Nullable protected final RowDataKeyGen keyGen; + @Nullable + protected final RowDataKeyGen keyGen; + private final int flushConcurrency; public BulkInsertWriterHelper(Configuration conf, HoodieTable hoodieTable, HoodieWriteConfig writeConfig, String instantTime, int taskPartitionId, long taskId, long taskEpochId, RowType rowType) { @@ -88,6 +97,7 @@ public BulkInsertWriterHelper(Configuration conf, HoodieTable hoodieTable, Hoodi this.isInputSorted = conf.getBoolean(FlinkOptions.WRITE_BULK_INSERT_SORT_INPUT); this.fileIdPrefix = UUID.randomUUID().toString(); this.keyGen = preserveHoodieMetadata ? null : RowDataKeyGen.instance(conf, rowType); + this.flushConcurrency = writeConfig.getFlushConcurrency(); } /** @@ -147,9 +157,25 @@ private HoodieRowDataCreateHandle getRowCreateHandle(String partitionPath) throw } public void close() throws IOException { - for (HoodieRowDataCreateHandle rowCreateHandle : handles.values()) { - LOG.info("Closing bulk insert file " + rowCreateHandle.getFileName()); - writeStatusList.add(rowCreateHandle.close()); + ExecutorService executor = Executors.newFixedThreadPool(flushConcurrency); + allOf(handles.values().stream() + .map(rowCreateHandle -> CompletableFuture.supplyAsync(() -> { + try { + LOG.info("Closing bulk insert file " + rowCreateHandle.getFileName()); + return rowCreateHandle.close(); + } catch (IOException e) { + throw new HoodieIOException("IOE during rowCreateHandle.close()", e); + } + }, executor)) + .collect(Collectors.toList()) + ).whenComplete((result, throwable) -> { + writeStatusList.addAll(result); + }).join(); + try { + executor.shutdown(); + executor.awaitTermination(24, TimeUnit.DAYS); + } catch (InterruptedException e) { + throw new RuntimeException(e); } handles.clear(); handle = null; diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/bulk/sort/SortOperatorGen.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/bulk/sort/SortOperatorGen.java index 0c7f8866deb3d..9988f51415490 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/bulk/sort/SortOperatorGen.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/bulk/sort/SortOperatorGen.java @@ -56,4 +56,12 @@ public SortCodeGenerator createSortCodeGenerator() { } return new SortCodeGeneratorAdapter(tableConfig, rowType, builder.build()); } + + public SortCodeGenerator createSortCodeGenerator(TableConfig tableConfig) { + SortSpec.SortSpecBuilder builder = SortSpec.builder(); + for (int sortIndex : sortIndices) { + builder.addField(sortIndex, true, true); + } + return new SortCodeGeneratorAdapter(tableConfig, rowType, builder.build()); + } } diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/ClusteringCommitSink.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/ClusteringCommitSink.java index 7fde3e87aa419..c0ac2d0959be4 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/ClusteringCommitSink.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/ClusteringCommitSink.java @@ -33,12 +33,14 @@ import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.configuration.FlinkOptions; import org.apache.hudi.exception.HoodieClusteringException; +import org.apache.hudi.exception.HoodieException; +import org.apache.hudi.metrics.FlinkClusteringMetrics; +import org.apache.hudi.metrics.enums.FlinkClusteringMetricEnum; import org.apache.hudi.sink.CleanFunction; import org.apache.hudi.table.HoodieFlinkTable; import org.apache.hudi.table.action.HoodieWriteMetadata; import org.apache.hudi.util.ClusteringUtil; import org.apache.hudi.util.FlinkWriteClients; -import org.apache.hudi.util.StreamerUtil; import org.apache.flink.configuration.Configuration; import org.slf4j.Logger; @@ -79,6 +81,17 @@ public class ClusteringCommitSink extends CleanFunction { */ private transient Map> commitBuffer; + /** + * Cache to store clustering plan for each instant. + * Stores the mapping of instant_time -> clusteringPlan. + */ + private transient Map clusteringPlanCache; + + /** + * clustering metric + */ + private transient FlinkClusteringMetrics clusteringMetrics; + public ClusteringCommitSink(Configuration conf) { super(conf); this.conf = conf; @@ -90,7 +103,9 @@ public void open(Configuration parameters) throws Exception { if (writeClient == null) { this.writeClient = FlinkWriteClients.createWriteClient(conf, getRuntimeContext()); } + registerMetrics(); this.commitBuffer = new HashMap<>(); + this.clusteringPlanCache = new HashMap<>(); this.table = writeClient.getHoodieTable(); } @@ -110,10 +125,15 @@ public void invoke(ClusteringCommitEvent event, Context context) throws Exceptio * @param events Commit events ever received for the instant */ private void commitIfNecessary(String instant, List events) { - HoodieInstant clusteringInstant = HoodieTimeline.getReplaceCommitInflightInstant(instant); - Option> clusteringPlanOption = ClusteringUtils.getClusteringPlan( - StreamerUtil.createMetaClient(this.conf), clusteringInstant); - HoodieClusteringPlan clusteringPlan = clusteringPlanOption.get().getRight(); + HoodieClusteringPlan clusteringPlan = clusteringPlanCache.computeIfAbsent(instant, k -> { + try { + Option> clusteringPlanOption = ClusteringUtils.getClusteringPlan( + this.writeClient.getHoodieTable().getMetaClient(), HoodieTimeline.getReplaceCommitInflightInstant(instant)); + return clusteringPlanOption.get().getRight(); + } catch (Exception e) { + throw new HoodieException(e); + } + }); boolean isReady = clusteringPlan.getInputGroups().size() == events.size(); if (!isReady) { return; @@ -124,6 +144,7 @@ private void commitIfNecessary(String instant, List event // handle failure case ClusteringUtil.rollbackClustering(table, writeClient, instant); } finally { + clusteringMetrics.markClusteringRolledBack(); // remove commitBuffer to avoid obsolete metadata commit reset(instant); } @@ -154,6 +175,7 @@ private void doCommit(String instant, HoodieClusteringPlan clusteringPlan, List< LOG.error("Got {} error records during clustering of instant {},\n" + "option '{}' is configured as false," + "rolls back the clustering", numErrorRecords, instant, FlinkOptions.IGNORE_FAILED.key()); + clusteringMetrics.markClusteringRolledBack(); ClusteringUtil.rollbackClustering(table, writeClient, instant); return; } @@ -178,6 +200,10 @@ private void doCommit(String instant, HoodieClusteringPlan clusteringPlan, List< this.writeClient.completeTableService( TableServiceType.CLUSTER, writeMetadata.getCommitMetadata().get(), table, instant); + // if clustering successful then update metric + this.clusteringMetrics.updateCommitMetrics(instant, writeMetadata.getCommitMetadata().get()); + this.clusteringMetrics.markClusteringCompleted(); + // whether to clean up the input base parquet files used for clustering if (!conf.getBoolean(FlinkOptions.CLEAN_ASYNC_ENABLED) && !isCleaning) { LOG.info("Running inline clean"); @@ -187,6 +213,7 @@ private void doCommit(String instant, HoodieClusteringPlan clusteringPlan, List< private void reset(String instant) { this.commitBuffer.remove(instant); + this.clusteringPlanCache.remove(instant); } /** @@ -203,7 +230,7 @@ private static void validateWriteResult(HoodieClusteringPlan clusteringPlan, Str } } - private static Map> getPartitionToReplacedFileIds( + protected Map> getPartitionToReplacedFileIds( HoodieClusteringPlan clusteringPlan, HoodieWriteMetadata> writeMetadata) { Set newFilesWritten = writeMetadata.getWriteStats().get().stream() @@ -212,4 +239,17 @@ private static Map> getPartitionToReplacedFileIds( .filter(fg -> !newFilesWritten.contains(fg)) .collect(Collectors.groupingBy(HoodieFileGroupId::getPartitionPath, Collectors.mapping(HoodieFileGroupId::getFileId, Collectors.toList()))); } + + private void registerMetrics() { + this.clusteringMetrics = new FlinkClusteringMetrics(FlinkWriteClients.getHoodieClientConfig(this.conf)); + this.clusteringMetrics.registerMetrics(FlinkClusteringMetricEnum.notClusteringPlanOperator.value, FlinkClusteringMetricEnum.notClusteringOperator.value); + } + + @Override + public void close() throws Exception { + super.close(); + if (this.clusteringMetrics != null) { + this.clusteringMetrics.shutDown(); + } + } } diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/ClusteringEndFileEvent.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/ClusteringEndFileEvent.java new file mode 100644 index 0000000000000..8125269bb57d5 --- /dev/null +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/ClusteringEndFileEvent.java @@ -0,0 +1,38 @@ +/* + * 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.clustering; + +import org.apache.hudi.common.model.lsm.HoodieLSMLogFile; +import org.apache.hudi.common.util.collection.Pair; + +import java.util.Set; + +/** + * Represents a cluster command from the clustering plan task {@link ClusteringPlanSourceFunction}. + */ +public class ClusteringEndFileEvent extends ClusteringFileEvent { + private static final long serialVersionUID = 1L; + + public ClusteringEndFileEvent() { + } + + public ClusteringEndFileEvent(HoodieLSMLogFile logFile, String fileID, String partitionPath, Pair, Set> missingAndCompletedInstants) { + super(logFile, fileID, partitionPath, null, missingAndCompletedInstants); + } +} diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/ClusteringFileEvent.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/ClusteringFileEvent.java new file mode 100644 index 0000000000000..d14add3740553 --- /dev/null +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/ClusteringFileEvent.java @@ -0,0 +1,101 @@ +/* + * 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.clustering; + +import org.apache.hadoop.fs.FileStatus; +import org.apache.hudi.common.model.lsm.HoodieLSMLogFile; +import org.apache.hudi.common.util.collection.Pair; + +import java.io.Serializable; +import java.util.Set; + +/** + * Represents a cluster command from the clustering plan task {@link ClusteringPlanSourceFunction}. + */ +public class ClusteringFileEvent implements Serializable { + private static final long serialVersionUID = 1L; + + private String fileID; + + private String partitionPath; + + private HoodieLSMLogFile logFile; + + private FileStatus fileStatus; + + private Pair, Set> missingAndCompletedInstants; + + public ClusteringFileEvent() { + } + + public ClusteringFileEvent(HoodieLSMLogFile logFile, String fileID, String partitionPath, FileStatus fileStatus) { + this.logFile = logFile; + this.fileID = fileID; + this.partitionPath = partitionPath; + this.fileStatus = fileStatus; + } + + public ClusteringFileEvent(HoodieLSMLogFile logFile, String fileID, String partitionPath, FileStatus fileStatus, Pair, Set> missingAndCompletedInstants) { + this.logFile = logFile; + this.fileID = fileID; + this.partitionPath = partitionPath; + this.fileStatus = fileStatus; + this.missingAndCompletedInstants = missingAndCompletedInstants; + } + + public HoodieLSMLogFile getLogFile() { + return logFile; + } + + public String getFileID() { + return fileID; + } + + public String getPartitionPath() { + return partitionPath; + } + + public void setFileID(String fileID) { + this.fileID = fileID; + } + + public void setPartitionPath(String partitionPath) { + this.partitionPath = partitionPath; + } + + public void setLogFile(HoodieLSMLogFile logFile) { + this.logFile = logFile; + } + + public FileStatus getFileStatus() { + return fileStatus; + } + + public void setFileStatus(FileStatus fileStatus) { + this.fileStatus = fileStatus; + } + + public Pair, Set> getMissingAndCompletedInstants() { + return missingAndCompletedInstants; + } + + public void setMissingAndCompletedInstants(Pair, Set> missingAndCompletedInstants) { + this.missingAndCompletedInstants = missingAndCompletedInstants; + } +} diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/ClusteringOperator.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/ClusteringOperator.java index 96462c2b345b6..ccf5c2bda94ea 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/ClusteringOperator.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/ClusteringOperator.java @@ -19,7 +19,6 @@ package org.apache.hudi.sink.clustering; import org.apache.hudi.adapter.MaskingOutputAdapter; -import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.client.FlinkTaskContextSupplier; import org.apache.hudi.client.HoodieFlinkWriteClient; import org.apache.hudi.client.WriteStatus; @@ -28,6 +27,7 @@ import org.apache.hudi.common.model.ClusteringGroupInfo; import org.apache.hudi.common.model.ClusteringOperation; import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.common.table.HoodieTableConfig; import org.apache.hudi.common.table.log.HoodieFileSliceReader; import org.apache.hudi.common.table.log.HoodieMergedLogRecordScanner; @@ -35,7 +35,6 @@ 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.HoodieWriteConfig; import org.apache.hudi.configuration.FlinkOptions; import org.apache.hudi.configuration.OptionsResolver; @@ -45,6 +44,8 @@ import org.apache.hudi.io.storage.HoodieAvroFileReader; import org.apache.hudi.io.storage.HoodieFileReader; import org.apache.hudi.io.storage.HoodieFileReaderFactory; +import org.apache.hudi.metrics.FlinkClusteringMetrics; +import org.apache.hudi.metrics.enums.FlinkClusteringMetricEnum; import org.apache.hudi.sink.bulk.BulkInsertWriterHelper; import org.apache.hudi.sink.bulk.sort.SortOperatorGen; import org.apache.hudi.sink.utils.NonThrownExecutor; @@ -55,7 +56,6 @@ import org.apache.avro.Schema; import org.apache.avro.generic.GenericRecord; -import org.apache.avro.generic.GenericRecordBuilder; import org.apache.avro.generic.IndexedRecord; import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.configuration.Configuration; @@ -93,8 +93,6 @@ import java.util.stream.Collectors; import java.util.stream.StreamSupport; -import static org.apache.hudi.table.format.FormatUtils.buildAvroRecordBySchema; - /** * Operator to execute the actual clustering task assigned by the clustering plan task. * In order to execute scalable, the input should shuffle by the clustering event {@link ClusteringPlanEvent}. @@ -104,11 +102,10 @@ public class ClusteringOperator extends TableStreamOperator table; + // private transient HoodieFlinkTable table; private transient Schema schema; private transient Schema readerSchema; private transient int[] requiredPos; @@ -132,12 +129,16 @@ public class ClusteringOperator extends TableStreamOperator containingTask, StreamConfig config, Output clusteringOps = clusteringGroupInfo.getOperations(); boolean hasLogFiles = clusteringOps.stream().anyMatch(op -> op.getDeltaFilePaths().size() > 0); + clusteringMetrics.startClustering(); + Iterator iterator; if (hasLogFiles) { // if there are log files, we read all records into memory for a file group and apply updates. - iterator = readRecordsForGroupWithLogs(clusteringOps, instantTime); + iterator = readRecordsForGroupWithLogs(clusteringOps, instantTime, table); } else { // We want to optimize reading records for case there are no log files. - iterator = readRecordsForGroupBaseFiles(clusteringOps); + iterator = readRecordsForGroupBaseFiles(clusteringOps, table); } RowDataSerializer rowDataSerializer = new RowDataSerializer(rowType); @@ -258,6 +270,8 @@ instantTime, this.taskID, getRuntimeContext().getNumberOfParallelSubtasks(), get } } + clusteringMetrics.endClustering(); + List writeStatuses = writerHelper.getWriteStatuses(this.taskID); collector.collect(new ClusteringCommitEvent(instantTime, writeStatuses, this.taskID)); writerHelper.close(); @@ -267,7 +281,7 @@ instantTime, this.taskID, getRuntimeContext().getNumberOfParallelSubtasks(), get * Read records from baseFiles, apply updates and convert to Iterator. */ @SuppressWarnings("unchecked") - private Iterator readRecordsForGroupWithLogs(List clusteringOps, String instantTime) { + private Iterator readRecordsForGroupWithLogs(List clusteringOps, String instantTime, HoodieFlinkTable table) { List> recordIterators = new ArrayList<>(); long maxMemoryPerCompaction = IOUtils.getMaxMemoryPerCompaction(new FlinkTaskContextSupplier(null), writeConfig); @@ -319,7 +333,7 @@ private Iterator readRecordsForGroupWithLogs(List /** * Read records from baseFiles and get iterator. */ - private Iterator readRecordsForGroupBaseFiles(List clusteringOps) { + private Iterator readRecordsForGroupBaseFiles(List clusteringOps, HoodieFlinkTable table) { List> iteratorsForPartition = clusteringOps.stream().map(clusteringOp -> { Iterable indexedRecords = () -> { try { @@ -343,9 +357,7 @@ private Iterator readRecordsForGroupBaseFiles(List * Transform IndexedRecord into HoodieRecord. */ private RowData transform(IndexedRecord indexedRecord) { - GenericRecord record = this.preserveHoodieMetadata - ? (GenericRecord) indexedRecord - : buildAvroRecordBySchema(indexedRecord, schema, requiredPos, new GenericRecordBuilder(schema)); + GenericRecord record = (GenericRecord) indexedRecord; return (RowData) avroToRowDataConverter.convert(record); } @@ -398,4 +410,9 @@ public void setExecutor(NonThrownExecutor executor) { public void setOutput(Output> output) { this.output = output; } + + private void registerMetrics() { + this.clusteringMetrics = new FlinkClusteringMetrics(FlinkWriteClients.getHoodieClientConfig(this.conf)); + this.clusteringMetrics.registerMetrics(FlinkClusteringMetricEnum.notClusteringPlanOperator.value, FlinkClusteringMetricEnum.isClusteringOperator.value); + } } diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/ClusteringPartitionEvent.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/ClusteringPartitionEvent.java new file mode 100644 index 0000000000000..0c097e331bde4 --- /dev/null +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/ClusteringPartitionEvent.java @@ -0,0 +1,56 @@ +/* + * 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.clustering; + +import org.apache.hudi.common.util.collection.Pair; + +import java.io.Serializable; +import java.util.Set; + +public class ClusteringPartitionEvent implements Serializable { + private static final long serialVersionUID = 1L; + + private String partitionPath; + + private Pair, Set> missingAndCompletedInstants; + + public ClusteringPartitionEvent() { + } + + public ClusteringPartitionEvent(String partitionPath, Pair, Set> missingAndCompletedInstants) { + this.partitionPath = partitionPath; + this.missingAndCompletedInstants = missingAndCompletedInstants; + } + + public String getPartitionPath() { + return partitionPath; + } + + public void setPartitionPath(String partitionPath) { + this.partitionPath = partitionPath; + } + + public Pair, Set> getMissingAndCompletedInstants() { + return missingAndCompletedInstants; + } + + public void setMissingAndCompletedInstants(Pair, Set> missingAndCompletedInstants) { + this.missingAndCompletedInstants = missingAndCompletedInstants; + } +} diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/ClusteringPlanGenerateOperator.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/ClusteringPlanGenerateOperator.java new file mode 100644 index 0000000000000..48c55dd4fba7e --- /dev/null +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/ClusteringPlanGenerateOperator.java @@ -0,0 +1,270 @@ +/* + * 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.clustering; + +import org.apache.hadoop.fs.FileStatus; +import org.apache.hudi.avro.AvroSchemaCache; +import org.apache.hudi.avro.model.HoodieClusteringGroup; +import org.apache.hudi.avro.model.HoodieClusteringPlan; +import org.apache.hudi.avro.model.HoodieRequestedReplaceMetadata; +import org.apache.hudi.common.model.FileSlice; +import org.apache.hudi.common.model.HoodieFileGroupId; +import org.apache.hudi.common.model.HoodieLogFile; +import org.apache.hudi.common.model.WriteOperationType; +import org.apache.hudi.common.model.lsm.HoodieLSMLogFile; +import org.apache.hudi.common.table.HoodieTableMetaClient; +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.common.table.timeline.TimelineMetadataUtils; +import org.apache.hudi.common.table.view.HoodieTableFileSystemView; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.configuration.FlinkOptions; +import org.apache.hudi.exception.HoodieIOException; +import org.apache.hudi.sink.bulk.BulkInsertWriterHelper; +import org.apache.hudi.sink.utils.NonThrownExecutor; +import org.apache.hudi.table.HoodieFlinkTable; +import org.apache.hudi.table.action.cluster.strategy.ClusteringPlanStrategy; +import org.apache.hudi.table.action.cluster.strategy.LsmBaseClusteringPlanStrategy; +import org.apache.hudi.util.AvroSchemaConverter; +import org.apache.hudi.util.FlinkTables; +import org.apache.hudi.util.FlinkWriteClients; +import org.apache.hudi.util.FsCacheCleanUtil; + +import org.apache.avro.Schema; +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.state.StateSnapshotContext; +import org.apache.flink.streaming.api.operators.AbstractStreamOperator; +import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.api.operators.Output; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.table.types.logical.RowType; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.TreeSet; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +public class ClusteringPlanGenerateOperator extends AbstractStreamOperator + implements OneInputStreamOperator { + + protected static final Logger LOG = LoggerFactory.getLogger(ClusteringPlanGenerateOperator.class); + + private final Configuration conf; + private final RowType rowType; + private final long interval; + private final long step; + private long checkLatency = 0; + private long lastCheckTs = -1; + private HoodieWriteConfig hoodieConfig; + private NonThrownExecutor executor; + // , > + private final Map, TreeSet> files = new ConcurrentHashMap<>(); + private final ConcurrentLinkedQueue groups = new ConcurrentLinkedQueue<>(); + // , max-level> + private final Map, Integer> flags = new ConcurrentHashMap<>(); + private final Set missingInstants = new HashSet<>(); + private final Set completedInstants = new HashSet<>(); + private final Set missingPartitions = new HashSet<>(); + private int numGroups; + private Schema schema; + private HoodieActiveTimeline activeTimeline; + private boolean skipOP; + + public ClusteringPlanGenerateOperator(Configuration conf, RowType rowType) { + this.conf = conf; + this.interval = conf.getLong(FlinkOptions.LSM_CLUSTERING_SCHEDULE_INTERVAL) * 1000; + this.step = (long) (Math.ceil(this.interval * 1.0) / 5); + this.rowType = BulkInsertWriterHelper.addMetadataFields(rowType, false); + } + + @Override + public void open() throws Exception { + super.open(); + this.hoodieConfig = FlinkWriteClients.getHoodieClientConfig(conf, true); + this.executor = NonThrownExecutor.builder(LOG).build(); + this.numGroups = hoodieConfig.getLsmClusteringMinNumGroups(); + this.schema = AvroSchemaCache.intern(AvroSchemaConverter.convertToSchema(rowType)); + this.activeTimeline = FlinkTables.createTable(conf, getRuntimeContext()).getActiveTimeline(); + this.skipOP = !this.hoodieConfig.isLsmReadFooterEnabled(); + } + + @Override + public void processElement(StreamRecord record) { + if (lastCheckTs == -1) { + this.lastCheckTs = System.currentTimeMillis(); + } + executor.execute(() -> { + long currentTs = System.currentTimeMillis(); + if (currentTs - lastCheckTs > interval + checkLatency) { + LOG.info("Start to schedule clustering plan " + " flags " + flags + ". groups: " + groups); + if (tryScheduleClusteringPlan()) { + lastCheckTs = currentTs; + checkLatency = 0; + LOG.info("Schedule clustering successful."); + } else { + checkLatency += step; + LOG.info("No clustering group to save. Wait " + step + " ms to save again."); + } + flags.clear(); + } else { + LOG.info("Still wait for interval " + (interval + checkLatency) + ". Gap " + (currentTs - lastCheckTs)); + } + + // event 可能来自于不同分区下的不同FileSlice + ClusteringFileEvent event = record.getValue(); + String fileID = event.getFileID(); + String partitionPath = event.getPartitionPath(); + Pair key = Pair.of(partitionPath, fileID); + + if (event instanceof ClusteringEndFileEvent && files.containsKey(key)) { + TreeSet logs = files.remove(key); + // 当前Key下的其他slice都不再接收 + flags.put(key, 1); + HoodieFlinkTable table = FlinkTables.createTable(conf, getRuntimeContext()); + LsmBaseClusteringPlanStrategy strategy = new LsmBaseClusteringPlanStrategy(table, table.getContext(), hoodieConfig, schema); + missingInstants.addAll(event.getMissingAndCompletedInstants().getLeft()); + completedInstants.addAll(event.getMissingAndCompletedInstants().getRight()); + buildAndCacheClusteringGroup(table, strategy, logs, key); + } else if (!flags.containsKey(key)) { + HoodieLSMLogFile logFile = event.getLogFile(); + logFile.setFileStatus(event.getFileStatus()); + if (files.containsKey(key)) { + TreeSet tree = files.get(key); + tree.add(logFile); + } else { + TreeSet tree = new TreeSet<>(HoodieLogFile.getReverseLogFileComparator()); + tree.add(logFile); + files.put(key, tree); + } + } + // drop current record directly, it doesn't mater will come again in next loop. + }, "Try to schedule clustering plan"); + } + + private boolean tryScheduleClusteringPlan() { + if (groups.size() > 0) { + Option plan = LsmBaseClusteringPlanStrategy.buildClusteringPlan(FlinkTables.createTable(conf, getRuntimeContext()).getMetaClient(), + groups.stream().collect(Collectors.toList()), hoodieConfig, missingInstants, completedInstants, missingPartitions, + new HashMap<>(), new HashMap<>(), ClusteringPlanStrategy.CLUSTERING_PLAN_VERSION_1); + saveClusteringPlan(plan); + groups.clear(); + missingPartitions.clear(); + missingInstants.clear(); + completedInstants.clear(); + return true; + } + return false; + } + + private void buildAndCacheClusteringGroup(HoodieFlinkTable table, LsmBaseClusteringPlanStrategy strategy, TreeSet logs, Pair partitionPathToFileId) { + String partitionPath = partitionPathToFileId.getLeft(); + String fileId = partitionPathToFileId.getRight(); + Pair, Pair> res = getEligibleLogFiles(table, strategy, logs, partitionPath, fileId); + Boolean level1Exists = res.getRight().getLeft(); + Boolean level1ExistsInPending = res.getRight().getRight(); + Pair> clusteringGroups = + (Pair>) strategy.buildClusteringGroups(res.getLeft(), level1Exists, level1ExistsInPending, partitionPath, skipOP); + if (clusteringGroups.getLeft()) { + missingPartitions.add(partitionPath); + } + groups.addAll(clusteringGroups.getRight().collect(Collectors.toList())); + } + + /** + * Get Eligible Log Files for Clutering Schedule + * + * @param table + * @param logs + * @param partitionPath + * @param fileId + * @return Pair> + */ + private Pair, Pair> getEligibleLogFiles(HoodieFlinkTable table, LsmBaseClusteringPlanStrategy strategy, + TreeSet logs, String partitionPath, String fileId) { + HoodieTableMetaClient metaClient = table.getMetaClient(); + FileStatus[] fileStatuses = logs.stream().map(HoodieLogFile::getFileStatus).toArray(FileStatus[]::new); + + // build fsView based on log files + HoodieTableFileSystemView fsView = new HoodieTableFileSystemView(metaClient, + metaClient.getCommitsAndCompactionTimeline().filterCompletedAndReplaceInstants(), + fileStatuses); + // use latest fileslice to drop pending and replaced files + Option fileSlice = fsView.getLatestFileSlice(partitionPath, fileId); + Set pendingClusteringFiles = fsView.getFileGroupsInPendingClustering() + .map(Pair::getKey).map(HoodieFileGroupId::getFileId).collect(Collectors.toSet()); + Set level1InPendingClustering = fsView.getLevel1FileIdInPendingClustering(partitionPath); + + if (fileSlice.isPresent() && !fileSlice.get().isFileSliceEmpty()) { + HashSet level1 = new HashSet<>(); + List finalLogs = strategy.dropPending(fileSlice.get().getLogFiles(), level1, pendingClusteringFiles); + // + return Pair.of(finalLogs, Pair.of(!level1.isEmpty(), level1InPendingClustering.contains(fileId))); + } + return Pair.of(Collections.EMPTY_LIST, Pair.of(false, false)); + } + + @Override + public void snapshotState(StateSnapshotContext context) throws Exception { + // no op + } + + @VisibleForTesting + public void setOutput(Output> output) { + this.output = output; + } + + private void saveClusteringPlan(Option planOption) { + if (planOption.isPresent()) { + String instantTime = HoodieActiveTimeline.createNewInstantTime(); + HoodieInstant clusteringInstant = + new HoodieInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.REPLACE_COMMIT_ACTION, instantTime); + try { + HoodieRequestedReplaceMetadata requestedReplaceMetadata = HoodieRequestedReplaceMetadata.newBuilder() + .setOperationType(WriteOperationType.CLUSTER.name()) + .setExtraMetadata(Collections.emptyMap()) + .setClusteringPlan(planOption.get()) + .build(); + activeTimeline.saveToPendingReplaceCommit(clusteringInstant, + TimelineMetadataUtils.serializeRequestedReplaceMetadata(requestedReplaceMetadata)); + LOG.info("Generate clustering plan at " + instantTime); + } catch (IOException ioe) { + throw new HoodieIOException("Exception scheduling clustering", ioe); + } + } + } + + @Override + public void close() throws Exception { + FsCacheCleanUtil.cleanChubaoFsCacheIfNecessary(); + } +} diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/ClusteringPlanListOperator.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/ClusteringPlanListOperator.java new file mode 100644 index 0000000000000..4b546313d4f5a --- /dev/null +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/ClusteringPlanListOperator.java @@ -0,0 +1,124 @@ +/* + * 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.clustering; + +import org.apache.avro.Schema; +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.state.StateSnapshotContext; +import org.apache.flink.streaming.api.graph.StreamConfig; +import org.apache.flink.streaming.api.operators.AbstractStreamOperator; +import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.api.operators.Output; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; + +import org.apache.flink.streaming.runtime.tasks.StreamTask; +import org.apache.flink.table.runtime.util.StreamRecordCollector; +import org.apache.hudi.adapter.MaskingOutputAdapter; +import org.apache.hudi.avro.AvroSchemaCache; +import org.apache.hudi.common.model.FileSlice; +import org.apache.hudi.common.model.lsm.HoodieLSMLogFile; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.sink.bulk.BulkInsertWriterHelper; +import org.apache.hudi.sink.utils.NonThrownExecutor; +import org.apache.hudi.table.HoodieFlinkTable; +import org.apache.hudi.table.action.cluster.strategy.LsmBaseClusteringPlanStrategy; +import org.apache.hudi.util.AvroSchemaConverter; +import org.apache.hudi.util.FlinkTables; +import org.apache.hudi.util.FlinkWriteClients; +import org.apache.hudi.util.FsCacheCleanUtil; + +import org.apache.flink.table.types.logical.RowType; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import java.util.List; +import java.util.Set; + +public class ClusteringPlanListOperator extends AbstractStreamOperator + implements OneInputStreamOperator { + + protected static final Logger LOG = LoggerFactory.getLogger(ClusteringPlanListOperator.class); + private final Configuration conf; + private final RowType rowType; + private HoodieWriteConfig hoodieConfig; + private NonThrownExecutor executor; + private Schema schema; + private transient StreamRecordCollector collector; + + public ClusteringPlanListOperator(Configuration conf, RowType rowType) { + this.conf = conf; + this.rowType = BulkInsertWriterHelper.addMetadataFields(rowType, false); + } + + @Override + public void open() throws Exception { + super.open(); + this.hoodieConfig = FlinkWriteClients.getHoodieClientConfig(conf, true); + this.executor = NonThrownExecutor.builder(LOG).build(); + this.schema = AvroSchemaCache.intern(AvroSchemaConverter.convertToSchema(rowType)); + this.collector = new StreamRecordCollector<>(output); + } + + @Override + public void setup(StreamTask containingTask, StreamConfig config, Output> output) { + super.setup(containingTask, config, new MaskingOutputAdapter<>(output)); + } + + @Override + public void processElement(StreamRecord record) { + executor.execute(() -> { + HoodieFlinkTable table = FlinkTables.createTable(conf, getRuntimeContext()); + LsmBaseClusteringPlanStrategy clusteringPlanStrategy = new LsmBaseClusteringPlanStrategy(table, table.getContext(), hoodieConfig, schema); + ClusteringPartitionEvent event = record.getValue(); + String partitionPath = event.getPartitionPath(); + Pair, Set> res = clusteringPlanStrategy.getFilesEligibleForClustering(partitionPath); + List fileSlices = res.getLeft(); + fileSlices.stream().filter(fileSlice -> !fileSlice.isFileSliceEmpty()).forEach(fileSlice -> { + String fileId = fileSlice.getFileId(); + fileSlice.getLogFiles().forEach(logfile -> { + HoodieLSMLogFile lsmLogFile = (HoodieLSMLogFile) logfile; + collector.collect(new ClusteringFileEvent(lsmLogFile, lsmLogFile.getFileId(), partitionPath, lsmLogFile.getFileStatus())); + }); + // 发送一个ClusteringEndFileEvent 代表当前Slice全部发送完了 + collector.collect(new ClusteringEndFileEvent(null, fileId, partitionPath, event.getMissingAndCompletedInstants())); + }); + }, "Get Files Eligible ForClustering"); + } + + @Override + public void snapshotState(StateSnapshotContext context) throws Exception { + // no op + } + + @Override + public void notifyCheckpointComplete(long checkpointId) { + // no op + } + + @VisibleForTesting + public void setOutput(Output> output) { + this.output = output; + } + + @Override + public void close() throws Exception { + FsCacheCleanUtil.cleanChubaoFsCacheIfNecessary(); + } +} \ No newline at end of file diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/ClusteringPlanOperator.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/ClusteringPlanOperator.java index 48b2a9becd436..a7393fcc69669 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/ClusteringPlanOperator.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/ClusteringPlanOperator.java @@ -20,16 +20,20 @@ import org.apache.hudi.avro.model.HoodieClusteringGroup; import org.apache.hudi.avro.model.HoodieClusteringPlan; +import org.apache.hudi.client.HoodieFlinkWriteClient; import org.apache.hudi.common.model.ClusteringGroupInfo; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.util.ClusteringUtils; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.metrics.FlinkClusteringMetrics; +import org.apache.hudi.metrics.enums.FlinkClusteringMetricEnum; import org.apache.hudi.table.HoodieFlinkTable; import org.apache.hudi.util.ClusteringUtil; import org.apache.hudi.util.FlinkTables; import org.apache.hudi.util.FlinkWriteClients; +import org.apache.hudi.util.FsCacheCleanUtil; import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.configuration.Configuration; @@ -37,6 +41,10 @@ import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.api.operators.Output; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.List; /** * Operator that generates the clustering plan with pluggable strategies on finished checkpoints. @@ -46,6 +54,8 @@ public class ClusteringPlanOperator extends AbstractStreamOperator implements OneInputStreamOperator { + protected static final Logger LOG = LoggerFactory.getLogger(ClusteringPlanOperator.class); + /** * Config options. */ @@ -56,6 +66,12 @@ public class ClusteringPlanOperator extends AbstractStreamOperator table, long checkpointId) { + + List pendingClusteringInstantTimes = ClusteringUtils.getPendingClusteringInstantTimes(table.getMetaClient()); + // the first instant takes the highest priority. Option firstRequested = Option.fromJavaOptional( - ClusteringUtils.getPendingClusteringInstantTimes(table.getMetaClient()).stream() + pendingClusteringInstantTimes.stream() .filter(instant -> instant.getState() == HoodieInstant.State.REQUESTED).findFirst()); + + if (clusteringMetrics != null) { + clusteringMetrics.setFirstPendingClusteringInstant(firstRequested); + clusteringMetrics.setPendingClusteringPlanCount(pendingClusteringInstantTimes.size()); + } + if (!firstRequested.isPresent()) { // do nothing. LOG.info("No clustering plan for checkpoint " + checkpointId); @@ -136,4 +163,18 @@ private void scheduleClustering(HoodieFlinkTable table, long checkpointId) { public void setOutput(Output> output) { this.output = output; } + + private void registerMetrics() { + this.clusteringMetrics = new FlinkClusteringMetrics(FlinkWriteClients.getHoodieClientConfig(this.conf)); + this.clusteringMetrics.registerMetrics(FlinkClusteringMetricEnum.isClusteringPlanOperator.value, FlinkClusteringMetricEnum.notClusteringOperator.value); + } + + @Override + public void close() throws Exception { + super.close(); + if (this.clusteringMetrics != null) { + this.clusteringMetrics.shutDown(); + } + FsCacheCleanUtil.cleanChubaoFsCacheIfNecessary(); + } } diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/ClusteringPlanPartitionFindOperator.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/ClusteringPlanPartitionFindOperator.java new file mode 100644 index 0000000000000..bd96934031b81 --- /dev/null +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/ClusteringPlanPartitionFindOperator.java @@ -0,0 +1,171 @@ +/* + * 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.clustering; + +import org.apache.flink.streaming.api.graph.StreamConfig; +import org.apache.flink.streaming.runtime.tasks.StreamTask; +import org.apache.flink.table.runtime.util.StreamRecordCollector; +import org.apache.hudi.adapter.MaskingOutputAdapter; +import org.apache.hudi.avro.AvroSchemaCache; +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.common.util.collection.Pair; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.configuration.FlinkOptions; +import org.apache.hudi.sink.bulk.BulkInsertWriterHelper; +import org.apache.hudi.sink.utils.NonThrownExecutor; +import org.apache.hudi.table.HoodieFlinkTable; +import org.apache.hudi.table.action.cluster.strategy.LsmBaseClusteringPlanStrategy; +import org.apache.hudi.util.AvroSchemaConverter; +import org.apache.hudi.util.FlinkTables; +import org.apache.hudi.util.FlinkWriteClients; +import org.apache.hudi.util.FsCacheCleanUtil; + +import org.apache.avro.Schema; +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.state.StateSnapshotContext; +import org.apache.flink.streaming.api.operators.AbstractStreamOperator; +import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.api.operators.Output; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.table.types.logical.RowType; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.List; +import java.util.Set; + +public class ClusteringPlanPartitionFindOperator extends AbstractStreamOperator + implements OneInputStreamOperator { + + protected static final Logger LOG = LoggerFactory.getLogger(ClusteringPlanPartitionFindOperator.class); + private final Configuration conf; + + private final long interval; + private final RowType rowType; + private long lastCheckTs; + private transient NonThrownExecutor executor; + private HoodieWriteConfig hoodieConfig; + private Schema schema; + private transient StreamRecordCollector collector; + + public ClusteringPlanPartitionFindOperator(Configuration conf, RowType rowType) { + this.conf = conf; + this.interval = conf.getLong(FlinkOptions.LSM_CLUSTERING_SCHEDULE_INTERVAL) * 1000; + this.lastCheckTs = System.currentTimeMillis(); + this.rowType = BulkInsertWriterHelper.addMetadataFields(rowType, false); + } + + @Override + public void setup(StreamTask containingTask, StreamConfig config, Output> output) { + super.setup(containingTask, config, new MaskingOutputAdapter<>(output)); + } + + @Override + public void open() throws Exception { + super.open(); + this.executor = NonThrownExecutor.builder(LOG).build(); + this.hoodieConfig = FlinkWriteClients.getHoodieClientConfig(conf, true); + this.schema = AvroSchemaCache.intern(AvroSchemaConverter.convertToSchema(rowType)); + this.collector = new StreamRecordCollector<>(output); + } + + @Override + public void processElement(StreamRecord record) { + // no op + } + + @Override + public void notifyCheckpointComplete(long checkpointId) { + executor.execute(() -> { + try { + HoodieFlinkTable table = FlinkTables.createTable(conf, getRuntimeContext()); + long currentTs = System.currentTimeMillis(); + if (currentTs - lastCheckTs > interval) { + if (scheduleClustering(table)) { + lastCheckTs = currentTs; + } + } else { + LOG.info("Skip current schedule lastCheckTs " + lastCheckTs + ", interval " + interval); + } + } catch (Throwable throwable) { + // make it fail-safe + LOG.error("Error while scheduling clustering plan for checkpoint: " + checkpointId, throwable); + } + }, "Get Clustering partitions"); + } + + @VisibleForTesting + public void setOutput(Output> output) { + this.output = output; + } + + @Override + public void snapshotState(StateSnapshotContext context) throws Exception { + // no op + } + + private boolean scheduleClustering(HoodieFlinkTable table) { + LsmBaseClusteringPlanStrategy clusteringPlanStrategy = new LsmBaseClusteringPlanStrategy(table, table.getContext(), hoodieConfig, schema); + List backtrackInstances = table.getMetaClient().getBacktrackInstances(); + + // get the pending insert overwrite instant + List pendingBackTrackInstances = table.getMetaClient().getPendingBacktrackInstances(); + int pendingClusteringNum = table.getActiveTimeline() + .filter(s -> s.getAction().equalsIgnoreCase(HoodieTimeline.REPLACE_COMMIT_ACTION) + && !backtrackInstances.contains(s.getTimestamp()) + && !pendingBackTrackInstances.contains(s.getTimestamp()) + && !s.isCompleted()) + .countInstants(); + if (pendingClusteringNum >= hoodieConfig.getLsmMaxOfPendingClustering()) { + LOG.info(String.format("The num of pending clustering is %s >= %s.", pendingClusteringNum, hoodieConfig.getMaxOfPendingClustering())); + return false; + } + + Option lastClusteringInstant = table.getActiveTimeline() + .filter(s -> s.getAction().equalsIgnoreCase(HoodieTimeline.REPLACE_COMMIT_ACTION) + && !backtrackInstances.contains(s.getTimestamp()) + && !pendingBackTrackInstances.contains(s.getTimestamp())).lastInstant(); + + int commitsSinceLastClustering = table.getActiveTimeline().getCommitsTimeline().filterCompletedInstants() + .findInstantsAfter(lastClusteringInstant.map(HoodieInstant::getTimestamp).orElse("0"), Integer.MAX_VALUE) + .countInstants(); + + if (commitsSinceLastClustering < 1) { + LOG.info("Not scheduling clustering as only " + commitsSinceLastClustering + + " commits was found since last clustering " + lastClusteringInstant + ". Waiting for at least 1 delta commit."); + return false; + } + + Pair, Pair, Set>> partitionsAndInstantsPair = clusteringPlanStrategy.getPartitionPathsToCluster(); + List partitions = partitionsAndInstantsPair.getLeft(); + LOG.info("Try to schedule clustering for partitions : " + partitions); + for (String partition : partitions) { + collector.collect(new ClusteringPartitionEvent(partition, partitionsAndInstantsPair.getRight())); + } + return true; + } + + @Override + public void close() throws Exception { + FsCacheCleanUtil.cleanChubaoFsCacheIfNecessary(); + } +} diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/ClusteringPlanReadFooterOperator.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/ClusteringPlanReadFooterOperator.java new file mode 100644 index 0000000000000..ae90bf1fb464a --- /dev/null +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/ClusteringPlanReadFooterOperator.java @@ -0,0 +1,113 @@ +/* + * 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.clustering; + +import org.apache.flink.streaming.api.graph.StreamConfig; +import org.apache.flink.streaming.runtime.tasks.StreamTask; +import org.apache.flink.table.runtime.util.StreamRecordCollector; +import org.apache.hudi.adapter.MaskingOutputAdapter; +import org.apache.hudi.common.model.HoodieFileFormat; +import org.apache.hudi.common.model.lsm.HoodieLSMLogFile; +import org.apache.hudi.common.util.BaseFileUtils; +import org.apache.hudi.sink.utils.NonThrownExecutor; +import org.apache.hudi.util.FlinkTables; +import org.apache.hudi.util.FsCacheCleanUtil; + +import com.github.benmanes.caffeine.cache.Cache; +import com.github.benmanes.caffeine.cache.Caffeine; +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.state.StateSnapshotContext; +import org.apache.flink.streaming.api.operators.AbstractStreamOperator; +import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.api.operators.Output; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class ClusteringPlanReadFooterOperator extends AbstractStreamOperator + implements OneInputStreamOperator { + + protected static final Logger LOG = LoggerFactory.getLogger(ClusteringPlanReadFooterOperator.class); + private final Configuration conf; + + public static final BaseFileUtils PARQUET_UTILS = BaseFileUtils.getInstance(HoodieFileFormat.PARQUET); + private org.apache.hadoop.conf.Configuration hadoopConf; + private NonThrownExecutor executor; + private Cache cache; // partitionPath + logFile.getFileName() + private transient StreamRecordCollector collector; + + public ClusteringPlanReadFooterOperator(Configuration conf) { + this.conf = conf; + } + + @Override + public void setup(StreamTask containingTask, StreamConfig config, Output> output) { + super.setup(containingTask, config, new MaskingOutputAdapter<>(output)); + } + + @Override + public void open() throws Exception { + super.open(); + this.hadoopConf = FlinkTables.createTable(conf, getRuntimeContext()).getHadoopConf(); + this.executor = NonThrownExecutor.builder(LOG).build(); + this.cache = Caffeine.newBuilder().weakValues().maximumSize(1024).build(); + this.collector = new StreamRecordCollector<>(output); + } + + @Override + public void processElement(StreamRecord record) { + executor.execute(() -> { + ClusteringFileEvent event = record.getValue(); + if (event instanceof ClusteringEndFileEvent) { + collector.collect(event); + } else { + HoodieLSMLogFile logFile = event.getLogFile(); + String fileID = event.getFileID(); + String partitionPath = event.getPartitionPath(); + String cacheKey = partitionPath + logFile.getFileName(); + HoodieLSMLogFile cacheLog = cache.getIfPresent(cacheKey); + if (cacheLog != null) { + collector.collect(new ClusteringFileEvent(cacheLog, fileID, partitionPath, event.getFileStatus())); + } else { + String[] minMax = PARQUET_UTILS.readMinMaxRecordKeys(hadoopConf, logFile.getPath()); + logFile.setMin(minMax[0]); + logFile.setMax(minMax[1]); + cache.put(cacheKey, logFile); + collector.collect(new ClusteringFileEvent(logFile, fileID, partitionPath, event.getFileStatus())); + } + } + }, "Read parquet footer"); + } + + @VisibleForTesting + public void setOutput(Output> output) { + this.output = output; + } + + @Override + public void snapshotState(StateSnapshotContext context) throws Exception { + // no op + } + + @Override + public void close() throws Exception { + FsCacheCleanUtil.cleanChubaoFsCacheIfNecessary(); + } +} diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/HoodieFlinkClusteringJob.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/HoodieFlinkClusteringJob.java index 5acbe55fe8b4e..12ccff2cfc3ce 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/HoodieFlinkClusteringJob.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/HoodieFlinkClusteringJob.java @@ -352,7 +352,9 @@ private void cluster() throws Exception { .addSink(new ClusteringCommitSink(conf)) .name("clustering_commit") .uid("uid_clustering_commit") - .setParallelism(1); + .setParallelism(1) + .getTransformation() + .setMaxParallelism(1); env.execute("flink_hudi_clustering_" + clusteringInstant.getTimestamp()); } diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/LSMClusteringCommitSink.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/LSMClusteringCommitSink.java new file mode 100644 index 0000000000000..8ac970ec7e69e --- /dev/null +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/LSMClusteringCommitSink.java @@ -0,0 +1,53 @@ +/* + * 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.clustering; + +import org.apache.hudi.avro.model.HoodieClusteringPlan; +import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.common.model.HoodieFileGroupId; +import org.apache.hudi.common.util.ClusteringUtils; +import org.apache.hudi.table.action.HoodieWriteMetadata; + +import org.apache.flink.configuration.Configuration; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; + +public class LSMClusteringCommitSink extends ClusteringCommitSink { + + public LSMClusteringCommitSink(Configuration conf) { + super(conf); + } + + /** + * new HoodieFileGroupId(s.getPartitionPath(), s.getPath())) + * use file path as replaced file ID in clustering commit + */ + @Override + protected Map> getPartitionToReplacedFileIds( + HoodieClusteringPlan clusteringPlan, + HoodieWriteMetadata> writeMetadata) { + Set newFilesWritten = writeMetadata.getWriteStats().get().stream() + .map(s -> new HoodieFileGroupId(s.getPartitionPath(), s.getPath())).collect(Collectors.toSet()); + return ClusteringUtils.getFileGroupsFromClusteringPlan(clusteringPlan) + .filter(fg -> !newFilesWritten.contains(fg)) + .collect(Collectors.groupingBy(HoodieFileGroupId::getPartitionPath, Collectors.mapping(HoodieFileGroupId::getFileId, Collectors.toList()))); + } +} diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/LSMClusteringOperator.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/LSMClusteringOperator.java new file mode 100644 index 0000000000000..96e00e1bcb200 --- /dev/null +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/LSMClusteringOperator.java @@ -0,0 +1,283 @@ +/* + * 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.clustering; + +import org.apache.flink.table.runtime.typeutils.RowDataSerializer; +import org.apache.hudi.adapter.MaskingOutputAdapter; +import org.apache.hudi.avro.AvroSchemaCache; +import org.apache.hudi.client.HoodieFlinkWriteClient; +import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.common.config.HoodieStorageConfig; +import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.model.ClusteringGroupInfo; +import org.apache.hudi.common.model.ClusteringOperation; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.WriteOperationType; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.util.ClosableIterator; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.configuration.FlinkOptions; +import org.apache.hudi.configuration.OptionsResolver; +import org.apache.hudi.exception.HoodieIOException; +import org.apache.hudi.io.lsm.RecordReader; +import org.apache.hudi.io.storage.row.LSMHoodieRowDataCreateHandle; +import org.apache.hudi.metrics.FlinkClusteringMetrics; +import org.apache.hudi.metrics.enums.FlinkClusteringMetricEnum; +import org.apache.hudi.sink.bulk.BulkInsertWriterHelper; +import org.apache.hudi.sink.utils.NonThrownExecutor; +import org.apache.hudi.table.HoodieFlinkTable; +import org.apache.hudi.table.format.InternalSchemaManager; +import org.apache.hudi.table.format.mor.lsm.FlinkLsmUtils; +import org.apache.hudi.table.format.mor.lsm.LsmMergeIterator; +import org.apache.hudi.util.AvroSchemaConverter; +import org.apache.hudi.util.FlinkWriteClients; +import org.apache.hudi.util.FsCacheCleanUtil; + +import org.apache.avro.Schema; +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.streaming.api.graph.StreamConfig; +import org.apache.flink.streaming.api.operators.BoundedOneInput; +import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.api.operators.Output; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.streaming.runtime.tasks.StreamTask; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.runtime.operators.TableStreamOperator; +import org.apache.flink.table.runtime.util.StreamRecordCollector; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.hadoop.fs.Path; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +import static org.apache.hudi.config.HoodieClusteringConfig.LSM_CLUSTERING_OUT_PUT_LEVEL; + +/** + * Operator to execute the actual clustering task assigned by the clustering plan task. + * In order to execute scalable, the input should shuffle by the clustering event {@link ClusteringPlanEvent}. + */ +public class LSMClusteringOperator extends TableStreamOperator implements + OneInputStreamOperator, BoundedOneInput { + private static final Logger LOG = LoggerFactory.getLogger(LSMClusteringOperator.class); + + private final Configuration conf; + private final RowType rowType; + private DataType rowDataType; + private int taskID; + private transient HoodieWriteConfig writeConfig; + private transient Schema schema; + private transient Schema readerSchema; + private transient int[] requiredPos; + private transient HoodieFlinkWriteClient writeClient; + private transient StreamRecordCollector collector; + private transient ClosableIterator iterator; + private int spillTreshold; + + /** + * Whether to execute clustering asynchronously. + */ + private final boolean asyncClustering; + + /** + * Executor service to execute the clustering task. + */ + private transient NonThrownExecutor executor; + + /** + * clustering metric + */ + private transient FlinkClusteringMetrics clusteringMetrics; + + private transient HoodieTableMetaClient metaClient; + + public LSMClusteringOperator(Configuration conf, RowType rowType) { + // copy a conf let following modification not to impact the global conf + this.conf = new Configuration(conf); + this.conf.setString(FlinkOptions.OPERATION.key(), WriteOperationType.CLUSTER.value()); + this.rowType = BulkInsertWriterHelper.addMetadataFields(rowType, false); + this.asyncClustering = OptionsResolver.needsAsyncClustering(conf); + this.spillTreshold = this.conf.getInteger(FlinkOptions.LSM_SORT_MERGE_SPILL_THRESHOLD); + + // enable parquet bloom filter for record key fields + this.conf.setBoolean(HoodieStorageConfig.PARQUET_RECORDKEY_BLOOM_FILTER_ENABLED.key(), + this.conf.getBoolean(FlinkOptions.PARQUET_RECORDKEY_CLUSTERING_BLOOM_FILTER_ENABLED)); + } + + @Override + public void setup(StreamTask containingTask, StreamConfig config, Output> output) { + super.setup(containingTask, config, new MaskingOutputAdapter<>(output)); + } + + @Override + public void open() throws Exception { + super.open(); + registerMetrics(); + this.taskID = getRuntimeContext().getIndexOfThisSubtask(); + this.writeConfig = FlinkWriteClients.getHoodieClientConfig(this.conf); + this.writeClient = FlinkWriteClients.createWriteClient(conf, getRuntimeContext()); + this.metaClient = writeClient.getHoodieTable().getMetaClient(); + this.schema = AvroSchemaCache.intern(AvroSchemaConverter.convertToSchema(rowType)); + this.rowDataType = AvroSchemaConverter.convertToDataType(schema); + this.readerSchema = this.schema; + this.requiredPos = getRequiredPositions(); + if (this.asyncClustering) { + this.executor = NonThrownExecutor.builder(LOG).build(); + } + this.collector = new StreamRecordCollector<>(output); + } + + @Override + public void processElement(StreamRecord element) throws Exception { + ClusteringPlanEvent event = element.getValue(); + final String instantTime = event.getClusteringInstantTime(); + if (this.asyncClustering) { + // executes the compaction task asynchronously to not block the checkpoint barrier propagate. + executor.execute( + () -> doClustering(instantTime, event), + (errMsg, t) -> collector.collect(new ClusteringCommitEvent(instantTime, taskID)), + "Execute clustering for instant %s from task %d", instantTime, taskID); + } else { + // executes the clustering task synchronously for batch mode. + LOG.info("Execute clustering for instant {} from task {}", instantTime, taskID); + doClustering(instantTime, event); + } + } + + @Override + public void close() throws Exception { + if (null != this.executor) { + this.executor.close(); + } + if (this.writeClient != null) { + this.writeClient.close(); + this.writeClient = null; + } + if (this.clusteringMetrics != null) { + this.clusteringMetrics.shutDown(); + } + if (iterator != null) { + iterator.close(); + iterator = null; + } + FsCacheCleanUtil.cleanChubaoFsCacheIfNecessary(); + } + + /** + * End input action for batch source. + */ + public void endInput() { + FsCacheCleanUtil.cleanChubaoFsCacheIfNecessary(); + } + + private void doClustering(String instantTime, ClusteringPlanEvent event) throws Exception { + final ClusteringGroupInfo clusteringGroupInfo = event.getClusteringGroupInfo(); + Map meta = clusteringGroupInfo.getExtraMeta(); + String outputLevel = meta.get(LSM_CLUSTERING_OUT_PUT_LEVEL); + String partitionPath = clusteringGroupInfo.getOperations().get(0).getPartitionPath(); + String fileID = FSUtils.getFileIdFromLogPath(new Path(clusteringGroupInfo.getOperations().get(0).getDataFilePath())); + + HoodieFlinkTable table = writeClient.getHoodieTable(); + LSMHoodieRowDataCreateHandle rowCreateHandle = new LSMHoodieRowDataCreateHandle(this.writeClient.getHoodieTable(), + this.writeClient.getConfig(), partitionPath, fileID, instantTime, + getRuntimeContext().getNumberOfParallelSubtasks(), taskID, getRuntimeContext().getAttemptNumber(), + rowType, true, Integer.parseInt(outputLevel), 0); + + clusteringMetrics.startClustering(); + + List clusteringOps = clusteringGroupInfo.getOperations(); + + // 记录Level 1文件的下标位置, 防止后续Level 1文件被溢写到磁盘 + int level1Index = -1; + for (int i = 0; i < clusteringOps.size(); i++) { + int level = FSUtils.getLevelNumFromLog(new Path(clusteringOps.get(i).getDataFilePath())); + if (level == 1) { + level1Index = i; + break; + } + } + + List> iterators = clusteringOps.stream().map(ClusteringOperation::getDataFilePath).map(path -> { + try { + return FlinkLsmUtils.getBaseFileIterator(path, FlinkLsmUtils.getLsmRequiredPositions(schema, schema), + rowType.getFieldNames(), rowDataType.getChildren(), conf.getString(FlinkOptions.PARTITION_DEFAULT_NAME), conf, + table.getHadoopConf(), table.getMetaClient().getTableConfig(), InternalSchemaManager.get(conf, table.getMetaClient()), + new ArrayList<>()); + } catch (IOException ioe) { + throw new HoodieIOException(ioe.getMessage(), ioe); + } + }).collect(Collectors.toList()); + + List> readers = FlinkLsmUtils.createLsmRecordReaders(iterators, spillTreshold, new RowDataSerializer(rowType), + getContainingTask().getEnvironment().getIOManager(), getContainingTask().getEnvironment().getMemoryManager().getPageSize(), + FlinkLsmUtils.getRecordKeyIndex(schema), level1Index); + + // clean spilled iterator by gc + iterators.clear(); + + boolean isIgnoreDelete = "1".equals(outputLevel); + iterator = new LsmMergeIterator(isIgnoreDelete, conf, writeConfig.getPayloadConfig().getProps(), + schema, rowType, table.getHadoopConf(), requiredPos, writeConfig.getBasePath(), readers, true); + + try { + while (iterator.hasNext()) { + RowData record = iterator.next(); + rowCreateHandle.write(record.getString(HoodieRecord.RECORD_KEY_META_FIELD_ORD).toString(), partitionPath, record, false); + } + + clusteringMetrics.endClustering(); + List writeStatuses = Collections.singletonList(rowCreateHandle.close().toWriteStatus()); + collector.collect(new ClusteringCommitEvent(instantTime, writeStatuses, this.taskID)); + } finally { + iterator.close(); + iterator = null; + } + } + + private int[] getRequiredPositions() { + final List fieldNames = readerSchema.getFields().stream().map(Schema.Field::name).collect(Collectors.toList()); + return schema.getFields().stream() + .map(field -> fieldNames.indexOf(field.name())) + .mapToInt(i -> i) + .toArray(); + } + + @VisibleForTesting + public void setExecutor(NonThrownExecutor executor) { + this.executor = executor; + } + + @VisibleForTesting + public void setOutput(Output> output) { + this.output = output; + } + + private void registerMetrics() { + this.clusteringMetrics = new FlinkClusteringMetrics(FlinkWriteClients.getHoodieClientConfig(this.conf)); + this.clusteringMetrics.registerMetrics(FlinkClusteringMetricEnum.notClusteringPlanOperator.value, FlinkClusteringMetricEnum.isClusteringOperator.value); + } + +} diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/LSMClusteringScheduleMode.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/LSMClusteringScheduleMode.java new file mode 100644 index 0000000000000..b71c8ae321d6a --- /dev/null +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/LSMClusteringScheduleMode.java @@ -0,0 +1,25 @@ +/* + * 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.clustering; + +public enum LSMClusteringScheduleMode { + STREAM, + ASYNC, + NONE +} \ No newline at end of file diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/utils/BufferUtils.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/utils/BufferUtils.java new file mode 100644 index 0000000000000..13c1df4edc7d6 --- /dev/null +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/utils/BufferUtils.java @@ -0,0 +1,79 @@ +/* + * 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.utils; + +import org.apache.hudi.sink.bulk.sort.SortOperatorGen; +import org.apache.hudi.sink.exception.MemoryPagesExhaustedException; + +import org.apache.flink.runtime.operators.sort.QuickSort; +import org.apache.flink.table.api.TableConfig; +import org.apache.flink.table.planner.codegen.sort.SortCodeGenerator; +import org.apache.flink.table.runtime.generated.NormalizedKeyComputer; +import org.apache.flink.table.runtime.generated.RecordComparator; +import org.apache.flink.table.runtime.operators.sort.BinaryInMemorySortBuffer; +import org.apache.flink.table.runtime.typeutils.BinaryRowDataSerializer; +import org.apache.flink.table.runtime.typeutils.RowDataSerializer; +import org.apache.flink.table.runtime.util.MemorySegmentPool; +import org.apache.flink.table.types.logical.RowType; + +/** + * Utilities to create binary buffer for writing functions. + */ +public class BufferUtils { + // minimum pages for a BinaryInMemorySortBuffer + private static final int MIN_REQUIRED_BUFFERS = 3; + + public static BinaryInMemorySortBuffer createBuffer(RowType rowType, + MemorySegmentPool memorySegmentPool, + ClassLoader cl, + String[] sortColumns) { + return createBuffer(rowType, memorySegmentPool, cl, sortColumns, new TableConfig()); + } + + public static BinaryInMemorySortBuffer createBuffer(RowType rowType, + MemorySegmentPool memorySegmentPool, + ClassLoader cl, + String[] sortColumns, + TableConfig tableConfig) { + if (memorySegmentPool.freePages() < MIN_REQUIRED_BUFFERS) { + // there is no enough free pages to create a binary buffer, may need flush first. + throw new MemoryPagesExhaustedException("Free pages are not enough to create a BinaryInMemorySortBuffer."); + } + // sort records based on record keys + NormalizedKeyComputer keyComputer = createSortCodeGenerator(rowType, sortColumns, tableConfig) + .generateNormalizedKeyComputer("SortComputer").newInstance(cl); + RecordComparator recordComparator = createSortCodeGenerator(rowType, sortColumns, tableConfig) + .generateRecordComparator("SortComparator").newInstance(cl); + return BinaryInMemorySortBuffer.createBuffer( + keyComputer, + new RowDataSerializer(rowType), + new BinaryRowDataSerializer(rowType.getFieldCount()), + recordComparator, + memorySegmentPool); + } + + private static SortCodeGenerator createSortCodeGenerator(RowType rowType, String[] sortColumns, TableConfig tableConfig) { + SortOperatorGen sortOperatorGen = new SortOperatorGen(rowType, sortColumns); + return sortOperatorGen.createSortCodeGenerator(tableConfig); + } + + public static void sort(BinaryInMemorySortBuffer dataBuffer) { + new QuickSort().sort(dataBuffer); + } +} \ No newline at end of file diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/utils/NaturalOrderKeyComputer.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/utils/NaturalOrderKeyComputer.java new file mode 100644 index 0000000000000..32b7af13fa3c0 --- /dev/null +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/utils/NaturalOrderKeyComputer.java @@ -0,0 +1,60 @@ +/* + * 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.utils; + +import org.apache.flink.core.memory.MemorySegment; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.runtime.generated.NormalizedKeyComputer; +import org.apache.flink.table.runtime.operators.sort.SortUtil; + +/** + * An implementation of @{code NormalizedKeyComputer} for {@code BinaryInMemorySortBuffer} in case sort is not needed, + * and the returned iterator follows natural order as inserted. + */ +public class NaturalOrderKeyComputer implements NormalizedKeyComputer { + @Override + public void putKey(RowData rowData, MemorySegment target, int offset) { + SortUtil.minNormalizedKey(target, offset, 1); + } + + @Override + public int compareKey(MemorySegment memorySegment, int i, MemorySegment target, int offset) { + return 0; + } + + @Override + public void swapKey(MemorySegment seg1, int index1, MemorySegment seg2, int index2) { + // do nothing. + } + + @Override + public int getNumKeyBytes() { + return 1; + } + + @Override + public boolean isKeyFullyDetermines() { + return true; + } + + @Override + public boolean invertKey() { + return false; + } +} diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/utils/NaturalOrderRecordComparator.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/utils/NaturalOrderRecordComparator.java new file mode 100644 index 0000000000000..9cd2807d096f3 --- /dev/null +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/utils/NaturalOrderRecordComparator.java @@ -0,0 +1,33 @@ +/* + * 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.utils; + +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.runtime.generated.RecordComparator; + +/** + * An implementation of @{code RecordComparator} for {@code BinaryInMemorySortBuffer} in case sort is not needed, + * and the returned iterator follows natural order as inserted. + */ +public class NaturalOrderRecordComparator implements RecordComparator { + @Override + public int compare(RowData rowData, RowData rowData1) { + return 0; + } +} diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/HoodieTableFactory.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/HoodieTableFactory.java index 6d9f666157942..85585722e55e9 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/HoodieTableFactory.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/HoodieTableFactory.java @@ -19,24 +19,34 @@ package org.apache.hudi.table; import org.apache.hudi.avro.AvroSchemaUtils; -import org.apache.hudi.common.model.DefaultHoodieRecordPayload; +import org.apache.hudi.client.model.ModelUtils; +import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy; +import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.table.HoodieTableConfig; +import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.StringUtils; +import org.apache.hudi.common.util.ValidationUtils; +import org.apache.hudi.config.HoodieClusteringConfig; +import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.configuration.FlinkOptions; import org.apache.hudi.configuration.HadoopConfigurations; import org.apache.hudi.configuration.OptionsResolver; import org.apache.hudi.exception.HoodieValidationException; import org.apache.hudi.index.HoodieIndex; -import org.apache.hudi.keygen.ComplexAvroKeyGenerator; +import org.apache.hudi.index.bucket.expression.utils.ExpressionUtils; import org.apache.hudi.keygen.NonpartitionedAvroKeyGenerator; import org.apache.hudi.keygen.TimestampBasedAvroKeyGenerator; import org.apache.hudi.keygen.constant.KeyGeneratorOptions; +import org.apache.hudi.metrics.FlinkEnvironmentContext; +import org.apache.hudi.sink.clustering.LSMClusteringScheduleMode; +import org.apache.hudi.table.format.mor.lsm.FlinkLsmUtils; import org.apache.hudi.util.AvroSchemaConverter; import org.apache.hudi.util.DataTypeUtils; import org.apache.hudi.util.StreamerUtil; import org.apache.flink.configuration.ConfigOption; import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.PipelineOptions; import org.apache.flink.table.api.ValidationException; import org.apache.flink.table.api.constraints.UniqueConstraint; import org.apache.flink.table.catalog.CatalogTable; @@ -55,11 +65,15 @@ import java.util.Arrays; import java.util.Collections; import java.util.List; +import java.util.Map; +import java.util.Properties; import java.util.Set; import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; import static org.apache.hudi.common.util.ValidationUtils.checkArgument; +import static org.apache.hudi.config.HoodieCleanConfig.FAILED_WRITES_CLEANER_POLICY; +import static org.apache.hudi.config.HoodieIndexConfig.BUCKET_INDEX_PARTITION_BUCKET_EXPR_PREFIX; /** * Hoodie data source/sink factory. @@ -72,11 +86,14 @@ public class HoodieTableFactory implements DynamicTableSourceFactory, DynamicTab @Override public DynamicTableSource createDynamicTableSource(Context context) { Configuration conf = FlinkOptions.fromMap(context.getCatalogTable().getOptions()); + String jobName = context.getConfiguration().get(PipelineOptions.NAME); Path path = new Path(conf.getOptional(FlinkOptions.PATH).orElseThrow(() -> new ValidationException("Option [path] should not be empty."))); setupTableOptions(conf.getString(FlinkOptions.PATH), conf); ResolvedSchema schema = context.getCatalogTable().getResolvedSchema(); + setupBaizeOptions(jobName, conf); setupConfOptions(conf, context.getObjectIdentifier(), context.getCatalogTable(), schema); + FlinkEnvironmentContext.init(); return new HoodieTableSource( schema, path, @@ -85,37 +102,108 @@ public DynamicTableSource createDynamicTableSource(Context context) { conf); } + private void setupBaizeOptions(String jobName, Configuration conf) { + if (!StringUtils.isNullOrEmpty(jobName)) { + conf.setString("hoodie.metrics.pushgateway.job.name", jobName); + conf.setBoolean("hoodie.metrics.on", conf.getBoolean(FlinkOptions.METRIC_ON)); + conf.setString("hoodie.metrics.reporter.type",conf.getString(FlinkOptions.REPORT_TYPE)); + } + } + @Override public DynamicTableSink createDynamicTableSink(Context context) { Configuration conf = FlinkOptions.fromMap(context.getCatalogTable().getOptions()); + String jobName = context.getConfiguration().get(PipelineOptions.NAME); checkArgument(!StringUtils.isNullOrEmpty(conf.getString(FlinkOptions.PATH)), "Option [path] should not be empty."); setupTableOptions(conf.getString(FlinkOptions.PATH), conf); + setupLSMWriteConfig(conf); ResolvedSchema schema = context.getCatalogTable().getResolvedSchema(); sanityCheck(conf, schema); + setupBaizeOptions(jobName, conf); setupConfOptions(conf, context.getObjectIdentifier(), context.getCatalogTable(), schema); + validateLSMClusteringConfig(conf); + FlinkEnvironmentContext.init(); return new HoodieTableSink(conf, schema); } + private void setupLSMWriteConfig(Configuration conf) { + if (OptionsResolver.isLSMBasedLogFormat(conf)) { + FlinkLsmUtils.setupLSMConfig(conf); + } + } + + private void validateLSMClusteringConfig(Configuration conf) { + Map confMap = conf.toMap(); + if (!confMap.containsKey(HoodieTableConfig.HOODIE_LOG_FORMAT.key())) { + return; + } + if (!confMap.get(HoodieTableConfig.HOODIE_LOG_FORMAT.key()).equalsIgnoreCase(HoodieTableConfig.LSM_HOODIE_TABLE_LOG_FORMAT)) { + return; + } + boolean inlineLSMClustering = Boolean.parseBoolean(confMap.get(HoodieClusteringConfig.LSM_INLINE_CLUSTERING.key())); + boolean scheduleInlineLSMClustering = Boolean.parseBoolean(confMap.get(HoodieClusteringConfig.LSM_SCHEDULE_INLINE_CLUSTERING.key())); + boolean asyncLSMClustering = conf.getBoolean(FlinkOptions.LSM_CLUSTERING_ASYNC_ENABLED); + String scheduleLSMClusteringMod = conf.getString(FlinkOptions.LSM_CLUSTERING_SCHEDULE_MODE); + boolean streamOrAsyncScheduleClustering = scheduleLSMClusteringMod.equalsIgnoreCase(LSMClusteringScheduleMode.STREAM.name()) + || scheduleLSMClusteringMod.equalsIgnoreCase(LSMClusteringScheduleMode.ASYNC.name()); + + // Can't turn on both async clustering and inline clustering in Flink job. + ValidationUtils.checkArgument(!(inlineLSMClustering && asyncLSMClustering), String.format("Either of inline clustering (%s) or " + + "async clustering (%s) can be enabled. Both can't be set to true at the same time in flink job. %s,%s", HoodieClusteringConfig.LSM_INLINE_CLUSTERING.key(), + FlinkOptions.LSM_CLUSTERING_ASYNC_ENABLED.key(), inlineLSMClustering, asyncLSMClustering)); + + // Can't turn on inline schedule clustering when the mod of lsm schedule clustering is async/sync in Flink job. + ValidationUtils.checkArgument(!(streamOrAsyncScheduleClustering && (inlineLSMClustering || scheduleInlineLSMClustering)), + String.format("Either of inline schedule clustering (%s/%s) or async/sync schedule clustering (%s) can be enabled. Both can't be set to true at the same time in flink job. %s,%s,%s", + HoodieClusteringConfig.LSM_INLINE_CLUSTERING.key(), HoodieClusteringConfig.LSM_SCHEDULE_INLINE_CLUSTERING.key(), + FlinkOptions.LSM_CLUSTERING_SCHEDULE_MODE.key(), inlineLSMClustering, scheduleInlineLSMClustering, scheduleLSMClusteringMod)); + } + /** * Supplement the table config options if not specified. */ private void setupTableOptions(String basePath, Configuration conf) { - StreamerUtil.getTableConfig(basePath, HadoopConfigurations.getHadoopConf(conf)) - .ifPresent(tableConfig -> { - if (tableConfig.contains(HoodieTableConfig.RECORDKEY_FIELDS) - && !conf.contains(FlinkOptions.RECORD_KEY_FIELD)) { - conf.setString(FlinkOptions.RECORD_KEY_FIELD, tableConfig.getString(HoodieTableConfig.RECORDKEY_FIELDS)); - } - if (tableConfig.contains(HoodieTableConfig.PRECOMBINE_FIELD) - && !conf.contains(FlinkOptions.PRECOMBINE_FIELD)) { - conf.setString(FlinkOptions.PRECOMBINE_FIELD, tableConfig.getString(HoodieTableConfig.PRECOMBINE_FIELD)); - } - if (tableConfig.contains(HoodieTableConfig.HIVE_STYLE_PARTITIONING_ENABLE) - && !conf.contains(FlinkOptions.HIVE_STYLE_PARTITIONING)) { - conf.setBoolean(FlinkOptions.HIVE_STYLE_PARTITIONING, tableConfig.getBoolean(HoodieTableConfig.HIVE_STYLE_PARTITIONING_ENABLE)); - } - }); + Option option = StreamerUtil.getTableConfig(basePath, HadoopConfigurations.getHadoopConf(conf)); + option.ifPresent(tableConfig -> { + if (tableConfig.contains(HoodieTableConfig.RECORDKEY_FIELDS) + && !conf.contains(FlinkOptions.RECORD_KEY_FIELD)) { + conf.setString(FlinkOptions.RECORD_KEY_FIELD, tableConfig.getString(HoodieTableConfig.RECORDKEY_FIELDS)); + } + if (tableConfig.contains(HoodieTableConfig.PRECOMBINE_FIELD) + && !conf.contains(FlinkOptions.PRECOMBINE_FIELD)) { + conf.setString(FlinkOptions.PRECOMBINE_FIELD, tableConfig.getString(HoodieTableConfig.PRECOMBINE_FIELD)); + } + if (tableConfig.contains(HoodieTableConfig.HIVE_STYLE_PARTITIONING_ENABLE) + && !conf.contains(FlinkOptions.HIVE_STYLE_PARTITIONING)) { + conf.setBoolean(FlinkOptions.HIVE_STYLE_PARTITIONING, tableConfig.getBoolean(HoodieTableConfig.HIVE_STYLE_PARTITIONING_ENABLE)); + } + if (tableConfig.contains(HoodieTableConfig.TYPE) && conf.contains(FlinkOptions.TABLE_TYPE)) { + if (!tableConfig.getString(HoodieTableConfig.TYPE).equals(conf.get(FlinkOptions.TABLE_TYPE))) { + LOG.warn( + String.format("Table type conflict : %s in %s and %s in table options. Fix the table type as to be in line with the hoodie.properties.", + tableConfig.getString(HoodieTableConfig.TYPE), HoodieTableConfig.HOODIE_PROPERTIES_FILE, + conf.get(FlinkOptions.TABLE_TYPE))); + conf.setString(FlinkOptions.TABLE_TYPE, tableConfig.getString(HoodieTableConfig.TYPE)); + } + } + if (tableConfig.contains(HoodieTableConfig.TYPE) + && !conf.contains(FlinkOptions.TABLE_TYPE)) { + conf.setString(FlinkOptions.TABLE_TYPE, tableConfig.getString(HoodieTableConfig.TYPE)); + } + if (tableConfig.isLSMBasedLogFormat() && tableConfig.contains(HoodieTableConfig.RECORD_MERGER_STRATEGY) + && !conf.contains(FlinkOptions.TABLE_RECORD_MERGER_STRATEGY)) { + conf.set(FlinkOptions.TABLE_RECORD_MERGER_STRATEGY, tableConfig.getString(HoodieTableConfig.RECORD_MERGER_STRATEGY)); + conf.set(FlinkOptions.RECORD_MERGER_STRATEGY, tableConfig.getString(HoodieTableConfig.RECORD_MERGER_STRATEGY)); + // 覆盖RECORD_MERGER_IMPLS,此时用户指定的impls不会生效 + conf.set(FlinkOptions.RECORD_MERGER_IMPLS, ModelUtils.getSupportedRecordMerger()); + } + }); + + // HoodieTable 表不存在 + if (!option.isPresent() && OptionsResolver.isLSMBasedLogFormat(conf)) { + FlinkLsmUtils.setupMergerConfig(conf); + } } @Override @@ -144,9 +232,26 @@ public Set> optionalOptions() { * @param schema The table schema */ private void sanityCheck(Configuration conf, ResolvedSchema schema) { + checkTableType(conf); if (!OptionsResolver.isAppendMode(conf)) { checkRecordKey(conf, schema); - checkPreCombineKey(conf, schema); + } + StreamerUtil.checkPreCombineKey(conf, schema.getColumnNames()); + StreamerUtil.checkBasePathAndStoragePath(conf); + } + + /** + * Validate the table type. + */ + private void checkTableType(Configuration conf) { + String tableType = conf.get(FlinkOptions.TABLE_TYPE); + if (StringUtils.nonEmpty(tableType)) { + try { + HoodieTableType.valueOf(tableType); + } catch (IllegalArgumentException e) { + throw new HoodieValidationException("Invalid table type: " + tableType + ". Table type should be either " + + HoodieTableType.MERGE_ON_READ + " or " + HoodieTableType.COPY_ON_WRITE + "."); + } } } @@ -174,26 +279,6 @@ private void checkRecordKey(Configuration conf, ResolvedSchema schema) { } } - /** - * Validate pre_combine key. - */ - private void checkPreCombineKey(Configuration conf, ResolvedSchema schema) { - List fields = schema.getColumnNames(); - String preCombineField = conf.get(FlinkOptions.PRECOMBINE_FIELD); - if (!fields.contains(preCombineField)) { - if (OptionsResolver.isDefaultHoodieRecordPayloadClazz(conf)) { - throw new HoodieValidationException("Option '" + FlinkOptions.PRECOMBINE_FIELD.key() - + "' is required for payload class: " + DefaultHoodieRecordPayload.class.getName()); - } - if (preCombineField.equals(FlinkOptions.PRECOMBINE_FIELD.defaultValue())) { - conf.setString(FlinkOptions.PRECOMBINE_FIELD, FlinkOptions.NO_PRE_COMBINE); - } else if (!preCombineField.equals(FlinkOptions.NO_PRE_COMBINE)) { - throw new HoodieValidationException("Field " + preCombineField + " does not exist in the table schema." - + "Please check '" + FlinkOptions.PRECOMBINE_FIELD.key() + "' option."); - } - } - } - /** * Sets up the config options based on the table definition, for e.g, the table name, primary key. * @@ -221,6 +306,23 @@ private static void setupConfOptions( setupWriteOptions(conf); // infer avro schema from physical DDL schema inferAvroSchema(conf, schema.toPhysicalRowDataType().notNull().getLogicalType()); + // 兼容base path + conf.setString(HoodieWriteConfig.BASE_PATH.key(), conf.getString(FlinkOptions.PATH)); + // set partition bucket expression + if (conf.getString(FlinkOptions.INDEX_TYPE).equals(HoodieIndex.IndexType.BUCKET.name()) + && conf.getBoolean(FlinkOptions.BUCKET_INDEX_PARTITION_LEVEL)) { + Properties props = new Properties(); + for (String key : table.getOptions().keySet()) { + if (key.startsWith(BUCKET_INDEX_PARTITION_BUCKET_EXPR_PREFIX)) { + props.setProperty(key, table.getOptions().get(key)); + } + } + conf.setString(FlinkOptions.BUCKET_INDEX_PARTITION_BUCKET_EXPR, ExpressionUtils.generateBucketExpression(props)); + } + // async rollback + if (conf.getBoolean(FlinkOptions.ROLLBACK_ASYNC_ENABLE)) { + conf.setString(FAILED_WRITES_CLEANER_POLICY.key(), HoodieFailedWritesCleaningPolicy.LAZY.name()); + } } /** @@ -275,11 +377,7 @@ private static void setupHoodieKeyOptions(Configuration conf, CatalogTable table } } boolean complexHoodieKey = pks.length > 1 || partitions.length > 1; - if (complexHoodieKey && FlinkOptions.isDefaultValueDefined(conf, FlinkOptions.KEYGEN_CLASS_NAME)) { - conf.setString(FlinkOptions.KEYGEN_CLASS_NAME, ComplexAvroKeyGenerator.class.getName()); - LOG.info("Table option [{}] is reset to {} because record key or partition path has two or more fields", - FlinkOptions.KEYGEN_CLASS_NAME.key(), ComplexAvroKeyGenerator.class.getName()); - } + StreamerUtil.checkKeygenGenerator(complexHoodieKey, conf); } /** diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/HoodieTableSink.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/HoodieTableSink.java index f8799d3ac940a..ba3aa903eca73 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/HoodieTableSink.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/HoodieTableSink.java @@ -24,6 +24,7 @@ import org.apache.hudi.configuration.FlinkOptions; import org.apache.hudi.configuration.OptionsInference; import org.apache.hudi.configuration.OptionsResolver; +import org.apache.hudi.sink.clustering.LSMClusteringScheduleMode; import org.apache.hudi.sink.utils.Pipelines; import org.apache.hudi.util.ChangelogModes; @@ -35,6 +36,7 @@ import org.apache.flink.table.connector.sink.DynamicTableSink; import org.apache.flink.table.connector.sink.abilities.SupportsOverwrite; import org.apache.flink.table.connector.sink.abilities.SupportsPartitioning; +import org.apache.flink.table.data.RowData; import org.apache.flink.table.types.logical.RowType; import java.util.Map; @@ -67,6 +69,11 @@ public SinkRuntimeProvider getSinkRuntimeProvider(Context context) { long ckpTimeout = dataStream.getExecutionEnvironment() .getCheckpointConfig().getCheckpointTimeout(); conf.setLong(FlinkOptions.WRITE_COMMIT_ACK_TIMEOUT, ckpTimeout); + + if (conf.get(FlinkOptions.LSM_CLUSTERING_SCHEDULE_INTERVAL) == -1L) { + conf.set(FlinkOptions.LSM_CLUSTERING_SCHEDULE_INTERVAL, dataStream.getExecutionEnvironment().getCheckpointConfig() + .getCheckpointInterval() / 1000 * conf.getInteger(FlinkOptions.LSM_CLUSTERING_DELTA_COMMITS)); + } // set up default parallelism OptionsInference.setupSinkTasks(conf, dataStream.getExecutionConfig().getParallelism()); @@ -81,6 +88,13 @@ public SinkRuntimeProvider getSinkRuntimeProvider(Context context) { // Append mode if (OptionsResolver.isAppendMode(conf)) { DataStream pipeline = Pipelines.append(conf, rowType, dataStream, context.isBounded()); + if (OptionsResolver.isAsyncRollback(conf)) { + pipeline = Pipelines.rollback(conf, pipeline); + } + // async archive + if (OptionsResolver.needsAsyncArchive(conf)) { + pipeline = Pipelines.archive(conf, pipeline); + } if (OptionsResolver.needsAsyncClustering(conf)) { return Pipelines.cluster(conf, rowType, pipeline); } else { @@ -89,18 +103,60 @@ public SinkRuntimeProvider getSinkRuntimeProvider(Context context) { } DataStream pipeline; - // bootstrap - final DataStream hoodieRecordDataStream = - Pipelines.bootstrap(conf, rowType, dataStream, context.isBounded(), overwrite); - // write pipeline - pipeline = Pipelines.hoodieStreamWrite(conf, hoodieRecordDataStream); + // Bucket Index and incremental rt chain + DataStream finalDataStream; + String rtChainMode = conf.get(FlinkOptions.RT_CHAIN_MODE); + if (rtChainMode == null) { + rtChainMode = dataStream.getExecutionConfig().getGlobalJobParameters().toMap().get(FlinkOptions.RT_CHAIN_MODE.key()); + } + if (rtChainMode != null && rtChainMode.equalsIgnoreCase("incremental")) { + finalDataStream = Pipelines.backFillDeletedRecordsInRTIncrementalChain(dataStream, conf, rowType); + } else { + finalDataStream = dataStream; + } + + if (OptionsResolver.isLSMBasedLogFormat(conf)) { + // in LSM Based format, skip bootstrap and do parquet create directly + pipeline = Pipelines.hoodieLsmStreamWrite(conf, rowType, dataStream); + if (OptionsResolver.areTableServicesEnabled(conf)) { + if (LSMClusteringScheduleMode.STREAM.name().equalsIgnoreCase(conf.getString(FlinkOptions.LSM_CLUSTERING_SCHEDULE_MODE))) { + pipeline = Pipelines.lsmClusterPlan(conf, pipeline, rowType); + } + + if (OptionsResolver.needsAsyncClustering(conf)) { + return Pipelines.lsmClusterExecute(conf, rowType, pipeline); + } else if (conf.getBoolean(FlinkOptions.CLEAN_ASYNC_ENABLED)) { + return Pipelines.clean(conf, pipeline); + } + } else { + return Pipelines.dummySink(pipeline); + } + } else { + // bootstrap + final DataStream hoodieRecordDataStream = + Pipelines.bootstrap(conf, rowType, finalDataStream, context.isBounded(), overwrite); + // write pipeline + pipeline = Pipelines.hoodieStreamWrite(conf, hoodieRecordDataStream); + } + // async archive + if (OptionsResolver.needsAsyncArchive(conf)) { + pipeline = Pipelines.archive(conf, pipeline); + } + + if (OptionsResolver.isAsyncRollback(conf)) { + pipeline = Pipelines.rollback(conf, pipeline); + } // compaction if (OptionsResolver.needsAsyncCompaction(conf)) { // use synchronous compaction for bounded source. if (context.isBounded()) { conf.setBoolean(FlinkOptions.COMPACTION_ASYNC_ENABLED, false); } - return Pipelines.compact(conf, pipeline); + if (conf.get(FlinkOptions.COMPACTION_SCANNER_TYPE).equals(FlinkOptions.SCANNER_WITH_FK)) { + return Pipelines.compactWithFK(conf, pipeline); + } else { + return Pipelines.compact(conf, pipeline); + } } else { return Pipelines.clean(conf, pipeline); } diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/mor/MergeOnReadInputFormat.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/mor/MergeOnReadInputFormat.java index 0f181bbd644c6..61184a7a52933 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/mor/MergeOnReadInputFormat.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/mor/MergeOnReadInputFormat.java @@ -18,18 +18,24 @@ package org.apache.hudi.table.format.mor; +import org.apache.flink.runtime.io.disk.iomanager.IOManager; +import org.apache.hadoop.fs.Path; +import org.apache.hudi.avro.AvroSchemaCache; import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.engine.EngineType; +import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodieAvroIndexedRecord; import org.apache.hudi.common.model.HoodieAvroRecord; import org.apache.hudi.common.model.HoodieOperation; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordMerger; +import org.apache.hudi.common.table.HoodieTableConfig; import org.apache.hudi.common.table.log.HoodieMergedLogRecordScanner; import org.apache.hudi.common.table.log.InstantRange; import org.apache.hudi.common.util.ClosableIterator; import org.apache.hudi.common.util.HoodieRecordUtils; 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.configuration.FlinkOptions; import org.apache.hudi.configuration.HadoopConfigurations; @@ -37,11 +43,17 @@ import org.apache.hudi.exception.HoodieException; import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.internal.schema.InternalSchema; +import org.apache.hudi.io.lsm.RecordReader; import org.apache.hudi.keygen.KeyGenUtils; +import org.apache.hudi.metrics.CollectStatistic; +import org.apache.hudi.source.ExpressionPredicates; import org.apache.hudi.table.format.FilePathUtils; import org.apache.hudi.table.format.FormatUtils; import org.apache.hudi.table.format.InternalSchemaManager; import org.apache.hudi.table.format.RecordIterators; +import org.apache.hudi.table.format.mor.lsm.LsmMergeIterator; +import org.apache.hudi.table.format.mor.lsm.FlinkLsmUtils; +import org.apache.hudi.util.AvroSchemaConverter; import org.apache.hudi.util.AvroToRowDataConverters; import org.apache.hudi.util.DataTypeUtils; import org.apache.hudi.util.RowDataProjection; @@ -61,20 +73,26 @@ import org.apache.flink.core.io.InputSplitAssigner; import org.apache.flink.table.data.GenericRowData; import org.apache.flink.table.data.RowData; +import org.apache.flink.table.expressions.CallExpression; +import org.apache.flink.table.runtime.typeutils.RowDataSerializer; import org.apache.flink.table.types.DataType; import org.apache.flink.table.types.logical.LogicalType; import org.apache.flink.table.types.logical.RowType; import org.apache.flink.types.RowKind; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.io.IOException; +import java.util.ArrayList; import java.util.Arrays; import java.util.stream.Collectors; import java.util.HashSet; import java.util.Iterator; import java.util.LinkedHashMap; import java.util.List; -import java.util.Properties; import java.util.Set; +import java.util.Map; +import java.util.HashMap; import java.util.function.Function; import java.util.stream.IntStream; @@ -93,6 +111,8 @@ public class MergeOnReadInputFormat private static final long serialVersionUID = 1L; + private static final Logger LOG = LoggerFactory.getLogger(MergeOnReadInputFormat.class); + protected final Configuration conf; protected transient org.apache.hadoop.conf.Configuration hadoopConf; @@ -123,7 +143,11 @@ public class MergeOnReadInputFormat /** * Required field positions. */ - private final int[] requiredPos; + protected final int[] requiredPos; + + private List predicates = new ArrayList<>(); + + private Option commitTimePredicate; // for limit push down /** @@ -140,20 +164,26 @@ public class MergeOnReadInputFormat * Flag saying whether to emit the deletes. In streaming read mode, downstream * operators need the DELETE messages to retract the legacy accumulator. */ - private boolean emitDelete; + protected boolean emitDelete; /** * Flag saying whether the input format has been closed. */ private boolean closed = true; - private final InternalSchemaManager internalSchemaManager; + protected final InternalSchemaManager internalSchemaManager; + protected HoodieTableConfig tableConfig; + + protected IOManager ioManager; + protected int memorySegmentSize; + protected int spillTreshold; protected MergeOnReadInputFormat( Configuration conf, MergeOnReadTableState tableState, List fieldTypes, String defaultPartName, + List predicates, long limit, boolean emitDelete, InternalSchemaManager internalSchemaManager) { @@ -165,9 +195,13 @@ protected MergeOnReadInputFormat( // Needs improvement: this requiredPos is only suitable for parquet reader, // because we need to this.requiredPos = tableState.getRequiredPositions(); + if (predicates != null && predicates.size() != 0) { + this.predicates.addAll(predicates); + } this.limit = limit; this.emitDelete = emitDelete; this.internalSchemaManager = internalSchemaManager; + this.spillTreshold = conf.getInteger(FlinkOptions.LSM_SORT_MERGE_SPILL_THRESHOLD); } /** @@ -182,11 +216,71 @@ public void open(MergeOnReadInputSplit split) throws IOException { this.currentReadCount = 0L; this.closed = false; this.hadoopConf = HadoopConfigurations.getHadoopConf(this.conf); + this.tableConfig = split.getTableConfig(); this.iterator = initIterator(split); mayShiftInputSplit(split); } + public void setIOManager(IOManager ioManager) { + this.ioManager = ioManager; + } + + public void setMemorySegmentSize(int memorySegmentSize) { + this.memorySegmentSize = memorySegmentSize; + } + + public void setSpillTreshold(int spillTreshold) { + this.spillTreshold = spillTreshold; + } + protected ClosableIterator initIterator(MergeOnReadInputSplit split) throws IOException { + if (tableConfig.isLSMBasedLogFormat()) { + this.commitTimePredicate = extractCommitTimePredicate(split.getInstantRange()); + if (this.commitTimePredicate.isPresent()) { + addCommitTimePredicate(this.commitTimePredicate.get()); + } + TypedProperties payloadProps = StreamerUtil.getPayloadConfig(conf).getProps(); + Schema tableSchema = new Schema.Parser().parse(tableState.getAvroSchema()); + Schema requiredSchemaWithMeta = AvroSchemaCache.intern(this.tableState.getLsmReadSchema(payloadProps, tableSchema, conf)); + RowType rowTypeWithMeta = (RowType) AvroSchemaConverter.convertToDataType(requiredSchemaWithMeta).getLogicalType(); + RowDataSerializer rowDataSerializer = new RowDataSerializer(rowTypeWithMeta); + int recordKeyIndex = FlinkLsmUtils.getRecordKeyIndex(requiredSchemaWithMeta); + + List lsmLogPaths = split.getLogPaths().get(); + // 标记level 1文件的下标位置, level 1文件不spill disk + int level1Index = -1; + for (int i = 0; i < lsmLogPaths.size(); i++) { + int level = FSUtils.getLevelNumFromLog(new Path(lsmLogPaths.get(i))); + if (level == 1) { + level1Index = i; + break; + } + } + + List> iterators = lsmLogPaths.stream().map(path -> { + try { + return FlinkLsmUtils.getBaseFileIterator(path, FlinkLsmUtils.getLsmRequiredPositions(requiredSchemaWithMeta, tableSchema), + fieldNames, fieldTypes, defaultPartName, conf, hadoopConf, tableConfig, internalSchemaManager, predicates); + } catch (IOException ioe) { + throw new HoodieIOException(ioe.getMessage(), ioe); + } + }).collect(Collectors.toList()); + + List> readers = FlinkLsmUtils.createLsmRecordReaders(iterators, spillTreshold, rowDataSerializer, ioManager, + memorySegmentSize, recordKeyIndex, level1Index); + return new LsmMergeIterator( + true, + conf, + payloadProps, + requiredSchemaWithMeta, + this.tableState.getRequiredRowType(), + hadoopConf, + this.requiredPos, + split.getTablePath(), + readers, + false); + } + if (!(split.getLogPaths().isPresent() && split.getLogPaths().get().size() > 0)) { if (split.getInstantRange().isPresent()) { // base file only with commit time filtering @@ -210,7 +304,12 @@ protected ClosableIterator initIterator(MergeOnReadInputSplit split) th return new SkipMergeIterator( getBaseFileIterator(split.getBasePath().get()), getLogFileIterator(split)); + } else if (split.getMergeType().equals(FlinkOptions.REALTIME_SKIP_COMBINE)) { + return new SkipMergeIterator( + getBaseFileIterator(split.getBasePath().get()), + getUnMergedLogFileIterator(split)); } else if (split.getMergeType().equals(FlinkOptions.REALTIME_PAYLOAD_COMBINE)) { + // conf contains all the configs including reading runtime env return new MergeIterator( conf, hadoopConf, @@ -224,6 +323,20 @@ protected ClosableIterator initIterator(MergeOnReadInputSplit split) th this.emitDelete, this.tableState.getOperationPos(), getBaseFileIteratorWithMetadata(split.getBasePath().get())); + } else if (split.getMergeType().equals(FlinkOptions.HISTORICAL_PAYLOAD_COMBINE)) { + return new MergeHistoricalIterator( + conf, + hadoopConf, + split, + this.tableState.getRowType(), + this.tableState.getRequiredRowType(), + new Schema.Parser().parse(this.tableState.getAvroSchema()), + new Schema.Parser().parse(this.tableState.getRequiredAvroSchema()), + internalSchemaManager.getQuerySchema(), + this.requiredPos, + this.emitDelete, + this.tableState.getOperationPos(), + getBaseFileIteratorWithMetadata(split.getBasePath().get())); } else { throw new HoodieException("Unable to select an Iterator to read the Hoodie MOR File Split for " + "file path: " + split.getBasePath() @@ -234,6 +347,42 @@ protected ClosableIterator initIterator(MergeOnReadInputSplit split) th } } + private Option extractCommitTimePredicate(Option instantRange) { + if (instantRange.isPresent()) { + InstantRange range = instantRange.get(); + if (range instanceof InstantRange.CloseCloseRange) { + CallExpression expression = FlinkLsmUtils.buildCloseCloseRangeCommitExpression(range); + return Option.of(ExpressionPredicates.fromExpression(expression)); + } else if (range instanceof InstantRange.CloseCloseRangeNullableBoundary) { + CallExpression expression = FlinkLsmUtils.buildCloseCloseRangeNullableBoundaryCommitExpression(range); + return Option.of(ExpressionPredicates.fromExpression(expression)); + } else if (range instanceof InstantRange.OpenCloseRange) { + CallExpression expression = FlinkLsmUtils.buildOpenCloseRangeCommitExpression(range); + return Option.of(ExpressionPredicates.fromExpression(expression)); + } else if (range instanceof InstantRange.OpenCloseRangeNullableBoundary) { + CallExpression expression = FlinkLsmUtils.buildOpenCloseRangeNullableBoundaryCommitExpression(range); + return Option.of(ExpressionPredicates.fromExpression(expression)); + } + } + return Option.empty(); + } + + private void addCommitTimePredicate(ExpressionPredicates.Predicate predicate) { + this.predicates.add(predicate); + } + + private void removeCommitTimePredicate(ExpressionPredicates.Predicate predicate) { + this.predicates.remove(predicate); + } + + public Map getStatistic() { + if (this.iterator instanceof CollectStatistic) { + return ((CollectStatistic) this.iterator).getStatisticMetrics(); + } else { + return new HashMap<>(); + } + } + @Override public void configure(Configuration configuration) { // no operation @@ -277,6 +426,10 @@ public void close() throws IOException { if (this.iterator != null) { this.iterator.close(); } + if (this.commitTimePredicate != null && this.commitTimePredicate.isPresent()) { + removeCommitTimePredicate(this.commitTimePredicate.get()); + } + this.commitTimePredicate = null; this.iterator = null; this.closed = true; } @@ -289,6 +442,10 @@ public boolean isClosed() { // Utilities // ------------------------------------------------------------------------- + public Configuration getConfiguration() { + return conf; + } + /** * Shifts the input split by its consumed records number. * @@ -316,7 +473,7 @@ protected ClosableIterator getBaseFileIterator(String path) throws IOEx return getBaseFileIterator(path, this.requiredPos); } - private ClosableIterator getBaseFileIterator(String path, int[] requiredPos) throws IOException { + protected ClosableIterator getBaseFileIterator(String path, int[] requiredPos) throws IOException { // generate partition specs. LinkedHashMap partSpec = FilePathUtils.extractPartitionKeyValues( new org.apache.hadoop.fs.Path(path).getParent(), @@ -337,12 +494,13 @@ private ClosableIterator getBaseFileIterator(String path, int[] require partObjects.put(k, DataTypeUtils.resolvePartition(defaultPartName.equals(v) ? null : v, fieldType)); } }); - + org.apache.hadoop.conf.Configuration parquetConf = HadoopConfigurations.getParquetConf(this.conf, hadoopConf); + FSUtils.addChubaoFsConfig2HadoopConf(parquetConf, tableConfig); return RecordIterators.getParquetRecordIterator( internalSchemaManager, this.conf.getBoolean(FlinkOptions.UTC_TIMEZONE), true, - HadoopConfigurations.getParquetConf(this.conf, hadoopConf), + parquetConf, fieldNames.toArray(new String[0]), fieldTypes.toArray(new DataType[0]), partObjects, @@ -350,10 +508,11 @@ private ClosableIterator getBaseFileIterator(String path, int[] require 2048, new org.apache.flink.core.fs.Path(path), 0, - Long.MAX_VALUE); // read the whole file + Long.MAX_VALUE, // read the whole file + predicates); } - private ClosableIterator getLogFileIterator(MergeOnReadInputSplit split) { + protected ClosableIterator getLogFileIterator(MergeOnReadInputSplit split) { final Schema tableSchema = new Schema.Parser().parse(tableState.getAvroSchema()); final Schema requiredSchema = new Schema.Parser().parse(tableState.getRequiredAvroSchema()); final GenericRecordBuilder recordBuilder = new GenericRecordBuilder(requiredSchema); @@ -369,7 +528,25 @@ private ClosableIterator getLogFileIterator(MergeOnReadInputSplit split final LogicalType[] pkTypes = pkSemanticLost ? null : tableState.getPkTypes(pkOffset); final StringToRowDataConverter converter = pkSemanticLost ? null : new StringToRowDataConverter(pkTypes); - return new ClosableIterator() { + return new CollectStatistic() { + @Override + public Map getStatisticMetrics() { + Map statistic = new HashMap<>(); + String finalBaseCommitTime = scanner.getFinalBaseCommitTime(); + if (StringUtils.nonEmpty(finalBaseCommitTime)) { + String baseCommitTime = finalBaseCommitTime.length() > 14 ? finalBaseCommitTime.substring(0, 14) : finalBaseCommitTime; + statistic.put("finalBaseCommitTime", Long.valueOf(baseCommitTime)); + } + statistic.put("diffBetweenLogPathSizeAndScannedLogSize", + scanner.getDiffBetweenLogPathSizeAndScannedLogSize()); + statistic.put("totalLogFiles", scanner.getTotalLogFiles()); + statistic.put("totalRollbacks",scanner.getTotalRollbacks()); + statistic.put("totalCorruptBlocks", scanner.getTotalCorruptBlocks()); + statistic.put("totalLogBlocks", scanner.getTotalLogBlocks()); + statistic.put("totalLogRecords", scanner.getTotalLogRecords()); + return statistic; + } + private RowData currentRecord; @Override @@ -595,7 +772,7 @@ public void close() { } } - protected static class LogFileOnlyIterator implements ClosableIterator { + protected static class LogFileOnlyIterator implements CollectStatistic { // iterator for log files private final ClosableIterator iterator; @@ -619,6 +796,15 @@ public void close() { this.iterator.close(); } } + + @Override + public Map getStatisticMetrics() { + if (iterator instanceof CollectStatistic) { + return ((CollectStatistic) iterator).getStatisticMetrics(); + } else { + return new HashMap<>(); + } + } } static class SkipMergeIterator implements ClosableIterator { @@ -669,37 +855,37 @@ public void close() { } } - protected static class MergeIterator implements ClosableIterator { + protected static class MergeIterator implements CollectStatistic { // base file record iterator - private final ClosableIterator nested; + protected final ClosableIterator nested; // log keys used for merging - private final Iterator logKeysIterator; + protected final Iterator logKeysIterator; // scanner - private final HoodieMergedLogRecordScanner scanner; + protected final HoodieMergedLogRecordScanner scanner; private final Schema tableSchema; - private final boolean emitDelete; - private final int operationPos; + protected final boolean emitDelete; + protected final int operationPos; private final RowDataToAvroConverters.RowDataToAvroConverter rowDataToAvroConverter; - private final AvroToRowDataConverters.AvroToRowDataConverter avroToRowDataConverter; + protected final AvroToRowDataConverters.AvroToRowDataConverter avroToRowDataConverter; - private final Option projection; - private final Option> avroProjection; + protected final Option projection; + protected final Option> avroProjection; - private final InstantRange instantRange; + protected final InstantRange instantRange; private final HoodieRecordMerger recordMerger; // add the flag because the flink ParquetColumnarRowSplitReader is buggy: // method #reachedEnd() returns false after it returns true. // refactor it out once FLINK-22370 is resolved. - private boolean readLogs = false; + protected boolean readLogs = false; - private final Set keyToSkip = new HashSet<>(); + protected final Set keyToSkip = new HashSet<>(); private final TypedProperties payloadProps; - private RowData currentRecord; + protected RowData currentRecord; public MergeIterator( Configuration flinkConf, @@ -767,7 +953,7 @@ public boolean hasNext() { final String curKey = currentRecord.getString(HOODIE_RECORD_KEY_COL_POS).toString(); if (scanner.getRecords().containsKey(curKey)) { keyToSkip.add(curKey); - Option mergedAvroRecord = mergeRowWithLog(currentRecord, curKey); + Option> mergedAvroRecord = mergeRowWithLog(currentRecord, curKey); if (!mergedAvroRecord.isPresent()) { // deleted continue; @@ -811,7 +997,7 @@ public boolean hasNext() { return false; } - private Option getInsertValue(String curKey) { + protected Option getInsertValue(String curKey) { final HoodieAvroRecord record = (HoodieAvroRecord) scanner.getRecords().get(curKey); if (!emitDelete && HoodieOperation.isDelete(record.getOperation())) { return Option.empty(); @@ -838,17 +1024,116 @@ public void close() { } } - private Option mergeRowWithLog(RowData curRow, String curKey) { - final HoodieAvroRecord record = (HoodieAvroRecord) scanner.getRecords().get(curKey); + @SuppressWarnings("unchecked") + protected Option> mergeRowWithLog(RowData curRow, String curKey) { + final HoodieRecord record = scanner.getRecords().get(curKey); GenericRecord historyAvroRecord = (GenericRecord) rowDataToAvroConverter.convert(tableSchema, curRow); HoodieAvroIndexedRecord hoodieAvroIndexedRecord = new HoodieAvroIndexedRecord(historyAvroRecord); try { - Option resultRecord = recordMerger.merge(hoodieAvroIndexedRecord, tableSchema, record, tableSchema, payloadProps).map(Pair::getLeft); - return resultRecord.get().toIndexedRecord(tableSchema, new Properties()); + return recordMerger.merge(hoodieAvroIndexedRecord, tableSchema, record, tableSchema, payloadProps).map(Pair::getLeft); } catch (IOException e) { throw new HoodieIOException("Merge base and delta payloads exception", e); } } + + @Override + public Map getStatisticMetrics() { + Map statistic = new HashMap<>(); + String finalBaseCommitTime = scanner.getFinalBaseCommitTime(); + if (StringUtils.nonEmpty(finalBaseCommitTime)) { + String baseCommitTime = finalBaseCommitTime.length() > 14 ? finalBaseCommitTime.substring(0, 14) : finalBaseCommitTime; + statistic.put("finalBaseCommitTime", Long.valueOf(baseCommitTime)); + } + statistic.put("diffBetweenLogPathSizeAndScannedLogSize", scanner.getDiffBetweenLogPathSizeAndScannedLogSize()); + statistic.put("totalLogFiles", scanner.getTotalLogFiles()); + statistic.put("totalRollbacks",scanner.getTotalRollbacks()); + statistic.put("totalCorruptBlocks", scanner.getTotalCorruptBlocks()); + statistic.put("totalLogBlocks", scanner.getTotalLogBlocks()); + statistic.put("totalLogRecords", scanner.getTotalLogRecords()); + return statistic; + } + } + + protected static class MergeHistoricalIterator extends MergeIterator { + public MergeHistoricalIterator( + Configuration flinkConf, + org.apache.hadoop.conf.Configuration hadoopConf, + MergeOnReadInputSplit split, + RowType tableRowType, + RowType requiredRowType, + Schema tableSchema, + Schema requiredSchema, + InternalSchema querySchema, + int[] requiredPos, + boolean emitDelete, + int operationPos, + ClosableIterator nested) { // the iterator should be with full schema + super(flinkConf, hadoopConf, split, tableRowType, requiredRowType, tableSchema, + querySchema, + Option.of(RowDataProjection.instance(requiredRowType, requiredPos)), + Option.of(record -> buildAvroRecordBySchema(record, requiredSchema, requiredPos, new GenericRecordBuilder(requiredSchema))), + emitDelete, operationPos, nested); + } + + @Override + public boolean hasNext() { + while (!readLogs && this.nested.hasNext()) { + currentRecord = this.nested.next(); + final String curKey = currentRecord.getString(HOODIE_RECORD_KEY_COL_POS).toString(); + if (scanner.getRecords().containsKey(curKey)) { + keyToSkip.add(curKey); + Option> mergedAvroRecord = mergeRowWithLog(currentRecord, curKey); + if (!mergedAvroRecord.isPresent()) { + // deleted + continue; + } else { + final RowKind rowKind = FormatUtils.getRowKindSafely(mergedAvroRecord.get().getData(), this.operationPos); + if (!emitDelete && rowKind == RowKind.DELETE) { + // deleted + continue; + } + IndexedRecord avroRecord = avroProjection.isPresent() + ? avroProjection.get().apply(mergedAvroRecord.get().getData()) + : mergedAvroRecord.get().getData(); + this.currentRecord = (RowData) avroToRowDataConverter.convert(avroRecord); + this.currentRecord.setRowKind(rowKind); + return true; + } + } + + if (instantRange != null) { + boolean isInRange = instantRange.isInRange(currentRecord.getString(HOODIE_COMMIT_TIME_COL_POS).toString()); + if (!isInRange) { + // filter base file by instant range + continue; + } + } + + // project the full record in base with required positions + if (projection.isPresent()) { + currentRecord = projection.get().project(currentRecord); + } + return true; + } + // read the logs + readLogs = true; + while (logKeysIterator.hasNext()) { + final String curKey = logKeysIterator.next(); + if (!keyToSkip.contains(curKey)) { + Option insertAvroRecord = getInsertValue(curKey); + if (insertAvroRecord.isPresent()) { + // the record is a DELETE if insertAvroRecord not present, skipping + IndexedRecord avroRecord = avroProjection.isPresent() + ? avroProjection.get().apply(insertAvroRecord.get()) + : insertAvroRecord.get(); + this.currentRecord = (RowData) avroToRowDataConverter.convert(avroRecord); + FormatUtils.setRowKind(this.currentRecord, insertAvroRecord.get(), this.operationPos); + return true; + } + } + } + return false; + } } /** @@ -859,6 +1144,7 @@ public static class Builder { protected MergeOnReadTableState tableState; protected List fieldTypes; protected String defaultPartName; + protected List predicates; protected long limit = -1; protected boolean emitDelete = false; protected InternalSchemaManager internalSchemaManager = InternalSchemaManager.DISABLED; @@ -883,6 +1169,11 @@ public Builder defaultPartName(String defaultPartName) { return this; } + public Builder predicates(List predicates) { + this.predicates = predicates; + return this; + } + public Builder limit(long limit) { this.limit = limit; return this; @@ -899,8 +1190,12 @@ public Builder internalSchemaManager(InternalSchemaManager internalSchemaManager } public MergeOnReadInputFormat build() { + if (conf.get(FlinkOptions.COMPACTION_SCANNER_TYPE).equals(FlinkOptions.SCANNER_WITH_FK)) { + return new MergeOnReadInputFormatWithFK(conf, tableState, fieldTypes, + defaultPartName, limit, emitDelete, internalSchemaManager); + } return new MergeOnReadInputFormat(conf, tableState, fieldTypes, - defaultPartName, limit, emitDelete, internalSchemaManager); + defaultPartName, predicates, limit, emitDelete, internalSchemaManager); } } @@ -908,7 +1203,7 @@ public MergeOnReadInputFormat build() { // Utilities // ------------------------------------------------------------------------- - private static int[] getRequiredPosWithCommitTime(int[] requiredPos) { + protected static int[] getRequiredPosWithCommitTime(int[] requiredPos) { if (getCommitTimePos(requiredPos) >= 0) { return requiredPos; } @@ -918,7 +1213,17 @@ private static int[] getRequiredPosWithCommitTime(int[] requiredPos) { return requiredPos2; } - private static int getCommitTimePos(int[] requiredPos) { + protected static int[] getRequiredPosWithRecordKey(int[] requiredPos) { + if (getRecordKeyPos(requiredPos) >= 0) { + return requiredPos; + } + int[] requiredPos2 = new int[requiredPos.length + 1]; + requiredPos2[0] = HOODIE_RECORD_KEY_COL_POS; + System.arraycopy(requiredPos, 0, requiredPos2, 1, requiredPos.length); + return requiredPos2; + } + + public static int getCommitTimePos(int[] requiredPos) { for (int i = 0; i < requiredPos.length; i++) { if (requiredPos[i] == HOODIE_COMMIT_TIME_COL_POS) { return i; @@ -927,6 +1232,15 @@ private static int getCommitTimePos(int[] requiredPos) { return -1; } + public static int getRecordKeyPos(int[] requiredPos) { + for (int i = 0; i < requiredPos.length; i++) { + if (requiredPos[i] == HOODIE_RECORD_KEY_COL_POS) { + return i; + } + } + return -1; + } + @VisibleForTesting public void isEmitDelete(boolean emitDelete) { this.emitDelete = emitDelete; diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/mor/MergeOnReadInputSplit.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/mor/MergeOnReadInputSplit.java index 7a54fa4b40db2..20408be680d8d 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/mor/MergeOnReadInputSplit.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/mor/MergeOnReadInputSplit.java @@ -18,6 +18,7 @@ package org.apache.hudi.table.format.mor; +import org.apache.hudi.common.table.HoodieTableConfig; import org.apache.hudi.common.table.log.InstantRange; import org.apache.hudi.common.util.Option; @@ -43,7 +44,9 @@ public class MergeOnReadInputSplit implements InputSplit { private final long maxCompactionMemoryInBytes; private final String mergeType; private final Option instantRange; + private HoodieTableConfig tableConfig; protected String fileId; + private Long savepointDateBoundary; // for streaming reader to record the consumed offset, // which is the start of next round reading. @@ -58,7 +61,8 @@ public MergeOnReadInputSplit( long maxCompactionMemoryInBytes, String mergeType, @Nullable InstantRange instantRange, - String fileId) { + String fileId, + HoodieTableConfig tableConfig) { this.splitNum = splitNum; this.basePath = Option.ofNullable(basePath); this.logPaths = logPaths; @@ -68,6 +72,32 @@ public MergeOnReadInputSplit( this.mergeType = mergeType; this.instantRange = Option.ofNullable(instantRange); this.fileId = fileId; + this.tableConfig = tableConfig; + } + + public MergeOnReadInputSplit( + int splitNum, + @Nullable String basePath, + Option> logPaths, + String latestCommit, + String tablePath, + long maxCompactionMemoryInBytes, + String mergeType, + @Nullable InstantRange instantRange, + String fileId, + Long savepointDateBoundary, + HoodieTableConfig tableConfig) { + this.splitNum = splitNum; + this.basePath = Option.ofNullable(basePath); + this.logPaths = logPaths; + this.latestCommit = latestCommit; + this.tablePath = tablePath; + this.maxCompactionMemoryInBytes = maxCompactionMemoryInBytes; + this.mergeType = mergeType; + this.instantRange = Option.ofNullable(instantRange); + this.fileId = fileId; + this.savepointDateBoundary = savepointDateBoundary; + this.tableConfig = tableConfig; } public String getFileId() { @@ -94,6 +124,14 @@ public String getTablePath() { return tablePath; } + public HoodieTableConfig getTableConfig() { + return tableConfig; + } + + public void setTableConfig(HoodieTableConfig tableConfig) { + this.tableConfig = tableConfig; + } + public long getMaxCompactionMemoryInBytes() { return maxCompactionMemoryInBytes; } @@ -119,6 +157,10 @@ public long getConsumed() { return consumed; } + public Long getSavepointDateBoundary() { + return this.savepointDateBoundary; + } + public boolean isConsumed() { return this.consumed != NUM_NO_CONSUMPTION; } diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/mor/MergeOnReadTableState.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/mor/MergeOnReadTableState.java index 36dfecbb79a5f..49c1b759b2301 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/mor/MergeOnReadTableState.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/mor/MergeOnReadTableState.java @@ -18,14 +18,23 @@ package org.apache.hudi.table.format.mor; +import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.MultiplePartialUpdateUnit; +import org.apache.hudi.common.util.StringUtils; +import org.apache.hudi.common.util.ValidationUtils; +import org.apache.hudi.config.HoodiePayloadConfig; +import org.apache.avro.Schema; +import org.apache.flink.configuration.Configuration; import org.apache.flink.table.types.logical.LogicalType; import org.apache.flink.table.types.logical.RowType; import java.io.Serializable; import java.util.Arrays; +import java.util.HashSet; import java.util.List; +import java.util.stream.Collectors; /** * Statistics for merge on read table source. @@ -90,6 +99,51 @@ public int[] getRequiredPositions() { .toArray(); } + // 这里可以乱序 + public HashSet getLsmRequiredColumns(TypedProperties payloadProps, Schema tableSchema, Configuration conf) { + HashSet requiredWithMeta = new HashSet<>(requiredRowType.getFieldNames()); + + requiredWithMeta.add(HoodieRecord.RECORD_KEY_METADATA_FIELD); + requiredWithMeta.add(HoodieRecord.COMMIT_TIME_METADATA_FIELD); + requiredWithMeta.add(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD); + // Avoid finding deleted data during query + if (tableSchema.getFields().stream().anyMatch(field -> HoodieRecord.HOODIE_IS_DELETED_FIELD.equals(field.name()))) { + requiredWithMeta.add(HoodieRecord.HOODIE_IS_DELETED_FIELD); + } + + String orderingField = payloadProps.getString(HoodiePayloadConfig.ORDERING_FIELD.key(), + HoodiePayloadConfig.ORDERING_FIELD.defaultValue()); + + if (!StringUtils.isNullOrEmpty(orderingField) && orderingField.split(":").length > 1) { + requiredWithMeta.addAll(new MultiplePartialUpdateUnit(orderingField).getAllOrderingFields()); + } else { + if (tableSchema.getField(orderingField) != null) { + requiredWithMeta.add(orderingField); + } + } + + return requiredWithMeta; + } + + // copy from HoodieBaseRelation#projectSchema + public Schema getLsmReadSchema(TypedProperties payloadProps, Schema tableSchema, Configuration conf) { + HashSet requiredColumns = getLsmRequiredColumns(payloadProps, tableSchema, conf); + + List requiredFields = tableSchema.getFields().stream().filter(f -> { + return requiredColumns.contains(f.name()); + }).map(f -> { + // We have to create a new [[Schema.Field]] since Avro schemas can't share field + // instances (and will throw "org.apache.avro.AvroRuntimeException: Field already used") + return new Schema.Field(f.name(), f.schema(), f.doc(), f.defaultVal(), f.order()); + }).collect(Collectors.toList()); + + ValidationUtils.checkArgument(requiredColumns.size() == requiredFields.size(), + "Miss match between required Col and table Col " + requiredFields + " VS " + requiredColumns); + + return Schema.createRecord(tableSchema.getName(), tableSchema.getDoc(), tableSchema.getNamespace(), + tableSchema.isError(), requiredFields); + } + /** * Get the primary key positions in required row type. */ diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/mor/lsm/ChannelReaderInputViewIteratorWrapper.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/mor/lsm/ChannelReaderInputViewIteratorWrapper.java new file mode 100644 index 0000000000000..927dfa7dda700 --- /dev/null +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/mor/lsm/ChannelReaderInputViewIteratorWrapper.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.table.format.mor.lsm; + +import org.apache.flink.runtime.io.disk.ChannelReaderInputViewIterator; +import org.apache.flink.table.data.RowData; + +import java.io.IOException; +import java.util.Iterator; + +public class ChannelReaderInputViewIteratorWrapper implements Iterator { + private final ChannelReaderInputViewIterator iterator; + private RowData next; + + public ChannelReaderInputViewIteratorWrapper(ChannelReaderInputViewIterator iterator) { + this.iterator = iterator; + } + + @Override + public boolean hasNext() { + try { + next = (RowData) iterator.next(); + if (next != null) { + return true; + } + } catch (IOException e) { + throw new RuntimeException(e); + } + return false; + } + + @Override + public RowData next() { + if (next != null) { + return next; + } + throw new RuntimeException("Reach End! No more spilled data to read."); + } +} diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/mor/lsm/FlinkExternalRecordReader.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/mor/lsm/FlinkExternalRecordReader.java new file mode 100644 index 0000000000000..07414596eccb1 --- /dev/null +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/mor/lsm/FlinkExternalRecordReader.java @@ -0,0 +1,147 @@ +/* + * 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.format.mor.lsm; + +import org.apache.flink.runtime.io.compression.BlockCompressionFactory; +import org.apache.flink.runtime.io.disk.ChannelReaderInputViewIterator; +import org.apache.flink.runtime.io.disk.iomanager.AbstractChannelReaderInputView; +import org.apache.flink.runtime.io.disk.iomanager.AbstractChannelWriterOutputView; +import org.apache.flink.runtime.io.disk.iomanager.FileIOChannel; +import org.apache.flink.runtime.io.disk.iomanager.IOManager; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.runtime.io.ChannelWithMeta; +import org.apache.flink.table.runtime.typeutils.RowDataSerializer; +import org.apache.flink.table.runtime.util.FileChannelUtil; +import org.apache.hudi.client.model.HoodieFlinkRecord; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.util.ClosableIterator; +import org.apache.hudi.common.util.ValidationUtils; +import org.apache.hudi.common.util.collection.MappingIterator; +import org.apache.hudi.exception.HoodieException; +import org.apache.hudi.io.lsm.ExternalRecordReader; +import org.jetbrains.annotations.Nullable; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Iterator; + +public class FlinkExternalRecordReader implements ExternalRecordReader { + + protected final Iterator iterator; + protected final RowDataSerializer rowDataSerializer; + protected final int recordKeyIndex; + protected final BlockCompressionFactory compressFactory; + protected final IOManager ioManager; + protected final int compressBlockSize = 64 * 1024; + protected final int memorySegmentSize; + protected AbstractChannelWriterOutputView outputView; + protected AbstractChannelReaderInputView inputView; + + public FlinkExternalRecordReader(ClosableIterator iterator, RowDataSerializer rowDataSerializer, + int recordKeyIndex, IOManager ioManager, int memorySegmentSize) throws IOException { + this.rowDataSerializer = rowDataSerializer; + this.recordKeyIndex = recordKeyIndex; + this.compressFactory = BlockCompressionFactory.createBlockCompressionFactory("zstd"); + this.ioManager = ioManager; + this.memorySegmentSize = memorySegmentSize; + this.iterator = spillToDisk(iterator); + } + + /** + * 返回FileChannel RecordReader Iterator + * + * @return + * @throws IOException + */ + @Nullable + @Override + public Iterator read() throws IOException { + return new MappingIterator<>(iterator, record -> { + return new HoodieFlinkRecord(record.getString(recordKeyIndex), record); + }); + } + + @Override + public void close() throws IOException { + if (inputView != null) { + inputView.getChannel().closeAndDelete(); + inputView = null; + } + } + + // 将数据溢写到FileChannel + @Override + public Iterator spillToDisk(Iterator iterator) throws IOException { + ValidationUtils.checkArgument(ioManager != null, "IOManager should not be null!"); + + ChannelWithMeta channelWithMeta; + try { + // write to channel + FileIOChannel.ID channel = ioManager.createChannel(); + outputView = FileChannelUtil.createOutputView( + ioManager, channel, true, compressFactory, compressBlockSize, memorySegmentSize); + + // no data, return an empty iterator + if (iterator.hasNext()) { + rowDataSerializer.serialize(iterator.next(), outputView); + } else { + return new Iterator() { + @Override + public boolean hasNext() { + return false; + } + + @Override + public RowData next() { + return null; + } + }; + } + + while (iterator.hasNext()) { + rowDataSerializer.serialize(iterator.next(), outputView); + } + + int bytesInLastBlock = outputView.close(); + channelWithMeta = new ChannelWithMeta(channel, outputView.getBlockCount(), bytesInLastBlock); + } catch (Exception e) { + // close and delete current file channel + if (outputView != null) { + outputView.getChannel().closeAndDelete(); + outputView = null; + } + throw new HoodieException(e.getMessage(), e); + } finally { + if (iterator instanceof ClosableIterator) { + ((ClosableIterator) iterator).close(); + } + } + + try { + inputView = FileChannelUtil.createInputView( + ioManager, channelWithMeta, new ArrayList<>(), true, compressFactory, compressBlockSize, memorySegmentSize); + ChannelReaderInputViewIterator inputViewIterator = new ChannelReaderInputViewIterator(inputView, null, rowDataSerializer); + return new ChannelReaderInputViewIteratorWrapper(inputViewIterator); + } catch (Exception e) { + // close and delete current file channel + close(); + throw new HoodieException(e.getMessage(), e); + } + } +} diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/mor/lsm/FlinkLsmUtils.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/mor/lsm/FlinkLsmUtils.java new file mode 100644 index 0000000000000..13d9cd9bacb13 --- /dev/null +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/mor/lsm/FlinkLsmUtils.java @@ -0,0 +1,433 @@ +/* + * 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.format.mor.lsm; + +import org.apache.hudi.client.model.CommitTimeFlinkRecordMerger; +import org.apache.hudi.client.model.HoodieFlinkRecord; +import org.apache.hudi.common.config.HoodieStorageConfig; +import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.model.HoodieAvroRecordMerger; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecordMerger; +import org.apache.hudi.common.table.HoodieTableConfig; +import org.apache.hudi.common.table.log.InstantRange; +import org.apache.hudi.common.util.ClosableIterator; +import org.apache.hudi.common.util.HoodieTimer; +import org.apache.hudi.common.util.ReflectionUtils; +import org.apache.hudi.common.util.ValidationUtils; +import org.apache.hudi.config.HoodieCleanConfig; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.configuration.FlinkOptions; +import org.apache.hudi.configuration.HadoopConfigurations; +import org.apache.hudi.io.lsm.RecordReader; +import org.apache.hudi.index.HoodieIndex; +import org.apache.hudi.source.ExpressionPredicates; +import org.apache.hudi.table.action.cluster.strategy.LsmBaseClusteringPlanStrategy; +import org.apache.hudi.table.format.FilePathUtils; +import org.apache.hudi.table.format.InternalSchemaManager; +import org.apache.hudi.table.format.RecordIterators; +import org.apache.hudi.util.DataTypeUtils; + +import org.apache.avro.Schema; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.io.disk.iomanager.IOManager; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.expressions.CallExpression; +import org.apache.flink.table.expressions.FieldReferenceExpression; +import org.apache.flink.table.expressions.ValueLiteralExpression; +import org.apache.flink.table.functions.BuiltInFunctionDefinitions; +import org.apache.flink.table.functions.FunctionIdentifier; +import org.apache.flink.table.runtime.typeutils.RowDataSerializer; +import org.apache.flink.table.types.DataType; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.Arrays; +import java.util.Comparator; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.stream.Collectors; + +import static org.apache.hudi.common.model.HoodieRecordMerger.COMMIT_TIME_BASED_MERGE_STRATEGY_UUID; +import static org.apache.hudi.hadoop.utils.HoodieInputFormatUtils.HOODIE_RECORD_KEY_COL_POS; + +public class FlinkLsmUtils { + + private static final Logger LOG = LoggerFactory.getLogger(FlinkLsmUtils.class); + + public static Comparator getHoodieFlinkRecordComparator() { + return new Comparator() { + @Override + public int compare(HoodieRecord o1, HoodieRecord o2) { + HoodieFlinkRecord record1 = (HoodieFlinkRecord) o1; + HoodieFlinkRecord record2 = (HoodieFlinkRecord) o2; + return record1.getRecordKeyStringData().compareTo(record2.getRecordKeyStringData()); + } + }; + } + + public static int[] getLsmRequiredPositions(Schema requiredSchemaWithMeta, Schema tableSchema) { + List fieldNames = tableSchema.getFields().stream().map(Schema.Field::name).collect(Collectors.toList()); + return requiredSchemaWithMeta.getFields().stream().map(Schema.Field::name).map(fieldNames::indexOf) + .mapToInt(i -> i) + .toArray(); + } + + public static ClosableIterator getBaseFileIterator(String path, + int[] requiredPos, + List fieldNames, + List fieldTypes, + String defaultPartName, + Configuration conf, + org.apache.hadoop.conf.Configuration hadoopConf, + HoodieTableConfig tableConfig, + InternalSchemaManager internalSchemaManager, + List predicates) throws IOException { + // generate partition specs. + LinkedHashMap partSpec = FilePathUtils.extractPartitionKeyValues( + new org.apache.hadoop.fs.Path(path).getParent(), + conf.getBoolean(FlinkOptions.HIVE_STYLE_PARTITIONING), + FilePathUtils.extractPartitionKeys(conf)); + LinkedHashMap partObjects = new LinkedHashMap<>(); + partSpec.forEach((k, v) -> { + final int idx = fieldNames.indexOf(k); + if (idx == -1) { + // for any rare cases that the partition field does not exist in schema, + // fallback to file read + return; + } + DataType fieldType = fieldTypes.get(idx); + if (!DataTypeUtils.isDatetimeType(fieldType)) { + // date time type partition field is formatted specifically, + // read directly from the data file to avoid format mismatch or precision loss + partObjects.put(k, DataTypeUtils.resolvePartition(defaultPartName.equals(v) ? null : v, fieldType)); + } + }); + org.apache.hadoop.conf.Configuration parquetConf = HadoopConfigurations.getParquetConf(conf, hadoopConf); + FSUtils.addChubaoFsConfig2HadoopConf(parquetConf, tableConfig); + return RecordIterators.getParquetRecordIterator( + internalSchemaManager, + conf.getBoolean(FlinkOptions.UTC_TIMEZONE), + true, + parquetConf, + fieldNames.toArray(new String[0]), + fieldTypes.toArray(new DataType[0]), + partObjects, + requiredPos, + 2048, + new org.apache.flink.core.fs.Path(path), + 0, + Long.MAX_VALUE, // read the whole file + predicates); + } + + public static CallExpression buildCloseCloseRangeCommitExpression(InstantRange instantRange) { + return new CallExpression( + FunctionIdentifier.of("AND"), + BuiltInFunctionDefinitions.AND, + Arrays.asList( + new CallExpression( + FunctionIdentifier.of("greaterThanOrEqual"), + BuiltInFunctionDefinitions.GREATER_THAN_OR_EQUAL, + Arrays.asList( + new FieldReferenceExpression( + HoodieRecord.COMMIT_TIME_METADATA_FIELD, + DataTypes.STRING(), + 0, + HOODIE_RECORD_KEY_COL_POS + ), + new ValueLiteralExpression(instantRange.getStartInstant()) + ), + DataTypes.BOOLEAN() + ), + new CallExpression( + FunctionIdentifier.of("lessThanOrEqual"), + BuiltInFunctionDefinitions.LESS_THAN_OR_EQUAL, + Arrays.asList( + new FieldReferenceExpression( + HoodieRecord.COMMIT_TIME_METADATA_FIELD, + DataTypes.STRING(), + 0, + HOODIE_RECORD_KEY_COL_POS + ), + new ValueLiteralExpression(instantRange.getEndInstant()) + ), + DataTypes.BOOLEAN() + ) + ), + DataTypes.BOOLEAN() + ); + } + + public static CallExpression buildCloseCloseRangeNullableBoundaryCommitExpression(InstantRange instantRange) { + if (instantRange.getStartInstant() == null) { + return new CallExpression( + FunctionIdentifier.of("lessThanOrEqual"), + BuiltInFunctionDefinitions.LESS_THAN_OR_EQUAL, + Arrays.asList( + new FieldReferenceExpression( + HoodieRecord.COMMIT_TIME_METADATA_FIELD, + DataTypes.STRING(), + 0, + HOODIE_RECORD_KEY_COL_POS + ), + new ValueLiteralExpression(instantRange.getEndInstant()) + ), + DataTypes.BOOLEAN() + ); + } else if (instantRange.getEndInstant() == null) { + return new CallExpression( + FunctionIdentifier.of("greaterThanOrEqual"), + BuiltInFunctionDefinitions.GREATER_THAN_OR_EQUAL, + Arrays.asList( + new FieldReferenceExpression( + HoodieRecord.COMMIT_TIME_METADATA_FIELD, + DataTypes.STRING(), + 0, + HOODIE_RECORD_KEY_COL_POS + ), + new ValueLiteralExpression(instantRange.getStartInstant()) + ), + DataTypes.BOOLEAN() + ); + } else { + return buildCloseCloseRangeCommitExpression(instantRange); + } + } + + public static CallExpression buildOpenCloseRangeCommitExpression(InstantRange instantRange) { + return new CallExpression( + FunctionIdentifier.of("greaterThan"), + BuiltInFunctionDefinitions.GREATER_THAN, + Arrays.asList( + new FieldReferenceExpression( + HoodieRecord.COMMIT_TIME_METADATA_FIELD, + DataTypes.STRING(), + 0, + HOODIE_RECORD_KEY_COL_POS + ), + new ValueLiteralExpression(instantRange.getStartInstant()) + ), + DataTypes.BOOLEAN() + ); + } + + public static CallExpression buildOpenCloseRangeNullableBoundaryCommitExpression(InstantRange instantRange) { + if (instantRange.getStartInstant() == null) { + return new CallExpression( + FunctionIdentifier.of("lessThanOrEqual"), + BuiltInFunctionDefinitions.LESS_THAN_OR_EQUAL, + Arrays.asList( + new FieldReferenceExpression( + HoodieRecord.COMMIT_TIME_METADATA_FIELD, + DataTypes.STRING(), + 0, + HOODIE_RECORD_KEY_COL_POS + ), + new ValueLiteralExpression(instantRange.getEndInstant()) + ), + DataTypes.BOOLEAN() + ); + } else if (instantRange.getEndInstant() == null) { + return new CallExpression( + FunctionIdentifier.of("greaterThan"), + BuiltInFunctionDefinitions.GREATER_THAN, + Arrays.asList( + new FieldReferenceExpression( + HoodieRecord.COMMIT_TIME_METADATA_FIELD, + DataTypes.STRING(), + 0, + HOODIE_RECORD_KEY_COL_POS + ), + new ValueLiteralExpression(instantRange.getStartInstant()) + ), + DataTypes.BOOLEAN() + ); + } else { + return new CallExpression( + FunctionIdentifier.of("AND"), + BuiltInFunctionDefinitions.AND, + Arrays.asList( + new CallExpression( + FunctionIdentifier.of("greaterThan"), + BuiltInFunctionDefinitions.GREATER_THAN, + Arrays.asList( + new FieldReferenceExpression( + HoodieRecord.COMMIT_TIME_METADATA_FIELD, + DataTypes.STRING(), + 0, + HOODIE_RECORD_KEY_COL_POS + ), + new ValueLiteralExpression(instantRange.getStartInstant()) + ), + DataTypes.BOOLEAN() + ), + new CallExpression( + FunctionIdentifier.of("lessThanOrEqual"), + BuiltInFunctionDefinitions.LESS_THAN_OR_EQUAL, + Arrays.asList( + new FieldReferenceExpression( + HoodieRecord.COMMIT_TIME_METADATA_FIELD, + DataTypes.STRING(), + 0, + HOODIE_RECORD_KEY_COL_POS + ), + new ValueLiteralExpression(instantRange.getEndInstant()) + ), + DataTypes.BOOLEAN() + ) + ), + DataTypes.BOOLEAN() + ); + } + } + + public static int getRecordKeyIndex(Schema requiredSchemaWithMeta) { + List fields = requiredSchemaWithMeta.getFields(); + int index = 0; + for (; index < fields.size(); index++) { + if (fields.get(index).name().equalsIgnoreCase(HoodieRecord.RECORD_KEY_METADATA_FIELD)) { + break; + } + } + return index; + } + + public static List> createLsmRecordReaders(List> iterators, + int spillThreshold, + RowDataSerializer rowDataSerializer, + IOManager ioManager, + int pageSize, + int recordKeyIndex, + int level1Index) throws IOException { + if (ioManager == null) { + return iterators.stream() + .map(iter -> new FlinkRecordReader(iter, recordKeyIndex)) + .collect(Collectors.toList()); + } + + int readDirectlySize = Math.min(iterators.size(), spillThreshold); + + List> readers = iterators.subList(0, readDirectlySize).stream().map(iter -> { + return new FlinkRecordReader(iter, recordKeyIndex); + }).collect(Collectors.toList()); + + if (iterators.size() > spillThreshold) { + int spillSize = iterators.size() - readDirectlySize; + LOG.info("Start spilling for clustering. " + spillSize + " files will be spilled."); + HoodieTimer timer = HoodieTimer.start(); + + for (int i = spillThreshold; i < iterators.size(); i++) { + if (level1Index == i) { + readers.add(new FlinkRecordReader(iterators.get(i), recordKeyIndex)); + } else { + try { + readers.add(new FlinkExternalRecordReader( + iterators.get(i), + rowDataSerializer, + recordKeyIndex, + ioManager, + pageSize) + ); + } catch (Exception e) { + LOG.error("Spill failed. Deleting all spilled files."); + // 清理其他已完成溢写的文件 + for (RecordReader reader : readers) { + reader.close(); + } + throw e; + } + } + } + + long spillDuration = timer.endTimer(); + LOG.info("Cost " + spillDuration + " ms to spill " + spillSize + " files to disk."); + } + + return readers; + } + + public static void setupMergerConfig(Configuration conf) { + // 考虑两种情况 1.用户指定了impls;2.用户没有指定impls + if (!conf.contains(FlinkOptions.RECORD_MERGER_IMPLS)) { + // 如果用户没有设置Record merger,则默认给定基于commit time的 record merger + conf.set(FlinkOptions.RECORD_MERGER_IMPLS, CommitTimeFlinkRecordMerger.class.getName()); + conf.set(FlinkOptions.RECORD_MERGER_STRATEGY, COMMIT_TIME_BASED_MERGE_STRATEGY_UUID); + conf.set(FlinkOptions.TABLE_RECORD_MERGER_STRATEGY, COMMIT_TIME_BASED_MERGE_STRATEGY_UUID); + } else { + // 如果用户指定的record merger,则1. 校验merger不能是avro merger;2. 检查用户是否设置的strategy,如果没有则自动补全 + String className = conf.get(FlinkOptions.RECORD_MERGER_IMPLS); + ValidationUtils.checkArgument(!className.equalsIgnoreCase(HoodieAvroRecordMerger.class.getName()), + "LSM 不支持 " + HoodieAvroRecordMerger.class.getName()); + HoodieRecordMerger merger = ReflectionUtils.loadClass(className); + conf.set(FlinkOptions.RECORD_MERGER_STRATEGY, merger.getMergingStrategy()); + conf.set(FlinkOptions.TABLE_RECORD_MERGER_STRATEGY, merger.getMergingStrategy()); + } + } + + public static void setupLSMConfig(Configuration conf) { + // Step1: parquet 相关参数 + if (!conf.containsKey(HoodieStorageConfig.PARQUET_COMPRESSION_CODEC_NAME.key())) { + // 默认是用ZSTD压缩算法 + conf.setString(HoodieStorageConfig.PARQUET_COMPRESSION_CODEC_NAME.key(), "zstd"); + } + + if (!conf.containsKey(HoodieStorageConfig.PARQUET_BLOCK_SIZE.key())) { + // Flink 写L0层的时候 默认block size 32M,避免流读和Clustering时候的OOM问题 + // 后续的Flink 的 L1 Major Compaction的时候 重新刷为128M LSMHoodieRowDataCreateHandle + conf.setString(HoodieStorageConfig.PARQUET_BLOCK_SIZE.key(), String.valueOf(32 * 1024 * 1024)); + } + + // Step2: 表服务相关参数 + // 强制关闭Compaction + conf.set(FlinkOptions.COMPACTION_SCHEDULE_ENABLED, false); + conf.set(FlinkOptions.COMPACTION_ASYNC_ENABLED, false); + // 强制关闭普通Clustering + conf.set(FlinkOptions.CLUSTERING_SCHEDULE_ENABLED, false); + conf.set(FlinkOptions.CLUSTERING_ASYNC_ENABLED, false); + if (!conf.contains(FlinkOptions.LSM_CLUSTERING_PLAN_STRATEGY_CLASS)) { + // 设置Flink LSM Clustering plan 策略 + conf.set(FlinkOptions.LSM_CLUSTERING_PLAN_STRATEGY_CLASS, LsmBaseClusteringPlanStrategy.class.getName()); + } + if (!conf.contains(FlinkOptions.CLEAN_RETAIN_COMMITS)) { + // 目前clean是基于clustering构成的版本而言,而不是常规的commit数,因此这里缩减下默认值 + conf.set(FlinkOptions.CLEAN_RETAIN_COMMITS, 10); + } + + // TODO read footer 只有在 支持流拷贝 且 event-time record merger下才会生效 + // if (conf.get(FlinkOptions.RECORD_MERGER_STRATEGY).equalsIgnoreCase(EVENT_TIME_BASED_MERGE_STRATEGY_UUID) + // && !conf.containsKey(HoodieClusteringConfig.LSM_CLUSTERING_READFOOTER_ENABLED.key())) { + // conf.set(FlinkOptions.READFOOTER_ENABLED, true); + // } + + // Step3:写时参数 + // 多写参数默认打开 + // 默认开启Lazy + if (!conf.containsKey(HoodieWriteConfig.MUTLIPLE_WRITE_ENABLE.key())) { + conf.setBoolean(HoodieWriteConfig.MUTLIPLE_WRITE_ENABLE.key(), true); + conf.setString(HoodieCleanConfig.FAILED_WRITES_CLEANER_POLICY.key(), "LAZY"); + } + + // Step4: 对于index类型的校验,LSM 目前只支持Bucket Index + String indexType = conf.get(FlinkOptions.INDEX_TYPE); + ValidationUtils.checkArgument(indexType.equalsIgnoreCase(HoodieIndex.IndexType.BUCKET.name()), "LSM 目前只能配合Bucket Index使用"); + } +} diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/mor/lsm/FlinkRecordMergerWrapper.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/mor/lsm/FlinkRecordMergerWrapper.java new file mode 100644 index 0000000000000..b2f576746f382 --- /dev/null +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/mor/lsm/FlinkRecordMergerWrapper.java @@ -0,0 +1,117 @@ +/* + * 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.format.mor.lsm; + +import org.apache.hudi.client.model.HoodieFlinkRecord; +import org.apache.hudi.common.config.TypedProperties; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecordMerger; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.exception.HoodieIOException; +import org.apache.hudi.io.lsm.RecordMergeWrapper; + +import org.apache.avro.Schema; +import org.apache.flink.table.data.RowData; + +import java.io.IOException; +import java.util.Iterator; +import java.util.List; +import java.util.Properties; + +public class FlinkRecordMergerWrapper implements RecordMergeWrapper { + + private final HoodieRecordMerger recordMerger; + private final Schema oldSchema; + private final Schema newSchema; + private final TypedProperties props; + + private final boolean isIgnoreDelete; + + private Option> nextRecord = Option.empty(); + + public FlinkRecordMergerWrapper( + boolean isIgnoreDelete, + HoodieRecordMerger recordMerger, + Schema oldSchema, + Schema newSchema, + TypedProperties props) { + this.isIgnoreDelete = isIgnoreDelete; + this.recordMerger = recordMerger; + this.oldSchema = oldSchema; + this.newSchema = newSchema; + this.props = props; + } + + @Override + public Option merge(List recordGroup) { + return null; + } + + @Override + public Option merge(Iterator sameKeyIterator) { + while (sameKeyIterator.hasNext()) { + HoodieFlinkRecord record = (HoodieFlinkRecord) sameKeyIterator.next(); + if (!nextRecord.isPresent()) { + nextRecord = Option.of(Pair.of(record, oldSchema)); + } else { + nextRecord = mergeInternal((HoodieFlinkRecord)nextRecord.get().getKey(), record); + } + } + + return nextRecord.map(pair -> { + return (RowData)pair.getLeft().getData(); + }); + } + + @Override + public void merge(HoodieRecord record) { + HoodieFlinkRecord flinkRecord = (HoodieFlinkRecord) record; + if (!nextRecord.isPresent()) { + nextRecord = Option.of(Pair.of(flinkRecord, oldSchema)); + } else { + nextRecord = mergeInternal((HoodieFlinkRecord) nextRecord.get().getKey(), flinkRecord); + } + } + + @Override + public Option getMergedResult() { + try { + if (this.isIgnoreDelete && nextRecord.get().getLeft().isDelete(newSchema, new Properties())) { + return Option.empty(); + } + } catch (IOException e) { + throw new HoodieIOException(e.getMessage()); + } + return nextRecord.map(pair -> (RowData) pair.getLeft().getData()); + } + + @Override + public void reset() { + this.nextRecord = Option.empty(); + } + + private Option> mergeInternal(HoodieFlinkRecord oldRecord, HoodieFlinkRecord newRecord) { + try { + return recordMerger.merge(oldRecord, oldSchema, newRecord, newSchema, props); + } catch (IOException e) { + throw new HoodieIOException(e.getMessage(), e); + } + } +} diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/mor/lsm/FlinkRecordReader.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/mor/lsm/FlinkRecordReader.java new file mode 100644 index 0000000000000..ebd1a2894a0df --- /dev/null +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/mor/lsm/FlinkRecordReader.java @@ -0,0 +1,63 @@ +/* + * 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.format.mor.lsm; + +import org.apache.hudi.client.model.HoodieFlinkRecord; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.util.ClosableIterator; +import org.apache.hudi.common.util.collection.MappingIterator; +import org.apache.hudi.io.lsm.RecordReader; + +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.runtime.typeutils.RowDataSerializer; +import org.jetbrains.annotations.Nullable; + +import java.io.IOException; +import java.util.Iterator; + +public class FlinkRecordReader implements RecordReader { + + private final ClosableIterator iterator; + private RowDataSerializer rowDataSerializer; + private final int recordKeyIndex; + + public FlinkRecordReader(ClosableIterator iterator, RowDataSerializer rowDataSerializer, int recordKeyIndex) { + this.iterator = iterator; + this.rowDataSerializer = rowDataSerializer; + this.recordKeyIndex = recordKeyIndex; + } + + public FlinkRecordReader(ClosableIterator iterator, int recordKeyIndex) { + this.iterator = iterator; + this.recordKeyIndex = recordKeyIndex; + } + + @Nullable + @Override + public Iterator read() throws IOException { + return new MappingIterator<>(iterator, record -> { + return new HoodieFlinkRecord(record.getString(recordKeyIndex), record); + }); + } + + @Override + public void close() throws IOException { + // no op + } +} diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/mor/lsm/LsmMergeIterator.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/mor/lsm/LsmMergeIterator.java new file mode 100644 index 0000000000000..8b3dd1496ab34 --- /dev/null +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/mor/lsm/LsmMergeIterator.java @@ -0,0 +1,119 @@ +/* + * 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.format.mor.lsm; + +import org.apache.hudi.common.config.TypedProperties; +import org.apache.hudi.common.engine.EngineType; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecordMerger; +import org.apache.hudi.common.util.ClosableIterator; +import org.apache.hudi.common.util.HoodieRecordUtils; +import org.apache.hudi.common.util.ValidationUtils; +import org.apache.hudi.configuration.FlinkOptions; +import org.apache.hudi.exception.HoodieIOException; +import org.apache.hudi.io.lsm.MergeSorter; +import org.apache.hudi.io.lsm.RecordReader; +import org.apache.hudi.util.RowDataProjection; + +import org.apache.avro.Schema; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.logical.RowType; + +import java.io.IOException; +import java.util.Arrays; +import java.util.Iterator; +import java.util.List; +import java.util.stream.Collectors; + +public class LsmMergeIterator implements ClosableIterator { + + private final Iterator iterator; + private final RowDataProjection projection; + private RecordReader reader; + private boolean skipProject; + + public LsmMergeIterator( + boolean isIgnoreDelete, + Configuration flinkConf, + TypedProperties payloadProps, + Schema requiredSchemaWithMeta, + RowType oriRequiredRowType, + org.apache.hadoop.conf.Configuration hadoopConf, + int[] oriRequiredPos, + String tablePath, + List> readers, + boolean skipProject) throws IOException { // the iterator should be with full schema + List mergers = Arrays.stream(flinkConf.getString(FlinkOptions.RECORD_MERGER_IMPLS).split(",")) + .map(String::trim) + .distinct() + .collect(Collectors.toList()); + HoodieRecordMerger recordMerger = HoodieRecordUtils.createRecordMerger(tablePath, + EngineType.FLINK, mergers, flinkConf.getString(FlinkOptions.RECORD_MERGER_STRATEGY)); + this.reader = new MergeSorter().mergeSort(readers, new FlinkRecordMergerWrapper(isIgnoreDelete, recordMerger, + requiredSchemaWithMeta, requiredSchemaWithMeta, payloadProps), FlinkLsmUtils.getHoodieFlinkRecordComparator()); + this.iterator = reader.read(); + int[] projectPos = getProjectPos(oriRequiredPos, oriRequiredRowType, requiredSchemaWithMeta); + this.projection = RowDataProjection.instance(oriRequiredRowType, projectPos); + this.skipProject = skipProject; + } + + private int[] getProjectPos(int[] requiredPos, RowType oriRequiredRowType, Schema requiredSchemaWithMeta) { + int[] res = new int[requiredPos.length]; + List oriFieldNames = oriRequiredRowType.getFieldNames(); + ValidationUtils.checkArgument(oriFieldNames.size() == requiredPos.length); + List fieldNameWithMeta = requiredSchemaWithMeta.getFields().stream().map(Schema.Field::name).collect(Collectors.toList()); + List indexes = oriFieldNames.stream().map(fieldNameWithMeta::indexOf).collect(Collectors.toList()); + + for (int i = 0; i < indexes.size(); i++) { + res[i] = indexes.get(i); + } + return res; + } + + @Override + public void close() { + try { + if (reader != null) { + reader.close(); + } + } catch (IOException e) { + throw new HoodieIOException(e.getMessage(), e); + } finally { + reader = null; + } + } + + @Override + public boolean hasNext() { + return iterator.hasNext(); + } + + @Override + public RowData next() { + RowData record = iterator.next(); + if (record == null) { + return null; + } + if (skipProject) { + return record; + } + return projection.project(record); + } +} diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/FlinkWriteClients.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/FlinkWriteClients.java index 9f61580da0b49..0b85025dd6c9c 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/FlinkWriteClients.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/FlinkWriteClients.java @@ -23,8 +23,8 @@ import org.apache.hudi.client.common.HoodieFlinkEngineContext; import org.apache.hudi.client.transaction.lock.FileSystemBasedLockProvider; import org.apache.hudi.common.config.HoodieMetadataConfig; -import org.apache.hudi.common.config.SerializableConfiguration; import org.apache.hudi.common.config.HoodieStorageConfig; +import org.apache.hudi.common.config.SerializableConfiguration; import org.apache.hudi.common.engine.EngineType; import org.apache.hudi.common.model.HoodieCleaningPolicy; import org.apache.hudi.common.model.WriteConcurrencyMode; @@ -34,12 +34,15 @@ import org.apache.hudi.config.HoodieCleanConfig; import org.apache.hudi.config.HoodieClusteringConfig; import org.apache.hudi.config.HoodieCompactionConfig; +import org.apache.hudi.config.HoodieIndexConfig; import org.apache.hudi.config.HoodieLockConfig; import org.apache.hudi.config.HoodieMemoryConfig; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.configuration.FlinkOptions; import org.apache.hudi.configuration.HadoopConfigurations; import org.apache.hudi.configuration.OptionsResolver; +import org.apache.hudi.index.HoodieIndex; +import org.apache.hudi.sink.clustering.LSMClusteringScheduleMode; import org.apache.hudi.table.action.cluster.ClusteringPlanPartitionFilterMode; import org.apache.hudi.table.action.compact.CompactionTriggerStrategy; @@ -159,10 +162,28 @@ public static HoodieWriteConfig getHoodieClientConfig( HoodieWriteConfig.newBuilder() .withEngineType(EngineType.FLINK) .withPath(conf.getString(FlinkOptions.PATH)) + .withRecordMergerImpls(conf.getString(FlinkOptions.RECORD_MERGER_IMPLS)) + .withRecordMergerStrategy(conf.getString(FlinkOptions.RECORD_MERGER_STRATEGY)) + .withFlushConcurrency(conf.getInteger(FlinkOptions.WRITE_BULK_INSERT_FLUSH_CONCURRENCY)) .combineInput(conf.getBoolean(FlinkOptions.PRE_COMBINE), true) .withMergeAllowDuplicateOnInserts(OptionsResolver.insertClustering(conf)) + .withOperation(conf.getString(FlinkOptions.OPERATION)) + .withStorageStrategyClass(conf.getString(FlinkOptions.STORAGE_STRATEGY)) + .withStoragePath(conf.getString(FlinkOptions.STORAGE_PATH)) .withClusteringConfig( HoodieClusteringConfig.newBuilder() + // clustering conf for LSM + .withLsmClusteringPlanStrategyClass(conf.getString(FlinkOptions.LSM_CLUSTERING_PLAN_STRATEGY_CLASS)) + .withLsmClusteringPlanNumSortRunTrigger(conf.getInteger(FlinkOptions.NUM_RUN_CLUSTERING_TRIGGER)) + .withLsmClusteringPlanMaxSizeAmp(conf.getInteger(FlinkOptions.MAX_SIZE_AMP)) + .withLsmClusteringReadFooterEnabled(conf.getBoolean(FlinkOptions.READFOOTER_ENABLED)) + .withLsmAsyncClusteringSchedule(conf.getString(FlinkOptions.LSM_CLUSTERING_SCHEDULE_MODE).equalsIgnoreCase(LSMClusteringScheduleMode.ASYNC.name())) + .withLsmAsyncClusteringMaxCommits(conf.getInteger(FlinkOptions.LSM_CLUSTERING_DELTA_COMMITS)) + .withLsmMaxOfPendingClustering(conf.getInteger(FlinkOptions.LSM_CLUSTERING_PENDING_MAX_NUM)) + .withLsmClusteringPlanSmallFileLimit(conf.getLong(FlinkOptions.LSM_CLUSTERING_PLAN_STRATEGY_SMALL_FILE_LIMIT)) + .withLsmClusteringMaxBytesInGroup(conf.getLong(FlinkOptions.LSM_PLAN_STRATEGY_MAX_BYTES_PER_OUTPUT_FILEGROUP)) + .withLsmClusteringMinNumGroups(conf.getInteger(FlinkOptions.LSM_CLUSTERING_MIN_NUM_GROUPS)) + // clustreing conf before .withAsyncClustering(conf.getBoolean(FlinkOptions.CLUSTERING_SCHEDULE_ENABLED)) .withClusteringPlanStrategyClass(conf.getString(FlinkOptions.CLUSTERING_PLAN_STRATEGY_CLASS)) .withClusteringPlanPartitionFilterMode( @@ -177,6 +198,8 @@ public static HoodieWriteConfig getHoodieClientConfig( .withClusteringPartitionRegexPattern(conf.get(FlinkOptions.CLUSTERING_PLAN_STRATEGY_PARTITION_REGEX_PATTERN)) .withClusteringPartitionSelected(conf.get(FlinkOptions.CLUSTERING_PLAN_STRATEGY_PARTITION_SELECTED)) .withAsyncClusteringMaxCommits(conf.getInteger(FlinkOptions.CLUSTERING_DELTA_COMMITS)) + .withClusteringSortColumns(conf.getString(FlinkOptions.CLUSTERING_SORT_COLUMNS)) + .withMaxOfPendingClustering(conf.getInteger(FlinkOptions.CLUSTERING_PENDING_MAX_NUM)) .build()) .withCleanConfig(HoodieCleanConfig.newBuilder() .withAsyncClean(conf.getBoolean(FlinkOptions.CLEAN_ASYNC_ENABLED)) @@ -188,8 +211,10 @@ public static HoodieWriteConfig getHoodieClientConfig( .withCleanerParallelism(20) .withCleanerPolicy(HoodieCleaningPolicy.valueOf(conf.getString(FlinkOptions.CLEAN_POLICY))) .build()) + .withAysncRollback(conf.getBoolean(FlinkOptions.ROLLBACK_ASYNC_ENABLE)) .withArchivalConfig(HoodieArchivalConfig.newBuilder() .archiveCommitsWith(conf.getInteger(FlinkOptions.ARCHIVE_MIN_COMMITS), conf.getInteger(FlinkOptions.ARCHIVE_MAX_COMMITS)) + .archiveCleanCommitsWith(conf.getInteger(FlinkOptions.ARCHIVE_MIN_COMMITS_CLEAN), conf.getInteger(FlinkOptions.ARCHIVE_MAX_COMMITS_CLEAN)) .build()) .withCompactionConfig(HoodieCompactionConfig.newBuilder() .withTargetIOPerCompactionInMB(conf.getLong(FlinkOptions.COMPACTION_TARGET_IO)) @@ -197,6 +222,8 @@ public static HoodieWriteConfig getHoodieClientConfig( 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)) + .withMaxOfPendingCompaction(conf.getInteger(FlinkOptions.COMPACTION_PENDING_MAX_NUM)) + .withIncrementalPartitions(conf.getBoolean(FlinkOptions.COMPACTION_SCHEDULE_INCREMENTAL_PARTITIONS)) .build()) .withMemoryConfig( HoodieMemoryConfig.newBuilder() @@ -208,9 +235,14 @@ public static HoodieWriteConfig getHoodieClientConfig( .withStorageConfig(HoodieStorageConfig.newBuilder() .logFileDataBlockMaxSize(conf.getInteger(FlinkOptions.WRITE_LOG_BLOCK_SIZE) * 1024 * 1024) .logFileMaxSize(conf.getLong(FlinkOptions.WRITE_LOG_MAX_SIZE) * 1024 * 1024) + .logFileRolloverDirectly(conf.getBoolean(FlinkOptions.WRITE_LOG_ROLLOVER_DIRECTLY)) .parquetBlockSize(conf.getInteger(FlinkOptions.WRITE_PARQUET_BLOCK_SIZE) * 1024 * 1024) .parquetPageSize(conf.getInteger(FlinkOptions.WRITE_PARQUET_PAGE_SIZE) * 1024 * 1024) .parquetMaxFileSize(conf.getInteger(FlinkOptions.WRITE_PARQUET_MAX_FILE_SIZE) * 1024 * 1024L) + .dataSketchEnabled(conf.getBoolean(FlinkOptions.DATASKETCH_ENABLED)) + .parquetRecordKeyBloomFilterEnabled(conf.getBoolean(FlinkOptions.PARQUET_RECORDKEY_BLOOM_FILTER_ENABLED)) + .parquetRecordKeyClusteringBloomFilterEnabled(conf.getBoolean(FlinkOptions.PARQUET_RECORDKEY_CLUSTERING_BLOOM_FILTER_ENABLED)) + .parquetRecordKeyCompactionBloomFilterEnabled(conf.getBoolean(FlinkOptions.PARQUET_RECORDKEY_COMPACTION_BLOOM_FILTER_ENABLED)) .build()) .withMetadataConfig(HoodieMetadataConfig.newBuilder() .enable(conf.getBoolean(FlinkOptions.METADATA_ENABLED)) @@ -222,7 +254,19 @@ public static HoodieWriteConfig getHoodieClientConfig( .withAutoCommit(false) .withAllowOperationMetadataField(conf.getBoolean(FlinkOptions.CHANGELOG_ENABLED)) .withProps(flinkConf2TypedProperties(conf)) - .withSchema(getSourceSchema(conf).toString()); + .withSchema(getSourceSchema(conf).toString()) + .withWriteIgnoreFailed(conf.get(FlinkOptions.IGNORE_FAILED)); + + if (conf.getString(FlinkOptions.INDEX_TYPE).equalsIgnoreCase(HoodieIndex.IndexType.BUCKET.name())) { + builder.withIndexConfig(HoodieIndexConfig.newBuilder() + .withIndexType(HoodieIndex.IndexType.BUCKET) + .withIndexKeyField(conf.getString(FlinkOptions.INDEX_KEY_FIELD)) + .withIndexRecordKeyField(conf.getString(FlinkOptions.RECORD_KEY_FIELD)) + .withBucketNum(String.valueOf(conf.getInteger(FlinkOptions.BUCKET_INDEX_NUM_BUCKETS))) + .withBucketIndexAtPartitionLevel(conf.getBoolean(FlinkOptions.BUCKET_INDEX_PARTITION_LEVEL)) + .withPartitionBucketExpr(conf.getString(FlinkOptions.BUCKET_INDEX_PARTITION_BUCKET_EXPR)) + .build()); + } if (conf.getBoolean(FlinkOptions.METADATA_ENABLED)) { builder.withWriteConcurrencyMode(WriteConcurrencyMode.OPTIMISTIC_CONCURRENCY_CONTROL); diff --git a/hudi-flink-datasource/hudi-flink1.16.x/src/main/java/org/apache/hudi/table/format/cow/ParquetSplitReaderUtil.java b/hudi-flink-datasource/hudi-flink1.16.x/src/main/java/org/apache/hudi/table/format/cow/ParquetSplitReaderUtil.java index 5eeb42514a2cc..4d7dfb9e1c3f8 100644 --- a/hudi-flink-datasource/hudi-flink1.16.x/src/main/java/org/apache/hudi/table/format/cow/ParquetSplitReaderUtil.java +++ b/hudi-flink-datasource/hudi-flink1.16.x/src/main/java/org/apache/hudi/table/format/cow/ParquetSplitReaderUtil.java @@ -19,11 +19,14 @@ package org.apache.hudi.table.format.cow; import org.apache.hudi.common.util.ValidationUtils; +import org.apache.hudi.table.format.cow.vector.HeapArrayGroupColumnVector; import org.apache.hudi.table.format.cow.vector.HeapArrayVector; +import org.apache.hudi.table.format.cow.vector.HeapDecimalVector; import org.apache.hudi.table.format.cow.vector.HeapMapColumnVector; import org.apache.hudi.table.format.cow.vector.HeapRowColumnVector; -import org.apache.hudi.table.format.cow.vector.ParquetDecimalVector; import org.apache.hudi.table.format.cow.vector.reader.ArrayColumnReader; +import org.apache.hudi.table.format.cow.vector.reader.ArrayGroupReader; +import org.apache.hudi.table.format.cow.vector.reader.EmptyColumnReader; import org.apache.hudi.table.format.cow.vector.reader.FixedLenBytesColumnReader; import org.apache.hudi.table.format.cow.vector.reader.Int64TimestampColumnReader; import org.apache.hudi.table.format.cow.vector.reader.MapColumnReader; @@ -61,16 +64,19 @@ import org.apache.flink.table.types.logical.IntType; import org.apache.flink.table.types.logical.LocalZonedTimestampType; import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.LogicalTypeFamily; +import org.apache.flink.table.types.logical.LogicalTypeRoot; import org.apache.flink.table.types.logical.MapType; import org.apache.flink.table.types.logical.RowType; import org.apache.flink.table.types.logical.TimestampType; -import org.apache.flink.table.types.logical.VarBinaryType; import org.apache.flink.util.Preconditions; import org.apache.hadoop.conf.Configuration; import org.apache.parquet.ParquetRuntimeException; import org.apache.parquet.column.ColumnDescriptor; import org.apache.parquet.column.page.PageReadStore; import org.apache.parquet.column.page.PageReader; +import org.apache.parquet.filter.UnboundRecordFilter; +import org.apache.parquet.filter2.predicate.FilterPredicate; import org.apache.parquet.schema.GroupType; import org.apache.parquet.schema.InvalidSchemaException; import org.apache.parquet.schema.OriginalType; @@ -114,7 +120,9 @@ public static ParquetColumnarRowSplitReader genPartColumnarRowReader( int batchSize, Path path, long splitStart, - long splitLength) throws IOException { + long splitLength, + FilterPredicate filterPredicate, + UnboundRecordFilter recordFilter) throws IOException { List selNonPartNames = Arrays.stream(selectedFields) .mapToObj(i -> fullFieldNames[i]) .filter(n -> !partitionSpec.containsKey(n)) @@ -147,7 +155,9 @@ public static ParquetColumnarRowSplitReader genPartColumnarRowReader( batchSize, new org.apache.hadoop.fs.Path(path.toUri()), splitStart, - splitLength); + splitLength, + filterPredicate, + recordFilter); } private static ColumnVector createVector( @@ -227,17 +237,18 @@ private static ColumnVector createVectorFromConstant( } return lv; case DECIMAL: - DecimalType decimalType = (DecimalType) type; - int precision = decimalType.getPrecision(); - int scale = decimalType.getScale(); - DecimalData decimal = value == null - ? null - : Preconditions.checkNotNull(DecimalData.fromBigDecimal((BigDecimal) value, precision, scale)); - ColumnVector internalVector = createVectorFromConstant( - new VarBinaryType(), - decimal == null ? null : decimal.toUnscaledBytes(), - batchSize); - return new ParquetDecimalVector(internalVector); + HeapDecimalVector decv = new HeapDecimalVector(batchSize); + if (value == null) { + decv.fillWithNulls(); + } else { + DecimalType decimalType = (DecimalType) type; + int precision = decimalType.getPrecision(); + int scale = decimalType.getScale(); + DecimalData decimal = Preconditions.checkNotNull( + DecimalData.fromBigDecimal((BigDecimal) value, precision, scale)); + decv.fill(decimal.toUnscaledBytes()); + } + return decv; case FLOAT: HeapFloatVector fv = new HeapFloatVector(batchSize); if (value == null) { @@ -270,6 +281,41 @@ private static ColumnVector createVectorFromConstant( tv.fill(TimestampData.fromLocalDateTime((LocalDateTime) value)); } return tv; + case ARRAY: + ArrayType arrayType = (ArrayType) type; + if (arrayType.getElementType().isAnyOf(LogicalTypeFamily.CONSTRUCTED)) { + HeapArrayGroupColumnVector arrayGroup = new HeapArrayGroupColumnVector(batchSize); + if (value == null) { + arrayGroup.fillWithNulls(); + return arrayGroup; + } else { + throw new UnsupportedOperationException("Unsupported create array with default value."); + } + } else { + HeapArrayVector arrayVector = new HeapArrayVector(batchSize); + if (value == null) { + arrayVector.fillWithNulls(); + return arrayVector; + } else { + throw new UnsupportedOperationException("Unsupported create array with default value."); + } + } + case MAP: + HeapMapColumnVector mapVector = new HeapMapColumnVector(batchSize, null, null); + if (value == null) { + mapVector.fillWithNulls(); + return mapVector; + } else { + throw new UnsupportedOperationException("Unsupported create map with default value."); + } + case ROW: + HeapRowColumnVector rowVector = new HeapRowColumnVector(batchSize); + if (value == null) { + rowVector.fillWithNulls(); + return rowVector; + } else { + throw new UnsupportedOperationException("Unsupported create row with default value."); + } default: throw new UnsupportedOperationException("Unsupported type: " + type); } @@ -359,12 +405,23 @@ private static ColumnReader createColumnReader( throw new AssertionError(); } case ARRAY: - return new ArrayColumnReader( - descriptor, - pageReader, - utcTimestamp, - descriptor.getPrimitiveType(), - fieldType); + ArrayType arrayType = (ArrayType) fieldType; + if (arrayType.getElementType().isAnyOf(LogicalTypeFamily.CONSTRUCTED)) { + return new ArrayGroupReader(createColumnReader( + utcTimestamp, + arrayType.getElementType(), + physicalType.asGroupType().getType(0), + descriptors, + pages, + depth + 1)); + } else { + return new ArrayColumnReader( + descriptor, + pageReader, + utcTimestamp, + descriptor.getPrimitiveType(), + fieldType); + } case MAP: MapType mapType = (MapType) fieldType; ArrayColumnReader keyReader = @@ -374,27 +431,71 @@ private static ColumnReader createColumnReader( utcTimestamp, descriptor.getPrimitiveType(), new ArrayType(mapType.getKeyType())); - ArrayColumnReader valueReader = - new ArrayColumnReader( - descriptors.get(1), - pages.getPageReader(descriptors.get(1)), - utcTimestamp, - descriptors.get(1).getPrimitiveType(), - new ArrayType(mapType.getValueType())); - return new MapColumnReader(keyReader, valueReader, fieldType); + ColumnReader valueReader; + if (mapType.getValueType().isAnyOf(LogicalTypeFamily.CONSTRUCTED)) { + valueReader = new ArrayGroupReader(createColumnReader( + utcTimestamp, + mapType.getValueType(), + physicalType.asGroupType().getType(0).asGroupType().getType(1), // Get the value physical type + descriptors.subList(1, descriptors.size()), // remove the key descriptor + pages, + depth + 2)); // increase the depth by 2, because there's a key_value entry in the path + } else { + valueReader = new ArrayColumnReader( + descriptors.get(1), + pages.getPageReader(descriptors.get(1)), + utcTimestamp, + descriptors.get(1).getPrimitiveType(), + new ArrayType(mapType.getValueType())); + } + return new MapColumnReader(keyReader, valueReader); case ROW: RowType rowType = (RowType) fieldType; GroupType groupType = physicalType.asGroupType(); List fieldReaders = new ArrayList<>(); for (int i = 0; i < rowType.getFieldCount(); i++) { - fieldReaders.add( - createColumnReader( - utcTimestamp, - rowType.getTypeAt(i), - groupType.getType(i), - descriptors, - pages, - depth + 1)); + // schema evolution: read the parquet file with a new extended field name. + int fieldIndex = getFieldIndexInPhysicalType(rowType.getFields().get(i).getName(), groupType); + if (fieldIndex < 0) { + fieldReaders.add(new EmptyColumnReader()); + } else { + // Check for nested row in array with atomic field type. + + // This is done to meet the Parquet field algorithm that pushes multiplicity and structures down to individual fields. + // In Parquet, an array of rows is stored as separate arrays for each field. + + // Limitations: It won't work for multiple nested arrays and maps. + // The main problem is that the Flink classes and interface don't follow that pattern. + if (getLegacyFieldIndexInPhysicalType(rowType.getFields().get(i).getName(), groupType) != -1) { + // This is a legacy field, which means it's an array of rows with atomic fields. + fieldReaders.add( + createColumnReader( + utcTimestamp, + new ArrayType(rowType.getTypeAt(i).isNullable(), rowType.getTypeAt(i)), + groupType.getType(0).asGroupType().getType(fieldIndex), + descriptors, + pages, + depth + 2)); + } else if (descriptors.get(fieldIndex).getMaxRepetitionLevel() > 0 && !rowType.getTypeAt(i).is(LogicalTypeRoot.ARRAY)) { + fieldReaders.add( + createColumnReader( + utcTimestamp, + new ArrayType(rowType.getTypeAt(i).isNullable(), rowType.getTypeAt(i)), + groupType.getType(fieldIndex), + descriptors, + pages, + depth + 1)); + } else { + fieldReaders.add( + createColumnReader( + utcTimestamp, + rowType.getTypeAt(i), + groupType.getType(fieldIndex), + descriptors, + pages, + depth + 1)); + } + } } return new RowColumnReader(fieldReaders); default: @@ -476,52 +577,134 @@ private static WritableColumnVector createWritableColumnVector( || typeName == PrimitiveType.PrimitiveTypeName.BINARY) && primitiveType.getOriginalType() == OriginalType.DECIMAL, "Unexpected type: %s", typeName); - return new HeapBytesVector(batchSize); + return new HeapDecimalVector(batchSize); case ARRAY: ArrayType arrayType = (ArrayType) fieldType; - return new HeapArrayVector( - batchSize, - createWritableColumnVector( - batchSize, - arrayType.getElementType(), - physicalType, - descriptors, - depth)); + if (arrayType.getElementType().isAnyOf(LogicalTypeFamily.CONSTRUCTED)) { + return new HeapArrayGroupColumnVector( + batchSize, + createWritableColumnVector( + batchSize, + arrayType.getElementType(), + physicalType.asGroupType().getType(0), + descriptors, + depth + 1)); + } else { + return new HeapArrayVector( + batchSize, + createWritableColumnVector( + batchSize, + arrayType.getElementType(), + physicalType, + descriptors, + depth)); + } case MAP: MapType mapType = (MapType) fieldType; GroupType repeatedType = physicalType.asGroupType().getType(0).asGroupType(); // the map column has three level paths. - return new HeapMapColumnVector( + WritableColumnVector keyColumnVector = createWritableColumnVector( batchSize, - createWritableColumnVector( - batchSize, - mapType.getKeyType(), - repeatedType.getType(0), - descriptors, - depth + 2), - createWritableColumnVector( - batchSize, - mapType.getValueType(), - repeatedType.getType(1), - descriptors, - depth + 2)); + new ArrayType(mapType.getKeyType().isNullable(), mapType.getKeyType()), + repeatedType.getType(0), + descriptors, + depth + 2); + WritableColumnVector valueColumnVector; + if (mapType.getValueType().isAnyOf(LogicalTypeFamily.CONSTRUCTED)) { + valueColumnVector = new HeapArrayGroupColumnVector( + batchSize, + createWritableColumnVector( + batchSize, + mapType.getValueType(), + repeatedType.getType(1).asGroupType(), + descriptors, + depth + 2)); + } else { + valueColumnVector = createWritableColumnVector( + batchSize, + new ArrayType(mapType.getValueType().isNullable(), mapType.getValueType()), + repeatedType.getType(1), + descriptors, + depth + 2); + } + return new HeapMapColumnVector(batchSize, keyColumnVector, valueColumnVector); case ROW: RowType rowType = (RowType) fieldType; GroupType groupType = physicalType.asGroupType(); - WritableColumnVector[] columnVectors = - new WritableColumnVector[rowType.getFieldCount()]; + WritableColumnVector[] columnVectors = new WritableColumnVector[rowType.getFieldCount()]; for (int i = 0; i < columnVectors.length; i++) { - columnVectors[i] = - createWritableColumnVector( - batchSize, - rowType.getTypeAt(i), - groupType.getType(i), - descriptors, - depth + 1); + // schema evolution: read the file with a new extended field name. + int fieldIndex = getFieldIndexInPhysicalType(rowType.getFields().get(i).getName(), groupType); + if (fieldIndex < 0) { + // Check for nested row in array with atomic field type. + + // This is done to meet the Parquet field algorithm that pushes multiplicity and structures down to individual fields. + // In Parquet, an array of rows is stored as separate arrays for each field. + + // Limitations: It won't work for multiple nested arrays and maps. + // The main problem is that the Flink classes and interface don't follow that pattern. + if (groupType.getRepetition().equals(Type.Repetition.REPEATED) && !rowType.getTypeAt(i).is(LogicalTypeRoot.ARRAY)) { + columnVectors[i] = (WritableColumnVector) createVectorFromConstant( + new ArrayType(rowType.getTypeAt(i).isNullable(), rowType.getTypeAt(i)), null, batchSize); + } else { + columnVectors[i] = (WritableColumnVector) createVectorFromConstant(rowType.getTypeAt(i), null, batchSize); + } + } else { + // Check for nested row in array with atomic field type. + + // This is done to meet the Parquet field algorithm that pushes multiplicity and structures down to individual fields. + // In Parquet, an array of rows is stored as separate arrays for each field. + + // Limitations: It won't work for multiple nested arrays and maps. + // The main problem is that the Flink classes and interface don't follow that pattern. + if (getLegacyFieldIndexInPhysicalType(rowType.getFields().get(i).getName(), groupType) != -1) { + // This is a legacy field, which means it's an array of rows with atomic fields. + columnVectors[i] = + createWritableColumnVector( + batchSize, + new ArrayType(rowType.getTypeAt(i).isNullable(), rowType.getTypeAt(i)), + groupType.getType(0).asGroupType().getType(fieldIndex), + descriptors, + depth + 2); + } else if (descriptors.get(fieldIndex).getMaxRepetitionLevel() > 0 && !rowType.getTypeAt(i).is(LogicalTypeRoot.ARRAY)) { + columnVectors[i] = + createWritableColumnVector( + batchSize, + new ArrayType(rowType.getTypeAt(i).isNullable(), rowType.getTypeAt(i)), + groupType.getType(fieldIndex), + descriptors, + depth + 1); + } else { + columnVectors[i] = + createWritableColumnVector( + batchSize, + rowType.getTypeAt(i), + groupType.getType(fieldIndex), + descriptors, + depth + 1); + } + } } return new HeapRowColumnVector(batchSize, columnVectors); default: throw new UnsupportedOperationException(fieldType + " is not supported now."); } } + + private static int getFieldIndexInPhysicalType(String fieldName, GroupType groupType) { + // get index from fileSchema type, else, return -1 + return groupType.containsField(fieldName) ? groupType.getFieldIndex(fieldName) : getLegacyFieldIndexInPhysicalType(fieldName, groupType); + } + + // Compatible with existing arrayfiles + private static int getLegacyFieldIndexInPhysicalType(String fieldName, GroupType groupType) { + if ("list".equals(groupType.getName()) && groupType.getFieldCount() == 1 && "element".equals(groupType.getFieldName(0))) { + Type elementType = groupType.getType("element"); + if (elementType instanceof GroupType) { + GroupType elementGroupType = elementType.asGroupType(); + return elementGroupType.containsField(fieldName) ? elementGroupType.getFieldIndex(fieldName) : -1; + } + } + return -1; + } } diff --git a/hudi-flink-datasource/hudi-flink1.16.x/src/main/java/org/apache/hudi/table/format/cow/vector/ColumnarGroupArrayData.java b/hudi-flink-datasource/hudi-flink1.16.x/src/main/java/org/apache/hudi/table/format/cow/vector/ColumnarGroupArrayData.java new file mode 100644 index 0000000000000..4c9275f3b0932 --- /dev/null +++ b/hudi-flink-datasource/hudi-flink1.16.x/src/main/java/org/apache/hudi/table/format/cow/vector/ColumnarGroupArrayData.java @@ -0,0 +1,179 @@ +/* + * 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.format.cow.vector; + +import org.apache.flink.table.data.ArrayData; +import org.apache.flink.table.data.DecimalData; +import org.apache.flink.table.data.MapData; +import org.apache.flink.table.data.RawValueData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.data.TimestampData; +import org.apache.flink.table.data.columnar.vector.writable.WritableColumnVector; + +public class ColumnarGroupArrayData implements ArrayData { + + WritableColumnVector vector; + int rowId; + + public ColumnarGroupArrayData(WritableColumnVector vector, int rowId) { + this.vector = vector; + this.rowId = rowId; + } + + @Override + public int size() { + if (vector == null) { + return 0; + } + + if (vector instanceof HeapRowColumnVector) { + // assume all fields have the same size + if (((HeapRowColumnVector) vector).vectors == null || ((HeapRowColumnVector) vector).vectors.length == 0) { + return 0; + } + return ((HeapArrayVector) ((HeapRowColumnVector) vector).vectors[0]).getArray(rowId).size(); + } + throw new UnsupportedOperationException(vector.getClass().getName() + " is not supported. Supported vector types: HeapRowColumnVector"); + } + + @Override + public boolean isNullAt(int index) { + if (vector == null) { + return true; + } + + if (vector instanceof HeapRowColumnVector) { + return ((HeapRowColumnVector) vector).vectors == null; + } + + throw new UnsupportedOperationException(vector.getClass().getName() + " is not supported. Supported vector types: HeapRowColumnVector"); + } + + @Override + public boolean getBoolean(int index) { + throw new UnsupportedOperationException("Not support the operation!"); + } + + @Override + public byte getByte(int index) { + throw new UnsupportedOperationException("Not support the operation!"); + } + + @Override + public short getShort(int index) { + throw new UnsupportedOperationException("Not support the operation!"); + } + + @Override + public int getInt(int index) { + throw new UnsupportedOperationException("Not support the operation!"); + } + + @Override + public long getLong(int index) { + throw new UnsupportedOperationException("Not support the operation!"); + } + + @Override + public float getFloat(int index) { + throw new UnsupportedOperationException("Not support the operation!"); + } + + @Override + public double getDouble(int index) { + throw new UnsupportedOperationException("Not support the operation!"); + } + + @Override + public StringData getString(int index) { + throw new UnsupportedOperationException("Not support the operation!"); + } + + @Override + public DecimalData getDecimal(int index, int precision, int scale) { + throw new UnsupportedOperationException("Not support the operation!"); + } + + @Override + public TimestampData getTimestamp(int index, int precision) { + throw new UnsupportedOperationException("Not support the operation!"); + } + + @Override + public RawValueData getRawValue(int index) { + throw new UnsupportedOperationException("Not support the operation!"); + } + + @Override + public byte[] getBinary(int index) { + throw new UnsupportedOperationException("Not support the operation!"); + } + + @Override + public ArrayData getArray(int index) { + throw new UnsupportedOperationException("Not support the operation!"); + } + + @Override + public MapData getMap(int index) { + throw new UnsupportedOperationException("Not support the operation!"); + } + + @Override + public RowData getRow(int index, int numFields) { + return new ColumnarGroupRowData((HeapRowColumnVector) vector, rowId, index); + } + + @Override + public boolean[] toBooleanArray() { + throw new UnsupportedOperationException("Not support the operation!"); + } + + @Override + public byte[] toByteArray() { + throw new UnsupportedOperationException("Not support the operation!"); + } + + @Override + public short[] toShortArray() { + throw new UnsupportedOperationException("Not support the operation!"); + } + + @Override + public int[] toIntArray() { + throw new UnsupportedOperationException("Not support the operation!"); + } + + @Override + public long[] toLongArray() { + throw new UnsupportedOperationException("Not support the operation!"); + } + + @Override + public float[] toFloatArray() { + throw new UnsupportedOperationException("Not support the operation!"); + } + + @Override + public double[] toDoubleArray() { + throw new UnsupportedOperationException("Not support the operation!"); + } + +} diff --git a/hudi-flink-datasource/hudi-flink1.16.x/src/main/java/org/apache/hudi/table/format/cow/vector/ColumnarGroupMapData.java b/hudi-flink-datasource/hudi-flink1.16.x/src/main/java/org/apache/hudi/table/format/cow/vector/ColumnarGroupMapData.java new file mode 100644 index 0000000000000..69cb6feca13e4 --- /dev/null +++ b/hudi-flink-datasource/hudi-flink1.16.x/src/main/java/org/apache/hudi/table/format/cow/vector/ColumnarGroupMapData.java @@ -0,0 +1,63 @@ +/* + * 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.format.cow.vector; + +import org.apache.flink.table.data.ArrayData; +import org.apache.flink.table.data.MapData; +import org.apache.flink.table.data.columnar.vector.writable.WritableColumnVector; + +public class ColumnarGroupMapData implements MapData { + + WritableColumnVector keyVector; + WritableColumnVector valueVector; + int rowId; + + public ColumnarGroupMapData(WritableColumnVector keyVector, WritableColumnVector valueVector, int rowId) { + this.keyVector = keyVector; + this.valueVector = valueVector; + this.rowId = rowId; + } + + @Override + public int size() { + if (keyVector == null) { + return 0; + } + + if (keyVector instanceof HeapArrayVector) { + return ((HeapArrayVector) keyVector).getArray(rowId).size(); + } + throw new UnsupportedOperationException(keyVector.getClass().getName() + " is not supported. Supported vector types: HeapArrayVector"); + } + + @Override + public ArrayData keyArray() { + return ((HeapArrayVector) keyVector).getArray(rowId); + } + + @Override + public ArrayData valueArray() { + if (valueVector instanceof HeapArrayVector) { + return ((HeapArrayVector) valueVector).getArray(rowId); + } else if (valueVector instanceof HeapArrayGroupColumnVector) { + return ((HeapArrayGroupColumnVector) valueVector).getArray(rowId); + } + throw new UnsupportedOperationException(valueVector.getClass().getName() + " is not supported. Supported vector types: HeapArrayVector, HeapArrayGroupColumnVector"); + } +} diff --git a/hudi-flink-datasource/hudi-flink1.16.x/src/main/java/org/apache/hudi/table/format/cow/vector/ColumnarGroupRowData.java b/hudi-flink-datasource/hudi-flink1.16.x/src/main/java/org/apache/hudi/table/format/cow/vector/ColumnarGroupRowData.java new file mode 100644 index 0000000000000..439c1880823f1 --- /dev/null +++ b/hudi-flink-datasource/hudi-flink1.16.x/src/main/java/org/apache/hudi/table/format/cow/vector/ColumnarGroupRowData.java @@ -0,0 +1,138 @@ +/* + * 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.format.cow.vector; + +import org.apache.flink.table.data.ArrayData; +import org.apache.flink.table.data.DecimalData; +import org.apache.flink.table.data.MapData; +import org.apache.flink.table.data.RawValueData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.data.TimestampData; +import org.apache.flink.types.RowKind; + +public class ColumnarGroupRowData implements RowData { + + HeapRowColumnVector vector; + int rowId; + int index; + + public ColumnarGroupRowData(HeapRowColumnVector vector, int rowId, int index) { + this.vector = vector; + this.rowId = rowId; + this.index = index; + } + + @Override + public int getArity() { + return vector.vectors.length; + } + + @Override + public RowKind getRowKind() { + return RowKind.INSERT; + } + + @Override + public void setRowKind(RowKind rowKind) { + throw new UnsupportedOperationException("Not support the operation!"); + } + + @Override + public boolean isNullAt(int pos) { + return + vector.vectors[pos].isNullAt(rowId) + || ((HeapArrayVector) (vector.vectors[pos])).getArray(rowId).isNullAt(index); + } + + @Override + public boolean getBoolean(int pos) { + return ((HeapArrayVector) (vector.vectors[pos])).getArray(rowId).getBoolean(index); + } + + @Override + public byte getByte(int pos) { + return ((HeapArrayVector) (vector.vectors[pos])).getArray(rowId).getByte(index); + } + + @Override + public short getShort(int pos) { + return ((HeapArrayVector) (vector.vectors[pos])).getArray(rowId).getShort(index); + } + + @Override + public int getInt(int pos) { + return ((HeapArrayVector) (vector.vectors[pos])).getArray(rowId).getInt(index); + } + + @Override + public long getLong(int pos) { + return ((HeapArrayVector) (vector.vectors[pos])).getArray(rowId).getLong(index); + } + + @Override + public float getFloat(int pos) { + return ((HeapArrayVector) (vector.vectors[pos])).getArray(rowId).getFloat(index); + } + + @Override + public double getDouble(int pos) { + return ((HeapArrayVector) (vector.vectors[pos])).getArray(rowId).getDouble(index); + } + + @Override + public StringData getString(int pos) { + return ((HeapArrayVector) (vector.vectors[pos])).getArray(rowId).getString(index); + } + + @Override + public DecimalData getDecimal(int pos, int i1, int i2) { + return ((HeapArrayVector) (vector.vectors[pos])).getArray(rowId).getDecimal(index, i1, i2); + } + + @Override + public TimestampData getTimestamp(int pos, int i1) { + return ((HeapArrayVector) (vector.vectors[pos])).getArray(rowId).getTimestamp(index, i1); + } + + @Override + public RawValueData getRawValue(int pos) { + return ((HeapArrayVector) (vector.vectors[pos])).getArray(rowId).getRawValue(index); + } + + @Override + public byte[] getBinary(int pos) { + return ((HeapArrayVector) (vector.vectors[pos])).getArray(rowId).getBinary(index); + } + + @Override + public ArrayData getArray(int pos) { + return ((HeapArrayVector) (vector.vectors[pos])).getArray(rowId).getArray(index); + } + + @Override + public MapData getMap(int pos) { + return ((HeapArrayVector) (vector.vectors[pos])).getArray(rowId).getMap(index); + } + + @Override + public RowData getRow(int pos, int numFields) { + return ((HeapArrayVector) (vector.vectors[pos])).getArray(rowId).getRow(index, numFields); + } +} diff --git a/hudi-flink-datasource/hudi-flink1.16.x/src/main/java/org/apache/hudi/table/format/cow/vector/HeapArrayGroupColumnVector.java b/hudi-flink-datasource/hudi-flink1.16.x/src/main/java/org/apache/hudi/table/format/cow/vector/HeapArrayGroupColumnVector.java new file mode 100644 index 0000000000000..3d7d8b1f0de0f --- /dev/null +++ b/hudi-flink-datasource/hudi-flink1.16.x/src/main/java/org/apache/hudi/table/format/cow/vector/HeapArrayGroupColumnVector.java @@ -0,0 +1,53 @@ +/* + * 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.format.cow.vector; + +import org.apache.flink.table.data.ArrayData; +import org.apache.flink.table.data.columnar.vector.ArrayColumnVector; +import org.apache.flink.table.data.columnar.vector.heap.AbstractHeapVector; +import org.apache.flink.table.data.columnar.vector.writable.WritableColumnVector; + +/** + * This class represents a nullable heap row column vector. + */ +public class HeapArrayGroupColumnVector extends AbstractHeapVector + implements WritableColumnVector, ArrayColumnVector { + + public WritableColumnVector vector; + + public HeapArrayGroupColumnVector(int len) { + super(len); + } + + public HeapArrayGroupColumnVector(int len, WritableColumnVector vector) { + super(len); + this.vector = vector; + } + + @Override + public ArrayData getArray(int rowId) { + return new ColumnarGroupArrayData(vector, rowId); + } + + @Override + public void reset() { + super.reset(); + vector.reset(); + } +} diff --git a/hudi-flink-datasource/hudi-flink1.16.x/src/main/java/org/apache/hudi/table/format/cow/vector/HeapDecimalVector.java b/hudi-flink-datasource/hudi-flink1.16.x/src/main/java/org/apache/hudi/table/format/cow/vector/HeapDecimalVector.java new file mode 100644 index 0000000000000..ba1936c1c41db --- /dev/null +++ b/hudi-flink-datasource/hudi-flink1.16.x/src/main/java/org/apache/hudi/table/format/cow/vector/HeapDecimalVector.java @@ -0,0 +1,39 @@ +/* + * 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.format.cow.vector; + +import org.apache.flink.table.data.DecimalData; +import org.apache.flink.table.data.columnar.vector.DecimalColumnVector; +import org.apache.flink.table.data.columnar.vector.heap.HeapBytesVector; + +/** + * This class represents a nullable heap map decimal vector. + */ +public class HeapDecimalVector extends HeapBytesVector implements DecimalColumnVector { + + public HeapDecimalVector(int len) { + super(len); + } + + @Override + public DecimalData getDecimal(int i, int precision, int scale) { + return DecimalData.fromUnscaledBytes( + this.getBytes(i).getBytes(), precision, scale); + } +} \ No newline at end of file diff --git a/hudi-flink-datasource/hudi-flink1.16.x/src/main/java/org/apache/hudi/table/format/cow/vector/HeapMapColumnVector.java b/hudi-flink-datasource/hudi-flink1.16.x/src/main/java/org/apache/hudi/table/format/cow/vector/HeapMapColumnVector.java index a379737169502..95d8fd720d300 100644 --- a/hudi-flink-datasource/hudi-flink1.16.x/src/main/java/org/apache/hudi/table/format/cow/vector/HeapMapColumnVector.java +++ b/hudi-flink-datasource/hudi-flink1.16.x/src/main/java/org/apache/hudi/table/format/cow/vector/HeapMapColumnVector.java @@ -19,8 +19,6 @@ package org.apache.hudi.table.format.cow.vector; import org.apache.flink.table.data.MapData; -import org.apache.flink.table.data.columnar.ColumnarMapData; -import org.apache.flink.table.data.columnar.vector.ColumnVector; import org.apache.flink.table.data.columnar.vector.MapColumnVector; import org.apache.flink.table.data.columnar.vector.heap.AbstractHeapVector; import org.apache.flink.table.data.columnar.vector.writable.WritableColumnVector; @@ -31,49 +29,25 @@ public class HeapMapColumnVector extends AbstractHeapVector implements WritableColumnVector, MapColumnVector { - private long[] offsets; - private long[] lengths; - private int size; - private ColumnVector keys; - private ColumnVector values; + private WritableColumnVector keys; + private WritableColumnVector values; - public HeapMapColumnVector(int len, ColumnVector keys, ColumnVector values) { + public HeapMapColumnVector(int len, WritableColumnVector keys, WritableColumnVector values) { super(len); - size = 0; - offsets = new long[len]; - lengths = new long[len]; this.keys = keys; this.values = values; } - public void setOffsets(long[] offsets) { - this.offsets = offsets; + public WritableColumnVector getKeys() { + return keys; } - public void setLengths(long[] lengths) { - this.lengths = lengths; - } - - public void setKeys(ColumnVector keys) { - this.keys = keys; - } - - public void setValues(ColumnVector values) { - this.values = values; - } - - public int getSize() { - return size; - } - - public void setSize(int size) { - this.size = size; + public WritableColumnVector getValues() { + return values; } @Override - public MapData getMap(int i) { - long offset = offsets[i]; - long length = lengths[i]; - return new ColumnarMapData(keys, values, (int) offset, (int) length); + public MapData getMap(int rowId) { + return new ColumnarGroupMapData(keys, values, rowId); } } diff --git a/hudi-flink-datasource/hudi-flink1.16.x/src/main/java/org/apache/hudi/table/format/cow/vector/reader/ArrayGroupReader.java b/hudi-flink-datasource/hudi-flink1.16.x/src/main/java/org/apache/hudi/table/format/cow/vector/reader/ArrayGroupReader.java new file mode 100644 index 0000000000000..df7c5d85bc4ab --- /dev/null +++ b/hudi-flink-datasource/hudi-flink1.16.x/src/main/java/org/apache/hudi/table/format/cow/vector/reader/ArrayGroupReader.java @@ -0,0 +1,45 @@ +/* + * 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.format.cow.vector.reader; + +import org.apache.hudi.table.format.cow.vector.HeapArrayGroupColumnVector; + +import org.apache.flink.formats.parquet.vector.reader.ColumnReader; +import org.apache.flink.table.data.columnar.vector.writable.WritableColumnVector; + +import java.io.IOException; + +/** + * Array of a Group type (Array, Map, Row, etc.) {@link ColumnReader}. + */ +public class ArrayGroupReader implements ColumnReader { + + private final ColumnReader fieldReader; + + public ArrayGroupReader(ColumnReader fieldReader) { + this.fieldReader = fieldReader; + } + + @Override + public void readToVector(int readNumber, WritableColumnVector vector) throws IOException { + HeapArrayGroupColumnVector rowColumnVector = (HeapArrayGroupColumnVector) vector; + + fieldReader.readToVector(readNumber, rowColumnVector.vector); + } +} diff --git a/hudi-flink-datasource/hudi-flink1.16.x/src/main/java/org/apache/hudi/table/format/cow/vector/reader/EmptyColumnReader.java b/hudi-flink-datasource/hudi-flink1.16.x/src/main/java/org/apache/hudi/table/format/cow/vector/reader/EmptyColumnReader.java new file mode 100644 index 0000000000000..40f3f8de1b834 --- /dev/null +++ b/hudi-flink-datasource/hudi-flink1.16.x/src/main/java/org/apache/hudi/table/format/cow/vector/reader/EmptyColumnReader.java @@ -0,0 +1,43 @@ +/* + * 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.format.cow.vector.reader; + +import org.apache.flink.formats.parquet.vector.reader.ColumnReader; +import org.apache.flink.table.data.columnar.vector.writable.WritableColumnVector; + +import java.io.IOException; + +/** + * Empty {@link ColumnReader}. + *

    + * This reader is to handle parquet files that have not been updated to the latest Schema. + * When reading a parquet file with the latest schema, parquet file might not have the new field. + * The EmptyColumnReader is used to handle such scenarios. + */ +public class EmptyColumnReader implements ColumnReader { + + public EmptyColumnReader() { + } + + @Override + public void readToVector(int readNumber, WritableColumnVector vector) throws IOException { + vector.fillWithNulls(); + } +} + diff --git a/hudi-flink-datasource/hudi-flink1.16.x/src/main/java/org/apache/hudi/table/format/cow/vector/reader/MapColumnReader.java b/hudi-flink-datasource/hudi-flink1.16.x/src/main/java/org/apache/hudi/table/format/cow/vector/reader/MapColumnReader.java index a6762d2e175c1..ee65dd22c4369 100644 --- a/hudi-flink-datasource/hudi-flink1.16.x/src/main/java/org/apache/hudi/table/format/cow/vector/reader/MapColumnReader.java +++ b/hudi-flink-datasource/hudi-flink1.16.x/src/main/java/org/apache/hudi/table/format/cow/vector/reader/MapColumnReader.java @@ -18,14 +18,11 @@ package org.apache.hudi.table.format.cow.vector.reader; -import org.apache.hudi.table.format.cow.vector.HeapArrayVector; import org.apache.hudi.table.format.cow.vector.HeapMapColumnVector; import org.apache.flink.formats.parquet.vector.reader.ColumnReader; -import org.apache.flink.table.data.columnar.vector.ColumnVector; +import org.apache.flink.table.data.columnar.vector.heap.AbstractHeapVector; import org.apache.flink.table.data.columnar.vector.writable.WritableColumnVector; -import org.apache.flink.table.types.logical.LogicalType; -import org.apache.flink.table.types.logical.MapType; import java.io.IOException; @@ -34,43 +31,26 @@ */ public class MapColumnReader implements ColumnReader { - private final LogicalType logicalType; private final ArrayColumnReader keyReader; - private final ArrayColumnReader valueReader; + private final ColumnReader valueReader; public MapColumnReader( - ArrayColumnReader keyReader, ArrayColumnReader valueReader, LogicalType logicalType) { + ArrayColumnReader keyReader, ColumnReader valueReader) { this.keyReader = keyReader; this.valueReader = valueReader; - this.logicalType = logicalType; } - public void readBatch(int total, ColumnVector column) throws IOException { - HeapMapColumnVector mapColumnVector = (HeapMapColumnVector) column; - MapType mapType = (MapType) logicalType; - // initialize 2 ListColumnVector for keys and values - HeapArrayVector keyArrayColumnVector = new HeapArrayVector(total); - HeapArrayVector valueArrayColumnVector = new HeapArrayVector(total); - // read the keys and values - keyReader.readToVector(total, keyArrayColumnVector); - valueReader.readToVector(total, valueArrayColumnVector); - - // set the related attributes according to the keys and values - mapColumnVector.setKeys(keyArrayColumnVector.child); - mapColumnVector.setValues(valueArrayColumnVector.child); - mapColumnVector.setOffsets(keyArrayColumnVector.offsets); - mapColumnVector.setLengths(keyArrayColumnVector.lengths); - mapColumnVector.setSize(keyArrayColumnVector.getSize()); + @Override + public void readToVector(int readNumber, WritableColumnVector vector) throws IOException { + HeapMapColumnVector mapColumnVector = (HeapMapColumnVector) vector; + AbstractHeapVector keyArrayColumnVector = (AbstractHeapVector) (mapColumnVector.getKeys()); + keyReader.readToVector(readNumber, mapColumnVector.getKeys()); + valueReader.readToVector(readNumber, mapColumnVector.getValues()); for (int i = 0; i < keyArrayColumnVector.getLen(); i++) { if (keyArrayColumnVector.isNullAt(i)) { mapColumnVector.setNullAt(i); } } } - - @Override - public void readToVector(int readNumber, WritableColumnVector vector) throws IOException { - readBatch(readNumber, vector); - } } diff --git a/hudi-flink-datasource/hudi-flink1.16.x/src/main/java/org/apache/hudi/table/format/cow/vector/reader/ParquetColumnarRowSplitReader.java b/hudi-flink-datasource/hudi-flink1.16.x/src/main/java/org/apache/hudi/table/format/cow/vector/reader/ParquetColumnarRowSplitReader.java index 1872ec385b4a9..56731fb24c312 100644 --- a/hudi-flink-datasource/hudi-flink1.16.x/src/main/java/org/apache/hudi/table/format/cow/vector/reader/ParquetColumnarRowSplitReader.java +++ b/hudi-flink-datasource/hudi-flink1.16.x/src/main/java/org/apache/hudi/table/format/cow/vector/reader/ParquetColumnarRowSplitReader.java @@ -33,7 +33,9 @@ import org.apache.hadoop.fs.Path; import org.apache.parquet.column.ColumnDescriptor; import org.apache.parquet.column.page.PageReadStore; +import org.apache.parquet.filter.UnboundRecordFilter; import org.apache.parquet.filter2.compat.FilterCompat; +import org.apache.parquet.filter2.predicate.FilterPredicate; import org.apache.parquet.hadoop.ParquetFileReader; import org.apache.parquet.hadoop.metadata.BlockMetaData; import org.apache.parquet.hadoop.metadata.ParquetMetadata; @@ -56,7 +58,6 @@ import static org.apache.parquet.filter2.compat.RowGroupFilter.filterRowGroups; import static org.apache.parquet.format.converter.ParquetMetadataConverter.range; import static org.apache.parquet.hadoop.ParquetFileReader.readFooter; -import static org.apache.parquet.hadoop.ParquetInputFormat.getFilter; /** * This reader is used to read a {@link VectorizedColumnBatch} from input split. @@ -123,13 +124,15 @@ public ParquetColumnarRowSplitReader( int batchSize, Path path, long splitStart, - long splitLength) throws IOException { + long splitLength, + FilterPredicate filterPredicate, + UnboundRecordFilter recordFilter) throws IOException { this.utcTimestamp = utcTimestamp; this.batchSize = batchSize; // then we need to apply the predicate push down filter ParquetMetadata footer = readFooter(conf, path, range(splitStart, splitStart + splitLength)); MessageType fileSchema = footer.getFileMetaData().getSchema(); - FilterCompat.Filter filter = getFilter(conf); + FilterCompat.Filter filter = FilterCompat.get(filterPredicate, recordFilter); List blocks = filterRowGroups(filter, footer.getBlocks(), fileSchema); this.fileSchema = footer.getFileMetaData().getSchema(); diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/CommitTimeSparkRecordMerger.java b/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/CommitTimeSparkRecordMerger.java new file mode 100644 index 0000000000000..db5a96ddaa8dc --- /dev/null +++ b/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/CommitTimeSparkRecordMerger.java @@ -0,0 +1,80 @@ +/* + * 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; + +import org.apache.hudi.common.config.TypedProperties; +import org.apache.hudi.common.model.HoodieRecord; +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.avro.Schema; +import org.apache.spark.unsafe.types.UTF8String; + +import java.io.IOException; + +public class CommitTimeSparkRecordMerger extends HoodieSparkRecordMerger { + + @Override + public String getMergingStrategy() { + return COMMIT_TIME_BASED_MERGE_STRATEGY_UUID; + } + + @Override + public Option> merge(HoodieRecord older, Schema oldSchema, HoodieRecord newer, Schema newSchema, TypedProperties props) throws IOException { + ValidationUtils.checkArgument(older.getRecordType() == HoodieRecord.HoodieRecordType.SPARK); + ValidationUtils.checkArgument(newer.getRecordType() == HoodieRecord.HoodieRecordType.SPARK); + + if (newer.getData() == null) { + // Delete record + return Option.of(Pair.of(newer, newSchema)); + } + if (older.getData() == null) { + // use natural order for delete record + return Option.of(Pair.of(newer, newSchema)); + } + + Comparable olderCommitTime = (Comparable)older.getColumnValues(oldSchema, new String[] {HoodieRecord.COMMIT_TIME_METADATA_FIELD}, false)[0]; + Comparable newerCommitTime = (Comparable)newer.getColumnValues(newSchema, new String[] {HoodieRecord.COMMIT_TIME_METADATA_FIELD}, false)[0]; + + if (olderCommitTime.compareTo(newerCommitTime) > 0) { + return Option.of(Pair.of(older, oldSchema)); + } else if (olderCommitTime.compareTo(newerCommitTime) < 0) { + return Option.of(Pair.of(newer, newSchema)); + } else { + UTF8String oldCommitSeq = (UTF8String) older.getColumnValues(oldSchema, + new String[] {HoodieRecord.COMMIT_SEQNO_METADATA_FIELD}, false)[0]; + UTF8String newCommitSeq = (UTF8String) newer.getColumnValues(newSchema, + new String[] {HoodieRecord.COMMIT_SEQNO_METADATA_FIELD}, false)[0]; + Long olderCommitSeqNoValue = Long.valueOf(oldCommitSeq.toString()); + Long newerCommitSeqNoValue = Long.valueOf(newCommitSeq.toString()); + if (olderCommitSeqNoValue.compareTo(newerCommitSeqNoValue) > 0) { + return Option.of(Pair.of(older, oldSchema)); + } else { + return Option.of(Pair.of(newer, newSchema)); + } + } + } + + @Override + public HoodieRecord.HoodieRecordType getRecordType() { + return HoodieRecord.HoodieRecordType.SPARK; + } +} diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/EventTimeHoodieSparkRecordMerger.java b/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/EventTimeHoodieSparkRecordMerger.java new file mode 100644 index 0000000000000..2ff88c8021883 --- /dev/null +++ b/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/EventTimeHoodieSparkRecordMerger.java @@ -0,0 +1,87 @@ +/* + * 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; + +import org.apache.hudi.common.config.TypedProperties; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecord.HoodieRecordType; +import org.apache.hudi.common.model.HoodieRecordMerger; +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.avro.Schema; +import org.apache.spark.unsafe.types.UTF8String; + +import java.io.IOException; + +public class EventTimeHoodieSparkRecordMerger extends HoodieSparkRecordMerger { + + @Override + public String getMergingStrategy() { + return HoodieRecordMerger.EVENT_TIME_BASED_MERGE_STRATEGY_UUID; + } + + @Override + public Option> merge(HoodieRecord older, Schema oldSchema, HoodieRecord newer, Schema newSchema, TypedProperties props) throws IOException { + ValidationUtils.checkArgument(older.getRecordType() == HoodieRecordType.SPARK); + ValidationUtils.checkArgument(newer.getRecordType() == HoodieRecordType.SPARK); + + if (newer.getData() == null) { + // Delete record + return Option.of(Pair.of(newer, newSchema)); + } + if (older.getData() == null) { + // use natural order for delete record + return Option.of(Pair.of(newer, newSchema)); + } + if (older.getOrderingValue(oldSchema, props).compareTo(newer.getOrderingValue(newSchema, props)) > 0) { + return Option.of(Pair.of(older, oldSchema)); + } else if (older.getOrderingValue(oldSchema, props).compareTo(newer.getOrderingValue(newSchema, props)) < 0) { + return Option.of(Pair.of(newer, newSchema)); + } else { + Comparable olderCommitTime = (Comparable)older.getColumnValues(oldSchema, + new String[]{HoodieRecord.COMMIT_TIME_METADATA_FIELD}, false)[0]; + Comparable newerCommitTime = (Comparable)newer.getColumnValues(newSchema, + new String[]{HoodieRecord.COMMIT_TIME_METADATA_FIELD}, false)[0]; + if (olderCommitTime.compareTo(newerCommitTime) > 0) { + return Option.of(Pair.of(older, oldSchema)); + } else if (olderCommitTime.compareTo(newerCommitTime) < 0) { + return Option.of(Pair.of(newer, newSchema)); + } else { + UTF8String oldCommitSeqNo = (UTF8String)older.getColumnValues(oldSchema, + new String[] {HoodieRecord.COMMIT_SEQNO_METADATA_FIELD}, false)[0]; + UTF8String newCommitSeqNo = (UTF8String)newer.getColumnValues(newSchema, + new String[] {HoodieRecord.COMMIT_SEQNO_METADATA_FIELD}, false)[0]; + Long olderCommitSeqNoValue = Long.valueOf(oldCommitSeqNo.toString()); + Long newerCommitSeqNoValue = Long.valueOf(newCommitSeqNo.toString()); + if (olderCommitSeqNoValue.compareTo(newerCommitSeqNoValue) > 0) { + return Option.of(Pair.of(older, oldSchema)); + } else { + return Option.of(Pair.of(newer, newSchema)); + } + } + } + } + + @Override + public HoodieRecordType getRecordType() { + return HoodieRecordType.SPARK; + } +} \ No newline at end of file diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/commit/LSMBucketRescaleCommitActionExecutor.java b/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/commit/LSMBucketRescaleCommitActionExecutor.java new file mode 100644 index 0000000000000..9574c6f9a5773 --- /dev/null +++ b/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/commit/LSMBucketRescaleCommitActionExecutor.java @@ -0,0 +1,73 @@ +/* + * 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.commit; + +import org.apache.hudi.client.SparkRDDWriteClient; +import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.common.data.HoodieData; +import org.apache.hudi.common.model.HoodiePartitionMetadata; +import org.apache.hudi.common.storage.HoodieStorageStrategy; +import org.apache.hudi.common.storage.HoodieStorageStrategyFactory; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.index.bucket.BucketStrategist; +import org.apache.hudi.index.bucket.BucketStrategistFactory; +import org.apache.hudi.table.action.HoodieWriteMetadata; +import org.apache.spark.api.java.JavaRDD; + +import java.util.List; +import java.util.Map; + +public class LSMBucketRescaleCommitActionExecutor extends LSMDatasetBulkInsertOverwriteCommitActionExecutor { + + private Map> partitionAndReplaceFiledIds = null; + private BucketStrategist bucketStrategist; + private HoodieStorageStrategy hoodieStorageStrategy; + + public LSMBucketRescaleCommitActionExecutor(HoodieWriteConfig config, SparkRDDWriteClient writeClient, String instantTime) { + super(config, writeClient, instantTime); + } + + @Override + protected void preExecute() { + super.preExecute(); + this.bucketStrategist = BucketStrategistFactory.getInstant(writeConfig, table.getMetaClient().getFs()); + this.hoodieStorageStrategy = HoodieStorageStrategyFactory.getInstant(table.getMetaClient()); + } + + @Override + protected Map> getPartitionToReplacedFileIds(HoodieData writeStatuses) { + partitionAndReplaceFiledIds = super.getPartitionToReplacedFileIds(writeStatuses); + return partitionAndReplaceFiledIds; + } + + @Override + protected void afterExecute(HoodieWriteMetadata> result) { + super.afterExecute(result); + // modify affected partitions related meta file. + if (partitionAndReplaceFiledIds != null) { + partitionAndReplaceFiledIds.keySet().forEach(partitionPath -> { + int bucketNumber = bucketStrategist.computeBucketNumber(partitionPath); + hoodieStorageStrategy.getAllLocations(partitionPath, true).forEach(path -> { + HoodiePartitionMetadata meta = new HoodiePartitionMetadata(table.getMetaClient().getFs(), path); + meta.updatePartitionBucketNumber(instantTime, bucketNumber); + }); + }); + } + } +} diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/commit/LSMDatasetBulkInsertCommitActionExecutor.java b/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/commit/LSMDatasetBulkInsertCommitActionExecutor.java new file mode 100644 index 0000000000000..358898da17022 --- /dev/null +++ b/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/commit/LSMDatasetBulkInsertCommitActionExecutor.java @@ -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.hudi.commit; + +import org.apache.hudi.DataSourceWriteOptions; +import org.apache.hudi.HoodieDatasetBulkInsertHelper; +import org.apache.hudi.HoodieSparkUtils; +import org.apache.hudi.client.HoodieWriteResult; +import org.apache.hudi.client.SparkRDDWriteClient; +import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.common.data.HoodieData; +import org.apache.hudi.common.model.WriteOperationType; +import org.apache.hudi.common.table.HoodieTableConfig; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.ValidationUtils; +import org.apache.hudi.config.HoodieInternalConfig; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.exception.HoodieException; +import org.apache.hudi.execution.bulkinsert.BucketIndexBulkInsertPartitionerWithRows; +import org.apache.hudi.execution.bulkinsert.LSMBucketIndexBulkInsertPartitionerWithRows; +import org.apache.hudi.index.HoodieIndex; +import org.apache.hudi.internal.DataSourceInternalWriterHelper; +import org.apache.hudi.table.action.HoodieWriteMetadata; + +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SaveMode; + +import java.util.HashMap; +import java.util.Map; +import java.util.stream.Collectors; + +public class LSMDatasetBulkInsertCommitActionExecutor extends BaseDatasetBulkInsertCommitActionExecutor { + + public LSMDatasetBulkInsertCommitActionExecutor(HoodieWriteConfig config, + SparkRDDWriteClient writeClient, + String instantTime) { + super(config, writeClient, instantTime); + } + + @Override + protected void preExecute() { + // no op + } + + // copy from DatasetBulkInsertCommitActionExecutor + // TODO zhangyue143 还要支持Spark3.x + @Override + protected Option> doExecute(Dataset records, boolean arePartitionRecordsSorted) { + Map opts = writeConfig.getProps().entrySet().stream().collect(Collectors.toMap( + e -> String.valueOf(e.getKey()), + e -> String.valueOf(e.getValue()))); + Map optsOverrides = new HashMap<>(); + optsOverrides.put(HoodieInternalConfig.BULKINSERT_ARE_PARTITIONER_RECORDS_SORTED, String.valueOf(arePartitionRecordsSorted)); + optsOverrides.put(HoodieTableConfig.HOODIE_LOG_FORMAT.key(), HoodieTableConfig.LSM_HOODIE_TABLE_LOG_FORMAT); + + String targetFormat; + Map customOpts = new HashMap<>(1); + if (HoodieSparkUtils.isSpark2()) { + targetFormat = "org.apache.hudi.internal"; + } else if (HoodieSparkUtils.isSpark3()) { + targetFormat = "org.apache.hudi.spark3.internal"; + customOpts.put(HoodieInternalConfig.BULKINSERT_INPUT_DATA_SCHEMA_DDL.key(), records.schema().json()); + } else { + throw new HoodieException("Bulk insert using row writer is not supported with current Spark version." + + " To use row writer please switch to spark 2 or spark 3"); + } + + records.write().format(targetFormat) + .option(DataSourceInternalWriterHelper.INSTANT_TIME_OPT_KEY, instantTime) + .options(opts) + .options(customOpts) + .options(optsOverrides) + .mode(SaveMode.Append) + .save(); + return Option.empty(); + } + + @Override + public HoodieWriteResult execute(Dataset records, boolean isTablePartitioned) { + table = writeClient.initTable(getWriteOperationType(), Option.ofNullable(instantTime)); + BucketIndexBulkInsertPartitionerWithRows bulkInsertPartitionerRows = getPartitioner(true, isTablePartitioned); + boolean shouldDropPartitionColumns = writeConfig.getBoolean(DataSourceWriteOptions.DROP_PARTITION_COLUMNS()); + ValidationUtils.checkArgument(writeConfig.getIndexType() == HoodieIndex.IndexType.BUCKET); + Dataset hoodieDF; + // do repartition and sort by + hoodieDF = HoodieDatasetBulkInsertHelper.prepareForBulkInsertWithLSM(records, writeConfig, bulkInsertPartitionerRows, shouldDropPartitionColumns); + preExecute(); + // always do sort(sort by instead of order by) + HoodieWriteMetadata> result = buildHoodieWriteMetadata(doExecute(hoodieDF, true)); + afterExecute(result); + + return new HoodieWriteResult(result.getWriteStatuses(), result.getPartitionToReplaceFileIds()); + } + + @Override + protected BucketIndexBulkInsertPartitionerWithRows getPartitioner(boolean populateMetaFields, boolean isTablePartitioned) { + + return new LSMBucketIndexBulkInsertPartitionerWithRows(table.getConfig()); + } + + @Override + protected void afterExecute(HoodieWriteMetadata> result) { + // no op + } + + @Override + public WriteOperationType getWriteOperationType() { + return WriteOperationType.BULK_INSERT; + } +} diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/commit/LSMDatasetBulkInsertOverwriteCommitActionExecutor.java b/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/commit/LSMDatasetBulkInsertOverwriteCommitActionExecutor.java new file mode 100644 index 0000000000000..e4c1b125501d7 --- /dev/null +++ b/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/commit/LSMDatasetBulkInsertOverwriteCommitActionExecutor.java @@ -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.hudi.commit; + +import org.apache.hudi.HoodieDatasetBulkInsertHelper; +import org.apache.hudi.client.HoodieTimelineSkipInstanceClient; +import org.apache.hudi.client.SparkRDDWriteClient; +import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.common.data.HoodieData; +import org.apache.hudi.common.model.HoodieLogFile; +import org.apache.hudi.common.model.WriteOperationType; +import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.data.HoodieJavaPairRDD; +import org.apache.hudi.table.action.HoodieWriteMetadata; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; + +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +public class LSMDatasetBulkInsertOverwriteCommitActionExecutor extends LSMDatasetBulkInsertCommitActionExecutor { + + public LSMDatasetBulkInsertOverwriteCommitActionExecutor(HoodieWriteConfig config, + SparkRDDWriteClient writeClient, + String instantTime) { + super(config, writeClient, instantTime); + } + + @Override + protected void preExecute() { + HoodieTimelineSkipInstanceClient.storeBacktrackInstantToTmpForInsertOverwrite( + table.getMetaClient().getFs(), table.getMetaClient().getBacktrackingPath(), instantTime); + + table.validateInsertSchema(); + writeClient.startCommitWithTime(instantTime, getCommitActionType()); + writeClient.preWrite(instantTime, getWriteOperationType(), table.getMetaClient()); + } + + @Override + protected Option> doExecute(Dataset records, boolean arePartitionRecordsSorted) { + table.getActiveTimeline().transitionRequestedToInflight(new HoodieInstant(HoodieInstant.State.REQUESTED, getCommitActionType(), instantTime), Option.empty()); + return Option.of(HoodieDatasetBulkInsertHelper + .bulkInsert(records, instantTime, table, writeConfig, arePartitionRecordsSorted, false, getWriteOperationType())); + } + + @Override + protected void afterExecute(HoodieWriteMetadata> result) { + writeClient.postWrite(result, instantTime, table); + HoodieTimelineSkipInstanceClient.deleteBacktrackInstantFromTmpForInsertOverwrite(table.getMetaClient().getFs(), table.getMetaClient().getBacktrackingTmpPath(), instantTime); + } + + @Override + public WriteOperationType getWriteOperationType() { + return WriteOperationType.INSERT_OVERWRITE; + } + + @Override + protected Map> getPartitionToReplacedFileIds(HoodieData writeStatuses) { + if (!writeConfig.getReplacePartitionRecords()) { + return Collections.emptyMap(); + } + + Map> partitionAndReplaceFiledIds; + if (canTakeFirstPartition(writeConfig, writeStatuses)) { + String partition = writeStatuses.first().getStat().getPartitionPath(); + partitionAndReplaceFiledIds = Collections.singletonMap(partition, getAllExistingFileIds(partition)); + } else { + partitionAndReplaceFiledIds = HoodieJavaPairRDD.getJavaPairRDD(writeStatuses.map(status -> status.getStat().getPartitionPath()).distinct().mapToPair(partitionPath -> + Pair.of(partitionPath, getAllExistingFileIds(partitionPath)))).collectAsMap(); + } + + // TODO 是否需要兼容backtrack逻辑 + Boolean needCreateBacktrackingInstance = true; + for (Map.Entry> partitionLevelFileIds : partitionAndReplaceFiledIds.entrySet()) { + if (partitionLevelFileIds.getValue() != null && partitionLevelFileIds.getValue().size() > 0) { + needCreateBacktrackingInstance = false; + break; + } + } + if (needCreateBacktrackingInstance) { + HoodieTimelineSkipInstanceClient.storeBacktrackInstanceForInsertOverwrite(table.getMetaClient().getFs(), table.getMetaClient().getBacktrackingPath(), instantTime); + } + return partitionAndReplaceFiledIds; + } + + protected List getAllExistingFileIds(String partitionPath) { + return table.getSliceView().getLatestFileSlices(partitionPath) + .flatMap(fileSlice -> fileSlice.getLogFiles().map(HoodieLogFile::getFileName)).distinct().collect(Collectors.toList()); + } +} diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/commit/LSMDatasetBulkInsertOverwriteTableCommitActionExecutor.java b/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/commit/LSMDatasetBulkInsertOverwriteTableCommitActionExecutor.java new file mode 100644 index 0000000000000..f0ad73bc52c5e --- /dev/null +++ b/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/commit/LSMDatasetBulkInsertOverwriteTableCommitActionExecutor.java @@ -0,0 +1,83 @@ +/* + * 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.commit; + +import org.apache.hudi.client.SparkRDDWriteClient; +import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.common.data.HoodieData; +import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.model.WriteOperationType; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.data.HoodieJavaPairRDD; +import org.apache.hudi.table.action.HoodieWriteMetadata; +import org.apache.spark.api.java.JavaRDD; + +import java.util.Collections; +import java.util.List; +import java.util.Map; + +public class LSMDatasetBulkInsertOverwriteTableCommitActionExecutor extends LSMDatasetBulkInsertOverwriteCommitActionExecutor { + public LSMDatasetBulkInsertOverwriteTableCommitActionExecutor(HoodieWriteConfig config, + SparkRDDWriteClient writeClient, + String instantTime) { + super(config, writeClient, instantTime); + } + + @Override + protected void preExecute() { + table.validateInsertSchema(); + writeClient.startCommitWithTime(instantTime, getCommitActionType()); + writeClient.preWrite(instantTime, getWriteOperationType(), table.getMetaClient()); + } + + protected void afterExecute(HoodieWriteMetadata> result) { + writeClient.postWrite(result, instantTime, table); + } + + @Override + public WriteOperationType getWriteOperationType() { + return WriteOperationType.INSERT_OVERWRITE_TABLE; + } + + @Override + protected Map> getPartitionToReplacedFileIds(HoodieData writeStatuses) { + if (!writeConfig.getReplacePartitionRecords()) { + return Collections.emptyMap(); + } + HoodieEngineContext context = writeClient.getEngineContext(); + List partitionPaths = FSUtils.getAllPartitionPaths(context, writeConfig.getMetadataConfig(), + table.getMetaClient().getBasePathV2().toString()); + + if (partitionPaths == null || partitionPaths.isEmpty()) { + return Collections.emptyMap(); + } + + context.setJobStatus(this.getClass().getSimpleName(), "Getting ExistingFileIds of all partitions"); + + if (canTakeFirstPartition(writeConfig, partitionPaths)) { + String partition = writeStatuses.first().getStat().getPartitionPath(); + return Collections.singletonMap(partition, getAllExistingFileIds(partition)); + } else { + return HoodieJavaPairRDD.getJavaPairRDD(context.parallelize(partitionPaths, partitionPaths.size()).mapToPair( + partitionPath -> Pair.of(partitionPath, getAllExistingFileIds(partitionPath)))).collectAsMap(); + } + } +} diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/commit/LSMDatasetUpsertCommitActionExecutor.java b/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/commit/LSMDatasetUpsertCommitActionExecutor.java new file mode 100644 index 0000000000000..4a2fb7c2708b4 --- /dev/null +++ b/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/commit/LSMDatasetUpsertCommitActionExecutor.java @@ -0,0 +1,63 @@ +/* + * 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.commit; + +import org.apache.hudi.HoodieDatasetBulkInsertHelper; +import org.apache.hudi.client.HoodieWriteResult; +import org.apache.hudi.client.SparkRDDWriteClient; +import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.common.model.WriteOperationType; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.ValidationUtils; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.index.HoodieIndex; +import org.apache.hudi.table.action.HoodieWriteMetadata; + +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; + +public class LSMDatasetUpsertCommitActionExecutor extends LSMDatasetBulkInsertCommitActionExecutor { + + public LSMDatasetUpsertCommitActionExecutor(HoodieWriteConfig config, + SparkRDDWriteClient writeClient, + String instantTime) { + super(config, writeClient, instantTime); + } + + @Override + public HoodieWriteResult execute(Dataset records, boolean isTablePartitioned) { + ValidationUtils.checkArgument(writeConfig.getIndexType() == HoodieIndex.IndexType.BUCKET); + Dataset hoodieDF; + // do repartition and sort by + hoodieDF = HoodieDatasetBulkInsertHelper.prepareForLsm(records, writeConfig); + table = writeClient.initTable(getWriteOperationType(), Option.ofNullable(instantTime)); + preExecute(); + // always do sort(sort by instead of order by) + HoodieWriteMetadata> result = buildHoodieWriteMetadata(doExecute(hoodieDF, true)); + afterExecute(result); + + return new HoodieWriteResult(result.getWriteStatuses(), result.getPartitionToReplaceFileIds()); + } + + @Override + public WriteOperationType getWriteOperationType() { + return WriteOperationType.UPSERT; + } +} diff --git a/hudi-spark-datasource/hudi-spark3-common/src/main/java/org/apache/hudi/spark3/internal/HoodieBulkInsertDataInternalWriter.java b/hudi-spark-datasource/hudi-spark3-common/src/main/java/org/apache/hudi/spark3/internal/HoodieBulkInsertDataInternalWriter.java index 24157c694ef21..d2b68c87ab890 100644 --- a/hudi-spark-datasource/hudi-spark3-common/src/main/java/org/apache/hudi/spark3/internal/HoodieBulkInsertDataInternalWriter.java +++ b/hudi-spark-datasource/hudi-spark3-common/src/main/java/org/apache/hudi/spark3/internal/HoodieBulkInsertDataInternalWriter.java @@ -21,6 +21,9 @@ import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.action.commit.BulkInsertDataInternalWriterHelper; +import org.apache.hudi.table.action.commit.BucketBulkInsertDataInternalWriterHelper; +import org.apache.hudi.table.action.commit.BucketLSMBulkInsertDataInternalWriterHelper; +import org.apache.hudi.index.HoodieIndex; import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.connector.write.DataWriter; @@ -39,8 +42,25 @@ public class HoodieBulkInsertDataInternalWriter implements DataWriter extraMetadata = new HashMap<>(); public HoodieDataSourceInternalBatchWrite(String instantTime, HoodieWriteConfig writeConfig, StructType structType, @@ -63,12 +70,13 @@ public HoodieDataSourceInternalBatchWrite(String instantTime, HoodieWriteConfig this.extraMetadata = DataSourceUtils.getExtraMetadata(properties); this.dataSourceInternalWriterHelper = new DataSourceInternalWriterHelper(instantTime, writeConfig, structType, jss, hadoopConfiguration, extraMetadata); + this.isLSM = dataSourceInternalWriterHelper.getHoodieTable().getMetaClient().getTableConfig().isLSMBasedLogFormat(); } @Override public DataWriterFactory createBatchWriterFactory(PhysicalWriteInfo info) { dataSourceInternalWriterHelper.createInflightCommit(); - if (WriteOperationType.BULK_INSERT == dataSourceInternalWriterHelper.getWriteOperationType()) { + if (WriteOperationType.BULK_INSERT == dataSourceInternalWriterHelper.getWriteOperationType() || isLSM) { return new HoodieBulkInsertDataInternalWriterFactory(dataSourceInternalWriterHelper.getHoodieTable(), writeConfig, instantTime, structType, populateMetaFields, arePartitionRecordsSorted); } else { @@ -90,6 +98,31 @@ public void onDataWriterCommit(WriterCommitMessage message) { public void commit(WriterCommitMessage[] messages) { List writeStatList = Arrays.stream(messages).map(m -> (HoodieWriterCommitMessage) m) .flatMap(m -> m.getWriteStatuses().stream().map(HoodieInternalWriteStatus::getStat)).collect(Collectors.toList()); + + if (writeConfig.getOperationType().equalsIgnoreCase(WriteOperationType.BULK_INSERT.value()) && writeConfig.isForcingWriteToL1() && isLSM) { + Set> partitionAndFileIdSet = writeStatList.stream() + .map(stat -> Pair.of(stat.getPartitionPath(), stat.getFileId())) + .collect(Collectors.toSet()); + + for (Pair partitionAndFileId : partitionAndFileIdSet) { + String partitionPath = partitionAndFileId.getLeft(); + String fileId = partitionAndFileId.getRight(); + + Option fileSliceOpt = dataSourceInternalWriterHelper.getHoodieTable().getSliceView() + .getLatestFileSlice(partitionPath, fileId); + + if (fileSliceOpt.isPresent()) { + FileSlice fileSlice = fileSliceOpt.get(); + long l1Num = fileSlice.getLogFiles() + .filter(l1LogFile -> ((HoodieLSMLogFile) l1LogFile).getLevelNumber() == 1) + .count(); + if (l1Num != 0) { + throw new HoodieException("There already exist L1 layer files in partition: " + partitionPath + + ", fileId: " + fileId + " when the operation type is bulk insert and hoodie.lsm.write.force.l1 is set to true."); + } + } + } + } dataSourceInternalWriterHelper.commit(writeStatList); }