From c11e0e2fdc759260aa24f9ae1e2f599a3245682c Mon Sep 17 00:00:00 2001 From: yanenze Date: Wed, 20 Apr 2022 00:04:59 +0800 Subject: [PATCH 01/44] #1 --- .../ScheduleCompactionsActionExecutor.java | 184 ++++++++++++++++++ .../action/compact/ScheduleConstant.java | 12 ++ 2 files changed, 196 insertions(+) create mode 100644 hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/compact/ScheduleCompactionsActionExecutor.java create mode 100644 hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/compact/ScheduleConstant.java diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/compact/ScheduleCompactionsActionExecutor.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/compact/ScheduleCompactionsActionExecutor.java new file mode 100644 index 0000000000000..e3e732eae9ef8 --- /dev/null +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/compact/ScheduleCompactionsActionExecutor.java @@ -0,0 +1,184 @@ +package org.apache.hudi.table.action.compact; + +import org.apache.hudi.avro.model.HoodieCompactionPlan; +import org.apache.hudi.common.engine.EngineType; +import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.model.HoodieFileGroupId; +import org.apache.hudi.common.model.HoodieRecordPayload; +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.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.HoodieCompactionException; +import org.apache.hudi.exception.HoodieIOException; +import org.apache.hudi.table.HoodieTable; +import org.apache.hudi.table.action.BaseActionExecutor; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; + +import java.io.IOException; +import java.text.ParseException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.atomic.AtomicLong; +import java.util.stream.Collectors; + +public class ScheduleCompactionsActionExecutor extends BaseActionExecutor>> { + + private static final Logger LOG = LogManager.getLogger(ScheduleCompactionActionExecutor.class); + + private final Option> extraMetadata; + private final HoodieFlinkMergeOnReadTableCompactor compactor; + + public ScheduleCompactionsActionExecutor(HoodieEngineContext context, + HoodieWriteConfig config, + HoodieTable table, + String instantTime, + Option> extraMetadata, + HoodieFlinkMergeOnReadTableCompactor compactor) { + super(context, config, table, instantTime); + this.extraMetadata = extraMetadata; + this.compactor = compactor; + } + + @Override + public Option> execute() { + if (!config.getWriteConcurrencyMode().supportsOptimisticConcurrencyControl() + && !config.getFailedWritesCleanPolicy().isLazy()) { + // TODO(yihua): this validation is removed for Java client used by kafka-connect. Need to revisit this. + if (config.getEngineType() != EngineType.JAVA) { + // if there are inflight writes, their instantTime must not be less than that of compaction instant time + table.getActiveTimeline().getCommitsTimeline().filterPendingExcludingCompaction().firstInstant() + .ifPresent(earliestInflight -> ValidationUtils.checkArgument( + HoodieTimeline.compareTimestamps(earliestInflight.getTimestamp(), HoodieTimeline.GREATER_THAN, instantTime), + "Earliest write inflight instant time must be later than compaction time. Earliest :" + earliestInflight + + ", Compaction scheduled at " + instantTime)); + } + // Committed and pending compaction instants should have strictly lower timestamps + List conflictingInstants = table.getActiveTimeline() + .getWriteTimeline().filterCompletedAndCompactionInstants().getInstants() + .filter(instant -> HoodieTimeline.compareTimestamps( + instant.getTimestamp(), HoodieTimeline.GREATER_THAN_OR_EQUALS, instantTime)) + .collect(Collectors.toList()); + ValidationUtils.checkArgument(conflictingInstants.isEmpty(), + "Following instants have timestamps >= compactionInstant (" + instantTime + ") Instants :" + + conflictingInstants); + } + + List plans = scheduleCompaction(); + AtomicLong num = new AtomicLong(0); + plans.stream().forEach(plan -> { + if (plan != null && (plan.getOperations() != null) && (!plan.getOperations().isEmpty())) { + extraMetadata.ifPresent(plan::setExtraMetadata); + HoodieInstant compactionInstant = + new HoodieInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, String.valueOf(Long.valueOf(instantTime) + num.getAndAdd(1))); + try { + table.getActiveTimeline().saveToCompactionRequested(compactionInstant, + TimelineMetadataUtils.serializeCompactionPlan(plan)); + } catch (IOException ioe) { + throw new HoodieIOException("Exception scheduling compaction", ioe); + } + } + }); + return Option.of(plans); + } + + private List scheduleCompaction() { + LOG.info("Checking if compaction needs to be run on " + config.getBasePath()); + // judge if we need to compact according to num delta commits and time elapsed + boolean compactable = needCompact(config.getInlineCompactTriggerStrategy()); + if (compactable) { + LOG.info("Generating compaction plan for merge on read table " + config.getBasePath()); + try { + SyncableFileSystemView fileSystemView = (SyncableFileSystemView) table.getSliceView(); + Set fgInPendingCompactionAndClustering = fileSystemView.getPendingCompactionOperations() + .map(instantTimeOpPair -> instantTimeOpPair.getValue().getFileGroupId()) + .collect(Collectors.toSet()); + // exclude files in pending clustering from compaction. + fgInPendingCompactionAndClustering.addAll(fileSystemView.getFileGroupsInPendingClustering().map(Pair::getLeft).collect(Collectors.toSet())); + context.setJobStatus(this.getClass().getSimpleName(), "Compaction: generating compaction plan"); + return compactor.generateCompactionPlans(context, table, config, instantTime, fgInPendingCompactionAndClustering); + } catch (IOException e) { + throw new HoodieCompactionException("Could not schedule compaction " + config.getBasePath(), e); + } + } + return new ArrayList(); + } + + + private Pair getLatestDeltaCommitInfo() { + Option lastCompaction = table.getActiveTimeline().getCommitTimeline() + .filterCompletedInstants().lastInstant(); + HoodieTimeline deltaCommits = table.getActiveTimeline().getDeltaCommitTimeline(); + + String latestInstantTs; + final int deltaCommitsSinceLastCompaction; + if (lastCompaction.isPresent()) { + latestInstantTs = lastCompaction.get().getTimestamp(); + deltaCommitsSinceLastCompaction = deltaCommits.findInstantsAfter(latestInstantTs, Integer.MAX_VALUE).countInstants(); + } else { + + latestInstantTs = deltaCommits.firstInstant().get().getTimestamp(); + deltaCommitsSinceLastCompaction = deltaCommits.findInstantsAfterOrEquals(latestInstantTs, Integer.MAX_VALUE).countInstants(); + } + return Pair.of(deltaCommitsSinceLastCompaction, latestInstantTs); + } + + private boolean needCompact(CompactionTriggerStrategy compactionTriggerStrategy) { + boolean compactable; + // get deltaCommitsSinceLastCompaction and lastCompactionTs + Pair latestDeltaCommitInfo = getLatestDeltaCommitInfo(); + int inlineCompactDeltaCommitMax = config.getInlineCompactDeltaCommitMax(); + int inlineCompactDeltaSecondsMax = config.getInlineCompactDeltaSecondsMax(); + switch (compactionTriggerStrategy) { + case NUM_COMMITS: + compactable = inlineCompactDeltaCommitMax <= latestDeltaCommitInfo.getLeft(); + if (compactable) { + LOG.info(String.format("The delta commits >= %s, trigger compaction scheduler.", inlineCompactDeltaCommitMax)); + } + break; + case TIME_ELAPSED: + compactable = inlineCompactDeltaSecondsMax <= parsedToSeconds(instantTime) - parsedToSeconds(latestDeltaCommitInfo.getRight()); + if (compactable) { + LOG.info(String.format("The elapsed time >=%ss, trigger compaction scheduler.", inlineCompactDeltaSecondsMax)); + } + break; + case NUM_OR_TIME: + compactable = inlineCompactDeltaCommitMax <= latestDeltaCommitInfo.getLeft() + || inlineCompactDeltaSecondsMax <= parsedToSeconds(instantTime) - parsedToSeconds(latestDeltaCommitInfo.getRight()); + if (compactable) { + LOG.info(String.format("The delta commits >= %s or elapsed_time >=%ss, trigger compaction scheduler.", inlineCompactDeltaCommitMax, + inlineCompactDeltaSecondsMax)); + } + break; + case NUM_AND_TIME: + compactable = inlineCompactDeltaCommitMax <= latestDeltaCommitInfo.getLeft() + && inlineCompactDeltaSecondsMax <= parsedToSeconds(instantTime) - parsedToSeconds(latestDeltaCommitInfo.getRight()); + if (compactable) { + LOG.info(String.format("The delta commits >= %s and elapsed_time >=%ss, trigger compaction scheduler.", inlineCompactDeltaCommitMax, + inlineCompactDeltaSecondsMax)); + } + break; + default: + throw new HoodieCompactionException("Unsupported compaction trigger strategy: " + config.getInlineCompactTriggerStrategy()); + } + return compactable; + } + + private Long parsedToSeconds(String time) { + long timestamp; + try { + timestamp = HoodieActiveTimeline.parseDateFromInstantTime(time).getTime() / 1000; + } catch (ParseException e) { + throw new HoodieCompactionException(e.getMessage(), e); + } + return timestamp; + } +} diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/compact/ScheduleConstant.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/compact/ScheduleConstant.java new file mode 100644 index 0000000000000..aae99723aec0e --- /dev/null +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/compact/ScheduleConstant.java @@ -0,0 +1,12 @@ +package org.apache.hudi.table.action.compact; + +import java.util.concurrent.atomic.AtomicInteger; + +/** + * @Author ez + * @Description + * @createTime 2022/3/17 + */ +public class ScheduleConstant { + static AtomicInteger countDeltaCommit = new AtomicInteger(0); +} From cbb03121d98291c8db1e1770cc3b325c5251fb45 Mon Sep 17 00:00:00 2001 From: yanenze Date: Wed, 20 Apr 2022 00:15:14 +0800 Subject: [PATCH 02/44] #1 --- .../ScheduleCompactionsActionExecutor.java | 184 ------------------ .../action/compact/ScheduleConstant.java | 12 -- 2 files changed, 196 deletions(-) delete mode 100644 hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/compact/ScheduleCompactionsActionExecutor.java delete mode 100644 hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/compact/ScheduleConstant.java diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/compact/ScheduleCompactionsActionExecutor.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/compact/ScheduleCompactionsActionExecutor.java deleted file mode 100644 index e3e732eae9ef8..0000000000000 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/compact/ScheduleCompactionsActionExecutor.java +++ /dev/null @@ -1,184 +0,0 @@ -package org.apache.hudi.table.action.compact; - -import org.apache.hudi.avro.model.HoodieCompactionPlan; -import org.apache.hudi.common.engine.EngineType; -import org.apache.hudi.common.engine.HoodieEngineContext; -import org.apache.hudi.common.model.HoodieFileGroupId; -import org.apache.hudi.common.model.HoodieRecordPayload; -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.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.HoodieCompactionException; -import org.apache.hudi.exception.HoodieIOException; -import org.apache.hudi.table.HoodieTable; -import org.apache.hudi.table.action.BaseActionExecutor; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; - -import java.io.IOException; -import java.text.ParseException; -import java.util.ArrayList; -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.concurrent.atomic.AtomicLong; -import java.util.stream.Collectors; - -public class ScheduleCompactionsActionExecutor extends BaseActionExecutor>> { - - private static final Logger LOG = LogManager.getLogger(ScheduleCompactionActionExecutor.class); - - private final Option> extraMetadata; - private final HoodieFlinkMergeOnReadTableCompactor compactor; - - public ScheduleCompactionsActionExecutor(HoodieEngineContext context, - HoodieWriteConfig config, - HoodieTable table, - String instantTime, - Option> extraMetadata, - HoodieFlinkMergeOnReadTableCompactor compactor) { - super(context, config, table, instantTime); - this.extraMetadata = extraMetadata; - this.compactor = compactor; - } - - @Override - public Option> execute() { - if (!config.getWriteConcurrencyMode().supportsOptimisticConcurrencyControl() - && !config.getFailedWritesCleanPolicy().isLazy()) { - // TODO(yihua): this validation is removed for Java client used by kafka-connect. Need to revisit this. - if (config.getEngineType() != EngineType.JAVA) { - // if there are inflight writes, their instantTime must not be less than that of compaction instant time - table.getActiveTimeline().getCommitsTimeline().filterPendingExcludingCompaction().firstInstant() - .ifPresent(earliestInflight -> ValidationUtils.checkArgument( - HoodieTimeline.compareTimestamps(earliestInflight.getTimestamp(), HoodieTimeline.GREATER_THAN, instantTime), - "Earliest write inflight instant time must be later than compaction time. Earliest :" + earliestInflight - + ", Compaction scheduled at " + instantTime)); - } - // Committed and pending compaction instants should have strictly lower timestamps - List conflictingInstants = table.getActiveTimeline() - .getWriteTimeline().filterCompletedAndCompactionInstants().getInstants() - .filter(instant -> HoodieTimeline.compareTimestamps( - instant.getTimestamp(), HoodieTimeline.GREATER_THAN_OR_EQUALS, instantTime)) - .collect(Collectors.toList()); - ValidationUtils.checkArgument(conflictingInstants.isEmpty(), - "Following instants have timestamps >= compactionInstant (" + instantTime + ") Instants :" - + conflictingInstants); - } - - List plans = scheduleCompaction(); - AtomicLong num = new AtomicLong(0); - plans.stream().forEach(plan -> { - if (plan != null && (plan.getOperations() != null) && (!plan.getOperations().isEmpty())) { - extraMetadata.ifPresent(plan::setExtraMetadata); - HoodieInstant compactionInstant = - new HoodieInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, String.valueOf(Long.valueOf(instantTime) + num.getAndAdd(1))); - try { - table.getActiveTimeline().saveToCompactionRequested(compactionInstant, - TimelineMetadataUtils.serializeCompactionPlan(plan)); - } catch (IOException ioe) { - throw new HoodieIOException("Exception scheduling compaction", ioe); - } - } - }); - return Option.of(plans); - } - - private List scheduleCompaction() { - LOG.info("Checking if compaction needs to be run on " + config.getBasePath()); - // judge if we need to compact according to num delta commits and time elapsed - boolean compactable = needCompact(config.getInlineCompactTriggerStrategy()); - if (compactable) { - LOG.info("Generating compaction plan for merge on read table " + config.getBasePath()); - try { - SyncableFileSystemView fileSystemView = (SyncableFileSystemView) table.getSliceView(); - Set fgInPendingCompactionAndClustering = fileSystemView.getPendingCompactionOperations() - .map(instantTimeOpPair -> instantTimeOpPair.getValue().getFileGroupId()) - .collect(Collectors.toSet()); - // exclude files in pending clustering from compaction. - fgInPendingCompactionAndClustering.addAll(fileSystemView.getFileGroupsInPendingClustering().map(Pair::getLeft).collect(Collectors.toSet())); - context.setJobStatus(this.getClass().getSimpleName(), "Compaction: generating compaction plan"); - return compactor.generateCompactionPlans(context, table, config, instantTime, fgInPendingCompactionAndClustering); - } catch (IOException e) { - throw new HoodieCompactionException("Could not schedule compaction " + config.getBasePath(), e); - } - } - return new ArrayList(); - } - - - private Pair getLatestDeltaCommitInfo() { - Option lastCompaction = table.getActiveTimeline().getCommitTimeline() - .filterCompletedInstants().lastInstant(); - HoodieTimeline deltaCommits = table.getActiveTimeline().getDeltaCommitTimeline(); - - String latestInstantTs; - final int deltaCommitsSinceLastCompaction; - if (lastCompaction.isPresent()) { - latestInstantTs = lastCompaction.get().getTimestamp(); - deltaCommitsSinceLastCompaction = deltaCommits.findInstantsAfter(latestInstantTs, Integer.MAX_VALUE).countInstants(); - } else { - - latestInstantTs = deltaCommits.firstInstant().get().getTimestamp(); - deltaCommitsSinceLastCompaction = deltaCommits.findInstantsAfterOrEquals(latestInstantTs, Integer.MAX_VALUE).countInstants(); - } - return Pair.of(deltaCommitsSinceLastCompaction, latestInstantTs); - } - - private boolean needCompact(CompactionTriggerStrategy compactionTriggerStrategy) { - boolean compactable; - // get deltaCommitsSinceLastCompaction and lastCompactionTs - Pair latestDeltaCommitInfo = getLatestDeltaCommitInfo(); - int inlineCompactDeltaCommitMax = config.getInlineCompactDeltaCommitMax(); - int inlineCompactDeltaSecondsMax = config.getInlineCompactDeltaSecondsMax(); - switch (compactionTriggerStrategy) { - case NUM_COMMITS: - compactable = inlineCompactDeltaCommitMax <= latestDeltaCommitInfo.getLeft(); - if (compactable) { - LOG.info(String.format("The delta commits >= %s, trigger compaction scheduler.", inlineCompactDeltaCommitMax)); - } - break; - case TIME_ELAPSED: - compactable = inlineCompactDeltaSecondsMax <= parsedToSeconds(instantTime) - parsedToSeconds(latestDeltaCommitInfo.getRight()); - if (compactable) { - LOG.info(String.format("The elapsed time >=%ss, trigger compaction scheduler.", inlineCompactDeltaSecondsMax)); - } - break; - case NUM_OR_TIME: - compactable = inlineCompactDeltaCommitMax <= latestDeltaCommitInfo.getLeft() - || inlineCompactDeltaSecondsMax <= parsedToSeconds(instantTime) - parsedToSeconds(latestDeltaCommitInfo.getRight()); - if (compactable) { - LOG.info(String.format("The delta commits >= %s or elapsed_time >=%ss, trigger compaction scheduler.", inlineCompactDeltaCommitMax, - inlineCompactDeltaSecondsMax)); - } - break; - case NUM_AND_TIME: - compactable = inlineCompactDeltaCommitMax <= latestDeltaCommitInfo.getLeft() - && inlineCompactDeltaSecondsMax <= parsedToSeconds(instantTime) - parsedToSeconds(latestDeltaCommitInfo.getRight()); - if (compactable) { - LOG.info(String.format("The delta commits >= %s and elapsed_time >=%ss, trigger compaction scheduler.", inlineCompactDeltaCommitMax, - inlineCompactDeltaSecondsMax)); - } - break; - default: - throw new HoodieCompactionException("Unsupported compaction trigger strategy: " + config.getInlineCompactTriggerStrategy()); - } - return compactable; - } - - private Long parsedToSeconds(String time) { - long timestamp; - try { - timestamp = HoodieActiveTimeline.parseDateFromInstantTime(time).getTime() / 1000; - } catch (ParseException e) { - throw new HoodieCompactionException(e.getMessage(), e); - } - return timestamp; - } -} diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/compact/ScheduleConstant.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/compact/ScheduleConstant.java deleted file mode 100644 index aae99723aec0e..0000000000000 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/compact/ScheduleConstant.java +++ /dev/null @@ -1,12 +0,0 @@ -package org.apache.hudi.table.action.compact; - -import java.util.concurrent.atomic.AtomicInteger; - -/** - * @Author ez - * @Description - * @createTime 2022/3/17 - */ -public class ScheduleConstant { - static AtomicInteger countDeltaCommit = new AtomicInteger(0); -} From 20b918eee4bc084a1808d9ed56b22e5ae2047b39 Mon Sep 17 00:00:00 2001 From: yanenze Date: Mon, 23 May 2022 15:47:30 +0800 Subject: [PATCH 03/44] #improve the flink sink operator name for better identify tables to write --- .../org/apache/hudi/sink/utils/Pipelines.java | 18 +++++++++--------- 1 file changed, 9 insertions(+), 9 deletions(-) diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/utils/Pipelines.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/utils/Pipelines.java index 91ac2beadc080..94b3132481f91 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/utils/Pipelines.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/utils/Pipelines.java @@ -114,7 +114,7 @@ public static DataStreamSink bulkInsert(Configuration conf, RowType rowT conf.getInteger(FlinkOptions.WRITE_SORT_MEMORY) * 1024L * 1024L); } return dataStream - .transform("bucket_bulk_insert", TypeInformation.of(Object.class), operatorFactory) + .transform("bucket_bulk_insert" + ":" + conf.getString(FlinkOptions.TABLE_NAME), TypeInformation.of(Object.class), operatorFactory) .uid("uid_bucket_bulk_insert" + conf.getString(FlinkOptions.TABLE_NAME)) .setParallelism(conf.getInteger(FlinkOptions.WRITE_TASKS)) .addSink(DummySink.INSTANCE) @@ -137,7 +137,7 @@ public static DataStreamSink bulkInsert(Configuration conf, RowType rowT SortOperatorGen sortOperatorGen = new SortOperatorGen(rowType, partitionFields); // sort by partition keys dataStream = dataStream - .transform("partition_key_sorter", + .transform("partition_key_sorter" + ":" + conf.getString(FlinkOptions.TABLE_NAME), TypeInformation.of(RowData.class), sortOperatorGen.createSortOperator()) .setParallelism(conf.getInteger(FlinkOptions.WRITE_TASKS)); @@ -146,7 +146,7 @@ public static DataStreamSink bulkInsert(Configuration conf, RowType rowT } } return dataStream - .transform("hoodie_bulk_insert_write", + .transform("hoodie_bulk_insert_write" + ":" + conf.getString(FlinkOptions.TABLE_NAME), TypeInformation.of(Object.class), operatorFactory) // follow the parallelism of upstream operators to avoid shuffle @@ -190,7 +190,7 @@ public static DataStreamSink append( WriteOperatorFactory operatorFactory = AppendWriteOperator.getFactory(conf, rowType); return dataStream - .transform("hoodie_append_write", TypeInformation.of(Object.class), operatorFactory) + .transform("hoodie_append_write" + ":" + conf.getString(FlinkOptions.TABLE_NAME), TypeInformation.of(Object.class), operatorFactory) .uid("uid_hoodie_stream_write" + conf.getString(FlinkOptions.TABLE_NAME)) .setParallelism(conf.getInteger(FlinkOptions.WRITE_TASKS)) .addSink(DummySink.INSTANCE) @@ -250,7 +250,7 @@ private static DataStream streamBootstrap( if (conf.getBoolean(FlinkOptions.INDEX_BOOTSTRAP_ENABLED) || bounded) { dataStream1 = dataStream1 .transform( - "index_bootstrap", + "index_bootstrap" + ":" + conf.getString(FlinkOptions.TABLE_NAME), TypeInformation.of(HoodieRecord.class), new BootstrapOperator<>(conf)) .setParallelism(conf.getOptional(FlinkOptions.INDEX_BOOTSTRAP_TASKS).orElse(defaultParallelism)) @@ -277,7 +277,7 @@ private static DataStream boundedBootstrap( return rowDataToHoodieRecord(conf, rowType, dataStream) .transform( - "batch_index_bootstrap", + "batch_index_bootstrap" + ":" + conf.getString(FlinkOptions.TABLE_NAME), TypeInformation.of(HoodieRecord.class), new BatchBootstrapOperator<>(conf)) .setParallelism(conf.getOptional(FlinkOptions.INDEX_BOOTSTRAP_TASKS).orElse(defaultParallelism)) @@ -322,7 +322,7 @@ public static DataStream hoodieStreamWrite(Configuration conf, int defau String indexKeyFields = conf.getString(FlinkOptions.INDEX_KEY_FIELD); BucketIndexPartitioner partitioner = new BucketIndexPartitioner<>(bucketNum, indexKeyFields); return dataStream.partitionCustom(partitioner, HoodieRecord::getKey) - .transform("bucket_write", TypeInformation.of(Object.class), operatorFactory) + .transform("bucket_write" + ":" + conf.getString(FlinkOptions.TABLE_NAME), TypeInformation.of(Object.class), operatorFactory) .uid("uid_bucket_write" + conf.getString(FlinkOptions.TABLE_NAME)) .setParallelism(conf.getInteger(FlinkOptions.WRITE_TASKS)); } else { @@ -331,7 +331,7 @@ public static DataStream hoodieStreamWrite(Configuration conf, int defau // Key-by record key, to avoid multiple subtasks write to a bucket at the same time .keyBy(HoodieRecord::getRecordKey) .transform( - "bucket_assigner", + "bucket_assigner" + ":" + conf.getString(FlinkOptions.TABLE_NAME), TypeInformation.of(HoodieRecord.class), new KeyedProcessOperator<>(new BucketAssignFunction<>(conf))) .uid("uid_bucket_assigner_" + conf.getString(FlinkOptions.TABLE_NAME)) @@ -365,7 +365,7 @@ public static DataStream hoodieStreamWrite(Configuration conf, int defau * @return the compaction pipeline */ public static DataStreamSink compact(Configuration conf, DataStream dataStream) { - return dataStream.transform("compact_plan_generate", + return dataStream.transform("compact_plan_generate" + ":" + conf.getString(FlinkOptions.TABLE_NAME), TypeInformation.of(CompactionPlanEvent.class), new CompactionPlanOperator(conf)) .setParallelism(1) // plan generate must be singleton From 42c7129e25ef42a74dd32d17d2816946cd818ab9 Mon Sep 17 00:00:00 2001 From: Sagar Sumit Date: Mon, 23 May 2022 18:10:07 +0530 Subject: [PATCH 04/44] [HUDI-4142] Claim RFC-54 for new table APIs (#5665) --- rfc/README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/rfc/README.md b/rfc/README.md index 0ccf7b1bbe285..5040860423014 100644 --- a/rfc/README.md +++ b/rfc/README.md @@ -89,4 +89,4 @@ The list of all RFCs can be found here. | 51 | [Change Data Capture](./rfc-51/rfc-51.md) | `UNDER REVIEW` | | 52 | [Introduce Secondary Index to Improve HUDI Query Performance](./rfc-52/rfc-52.md) | `UNDER REVIEW` | | 53 | [Use Lock-Free Message Queue Improving Hoodie Writing Efficiency](./rfc-53/rfc-53.md) | `UNDER REVIEW` | - +| 54 | [New Table APIs and Streamline Hudi Configs](./rfc-54/rfc-54.md) | `UNDER REVIEW` | From 752f956f036e5aaf8dcc480ab0baf0c1e3d7cec9 Mon Sep 17 00:00:00 2001 From: Y Ethan Guo Date: Mon, 23 May 2022 06:48:09 -0700 Subject: [PATCH 05/44] [HUDI-3933] Add UT cases to cover different key gen (#5638) --- .../functional/TestCOWDataSourceStorage.scala | 66 +++++++++---------- 1 file changed, 32 insertions(+), 34 deletions(-) diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestCOWDataSourceStorage.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestCOWDataSourceStorage.scala index 48bb46f81b1b0..6f13dbc82f4d9 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestCOWDataSourceStorage.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestCOWDataSourceStorage.scala @@ -26,14 +26,14 @@ import org.apache.hudi.common.table.timeline.{HoodieInstant, HoodieTimeline} import org.apache.hudi.common.testutils.HoodieTestDataGenerator import org.apache.hudi.common.testutils.RawTripTestPayload.recordsToStrings import org.apache.hudi.config.HoodieWriteConfig +import org.apache.hudi.keygen.TimestampBasedKeyGenerator import org.apache.hudi.keygen.constant.KeyGeneratorOptions.Config -import org.apache.hudi.keygen.{ComplexKeyGenerator, TimestampBasedKeyGenerator} import org.apache.hudi.testutils.SparkClientFunctionalTestHarness import org.apache.hudi.{DataSourceReadOptions, DataSourceWriteOptions, HoodieDataSourceHelpers} import org.apache.spark.sql._ import org.apache.spark.sql.functions.{col, lit} import org.junit.jupiter.api.Assertions.{assertEquals, assertFalse, assertTrue} -import org.junit.jupiter.api.{Disabled, Tag} +import org.junit.jupiter.api.Tag import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.params.provider.{CsvSource, ValueSource} @@ -51,31 +51,33 @@ class TestCOWDataSourceStorage extends SparkClientFunctionalTestHarness { DataSourceWriteOptions.RECORDKEY_FIELD.key -> "_row_key", DataSourceWriteOptions.PARTITIONPATH_FIELD.key -> "partition", DataSourceWriteOptions.PRECOMBINE_FIELD.key -> "timestamp", - HoodieWriteConfig.TBL_NAME.key -> "hoodie_test" + HoodieWriteConfig.TBL_NAME.key -> "hoodie_test", + DataSourceWriteOptions.HIVE_STYLE_PARTITIONING.key -> "false" ) val verificationCol: String = "driver" val updatedVerificationVal: String = "driver_update" @ParameterizedTest - @CsvSource(Array( - "true,org.apache.hudi.keygen.SimpleKeyGenerator", - "true,org.apache.hudi.keygen.ComplexKeyGenerator", - "true,org.apache.hudi.keygen.TimestampBasedKeyGenerator", - "false,org.apache.hudi.keygen.SimpleKeyGenerator", - "false,org.apache.hudi.keygen.ComplexKeyGenerator", - "false,org.apache.hudi.keygen.TimestampBasedKeyGenerator" - )) - def testCopyOnWriteStorage(isMetadataEnabled: Boolean, keyGenClass: String): Unit = { - commonOpts += DataSourceWriteOptions.KEYGENERATOR_CLASS_NAME.key() -> keyGenClass - if (classOf[ComplexKeyGenerator].getName.equals(keyGenClass)) { - commonOpts += DataSourceWriteOptions.RECORDKEY_FIELD.key() -> "_row_key, pii_col" - } - if (classOf[TimestampBasedKeyGenerator].getName.equals(keyGenClass)) { - commonOpts += DataSourceWriteOptions.RECORDKEY_FIELD.key() -> "_row_key" - commonOpts += DataSourceWriteOptions.PARTITIONPATH_FIELD.key() -> "current_ts" - commonOpts += Config.TIMESTAMP_TYPE_FIELD_PROP -> "EPOCHMILLISECONDS" - commonOpts += Config.TIMESTAMP_OUTPUT_DATE_FORMAT_PROP -> "yyyyMMdd" + @CsvSource(value = Array( + "true|org.apache.hudi.keygen.SimpleKeyGenerator|_row_key", + "true|org.apache.hudi.keygen.ComplexKeyGenerator|_row_key,nation.bytes", + "true|org.apache.hudi.keygen.TimestampBasedKeyGenerator|_row_key", + "false|org.apache.hudi.keygen.SimpleKeyGenerator|_row_key", + "false|org.apache.hudi.keygen.ComplexKeyGenerator|_row_key,nation.bytes", + "false|org.apache.hudi.keygen.TimestampBasedKeyGenerator|_row_key" + ), delimiter = '|') + def testCopyOnWriteStorage(isMetadataEnabled: Boolean, keyGenClass: String, recordKeys: String): Unit = { + var options: Map[String, String] = commonOpts + + (HoodieMetadataConfig.ENABLE.key -> String.valueOf(isMetadataEnabled)) + + (DataSourceWriteOptions.KEYGENERATOR_CLASS_NAME.key() -> keyGenClass) + + (DataSourceWriteOptions.RECORDKEY_FIELD.key() -> recordKeys) + val isTimestampBasedKeyGen: Boolean = classOf[TimestampBasedKeyGenerator].getName.equals(keyGenClass) + if (isTimestampBasedKeyGen) { + options += DataSourceWriteOptions.RECORDKEY_FIELD.key() -> "_row_key" + options += Config.TIMESTAMP_TYPE_FIELD_PROP -> "DATE_STRING" + options += Config.TIMESTAMP_INPUT_DATE_FORMAT_PROP -> "yyyy/MM/dd" + options += Config.TIMESTAMP_OUTPUT_DATE_FORMAT_PROP -> "yyyyMMdd" } val dataGen = new HoodieTestDataGenerator(0xDEED) val fs = FSUtils.getFs(basePath, spark.sparkContext.hadoopConfiguration) @@ -83,14 +85,12 @@ class TestCOWDataSourceStorage extends SparkClientFunctionalTestHarness { val records0 = recordsToStrings(dataGen.generateInserts("000", 100)).toList val inputDF0 = spark.read.json(spark.sparkContext.parallelize(records0, 2)) inputDF0.write.format("org.apache.hudi") - .options(commonOpts) + .options(options) .option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL) - .option(HoodieMetadataConfig.ENABLE.key, isMetadataEnabled) .mode(SaveMode.Overwrite) .save(basePath) assertTrue(HoodieDataSourceHelpers.hasNewCommits(fs, basePath, "000")) - val commitInstantTime1 = HoodieDataSourceHelpers.latestCommit(fs, basePath) // Snapshot query val snapshotDF1 = spark.read.format("org.apache.hudi") @@ -102,7 +102,7 @@ class TestCOWDataSourceStorage extends SparkClientFunctionalTestHarness { val inputDF1 = spark.read.json(spark.sparkContext.parallelize(records1, 2)) val verificationRowKey = inputDF1.limit(1).select("_row_key").first.getString(0) var updateDf: DataFrame = null - if (classOf[TimestampBasedKeyGenerator].getName.equals(keyGenClass)) { + if (isTimestampBasedKeyGen) { // update current_ts to be same as original record so that partition path does not change with timestamp based key gen val originalRow = snapshotDF1.filter(col("_row_key") === verificationRowKey).collectAsList().get(0) updateDf = inputDF1.filter(col("_row_key") === verificationRowKey) @@ -116,8 +116,7 @@ class TestCOWDataSourceStorage extends SparkClientFunctionalTestHarness { } updateDf.write.format("org.apache.hudi") - .options(commonOpts) - .option(HoodieMetadataConfig.ENABLE.key, isMetadataEnabled) + .options(options) .mode(SaveMode.Append) .save(basePath) val commitInstantTime2 = HoodieDataSourceHelpers.latestCommit(fs, basePath) @@ -132,7 +131,7 @@ class TestCOWDataSourceStorage extends SparkClientFunctionalTestHarness { val records2 = recordsToStrings(dataGen.generateUpdates("002", 100)).toList var inputDF2 = spark.read.json(spark.sparkContext.parallelize(records2, 2)) - if (classOf[TimestampBasedKeyGenerator].getName.equals(keyGenClass)) { + if (isTimestampBasedKeyGen) { // incase of Timestamp based key gen, current_ts should not be updated. but dataGen.generateUpdates() would have updated // the value of current_ts. So, we need to revert it back to original value. // here is what we are going to do. Copy values to temp columns, join with original df and update the current_ts @@ -152,8 +151,7 @@ class TestCOWDataSourceStorage extends SparkClientFunctionalTestHarness { val uniqueKeyCnt = inputDF2.select("_row_key").distinct().count() inputDF2.write.format("org.apache.hudi") - .options(commonOpts) - .option(HoodieMetadataConfig.ENABLE.key, isMetadataEnabled) + .options(options) .mode(SaveMode.Append) .save(basePath) @@ -191,8 +189,7 @@ class TestCOWDataSourceStorage extends SparkClientFunctionalTestHarness { val emptyRecords = recordsToStrings(dataGen.generateUpdates("003", 0)).toList val emptyDF = spark.read.json(spark.sparkContext.parallelize(emptyRecords, 1)) emptyDF.write.format("org.apache.hudi") - .options(commonOpts) - .option(HoodieMetadataConfig.ENABLE.key, isMetadataEnabled) + .options(options) .mode(SaveMode.Append) .save(basePath) @@ -211,9 +208,10 @@ class TestCOWDataSourceStorage extends SparkClientFunctionalTestHarness { val hoodieIncViewDF3 = spark.read.format("org.apache.hudi") .option(DataSourceReadOptions.QUERY_TYPE.key, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL) .option(DataSourceReadOptions.BEGIN_INSTANTTIME.key, commitInstantTime2) - .option(DataSourceReadOptions.INCR_PATH_GLOB.key, "/2016/*/*/*") + .option(DataSourceReadOptions.INCR_PATH_GLOB.key, if (isTimestampBasedKeyGen) "/2016*/*" else "/2016/*/*/*") .load(basePath) - assertEquals(hoodieIncViewDF2.filter(col("_hoodie_partition_path").contains("2016")).count(), hoodieIncViewDF3.count()) + assertEquals(hoodieIncViewDF2 + .filter(col("_hoodie_partition_path").startsWith("2016")).count(), hoodieIncViewDF3.count()) val timeTravelDF = spark.read.format("org.apache.hudi") .option(DataSourceReadOptions.QUERY_TYPE.key, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL) From 716e995a3869f1b84e4182957948c704aa2a162b Mon Sep 17 00:00:00 2001 From: felixYyu Date: Tue, 24 May 2022 06:26:36 +0800 Subject: [PATCH 06/44] [MINOR] Removing redundant semicolons and line breaks (#5662) --- .../org/apache/hudi/common/model/HoodieCleaningPolicy.java | 2 +- .../src/main/java/org/apache/hudi/sql/InsertMode.java | 3 +-- .../src/main/java/org/apache/hudi/hive/ddl/HiveSyncMode.java | 3 +-- 3 files changed, 3 insertions(+), 5 deletions(-) diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieCleaningPolicy.java b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieCleaningPolicy.java index 58b9f7475a35f..3eb8f784dbab7 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieCleaningPolicy.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieCleaningPolicy.java @@ -22,5 +22,5 @@ * Hoodie cleaning policies. */ public enum HoodieCleaningPolicy { - KEEP_LATEST_FILE_VERSIONS, KEEP_LATEST_COMMITS, KEEP_LATEST_BY_HOURS; + KEEP_LATEST_FILE_VERSIONS, KEEP_LATEST_COMMITS, KEEP_LATEST_BY_HOURS } diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/sql/InsertMode.java b/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/sql/InsertMode.java index 4b44ae4385ad3..c68bd60ba6344 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/sql/InsertMode.java +++ b/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/sql/InsertMode.java @@ -38,8 +38,7 @@ public enum InsertMode { * In non-strict mode for insert into, we use insert operation * to write data which allow writing the duplicate record. */ - NON_STRICT("non-strict") - ; + NON_STRICT("non-strict"); private String value; diff --git a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/ddl/HiveSyncMode.java b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/ddl/HiveSyncMode.java index abe044cb114f4..127fc7a438724 100644 --- a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/ddl/HiveSyncMode.java +++ b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/ddl/HiveSyncMode.java @@ -37,8 +37,7 @@ public enum HiveSyncMode { /** * The JDBC mode use hive jdbc to sync metadata. */ - JDBC - ; + JDBC; public static HiveSyncMode of(String syncMode) { return HiveSyncMode.valueOf(syncMode.toUpperCase(Locale.ROOT)); From 47b764ec333d1fcaf13ec94d013424faf3b42584 Mon Sep 17 00:00:00 2001 From: Heap <35054152+h1ap@users.noreply.github.com> Date: Tue, 24 May 2022 06:28:48 +0800 Subject: [PATCH 07/44] [HUDI-4134] Fix Method naming consistency issues in FSUtils (#5655) --- .../commands/TestFileSystemViewCommand.java | 4 +- .../apache/hudi/io/HoodieAppendHandle.java | 2 +- .../apache/hudi/io/HoodieCreateHandle.java | 2 +- .../org/apache/hudi/io/HoodieMergeHandle.java | 2 +- .../org/apache/hudi/io/HoodieWriteHandle.java | 2 +- .../upgrade/ZeroToOneUpgradeHandler.java | 2 +- .../hudi/HoodieTestCommitGenerator.java | 2 +- .../org/apache/hudi/io/FlinkCreateHandle.java | 4 +- .../hudi/io/FlinkMergeAndReplaceHandle.java | 4 +- .../org/apache/hudi/io/FlinkMergeHandle.java | 4 +- .../row/HoodieRowDataCreateHandle.java | 4 +- .../TestJavaCopyOnWriteActionExecutor.java | 2 +- .../io/storage/row/HoodieRowCreateHandle.java | 4 +- .../TestHoodieClientOnCopyOnWriteStorage.java | 2 +- .../commit/TestCopyOnWriteActionExecutor.java | 2 +- .../org/apache/hudi/common/fs/FSUtils.java | 4 +- .../apache/hudi/common/fs/TestFSUtils.java | 8 +- .../common/model/TestHoodieWriteStat.java | 2 +- .../view/TestHoodieTableFileSystemView.java | 164 +++++++++--------- .../table/view/TestIncrementalFSViewSync.java | 2 +- .../common/testutils/FileCreateUtils.java | 2 +- .../hudi/common/util/TestClusteringUtils.java | 2 +- .../hadoop/testutils/InputFormatTestUtil.java | 10 +- .../hudi/hive/testutils/HiveTestUtil.java | 4 +- .../hudi/hive/testutils/TestCluster.java | 2 +- .../functional/TestHoodieSnapshotCopier.java | 18 +- 26 files changed, 130 insertions(+), 130 deletions(-) diff --git a/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestFileSystemViewCommand.java b/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestFileSystemViewCommand.java index 1d2872edf267f..d5c535ebfe00c 100644 --- a/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestFileSystemViewCommand.java +++ b/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestFileSystemViewCommand.java @@ -88,11 +88,11 @@ public void init() throws IOException { // Write date files and log file String testWriteToken = "1-0-1"; Files.createFile(Paths.get(fullPartitionPath, FSUtils - .makeDataFileName(commitTime1, testWriteToken, fileId1))); + .makeBaseFileName(commitTime1, testWriteToken, fileId1))); Files.createFile(Paths.get(fullPartitionPath, FSUtils .makeLogFileName(fileId1, HoodieLogFile.DELTA_EXTENSION, commitTime1, 0, testWriteToken))); Files.createFile(Paths.get(fullPartitionPath, FSUtils - .makeDataFileName(commitTime2, testWriteToken, fileId1))); + .makeBaseFileName(commitTime2, testWriteToken, fileId1))); Files.createFile(Paths.get(fullPartitionPath, FSUtils .makeLogFileName(fileId1, HoodieLogFile.DELTA_EXTENSION, commitTime2, 0, testWriteToken))); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieAppendHandle.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieAppendHandle.java index 7fc46e8b9bbc4..426e20f83b034 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieAppendHandle.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieAppendHandle.java @@ -180,7 +180,7 @@ private void init(HoodieRecord record) { // base file to denote some log appends happened on a slice. writeToken will still fence concurrent // writers. // https://issues.apache.org/jira/browse/HUDI-1517 - createMarkerFile(partitionPath, FSUtils.makeDataFileName(baseInstantTime, writeToken, fileId, hoodieTable.getBaseFileExtension())); + createMarkerFile(partitionPath, FSUtils.makeBaseFileName(baseInstantTime, writeToken, fileId, hoodieTable.getBaseFileExtension())); this.writer = createLogWriter(fileSlice, baseInstantTime); } catch (Exception e) { diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieCreateHandle.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieCreateHandle.java index 43a8c12324136..738e2d6b48d13 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieCreateHandle.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieCreateHandle.java @@ -100,7 +100,7 @@ public HoodieCreateHandle(HoodieWriteConfig config, String instantTime, HoodieTa new Path(config.getBasePath()), FSUtils.getPartitionPath(config.getBasePath(), partitionPath), hoodieTable.getPartitionMetafileFormat()); partitionMetadata.trySave(getPartitionId()); - createMarkerFile(partitionPath, FSUtils.makeDataFileName(this.instantTime, this.writeToken, this.fileId, hoodieTable.getBaseFileExtension())); + createMarkerFile(partitionPath, FSUtils.makeBaseFileName(this.instantTime, this.writeToken, this.fileId, hoodieTable.getBaseFileExtension())); this.fileWriter = HoodieFileWriterFactory.getFileWriter(instantTime, path, hoodieTable, config, writeSchemaWithMetaFields, this.taskContextSupplier); } catch (IOException e) { diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieMergeHandle.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieMergeHandle.java index b999cc6906406..92fa5c28394a5 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieMergeHandle.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieMergeHandle.java @@ -174,7 +174,7 @@ private void init(String fileId, String partitionPath, HoodieBaseFile baseFileTo hoodieTable.getPartitionMetafileFormat()); partitionMetadata.trySave(getPartitionId()); - String newFileName = FSUtils.makeDataFileName(instantTime, writeToken, fileId, hoodieTable.getBaseFileExtension()); + String newFileName = FSUtils.makeBaseFileName(instantTime, writeToken, fileId, hoodieTable.getBaseFileExtension()); makeOldAndNewFilePaths(partitionPath, latestValidFilePath, newFileName); LOG.info(String.format("Merging new data into oldPath %s, as newPath %s", oldFilePath.toString(), diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieWriteHandle.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieWriteHandle.java index 5d5760961a461..b7fdbecfd56d1 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieWriteHandle.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieWriteHandle.java @@ -164,7 +164,7 @@ public Path makeNewPath(String partitionPath) { throw new HoodieIOException("Failed to make dir " + path, e); } - return new Path(path.toString(), FSUtils.makeDataFileName(instantTime, writeToken, fileId, + return new Path(path.toString(), FSUtils.makeBaseFileName(instantTime, writeToken, fileId, hoodieTable.getMetaClient().getTableConfig().getBaseFileFormat().getFileExtension())); } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/ZeroToOneUpgradeHandler.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/ZeroToOneUpgradeHandler.java index 42add690f29ea..95f22bba27d5f 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/ZeroToOneUpgradeHandler.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/ZeroToOneUpgradeHandler.java @@ -137,6 +137,6 @@ private static String getFileNameForMarkerFromLogFile(String logFilePath, Hoodie String baseInstant = FSUtils.getBaseCommitTimeFromLogPath(logPath); String writeToken = FSUtils.getWriteTokenFromLogPath(logPath); - return FSUtils.makeDataFileName(baseInstant, writeToken, fileId, table.getBaseFileFormat().getFileExtension()); + return FSUtils.makeBaseFileName(baseInstant, writeToken, fileId, table.getBaseFileFormat().getFileExtension()); } } diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/HoodieTestCommitGenerator.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/HoodieTestCommitGenerator.java index 0c4a971ac8055..3146c9d6b4928 100644 --- a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/HoodieTestCommitGenerator.java +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/HoodieTestCommitGenerator.java @@ -103,7 +103,7 @@ public static void setupTimelineInFS( } public static String getBaseFilename(String instantTime, String fileId) { - return FSUtils.makeDataFileName(instantTime, BASE_FILE_WRITE_TOKEN, fileId); + return FSUtils.makeBaseFileName(instantTime, BASE_FILE_WRITE_TOKEN, fileId); } public static String getLogFilename(String instantTime, String fileId) { diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/FlinkCreateHandle.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/FlinkCreateHandle.java index 3005f40352afb..777e228c9510d 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/FlinkCreateHandle.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/FlinkCreateHandle.java @@ -88,7 +88,7 @@ public FlinkCreateHandle(HoodieWriteConfig config, String instantTime, HoodieTab */ private void deleteInvalidDataFile(long lastAttemptId) { final String lastWriteToken = FSUtils.makeWriteToken(getPartitionId(), getStageId(), lastAttemptId); - final String lastDataFileName = FSUtils.makeDataFileName(instantTime, + final String lastDataFileName = FSUtils.makeBaseFileName(instantTime, lastWriteToken, this.fileId, hoodieTable.getBaseFileExtension()); final Path path = makeNewFilePath(partitionPath, lastDataFileName); try { @@ -136,7 +136,7 @@ public boolean canWrite(HoodieRecord record) { * Use the writeToken + "-" + rollNumber as the new writeToken of a mini-batch write. */ private Path newFilePathWithRollover(int rollNumber) { - final String dataFileName = FSUtils.makeDataFileName(instantTime, writeToken + "-" + rollNumber, fileId, + final String dataFileName = FSUtils.makeBaseFileName(instantTime, writeToken + "-" + rollNumber, fileId, hoodieTable.getBaseFileExtension()); return makeNewFilePath(partitionPath, dataFileName); } diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/FlinkMergeAndReplaceHandle.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/FlinkMergeAndReplaceHandle.java index a16e9cc738633..24da25b20be1e 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/FlinkMergeAndReplaceHandle.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/FlinkMergeAndReplaceHandle.java @@ -90,7 +90,7 @@ public FlinkMergeAndReplaceHandle(HoodieWriteConfig config, String instantTime, */ private void deleteInvalidDataFile(long lastAttemptId) { final String lastWriteToken = FSUtils.makeWriteToken(getPartitionId(), getStageId(), lastAttemptId); - final String lastDataFileName = FSUtils.makeDataFileName(instantTime, + final String lastDataFileName = FSUtils.makeBaseFileName(instantTime, lastWriteToken, this.fileId, hoodieTable.getBaseFileExtension()); final Path path = makeNewFilePath(partitionPath, lastDataFileName); try { @@ -139,7 +139,7 @@ protected void makeOldAndNewFilePaths(String partitionPath, String oldFileName, protected String newFileNameWithRollover(int rollNumber) { // make the intermediate file as hidden final String fileID = "." + this.fileId; - return FSUtils.makeDataFileName(instantTime, writeToken + "-" + rollNumber, + return FSUtils.makeBaseFileName(instantTime, writeToken + "-" + rollNumber, fileID, hoodieTable.getBaseFileExtension()); } diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/FlinkMergeHandle.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/FlinkMergeHandle.java index fbb7dd7b5a91a..e1117712634cf 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/FlinkMergeHandle.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/FlinkMergeHandle.java @@ -94,7 +94,7 @@ public FlinkMergeHandle(HoodieWriteConfig config, String instantTime, HoodieTabl */ private void deleteInvalidDataFile(long lastAttemptId) { final String lastWriteToken = FSUtils.makeWriteToken(getPartitionId(), getStageId(), lastAttemptId); - final String lastDataFileName = FSUtils.makeDataFileName(instantTime, + final String lastDataFileName = FSUtils.makeBaseFileName(instantTime, lastWriteToken, this.fileId, hoodieTable.getBaseFileExtension()); final Path path = makeNewFilePath(partitionPath, lastDataFileName); if (path.equals(oldFilePath)) { @@ -159,7 +159,7 @@ protected void makeOldAndNewFilePaths(String partitionPath, String oldFileName, */ protected String newFileNameWithRollover(int rollNumber) { // make the intermediate file as hidden - return FSUtils.makeDataFileName(instantTime, writeToken + "-" + rollNumber, + return FSUtils.makeBaseFileName(instantTime, writeToken + "-" + rollNumber, this.fileId, hoodieTable.getBaseFileExtension()); } 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 486a5cc54b69a..f65e6cf215480 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 @@ -97,7 +97,7 @@ public HoodieRowDataCreateHandle(HoodieTable table, HoodieWriteConfig writeConfi FSUtils.getPartitionPath(writeConfig.getBasePath(), partitionPath), table.getPartitionMetafileFormat()); partitionMetadata.trySave(taskPartitionId); - createMarkerFile(partitionPath, FSUtils.makeDataFileName(this.instantTime, getWriteToken(), this.fileId, table.getBaseFileExtension())); + createMarkerFile(partitionPath, FSUtils.makeBaseFileName(this.instantTime, getWriteToken(), this.fileId, table.getBaseFileExtension())); this.fileWriter = createNewFileWriter(path, table, writeConfig, rowType); } catch (IOException e) { throw new HoodieInsertException("Failed to initialize file writer for path " + path, e); @@ -180,7 +180,7 @@ private Path makeNewPath(String partitionPath) { throw new HoodieIOException("Failed to make dir " + path, e); } HoodieTableConfig tableConfig = table.getMetaClient().getTableConfig(); - return new Path(path.toString(), FSUtils.makeDataFileName(instantTime, getWriteToken(), fileId, + return new Path(path.toString(), FSUtils.makeBaseFileName(instantTime, getWriteToken(), fileId, tableConfig.getBaseFileFormat().getFileExtension())); } diff --git a/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/table/action/commit/TestJavaCopyOnWriteActionExecutor.java b/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/table/action/commit/TestJavaCopyOnWriteActionExecutor.java index 7b0c4dbdf2a96..28d3ac6e9bd6e 100644 --- a/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/table/action/commit/TestJavaCopyOnWriteActionExecutor.java +++ b/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/table/action/commit/TestJavaCopyOnWriteActionExecutor.java @@ -103,7 +103,7 @@ public void testMakeNewPath() { }).collect(Collectors.toList()).get(0); assertEquals(newPathWithWriteToken.getKey().toString(), Paths.get(this.basePath, partitionPath, - FSUtils.makeDataFileName(instantTime, newPathWithWriteToken.getRight(), fileName)).toString()); + FSUtils.makeBaseFileName(instantTime, newPathWithWriteToken.getRight(), fileName)).toString()); } private HoodieWriteConfig makeHoodieClientConfig() { 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 4db7eb26e64ba..916b31d2931e5 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 @@ -96,7 +96,7 @@ public HoodieRowCreateHandle(HoodieTable table, HoodieWriteConfig writeConfig, S FSUtils.getPartitionPath(writeConfig.getBasePath(), partitionPath), table.getPartitionMetafileFormat()); partitionMetadata.trySave(taskPartitionId); - createMarkerFile(partitionPath, FSUtils.makeDataFileName(this.instantTime, getWriteToken(), this.fileId, table.getBaseFileExtension())); + createMarkerFile(partitionPath, FSUtils.makeBaseFileName(this.instantTime, getWriteToken(), this.fileId, table.getBaseFileExtension())); this.fileWriter = createNewFileWriter(path, table, writeConfig, structType); } catch (IOException e) { throw new HoodieInsertException("Failed to initialize file writer for path " + path, e); @@ -178,7 +178,7 @@ private Path makeNewPath(String partitionPath) { throw new HoodieIOException("Failed to make dir " + path, e); } HoodieTableConfig tableConfig = table.getMetaClient().getTableConfig(); - return new Path(path.toString(), FSUtils.makeDataFileName(instantTime, getWriteToken(), fileId, + return new Path(path.toString(), FSUtils.makeBaseFileName(instantTime, getWriteToken(), fileId, tableConfig.getBaseFileFormat().getFileExtension())); } diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieClientOnCopyOnWriteStorage.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieClientOnCopyOnWriteStorage.java index a6a37030e8a69..bf3063c5d4733 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieClientOnCopyOnWriteStorage.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieClientOnCopyOnWriteStorage.java @@ -2496,7 +2496,7 @@ private Pair> testConsistencyCheck(HoodieTableMetaCli Option markerFilePath = WriteMarkersFactory.get( cfg.getMarkersType(), getHoodieTable(metaClient, cfg), instantTime) .create(partitionPath, - FSUtils.makeDataFileName(instantTime, "1-0-1", UUID.randomUUID().toString()), + FSUtils.makeBaseFileName(instantTime, "1-0-1", UUID.randomUUID().toString()), IOType.MERGE); LOG.info("Created a dummy marker path=" + markerFilePath.get()); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/commit/TestCopyOnWriteActionExecutor.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/commit/TestCopyOnWriteActionExecutor.java index 30f7ad66543d1..7471d26cdfb56 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/commit/TestCopyOnWriteActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/commit/TestCopyOnWriteActionExecutor.java @@ -127,7 +127,7 @@ public void testMakeNewPath() { }).collect().get(0); assertEquals(newPathWithWriteToken.getKey().toString(), Paths.get(this.basePath, partitionPath, - FSUtils.makeDataFileName(instantTime, newPathWithWriteToken.getRight(), fileName)).toString()); + FSUtils.makeBaseFileName(instantTime, newPathWithWriteToken.getRight(), fileName)).toString()); } private HoodieWriteConfig makeHoodieClientConfig() { 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 aa0cadf5b9354..cfc143e3d0caa 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 @@ -159,12 +159,12 @@ public static String makeWriteToken(int taskPartitionId, int stageId, long taskA } // TODO: this should be removed - public static String makeDataFileName(String instantTime, String writeToken, String fileId) { + public static String makeBaseFileName(String instantTime, String writeToken, String fileId) { return String.format("%s_%s_%s%s", fileId, writeToken, instantTime, HoodieTableConfig.BASE_FILE_FORMAT.defaultValue().getFileExtension()); } - public static String makeDataFileName(String instantTime, String writeToken, String fileId, String fileExtension) { + public static String makeBaseFileName(String instantTime, String writeToken, String fileId, String fileExtension) { return String.format("%s_%s_%s%s", fileId, writeToken, instantTime, fileExtension); } diff --git a/hudi-common/src/test/java/org/apache/hudi/common/fs/TestFSUtils.java b/hudi-common/src/test/java/org/apache/hudi/common/fs/TestFSUtils.java index f51702a447258..7506e659c9254 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/fs/TestFSUtils.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/fs/TestFSUtils.java @@ -84,7 +84,7 @@ public void setUp() throws IOException { public void testMakeDataFileName() { String instantTime = HoodieActiveTimeline.formatDate(new Date()); String fileName = UUID.randomUUID().toString(); - assertEquals(FSUtils.makeDataFileName(instantTime, TEST_WRITE_TOKEN, fileName), fileName + "_" + TEST_WRITE_TOKEN + "_" + instantTime + BASE_FILE_EXTENSION); + assertEquals(FSUtils.makeBaseFileName(instantTime, TEST_WRITE_TOKEN, fileName), fileName + "_" + TEST_WRITE_TOKEN + "_" + instantTime + BASE_FILE_EXTENSION); } @Test @@ -159,7 +159,7 @@ public void testProcessFiles() throws Exception { public void testGetCommitTime() { String instantTime = HoodieActiveTimeline.formatDate(new Date()); String fileName = UUID.randomUUID().toString(); - String fullFileName = FSUtils.makeDataFileName(instantTime, TEST_WRITE_TOKEN, fileName); + String fullFileName = FSUtils.makeBaseFileName(instantTime, TEST_WRITE_TOKEN, fileName); assertEquals(instantTime, FSUtils.getCommitTime(fullFileName)); // test log file name fullFileName = FSUtils.makeLogFileName(fileName, HOODIE_LOG.getFileExtension(), instantTime, 1, TEST_WRITE_TOKEN); @@ -170,7 +170,7 @@ public void testGetCommitTime() { public void testGetFileNameWithoutMeta() { String instantTime = HoodieActiveTimeline.formatDate(new Date()); String fileName = UUID.randomUUID().toString(); - String fullFileName = FSUtils.makeDataFileName(instantTime, TEST_WRITE_TOKEN, fileName); + String fullFileName = FSUtils.makeBaseFileName(instantTime, TEST_WRITE_TOKEN, fileName); assertEquals(fileName, FSUtils.getFileId(fullFileName)); } @@ -304,7 +304,7 @@ public void testFileNameRelatedFunctions() throws Exception { final String LOG_EXTENTION = "." + LOG_STR; // data file name - String dataFileName = FSUtils.makeDataFileName(instantTime, writeToken, fileId); + String dataFileName = FSUtils.makeBaseFileName(instantTime, writeToken, fileId); assertEquals(instantTime, FSUtils.getCommitTime(dataFileName)); assertEquals(fileId, FSUtils.getFileId(dataFileName)); diff --git a/hudi-common/src/test/java/org/apache/hudi/common/model/TestHoodieWriteStat.java b/hudi-common/src/test/java/org/apache/hudi/common/model/TestHoodieWriteStat.java index f8995ab4c07ac..631c7cd41a385 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/model/TestHoodieWriteStat.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/model/TestHoodieWriteStat.java @@ -45,7 +45,7 @@ public void testSetPaths() { Path basePath = new Path(basePathString); Path partitionPath = new Path(basePath, partitionPathString); - Path finalizeFilePath = new Path(partitionPath, FSUtils.makeDataFileName(instantTime, writeToken, fileName)); + Path finalizeFilePath = new Path(partitionPath, FSUtils.makeBaseFileName(instantTime, writeToken, fileName)); HoodieWriteStat writeStat = new HoodieWriteStat(); writeStat.setPath(basePath, finalizeFilePath); assertEquals(finalizeFilePath, new Path(basePath, writeStat.getPath())); diff --git a/hudi-common/src/test/java/org/apache/hudi/common/table/view/TestHoodieTableFileSystemView.java b/hudi-common/src/test/java/org/apache/hudi/common/table/view/TestHoodieTableFileSystemView.java index 54bc138fc8f84..b63b9df9746a3 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/table/view/TestHoodieTableFileSystemView.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/table/view/TestHoodieTableFileSystemView.java @@ -378,7 +378,7 @@ protected void testViewForFileSlicesWithAsyncCompaction(boolean skipCreatingData String dataFileName = null; if (!skipCreatingDataFile) { - dataFileName = FSUtils.makeDataFileName(instantTime1, TEST_WRITE_TOKEN, fileId); + dataFileName = FSUtils.makeBaseFileName(instantTime1, TEST_WRITE_TOKEN, fileId); new File(basePath + "/" + partitionPath + "/" + dataFileName).createNewFile(); } String fileName1 = @@ -417,7 +417,7 @@ protected void testViewForFileSlicesWithAsyncCompaction(boolean skipCreatingData checkExternalFile(srcFileStatus, fileSlice.getBaseFile().get().getBootstrapBaseFile(), testBootstrap); } String compactionRequestedTime = "4"; - String compactDataFileName = FSUtils.makeDataFileName(compactionRequestedTime, TEST_WRITE_TOKEN, fileId); + String compactDataFileName = FSUtils.makeBaseFileName(compactionRequestedTime, TEST_WRITE_TOKEN, fileId); List> partitionFileSlicesPairs = new ArrayList<>(); partitionFileSlicesPairs.add(Pair.of(partitionPath, fileSlices.get(0))); HoodieCompactionPlan compactionPlan = @@ -552,12 +552,12 @@ protected void testViewForFileSlicesWithAsyncCompaction(boolean skipCreatingData final String orphanFileId2 = UUID.randomUUID().toString(); final String invalidInstantId = "INVALIDTIME"; String inflightDeltaInstantTime = "7"; - String orphanDataFileName = FSUtils.makeDataFileName(invalidInstantId, TEST_WRITE_TOKEN, orphanFileId1); + String orphanDataFileName = FSUtils.makeBaseFileName(invalidInstantId, TEST_WRITE_TOKEN, orphanFileId1); new File(basePath + "/" + partitionPath + "/" + orphanDataFileName).createNewFile(); String orphanLogFileName = FSUtils.makeLogFileName(orphanFileId2, HoodieLogFile.DELTA_EXTENSION, invalidInstantId, 0, TEST_WRITE_TOKEN); new File(basePath + "/" + partitionPath + "/" + orphanLogFileName).createNewFile(); - String inflightDataFileName = FSUtils.makeDataFileName(inflightDeltaInstantTime, TEST_WRITE_TOKEN, inflightFileId1); + String inflightDataFileName = FSUtils.makeBaseFileName(inflightDeltaInstantTime, TEST_WRITE_TOKEN, inflightFileId1); new File(basePath + "/" + partitionPath + "/" + inflightDataFileName).createNewFile(); String inflightLogFileName = FSUtils.makeLogFileName(inflightFileId2, HoodieLogFile.DELTA_EXTENSION, inflightDeltaInstantTime, 0, TEST_WRITE_TOKEN); @@ -712,7 +712,7 @@ public void testGetLatestDataFilesForFileId() throws IOException { // Only one commit, but is not safe String commitTime1 = "1"; - String fileName1 = FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, fileId); + String fileName1 = FSUtils.makeBaseFileName(commitTime1, TEST_WRITE_TOKEN, fileId); new File(basePath + "/" + partitionPath + "/" + fileName1).createNewFile(); refreshFsView(); assertFalse(roView.getLatestBaseFiles(partitionPath).anyMatch(dfile -> dfile.getFileId().equals(fileId)), @@ -728,7 +728,7 @@ public void testGetLatestDataFilesForFileId() throws IOException { // Do another commit, but not safe String commitTime2 = "2"; - String fileName2 = FSUtils.makeDataFileName(commitTime2, TEST_WRITE_TOKEN, fileId); + String fileName2 = FSUtils.makeBaseFileName(commitTime2, TEST_WRITE_TOKEN, fileId); new File(basePath + "/" + partitionPath + "/" + fileName2).createNewFile(); refreshFsView(); assertEquals(fileName1, roView.getLatestBaseFiles(partitionPath) @@ -762,22 +762,22 @@ public void testStreamLatestVersionInPartition(boolean isLatestFileSliceOnly) th String fileId3 = UUID.randomUUID().toString(); String fileId4 = UUID.randomUUID().toString(); - new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, fileId1)).createNewFile(); - new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime4, TEST_WRITE_TOKEN, fileId1)).createNewFile(); + new File(fullPartitionPath + FSUtils.makeBaseFileName(commitTime1, TEST_WRITE_TOKEN, fileId1)).createNewFile(); + new File(fullPartitionPath + FSUtils.makeBaseFileName(commitTime4, TEST_WRITE_TOKEN, fileId1)).createNewFile(); new File(fullPartitionPath + FSUtils.makeLogFileName(fileId1, HoodieLogFile.DELTA_EXTENSION, commitTime4, 0, TEST_WRITE_TOKEN)) .createNewFile(); new File(fullPartitionPath + FSUtils.makeLogFileName(fileId1, HoodieLogFile.DELTA_EXTENSION, commitTime4, 1, TEST_WRITE_TOKEN)) .createNewFile(); - new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, fileId2)).createNewFile(); - new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime2, TEST_WRITE_TOKEN, fileId2)).createNewFile(); - new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, TEST_WRITE_TOKEN, fileId2)).createNewFile(); + new File(fullPartitionPath + FSUtils.makeBaseFileName(commitTime1, TEST_WRITE_TOKEN, fileId2)).createNewFile(); + new File(fullPartitionPath + FSUtils.makeBaseFileName(commitTime2, TEST_WRITE_TOKEN, fileId2)).createNewFile(); + new File(fullPartitionPath + FSUtils.makeBaseFileName(commitTime3, TEST_WRITE_TOKEN, fileId2)).createNewFile(); new File(fullPartitionPath + FSUtils.makeLogFileName(fileId2, HoodieLogFile.DELTA_EXTENSION, commitTime3, 0, TEST_WRITE_TOKEN)) .createNewFile(); - new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, TEST_WRITE_TOKEN, fileId3)).createNewFile(); - new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime4, TEST_WRITE_TOKEN, fileId3)).createNewFile(); + new File(fullPartitionPath + FSUtils.makeBaseFileName(commitTime3, TEST_WRITE_TOKEN, fileId3)).createNewFile(); + new File(fullPartitionPath + FSUtils.makeBaseFileName(commitTime4, TEST_WRITE_TOKEN, fileId3)).createNewFile(); new File(fullPartitionPath + FSUtils.makeLogFileName(fileId4, HoodieLogFile.DELTA_EXTENSION, commitTime4, 0, TEST_WRITE_TOKEN)) .createNewFile(); @@ -827,9 +827,9 @@ private void testStreamLatestVersionInPartition(boolean isLatestFileSliceOnly, S for (HoodieBaseFile status : dataFileList) { filenames.add(status.getFileName()); } - assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime4, TEST_WRITE_TOKEN, fileId1))); - assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime3, TEST_WRITE_TOKEN, fileId2))); - assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime4, TEST_WRITE_TOKEN, fileId3))); + assertTrue(filenames.contains(FSUtils.makeBaseFileName(commitTime4, TEST_WRITE_TOKEN, fileId1))); + assertTrue(filenames.contains(FSUtils.makeBaseFileName(commitTime3, TEST_WRITE_TOKEN, fileId2))); + assertTrue(filenames.contains(FSUtils.makeBaseFileName(commitTime4, TEST_WRITE_TOKEN, fileId3))); filenames = new HashSet<>(); List logFilesList = rtView.getLatestFileSlicesBeforeOrOn("2016/05/01", commitTime4, true) @@ -856,12 +856,12 @@ private void testStreamLatestVersionInPartition(boolean isLatestFileSliceOnly, S } if (!isLatestFileSliceOnly) { assertEquals(3, dataFiles.size()); - assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, fileId1))); - assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime3, TEST_WRITE_TOKEN, fileId2))); - assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime3, TEST_WRITE_TOKEN, fileId3))); + assertTrue(filenames.contains(FSUtils.makeBaseFileName(commitTime1, TEST_WRITE_TOKEN, fileId1))); + assertTrue(filenames.contains(FSUtils.makeBaseFileName(commitTime3, TEST_WRITE_TOKEN, fileId2))); + assertTrue(filenames.contains(FSUtils.makeBaseFileName(commitTime3, TEST_WRITE_TOKEN, fileId3))); } else { assertEquals(1, dataFiles.size()); - assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime3, TEST_WRITE_TOKEN, fileId2))); + assertTrue(filenames.contains(FSUtils.makeBaseFileName(commitTime3, TEST_WRITE_TOKEN, fileId2))); } logFilesList = rtView.getLatestFileSlicesBeforeOrOn("2016/05/01", commitTime3, true) @@ -887,13 +887,13 @@ protected void testStreamEveryVersionInPartition(boolean isLatestFileSliceOnly) String fileId2 = UUID.randomUUID().toString(); String fileId3 = UUID.randomUUID().toString(); - new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, fileId1)).createNewFile(); - new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime4, TEST_WRITE_TOKEN, fileId1)).createNewFile(); - new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, fileId2)).createNewFile(); - new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime2, TEST_WRITE_TOKEN, fileId2)).createNewFile(); - new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, TEST_WRITE_TOKEN, fileId2)).createNewFile(); - new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, TEST_WRITE_TOKEN, fileId3)).createNewFile(); - new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime4, TEST_WRITE_TOKEN, fileId3)).createNewFile(); + new File(fullPartitionPath + FSUtils.makeBaseFileName(commitTime1, TEST_WRITE_TOKEN, fileId1)).createNewFile(); + new File(fullPartitionPath + FSUtils.makeBaseFileName(commitTime4, TEST_WRITE_TOKEN, fileId1)).createNewFile(); + new File(fullPartitionPath + FSUtils.makeBaseFileName(commitTime1, TEST_WRITE_TOKEN, fileId2)).createNewFile(); + new File(fullPartitionPath + FSUtils.makeBaseFileName(commitTime2, TEST_WRITE_TOKEN, fileId2)).createNewFile(); + new File(fullPartitionPath + FSUtils.makeBaseFileName(commitTime3, TEST_WRITE_TOKEN, fileId2)).createNewFile(); + new File(fullPartitionPath + FSUtils.makeBaseFileName(commitTime3, TEST_WRITE_TOKEN, fileId3)).createNewFile(); + new File(fullPartitionPath + FSUtils.makeBaseFileName(commitTime4, TEST_WRITE_TOKEN, fileId3)).createNewFile(); new File(basePath + "/.hoodie/" + commitTime1 + ".commit").createNewFile(); new File(basePath + "/.hoodie/" + commitTime2 + ".commit").createNewFile(); @@ -918,22 +918,22 @@ protected void testStreamEveryVersionInPartition(boolean isLatestFileSliceOnly) Set expFileNames = new HashSet<>(); if (fileId.equals(fileId1)) { if (!isLatestFileSliceOnly) { - expFileNames.add(FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, fileId1)); + expFileNames.add(FSUtils.makeBaseFileName(commitTime1, TEST_WRITE_TOKEN, fileId1)); } - expFileNames.add(FSUtils.makeDataFileName(commitTime4, TEST_WRITE_TOKEN, fileId1)); + expFileNames.add(FSUtils.makeBaseFileName(commitTime4, TEST_WRITE_TOKEN, fileId1)); assertEquals(expFileNames, filenames); } else if (fileId.equals(fileId2)) { if (!isLatestFileSliceOnly) { - expFileNames.add(FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, fileId2)); - expFileNames.add(FSUtils.makeDataFileName(commitTime2, TEST_WRITE_TOKEN, fileId2)); + expFileNames.add(FSUtils.makeBaseFileName(commitTime1, TEST_WRITE_TOKEN, fileId2)); + expFileNames.add(FSUtils.makeBaseFileName(commitTime2, TEST_WRITE_TOKEN, fileId2)); } - expFileNames.add(FSUtils.makeDataFileName(commitTime3, TEST_WRITE_TOKEN, fileId2)); + expFileNames.add(FSUtils.makeBaseFileName(commitTime3, TEST_WRITE_TOKEN, fileId2)); assertEquals(expFileNames, filenames); } else { if (!isLatestFileSliceOnly) { - expFileNames.add(FSUtils.makeDataFileName(commitTime3, TEST_WRITE_TOKEN, fileId3)); + expFileNames.add(FSUtils.makeBaseFileName(commitTime3, TEST_WRITE_TOKEN, fileId3)); } - expFileNames.add(FSUtils.makeDataFileName(commitTime4, TEST_WRITE_TOKEN, fileId3)); + expFileNames.add(FSUtils.makeBaseFileName(commitTime4, TEST_WRITE_TOKEN, fileId3)); assertEquals(expFileNames, filenames); } } @@ -956,21 +956,21 @@ protected void testStreamLatestVersionInRange(boolean isLatestFileSliceOnly) thr String fileId2 = UUID.randomUUID().toString(); String fileId3 = UUID.randomUUID().toString(); - new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, fileId1)).createNewFile(); + new File(fullPartitionPath + FSUtils.makeBaseFileName(commitTime1, TEST_WRITE_TOKEN, fileId1)).createNewFile(); new File(fullPartitionPath + FSUtils.makeLogFileName(fileId1, HoodieLogFile.DELTA_EXTENSION, commitTime1, 0, TEST_WRITE_TOKEN)) .createNewFile(); - new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, TEST_WRITE_TOKEN, fileId1)).createNewFile(); + new File(fullPartitionPath + FSUtils.makeBaseFileName(commitTime3, TEST_WRITE_TOKEN, fileId1)).createNewFile(); - new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, fileId2)).createNewFile(); - new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime2, TEST_WRITE_TOKEN, fileId2)).createNewFile(); - new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, TEST_WRITE_TOKEN, fileId2)).createNewFile(); + new File(fullPartitionPath + FSUtils.makeBaseFileName(commitTime1, TEST_WRITE_TOKEN, fileId2)).createNewFile(); + new File(fullPartitionPath + FSUtils.makeBaseFileName(commitTime2, TEST_WRITE_TOKEN, fileId2)).createNewFile(); + new File(fullPartitionPath + FSUtils.makeBaseFileName(commitTime3, TEST_WRITE_TOKEN, fileId2)).createNewFile(); new File(fullPartitionPath + FSUtils.makeLogFileName(fileId2, HoodieLogFile.DELTA_EXTENSION, commitTime3, 0, TEST_WRITE_TOKEN)) .createNewFile(); - new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, TEST_WRITE_TOKEN, fileId3)).createNewFile(); - new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime4, TEST_WRITE_TOKEN, fileId3)).createNewFile(); + new File(fullPartitionPath + FSUtils.makeBaseFileName(commitTime3, TEST_WRITE_TOKEN, fileId3)).createNewFile(); + new File(fullPartitionPath + FSUtils.makeBaseFileName(commitTime4, TEST_WRITE_TOKEN, fileId3)).createNewFile(); new File(basePath + "/.hoodie/" + commitTime1 + ".commit").createNewFile(); new File(basePath + "/.hoodie/" + commitTime2 + ".commit").createNewFile(); @@ -993,10 +993,10 @@ protected void testStreamLatestVersionInRange(boolean isLatestFileSliceOnly) thr filenames.add(status.getFileName()); } - assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime3, TEST_WRITE_TOKEN, fileId1))); - assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime3, TEST_WRITE_TOKEN, fileId2))); + assertTrue(filenames.contains(FSUtils.makeBaseFileName(commitTime3, TEST_WRITE_TOKEN, fileId1))); + assertTrue(filenames.contains(FSUtils.makeBaseFileName(commitTime3, TEST_WRITE_TOKEN, fileId2))); if (!isLatestFileSliceOnly) { - assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime3, TEST_WRITE_TOKEN, fileId3))); + assertTrue(filenames.contains(FSUtils.makeBaseFileName(commitTime3, TEST_WRITE_TOKEN, fileId3))); } List slices = @@ -1037,13 +1037,13 @@ protected void testStreamLatestVersionsBefore(boolean isLatestFileSliceOnly) thr String fileId2 = UUID.randomUUID().toString(); String fileId3 = UUID.randomUUID().toString(); - new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, fileId1)).createNewFile(); - new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime4, TEST_WRITE_TOKEN, fileId1)).createNewFile(); - new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, fileId2)).createNewFile(); - new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime2, TEST_WRITE_TOKEN, fileId2)).createNewFile(); - new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, TEST_WRITE_TOKEN, fileId2)).createNewFile(); - new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, TEST_WRITE_TOKEN, fileId3)).createNewFile(); - new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime4, TEST_WRITE_TOKEN, fileId3)).createNewFile(); + new File(fullPartitionPath + FSUtils.makeBaseFileName(commitTime1, TEST_WRITE_TOKEN, fileId1)).createNewFile(); + new File(fullPartitionPath + FSUtils.makeBaseFileName(commitTime4, TEST_WRITE_TOKEN, fileId1)).createNewFile(); + new File(fullPartitionPath + FSUtils.makeBaseFileName(commitTime1, TEST_WRITE_TOKEN, fileId2)).createNewFile(); + new File(fullPartitionPath + FSUtils.makeBaseFileName(commitTime2, TEST_WRITE_TOKEN, fileId2)).createNewFile(); + new File(fullPartitionPath + FSUtils.makeBaseFileName(commitTime3, TEST_WRITE_TOKEN, fileId2)).createNewFile(); + new File(fullPartitionPath + FSUtils.makeBaseFileName(commitTime3, TEST_WRITE_TOKEN, fileId3)).createNewFile(); + new File(fullPartitionPath + FSUtils.makeBaseFileName(commitTime4, TEST_WRITE_TOKEN, fileId3)).createNewFile(); new File(basePath + "/.hoodie/" + commitTime1 + ".commit").createNewFile(); new File(basePath + "/.hoodie/" + commitTime2 + ".commit").createNewFile(); @@ -1063,8 +1063,8 @@ protected void testStreamLatestVersionsBefore(boolean isLatestFileSliceOnly) thr for (HoodieBaseFile status : dataFiles) { filenames.add(status.getFileName()); } - assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, fileId1))); - assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime2, TEST_WRITE_TOKEN, fileId2))); + assertTrue(filenames.contains(FSUtils.makeBaseFileName(commitTime1, TEST_WRITE_TOKEN, fileId1))); + assertTrue(filenames.contains(FSUtils.makeBaseFileName(commitTime2, TEST_WRITE_TOKEN, fileId2))); } else { assertEquals(0, dataFiles.size()); } @@ -1088,30 +1088,30 @@ protected void testStreamLatestVersions(boolean isLatestFileSliceOnly) throws IO String fileId2 = UUID.randomUUID().toString(); String fileId3 = UUID.randomUUID().toString(); - new File(fullPartitionPath + "/" + FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, fileId1)) + new File(fullPartitionPath + "/" + FSUtils.makeBaseFileName(commitTime1, TEST_WRITE_TOKEN, fileId1)) .createNewFile(); new File(fullPartitionPath + "/" + FSUtils.makeLogFileName(fileId1, HoodieLogFile.DELTA_EXTENSION, commitTime1, 0, TEST_WRITE_TOKEN)) .createNewFile(); - new File(fullPartitionPath + "/" + FSUtils.makeDataFileName(commitTime4, TEST_WRITE_TOKEN, fileId1)) + new File(fullPartitionPath + "/" + FSUtils.makeBaseFileName(commitTime4, TEST_WRITE_TOKEN, fileId1)) .createNewFile(); new File(fullPartitionPath + "/" + FSUtils.makeLogFileName(fileId1, HoodieLogFile.DELTA_EXTENSION, commitTime4, 0, TEST_WRITE_TOKEN)) .createNewFile(); - new File(fullPartitionPath + "/" + FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, fileId2)) + new File(fullPartitionPath + "/" + FSUtils.makeBaseFileName(commitTime1, TEST_WRITE_TOKEN, fileId2)) .createNewFile(); - new File(fullPartitionPath + "/" + FSUtils.makeDataFileName(commitTime2, TEST_WRITE_TOKEN, fileId2)) + new File(fullPartitionPath + "/" + FSUtils.makeBaseFileName(commitTime2, TEST_WRITE_TOKEN, fileId2)) .createNewFile(); new File(fullPartitionPath + "/" + FSUtils.makeLogFileName(fileId2, HoodieLogFile.DELTA_EXTENSION, commitTime2, 0, TEST_WRITE_TOKEN)) .createNewFile(); - new File(fullPartitionPath + "/" + FSUtils.makeDataFileName(commitTime3, TEST_WRITE_TOKEN, fileId2)) + new File(fullPartitionPath + "/" + FSUtils.makeBaseFileName(commitTime3, TEST_WRITE_TOKEN, fileId2)) .createNewFile(); - new File(fullPartitionPath + "/" + FSUtils.makeDataFileName(commitTime3, TEST_WRITE_TOKEN, fileId3)) + new File(fullPartitionPath + "/" + FSUtils.makeBaseFileName(commitTime3, TEST_WRITE_TOKEN, fileId3)) .createNewFile(); - new File(fullPartitionPath + "/" + FSUtils.makeDataFileName(commitTime4, TEST_WRITE_TOKEN, fileId3)) + new File(fullPartitionPath + "/" + FSUtils.makeBaseFileName(commitTime4, TEST_WRITE_TOKEN, fileId3)) .createNewFile(); new File(basePath + "/.hoodie/" + commitTime1 + ".commit").createNewFile(); @@ -1158,9 +1158,9 @@ protected void testStreamLatestVersions(boolean isLatestFileSliceOnly) throws IO for (HoodieBaseFile status : statuses1) { filenames.add(status.getFileName()); } - assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime4, TEST_WRITE_TOKEN, fileId1))); - assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime3, TEST_WRITE_TOKEN, fileId2))); - assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime4, TEST_WRITE_TOKEN, fileId3))); + assertTrue(filenames.contains(FSUtils.makeBaseFileName(commitTime4, TEST_WRITE_TOKEN, fileId1))); + assertTrue(filenames.contains(FSUtils.makeBaseFileName(commitTime3, TEST_WRITE_TOKEN, fileId2))); + assertTrue(filenames.contains(FSUtils.makeBaseFileName(commitTime4, TEST_WRITE_TOKEN, fileId3))); } @Test @@ -1181,15 +1181,15 @@ public void testPendingCompactionWithDuplicateFileIdsAcrossPartitions() throws E String deltaInstantTime2 = "3"; String fileId = UUID.randomUUID().toString(); - String dataFileName = FSUtils.makeDataFileName(instantTime1, TEST_WRITE_TOKEN, fileId); + String dataFileName = FSUtils.makeBaseFileName(instantTime1, TEST_WRITE_TOKEN, fileId); new File(fullPartitionPath1 + dataFileName).createNewFile(); String fileName1 = FSUtils.makeLogFileName(fileId, HoodieLogFile.DELTA_EXTENSION, instantTime1, 0, TEST_WRITE_TOKEN); new File(fullPartitionPath1 + fileName1).createNewFile(); - new File(fullPartitionPath2 + FSUtils.makeDataFileName(instantTime1, TEST_WRITE_TOKEN, fileId)).createNewFile(); + new File(fullPartitionPath2 + FSUtils.makeBaseFileName(instantTime1, TEST_WRITE_TOKEN, fileId)).createNewFile(); new File(fullPartitionPath2 + fileName1).createNewFile(); - new File(fullPartitionPath3 + FSUtils.makeDataFileName(instantTime1, TEST_WRITE_TOKEN, fileId)).createNewFile(); + new File(fullPartitionPath3 + FSUtils.makeBaseFileName(instantTime1, TEST_WRITE_TOKEN, fileId)).createNewFile(); new File(fullPartitionPath3 + fileName1).createNewFile(); HoodieActiveTimeline commitTimeline = metaClient.getActiveTimeline(); @@ -1228,7 +1228,7 @@ public void testPendingCompactionWithDuplicateFileIdsAcrossPartitions() throws E partitionFileSlicesPairs.add(Pair.of(partitionPath3, fileSlices.get(0))); String compactionRequestedTime = "2"; - String compactDataFileName = FSUtils.makeDataFileName(compactionRequestedTime, TEST_WRITE_TOKEN, fileId); + String compactDataFileName = FSUtils.makeBaseFileName(compactionRequestedTime, TEST_WRITE_TOKEN, fileId); HoodieCompactionPlan compactionPlan = CompactionUtils.buildFromFileSlices(partitionFileSlicesPairs, Option.empty(), Option.empty()); @@ -1345,8 +1345,8 @@ public void testReplaceWithTimeTravel() throws IOException { "No commit, should not find any data file"); // Only one commit String commitTime1 = "1"; - String fileName1 = FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, fileId1); - String fileName2 = FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, fileId2); + String fileName1 = FSUtils.makeBaseFileName(commitTime1, TEST_WRITE_TOKEN, fileId1); + String fileName2 = FSUtils.makeBaseFileName(commitTime1, TEST_WRITE_TOKEN, fileId2); new File(basePath + "/" + partitionPath1 + "/" + fileName1).createNewFile(); new File(basePath + "/" + partitionPath1 + "/" + fileName2).createNewFile(); @@ -1362,8 +1362,8 @@ public void testReplaceWithTimeTravel() throws IOException { // create commit2 - fileId1 is replaced. new file groups fileId3,fileId4 are created. String fileId3 = UUID.randomUUID().toString(); String fileId4 = UUID.randomUUID().toString(); - String fileName3 = FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, fileId3); - String fileName4 = FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, fileId4); + String fileName3 = FSUtils.makeBaseFileName(commitTime1, TEST_WRITE_TOKEN, fileId3); + String fileName4 = FSUtils.makeBaseFileName(commitTime1, TEST_WRITE_TOKEN, fileId4); new File(basePath + "/" + partitionPath1 + "/" + fileName3).createNewFile(); new File(basePath + "/" + partitionPath1 + "/" + fileName4).createNewFile(); @@ -1440,10 +1440,10 @@ public void testReplaceFileIdIsExcludedInView() throws IOException { // Only one commit String commitTime1 = "1"; - String fileName1 = FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, fileId1); - String fileName2 = FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, fileId2); - String fileName3 = FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, fileId3); - String fileName4 = FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, fileId4); + String fileName1 = FSUtils.makeBaseFileName(commitTime1, TEST_WRITE_TOKEN, fileId1); + String fileName2 = FSUtils.makeBaseFileName(commitTime1, TEST_WRITE_TOKEN, fileId2); + String fileName3 = FSUtils.makeBaseFileName(commitTime1, TEST_WRITE_TOKEN, fileId3); + String fileName4 = FSUtils.makeBaseFileName(commitTime1, TEST_WRITE_TOKEN, fileId4); new File(basePath + "/" + partitionPath1 + "/" + fileName1).createNewFile(); new File(basePath + "/" + partitionPath1 + "/" + fileName2).createNewFile(); new File(basePath + "/" + partitionPath2 + "/" + fileName3).createNewFile(); @@ -1500,9 +1500,9 @@ public void testPendingClusteringOperations() throws IOException { "No commit, should not find any data file"); // Only one commit String commitTime1 = "1"; - String fileName1 = FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, fileId1); - String fileName2 = FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, fileId2); - String fileName3 = FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, fileId3); + String fileName1 = FSUtils.makeBaseFileName(commitTime1, TEST_WRITE_TOKEN, fileId1); + String fileName2 = FSUtils.makeBaseFileName(commitTime1, TEST_WRITE_TOKEN, fileId2); + String fileName3 = FSUtils.makeBaseFileName(commitTime1, TEST_WRITE_TOKEN, fileId3); new File(basePath + "/" + partitionPath1 + "/" + fileName1).createNewFile(); new File(basePath + "/" + partitionPath1 + "/" + fileName2).createNewFile(); new File(basePath + "/" + partitionPath1 + "/" + fileName3).createNewFile(); @@ -1614,8 +1614,8 @@ public void testHoodieTableFileSystemViewWithPendingClustering() throws IOExcept // first insert commit String commitTime1 = "1"; - String fileName1 = FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, fileId1); - String fileName2 = FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, fileId2); + String fileName1 = FSUtils.makeBaseFileName(commitTime1, TEST_WRITE_TOKEN, fileId1); + String fileName2 = FSUtils.makeBaseFileName(commitTime1, TEST_WRITE_TOKEN, fileId2); new File(basePath + "/" + partitionPath + "/" + fileName1).createNewFile(); new File(basePath + "/" + partitionPath + "/" + fileName2).createNewFile(); @@ -1636,7 +1636,7 @@ public void testHoodieTableFileSystemViewWithPendingClustering() throws IOExcept // replace commit String commitTime2 = "2"; - String fileName3 = FSUtils.makeDataFileName(commitTime2, TEST_WRITE_TOKEN, fileId3); + String fileName3 = FSUtils.makeBaseFileName(commitTime2, TEST_WRITE_TOKEN, fileId3); new File(basePath + "/" + partitionPath + "/" + fileName3).createNewFile(); HoodieInstant instant2 = new HoodieInstant(true, HoodieTimeline.REPLACE_COMMIT_ACTION, commitTime2); @@ -1658,7 +1658,7 @@ public void testHoodieTableFileSystemViewWithPendingClustering() throws IOExcept // another insert commit String commitTime3 = "3"; - String fileName4 = FSUtils.makeDataFileName(commitTime3, TEST_WRITE_TOKEN, fileId4); + String fileName4 = FSUtils.makeBaseFileName(commitTime3, TEST_WRITE_TOKEN, fileId4); new File(basePath + "/" + partitionPath + "/" + fileName4).createNewFile(); HoodieInstant instant3 = new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, commitTime3); diff --git a/hudi-common/src/test/java/org/apache/hudi/common/table/view/TestIncrementalFSViewSync.java b/hudi-common/src/test/java/org/apache/hudi/common/table/view/TestIncrementalFSViewSync.java index a9c9db303f328..1c59558c94ce7 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/table/view/TestIncrementalFSViewSync.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/table/view/TestIncrementalFSViewSync.java @@ -826,7 +826,7 @@ private List> generateDataForInstant(String baseIn File file = new File(basePath + "/" + p + "/" + (deltaCommit ? FSUtils.makeLogFileName(f, ".log", baseInstant, Integer.parseInt(instant), TEST_WRITE_TOKEN) - : FSUtils.makeDataFileName(instant, TEST_WRITE_TOKEN, f))); + : FSUtils.makeBaseFileName(instant, TEST_WRITE_TOKEN, f))); file.createNewFile(); HoodieWriteStat w = new HoodieWriteStat(); w.setFileId(f); diff --git a/hudi-common/src/test/java/org/apache/hudi/common/testutils/FileCreateUtils.java b/hudi-common/src/test/java/org/apache/hudi/common/testutils/FileCreateUtils.java index 231915072914d..290753ef52006 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/testutils/FileCreateUtils.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/testutils/FileCreateUtils.java @@ -80,7 +80,7 @@ public static String baseFileName(String instantTime, String fileId) { } public static String baseFileName(String instantTime, String fileId, String fileExtension) { - return FSUtils.makeDataFileName(instantTime, WRITE_TOKEN, fileId, fileExtension); + return FSUtils.makeBaseFileName(instantTime, WRITE_TOKEN, fileId, fileExtension); } public static String logFileName(String instantTime, String fileId, int version) { diff --git a/hudi-common/src/test/java/org/apache/hudi/common/util/TestClusteringUtils.java b/hudi-common/src/test/java/org/apache/hudi/common/util/TestClusteringUtils.java index 54ca072651e07..a5d45d1184f9b 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/util/TestClusteringUtils.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/util/TestClusteringUtils.java @@ -147,7 +147,7 @@ private HoodieInstant createRequestedReplaceInstant(String partitionPath1, Strin private FileSlice generateFileSlice(String partitionPath, String fileId, String baseInstant) { FileSlice fs = new FileSlice(new HoodieFileGroupId(partitionPath, fileId), baseInstant); - fs.setBaseFile(new HoodieBaseFile(FSUtils.makeDataFileName(baseInstant, "1-0-1", fileId))); + fs.setBaseFile(new HoodieBaseFile(FSUtils.makeBaseFileName(baseInstant, "1-0-1", fileId))); return fs; } diff --git a/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/testutils/InputFormatTestUtil.java b/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/testutils/InputFormatTestUtil.java index ccd85d382930a..1081e43175630 100644 --- a/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/testutils/InputFormatTestUtil.java +++ b/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/testutils/InputFormatTestUtil.java @@ -101,7 +101,7 @@ public static File simulateInserts(File partitionPath, String baseFileExtension, throws IOException { for (int i = 0; i < numberOfFiles; i++) { Files.createFile(partitionPath.toPath() - .resolve(FSUtils.makeDataFileName(commitNumber, TEST_WRITE_TOKEN, fileId + i, baseFileExtension))); + .resolve(FSUtils.makeBaseFileName(commitNumber, TEST_WRITE_TOKEN, fileId + i, baseFileExtension))); } return partitionPath; } @@ -118,7 +118,7 @@ public static void simulateUpdates(File directory, String baseFileExtension, fin List toUpdateList = dataFiles.subList(0, Math.min(numberOfFilesUpdated, dataFiles.size())); for (File file : toUpdateList) { String fileId = FSUtils.getFileId(file.getName()); - Files.createFile(directory.toPath().resolve(FSUtils.makeDataFileName(newCommit, TEST_WRITE_TOKEN, fileId, + Files.createFile(directory.toPath().resolve(FSUtils.makeBaseFileName(newCommit, TEST_WRITE_TOKEN, fileId, baseFileExtension))); } } @@ -270,7 +270,7 @@ private static void createData(Schema schema, java.nio.file.Path partitionPath, String commitNumber) throws IOException { AvroParquetWriter parquetWriter; for (int i = 0; i < numberOfFiles; i++) { - String fileId = FSUtils.makeDataFileName(commitNumber, TEST_WRITE_TOKEN, "fileid" + i, HoodieFileFormat.PARQUET.getFileExtension()); + String fileId = FSUtils.makeBaseFileName(commitNumber, TEST_WRITE_TOKEN, "fileid" + i, HoodieFileFormat.PARQUET.getFileExtension()); parquetWriter = new AvroParquetWriter(new Path(partitionPath.resolve(fileId).toString()), schema); try { for (GenericRecord record : generateAvroRecords(schema, numberOfRecords, commitNumber, fileId)) { @@ -286,7 +286,7 @@ private static void createSimpleData(Schema schema, java.nio.file.Path partition String commitNumber) throws Exception { AvroParquetWriter parquetWriter; for (int i = 0; i < numberOfFiles; i++) { - String fileId = FSUtils.makeDataFileName(commitNumber, "1", "fileid" + i, HoodieFileFormat.PARQUET.getFileExtension()); + String fileId = FSUtils.makeBaseFileName(commitNumber, "1", "fileid" + i, HoodieFileFormat.PARQUET.getFileExtension()); parquetWriter = new AvroParquetWriter(new Path(partitionPath.resolve(fileId).toString()), schema); try { List records = SchemaTestUtil.generateTestRecords(0, numberOfRecords); @@ -318,7 +318,7 @@ public static void simulateParquetUpdates(File directory, Schema schema, String File fileToUpdate = Objects.requireNonNull(directory.listFiles((dir, name) -> name.endsWith("parquet")))[0]; String fileId = FSUtils.getFileId(fileToUpdate.getName()); File dataFile = new File(directory, - FSUtils.makeDataFileName(newCommit, TEST_WRITE_TOKEN, fileId, HoodieFileFormat.PARQUET.getFileExtension())); + FSUtils.makeBaseFileName(newCommit, TEST_WRITE_TOKEN, fileId, HoodieFileFormat.PARQUET.getFileExtension())); try (AvroParquetWriter parquetWriter = new AvroParquetWriter(new Path(dataFile.getAbsolutePath()), schema)) { for (GenericRecord record : generateAvroRecords(schema, totalNumberOfRecords, originalCommit, fileId)) { if (numberOfRecordsToUpdate > 0) { diff --git a/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/testutils/HiveTestUtil.java b/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/testutils/HiveTestUtil.java index 3cdbe0d8bb757..8be2ace89f8f1 100644 --- a/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/testutils/HiveTestUtil.java +++ b/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/testutils/HiveTestUtil.java @@ -234,7 +234,7 @@ public static void createCOWTableWithSchema(String instantTime, String schemaFil fileSystem.mkdirs(partPath); List writeStats = new ArrayList<>(); String fileId = UUID.randomUUID().toString(); - Path filePath = new Path(partPath.toString() + "/" + FSUtils.makeDataFileName(instantTime, "1-0-1", fileId)); + Path filePath = new Path(partPath.toString() + "/" + FSUtils.makeBaseFileName(instantTime, "1-0-1", fileId)); Schema schema = SchemaTestUtil.getSchemaFromResource(HiveTestUtil.class, schemaFileName); generateParquetDataWithSchema(filePath, schema); HoodieWriteStat writeStat = new HoodieWriteStat(); @@ -371,7 +371,7 @@ private static List createTestData(Path partPath, boolean isPar for (int i = 0; i < 5; i++) { // Create 5 files String fileId = UUID.randomUUID().toString(); - Path filePath = new Path(partPath.toString() + "/" + FSUtils.makeDataFileName(instantTime, "1-0-1", fileId)); + Path filePath = new Path(partPath.toString() + "/" + FSUtils.makeBaseFileName(instantTime, "1-0-1", fileId)); generateParquetData(filePath, isParquetSchemaSimple); HoodieWriteStat writeStat = new HoodieWriteStat(); writeStat.setFileId(fileId); diff --git a/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/testutils/TestCluster.java b/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/testutils/TestCluster.java index c059c63a6a6f9..c1f891fce8431 100644 --- a/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/testutils/TestCluster.java +++ b/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/testutils/TestCluster.java @@ -211,7 +211,7 @@ private List createTestData(Path partPath, boolean isParquetSch // Create 5 files String fileId = UUID.randomUUID().toString(); Path filePath = new Path(partPath.toString() + "/" + FSUtils - .makeDataFileName(commitTime, "1-0-1", fileId)); + .makeBaseFileName(commitTime, "1-0-1", fileId)); generateParquetData(filePath, isParquetSchemaSimple); HoodieWriteStat writeStat = new HoodieWriteStat(); writeStat.setFileId(fileId); diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieSnapshotCopier.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieSnapshotCopier.java index dd25e7f8bebad..9d4ce71d8f25b 100644 --- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieSnapshotCopier.java +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieSnapshotCopier.java @@ -99,27 +99,27 @@ public void testSnapshotCopy() throws Exception { HoodieTestDataGenerator.writePartitionMetadataDeprecated(fs, new String[] {"2016/05/01", "2016/05/02", "2016/05/06"}, basePath); // Make commit1 - File file11 = new File(basePath + "/2016/05/01/" + FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, "id11")); + File file11 = new File(basePath + "/2016/05/01/" + FSUtils.makeBaseFileName(commitTime1, TEST_WRITE_TOKEN, "id11")); file11.createNewFile(); - File file12 = new File(basePath + "/2016/05/02/" + FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, "id12")); + File file12 = new File(basePath + "/2016/05/02/" + FSUtils.makeBaseFileName(commitTime1, TEST_WRITE_TOKEN, "id12")); file12.createNewFile(); - File file13 = new File(basePath + "/2016/05/06/" + FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, "id13")); + File file13 = new File(basePath + "/2016/05/06/" + FSUtils.makeBaseFileName(commitTime1, TEST_WRITE_TOKEN, "id13")); file13.createNewFile(); // Make commit2 - File file21 = new File(basePath + "/2016/05/01/" + FSUtils.makeDataFileName(commitTime2, TEST_WRITE_TOKEN, "id21")); + File file21 = new File(basePath + "/2016/05/01/" + FSUtils.makeBaseFileName(commitTime2, TEST_WRITE_TOKEN, "id21")); file21.createNewFile(); - File file22 = new File(basePath + "/2016/05/02/" + FSUtils.makeDataFileName(commitTime2, TEST_WRITE_TOKEN, "id22")); + File file22 = new File(basePath + "/2016/05/02/" + FSUtils.makeBaseFileName(commitTime2, TEST_WRITE_TOKEN, "id22")); file22.createNewFile(); - File file23 = new File(basePath + "/2016/05/06/" + FSUtils.makeDataFileName(commitTime2, TEST_WRITE_TOKEN, "id23")); + File file23 = new File(basePath + "/2016/05/06/" + FSUtils.makeBaseFileName(commitTime2, TEST_WRITE_TOKEN, "id23")); file23.createNewFile(); // Make commit3 - File file31 = new File(basePath + "/2016/05/01/" + FSUtils.makeDataFileName(commitTime3, TEST_WRITE_TOKEN, "id31")); + File file31 = new File(basePath + "/2016/05/01/" + FSUtils.makeBaseFileName(commitTime3, TEST_WRITE_TOKEN, "id31")); file31.createNewFile(); - File file32 = new File(basePath + "/2016/05/02/" + FSUtils.makeDataFileName(commitTime3, TEST_WRITE_TOKEN, "id32")); + File file32 = new File(basePath + "/2016/05/02/" + FSUtils.makeBaseFileName(commitTime3, TEST_WRITE_TOKEN, "id32")); file32.createNewFile(); - File file33 = new File(basePath + "/2016/05/06/" + FSUtils.makeDataFileName(commitTime3, TEST_WRITE_TOKEN, "id33")); + File file33 = new File(basePath + "/2016/05/06/" + FSUtils.makeBaseFileName(commitTime3, TEST_WRITE_TOKEN, "id33")); file33.createNewFile(); // Do a snapshot copy From af1128acf95ade0e52920638c2a7a0badaf53869 Mon Sep 17 00:00:00 2001 From: Sivabalan Narayanan Date: Mon, 23 May 2022 23:05:56 -0400 Subject: [PATCH 08/44] [HUDI-4084] Add support to test async table services with integ test suite framework (#5557) * Add support to test async table services with integ test suite framework * Make await time for validation configurable --- ...er-long-running-multi-partitions-hive.yaml | 2 + ...ong-running-multi-partitions-metadata.yaml | 2 + ...treamer-long-running-multi-partitions.yaml | 2 + .../deltastreamer-medium-clustering.yaml | 2 + ...reamer-medium-full-dataset-validation.yaml | 2 + .../detlastreamer-long-running-example.yaml | 2 + .../spark-long-running-non-partitioned.yaml | 2 + .../config/test-suite/spark-long-running.yaml | 2 + hudi-integ-test/README.md | 50 ++++ .../HoodieContinousTestSuiteWriter.java | 157 ++++++++++++ .../HoodieInlineTestSuiteWriter.java | 225 ++++++++++++++++++ .../integ/testsuite/HoodieTestSuiteJob.java | 9 +- .../testsuite/HoodieTestSuiteWriter.java | 206 +++------------- .../testsuite/configuration/DeltaConfig.java | 5 + .../integ/testsuite/dag/WriterContext.java | 43 +++- .../dag/nodes/BaseValidateDatasetNode.java | 66 ++++- .../testsuite/generator/DeltaGenerator.java | 23 +- 17 files changed, 612 insertions(+), 188 deletions(-) create mode 100644 hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieContinousTestSuiteWriter.java create mode 100644 hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieInlineTestSuiteWriter.java diff --git a/docker/demo/config/test-suite/deltastreamer-long-running-multi-partitions-hive.yaml b/docker/demo/config/test-suite/deltastreamer-long-running-multi-partitions-hive.yaml index 76172203866b0..8b82415982f90 100644 --- a/docker/demo/config/test-suite/deltastreamer-long-running-multi-partitions-hive.yaml +++ b/docker/demo/config/test-suite/deltastreamer-long-running-multi-partitions-hive.yaml @@ -74,10 +74,12 @@ dag_content: validate_once_every_itr : 5 validate_hive: true delete_input_data: true + max_wait_time_for_deltastreamer_catch_up_ms: 600000 type: ValidateDatasetNode deps: second_hive_sync last_validate: config: execute_itr_count: 50 + max_wait_time_for_deltastreamer_catch_up_ms: 600000 type: ValidateAsyncOperations deps: second_validate diff --git a/docker/demo/config/test-suite/deltastreamer-long-running-multi-partitions-metadata.yaml b/docker/demo/config/test-suite/deltastreamer-long-running-multi-partitions-metadata.yaml index dc1e99a431209..031664cd15c99 100644 --- a/docker/demo/config/test-suite/deltastreamer-long-running-multi-partitions-metadata.yaml +++ b/docker/demo/config/test-suite/deltastreamer-long-running-multi-partitions-metadata.yaml @@ -62,10 +62,12 @@ dag_content: validate_once_every_itr : 5 validate_hive: false delete_input_data: true + max_wait_time_for_deltastreamer_catch_up_ms: 600000 type: ValidateDatasetNode deps: first_delete last_validate: config: execute_itr_count: 30 + max_wait_time_for_deltastreamer_catch_up_ms: 600000 type: ValidateAsyncOperations deps: second_validate diff --git a/docker/demo/config/test-suite/deltastreamer-long-running-multi-partitions.yaml b/docker/demo/config/test-suite/deltastreamer-long-running-multi-partitions.yaml index eca4eac1c710a..c23775b2ce546 100644 --- a/docker/demo/config/test-suite/deltastreamer-long-running-multi-partitions.yaml +++ b/docker/demo/config/test-suite/deltastreamer-long-running-multi-partitions.yaml @@ -62,10 +62,12 @@ dag_content: validate_once_every_itr : 5 validate_hive: false delete_input_data: true + max_wait_time_for_deltastreamer_catch_up_ms: 600000 type: ValidateDatasetNode deps: first_delete last_validate: config: execute_itr_count: 50 + max_wait_time_for_deltastreamer_catch_up_ms: 600000 type: ValidateAsyncOperations deps: second_validate diff --git a/docker/demo/config/test-suite/deltastreamer-medium-clustering.yaml b/docker/demo/config/test-suite/deltastreamer-medium-clustering.yaml index 81c21a7be67c8..2fc68596d84a4 100644 --- a/docker/demo/config/test-suite/deltastreamer-medium-clustering.yaml +++ b/docker/demo/config/test-suite/deltastreamer-medium-clustering.yaml @@ -64,10 +64,12 @@ dag_content: config: validate_hive: false delete_input_data: true + max_wait_time_for_deltastreamer_catch_up_ms: 600000 type: ValidateDatasetNode deps: first_delete last_validate: config: execute_itr_count: 20 + max_wait_time_for_deltastreamer_catch_up_ms: 600000 type: ValidateAsyncOperations deps: second_validate diff --git a/docker/demo/config/test-suite/deltastreamer-medium-full-dataset-validation.yaml b/docker/demo/config/test-suite/deltastreamer-medium-full-dataset-validation.yaml index a2d85a7a4d0f5..db7edb8f8f28c 100644 --- a/docker/demo/config/test-suite/deltastreamer-medium-full-dataset-validation.yaml +++ b/docker/demo/config/test-suite/deltastreamer-medium-full-dataset-validation.yaml @@ -65,10 +65,12 @@ dag_content: validate_once_every_itr : 5 validate_hive: false delete_input_data: false + max_wait_time_for_deltastreamer_catch_up_ms: 600000 type: ValidateDatasetNode deps: first_delete last_validate: config: execute_itr_count: 20 + max_wait_time_for_deltastreamer_catch_up_ms: 600000 type: ValidateAsyncOperations deps: second_validate diff --git a/docker/demo/config/test-suite/detlastreamer-long-running-example.yaml b/docker/demo/config/test-suite/detlastreamer-long-running-example.yaml index 1c2f44b060036..102807ec435be 100644 --- a/docker/demo/config/test-suite/detlastreamer-long-running-example.yaml +++ b/docker/demo/config/test-suite/detlastreamer-long-running-example.yaml @@ -62,10 +62,12 @@ dag_content: validate_once_every_itr : 5 validate_hive: false delete_input_data: true + max_wait_time_for_deltastreamer_catch_up_ms: 600000 type: ValidateDatasetNode deps: first_delete last_validate: config: execute_itr_count: 50 + max_wait_time_for_deltastreamer_catch_up_ms: 600000 type: ValidateAsyncOperations deps: second_validate diff --git a/docker/demo/config/test-suite/spark-long-running-non-partitioned.yaml b/docker/demo/config/test-suite/spark-long-running-non-partitioned.yaml index dfbfba0a15700..947bbdab86b43 100644 --- a/docker/demo/config/test-suite/spark-long-running-non-partitioned.yaml +++ b/docker/demo/config/test-suite/spark-long-running-non-partitioned.yaml @@ -45,10 +45,12 @@ dag_content: config: validate_hive: false delete_input_data: true + max_wait_time_for_deltastreamer_catch_up_ms: 600000 type: ValidateDatasetNode deps: first_delete last_validate: config: execute_itr_count: 6 + max_wait_time_for_deltastreamer_catch_up_ms: 600000 type: ValidateAsyncOperations deps: second_validate diff --git a/docker/demo/config/test-suite/spark-long-running.yaml b/docker/demo/config/test-suite/spark-long-running.yaml index 00fea43f4578e..2ffef557815c7 100644 --- a/docker/demo/config/test-suite/spark-long-running.yaml +++ b/docker/demo/config/test-suite/spark-long-running.yaml @@ -46,10 +46,12 @@ dag_content: validate_once_every_itr : 5 validate_hive: false delete_input_data: true + max_wait_time_for_deltastreamer_catch_up_ms: 600000 type: ValidateDatasetNode deps: first_delete last_validate: config: execute_itr_count: 30 + max_wait_time_for_deltastreamer_catch_up_ms: 600000 type: ValidateAsyncOperations deps: second_validate diff --git a/hudi-integ-test/README.md b/hudi-integ-test/README.md index 5d26d03a20a89..687ad9a2a90d2 100644 --- a/hudi-integ-test/README.md +++ b/hudi-integ-test/README.md @@ -593,6 +593,56 @@ Sample spark-submit command to test one delta streamer and a spark data source w --use-hudi-data-to-generate-updates ``` +======= +### Testing async table services +We can test async table services with deltastreamer using below command. 3 additional arguments are required to test async +table services comapared to previous command. + +```shell +--continuous \ +--test-continuous-mode \ +--min-sync-interval-seconds 20 +``` + +Here is the full command: +```shell +./bin/spark-submit --packages org.apache.spark:spark-avro_2.11:2.4.4 \ + --conf spark.task.cpus=1 --conf spark.executor.cores=1 \ +--conf spark.task.maxFailures=100 \ +--conf spark.memory.fraction=0.4 \ +--conf spark.rdd.compress=true \ +--conf spark.kryoserializer.buffer.max=2000m \ +--conf spark.serializer=org.apache.spark.serializer.KryoSerializer \ +--conf spark.memory.storageFraction=0.1 \ +--conf spark.shuffle.service.enabled=true \ +--conf spark.sql.hive.convertMetastoreParquet=false \ +--conf spark.driver.maxResultSize=12g \ +--conf spark.executor.heartbeatInterval=120s \ +--conf spark.network.timeout=600s \ +--conf spark.yarn.max.executor.failures=10 \ +--conf spark.sql.catalogImplementation=hive \ +--class org.apache.hudi.integ.testsuite.HoodieTestSuiteJob /hudi-integ-test-bundle-0.12.0-SNAPSHOT.jar \ +--source-ordering-field test_suite_source_ordering_field \ +--use-deltastreamer \ +--target-base-path /tmp/hudi/output \ +--input-base-path /tmp/hudi/input \ +--target-table table1 \ +-props file:/tmp/test.properties \ +--schemaprovider-class org.apache.hudi.integ.testsuite.schema.TestSuiteFileBasedSchemaProvider \ +--source-class org.apache.hudi.utilities.sources.AvroDFSSource \ +--input-file-size 125829120 \ +--workload-yaml-path file:/tmp/simple-deltastreamer.yaml \ +--workload-generator-classname org.apache.hudi.integ.testsuite.dag.WorkflowDagGenerator \ +--table-type COPY_ON_WRITE \ +--compact-scheduling-minshare 1 \ +--clean-input \ +--clean-output \ +--continuous \ +--test-continuous-mode \ +--min-sync-interval-seconds 20 +``` + +We can use any yaml and properties file w/ above spark-submit command to test deltastreamer w/ async table services. ## Automated tests for N no of yamls in Local Docker environment diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieContinousTestSuiteWriter.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieContinousTestSuiteWriter.java new file mode 100644 index 0000000000000..1bf69aaf836cc --- /dev/null +++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieContinousTestSuiteWriter.java @@ -0,0 +1,157 @@ +/* + * 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.integ.testsuite; + +import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.integ.testsuite.HoodieTestSuiteWriter; +import org.apache.hudi.integ.testsuite.writer.DeltaWriteStats; +import org.apache.hudi.utilities.schema.SchemaProvider; + +import org.apache.avro.generic.GenericRecord; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.rdd.RDD; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.Map; +import java.util.Properties; + +/** + * Test suite Writer that assists in testing async table operations with Deltastreamer continuous mode. + * + * Sample command + * ./bin/spark-submit --packages org.apache.spark:spark-avro_2.11:2.4.4 \ + * --conf spark.task.cpus=1 --conf spark.executor.cores=1 \ + * --conf spark.task.maxFailures=100 \ + * --conf spark.memory.fraction=0.4 \ + * --conf spark.rdd.compress=true \ + * --conf spark.kryoserializer.buffer.max=2000m \ + * --conf spark.serializer=org.apache.spark.serializer.KryoSerializer \ + * --conf spark.memory.storageFraction=0.1 \ + * --conf spark.shuffle.service.enabled=true \ + * --conf spark.sql.hive.convertMetastoreParquet=false \ + * --conf spark.driver.maxResultSize=12g \ + * --conf spark.executor.heartbeatInterval=120s \ + * --conf spark.network.timeout=600s \ + * --conf spark.yarn.max.executor.failures=10 \ + * --conf spark.sql.catalogImplementation=hive \ + * --class org.apache.hudi.integ.testsuite.HoodieTestSuiteJob /hudi-integ-test-bundle-0.12.0-SNAPSHOT.jar \ + * --source-ordering-field test_suite_source_ordering_field \ + * --use-deltastreamer \ + * --target-base-path /tmp/hudi/output \ + * --input-base-path /tmp/hudi/input \ + * --target-table table1 \ + * -props file:/tmp/test.properties \ + * --schemaprovider-class org.apache.hudi.integ.testsuite.schema.TestSuiteFileBasedSchemaProvider \ + * --source-class org.apache.hudi.utilities.sources.AvroDFSSource \ + * --input-file-size 125829120 \ + * --workload-yaml-path file:/tmp/simple-deltastreamer.yaml \ + * --workload-generator-classname org.apache.hudi.integ.testsuite.dag.WorkflowDagGenerator \ + * --table-type COPY_ON_WRITE \ + * --compact-scheduling-minshare 1 \ + * --clean-input \ + * --clean-output \ + * --continuous \ + * --test-continuous-mode \ + * --min-sync-interval-seconds 20 + */ +public class HoodieContinousTestSuiteWriter extends HoodieTestSuiteWriter { + + private static Logger log = LoggerFactory.getLogger(HoodieContinousTestSuiteWriter.class); + + public HoodieContinousTestSuiteWriter(JavaSparkContext jsc, Properties props, HoodieTestSuiteJob.HoodieTestSuiteConfig cfg, String schema) throws Exception { + super(jsc, props, cfg, schema); + } + + @Override + public void shutdownResources() { + log.info("Shutting down deltastreamer gracefully "); + this.deltaStreamerWrapper.shutdownGracefully(); + } + + @Override + public RDD getNextBatch() throws Exception { + return null; + } + + @Override + public Pair>> fetchSource() throws Exception { + return null; + } + + @Override + public Option startCommit() { + return null; + } + + public JavaRDD upsert(Option instantTime) throws Exception { + return null; + } + + @Override + public JavaRDD insert(Option instantTime) throws Exception { + return null; + } + + @Override + public JavaRDD insertOverwrite(Option instantTime) throws Exception { + return null; + } + + @Override + public JavaRDD insertOverwriteTable(Option instantTime) throws Exception { + return null; + } + + @Override + public JavaRDD bulkInsert(Option instantTime) throws Exception { + return null; + } + + @Override + public JavaRDD compact(Option instantTime) throws Exception { + return null; + } + + @Override + public void inlineClustering() { + } + + @Override + public Option scheduleCompaction(Option> previousCommitExtraMetadata) throws + Exception { + return Option.empty(); + } + + @Override + public void commit(JavaRDD records, JavaRDD generatedDataStats, + Option instantTime) { + } + + @Override + public void commitCompaction(JavaRDD records, JavaRDD generatedDataStats, + Option instantTime) throws IOException { + } +} diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieInlineTestSuiteWriter.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieInlineTestSuiteWriter.java new file mode 100644 index 0000000000000..63805e71a5645 --- /dev/null +++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieInlineTestSuiteWriter.java @@ -0,0 +1,225 @@ +/* + * 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.integ.testsuite; + +import org.apache.hudi.avro.model.HoodieCompactionPlan; +import org.apache.hudi.client.HoodieReadClient; +import org.apache.hudi.client.SparkRDDWriteClient; +import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.client.common.HoodieSparkEngineContext; +import org.apache.hudi.common.model.HoodieAvroRecord; +import org.apache.hudi.common.model.HoodieCommitMetadata; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.model.WriteOperationType; +import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.config.HoodieCompactionConfig; +import org.apache.hudi.config.HoodieIndexConfig; +import org.apache.hudi.config.HoodiePayloadConfig; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.data.HoodieJavaRDD; +import org.apache.hudi.index.HoodieIndex; +import org.apache.hudi.integ.testsuite.HoodieTestSuiteJob.HoodieTestSuiteConfig; +import org.apache.hudi.integ.testsuite.writer.DeltaWriteStats; +import org.apache.hudi.table.HoodieSparkTable; +import org.apache.hudi.table.action.HoodieWriteMetadata; +import org.apache.hudi.table.action.compact.CompactHelpers; +import org.apache.hudi.utilities.schema.SchemaProvider; + +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericRecord; +import org.apache.hadoop.conf.Configuration; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.rdd.RDD; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; +import java.util.Properties; + +/** + * A writer abstraction for the Hudi test suite. This class wraps different implementations of writers used to perform write operations into the target hudi dataset. Current supported writers are + * {@link HoodieDeltaStreamerWrapper} and {@link SparkRDDWriteClient}. + */ +public class HoodieInlineTestSuiteWriter extends HoodieTestSuiteWriter { + + private static Logger log = LoggerFactory.getLogger(HoodieInlineTestSuiteWriter.class); + + private static final String GENERATED_DATA_PATH = "generated.data.path"; + + public HoodieInlineTestSuiteWriter(JavaSparkContext jsc, Properties props, HoodieTestSuiteConfig cfg, String schema) throws Exception { + super(jsc, props, cfg, schema); + } + + public void shutdownResources() { + // no-op for non continuous mode test suite writer. + } + + public RDD getNextBatch() throws Exception { + Pair>> nextBatch = fetchSource(); + lastCheckpoint = Option.of(nextBatch.getValue().getLeft()); + JavaRDD inputRDD = nextBatch.getRight().getRight(); + return inputRDD.map(r -> (GenericRecord) ((HoodieAvroRecord) r).getData() + .getInsertValue(new Schema.Parser().parse(schema)).get()).rdd(); + } + + public Pair>> fetchSource() throws Exception { + return this.deltaStreamerWrapper.fetchSource(); + } + + public Option startCommit() { + if (cfg.useDeltaStreamer) { + return Option.of(HoodieActiveTimeline.createNewInstantTime()); + } else { + return Option.of(writeClient.startCommit()); + } + } + + public JavaRDD upsert(Option instantTime) throws Exception { + if (cfg.useDeltaStreamer) { + return deltaStreamerWrapper.upsert(WriteOperationType.UPSERT); + } else { + Pair>> nextBatch = fetchSource(); + lastCheckpoint = Option.of(nextBatch.getValue().getLeft()); + return writeClient.upsert(nextBatch.getRight().getRight(), instantTime.get()); + } + } + + public JavaRDD insert(Option instantTime) throws Exception { + if (cfg.useDeltaStreamer) { + return deltaStreamerWrapper.insert(); + } else { + Pair>> nextBatch = fetchSource(); + lastCheckpoint = Option.of(nextBatch.getValue().getLeft()); + return writeClient.insert(nextBatch.getRight().getRight(), instantTime.get()); + } + } + + public JavaRDD insertOverwrite(Option instantTime) throws Exception { + if (cfg.useDeltaStreamer) { + return deltaStreamerWrapper.insertOverwrite(); + } else { + Pair>> nextBatch = fetchSource(); + lastCheckpoint = Option.of(nextBatch.getValue().getLeft()); + return writeClient.insertOverwrite(nextBatch.getRight().getRight(), instantTime.get()).getWriteStatuses(); + } + } + + public JavaRDD insertOverwriteTable(Option instantTime) throws Exception { + if (cfg.useDeltaStreamer) { + return deltaStreamerWrapper.insertOverwriteTable(); + } else { + Pair>> nextBatch = fetchSource(); + lastCheckpoint = Option.of(nextBatch.getValue().getLeft()); + return writeClient.insertOverwriteTable(nextBatch.getRight().getRight(), instantTime.get()).getWriteStatuses(); + } + } + + public JavaRDD bulkInsert(Option instantTime) throws Exception { + if (cfg.useDeltaStreamer) { + return deltaStreamerWrapper.bulkInsert(); + } else { + Pair>> nextBatch = fetchSource(); + lastCheckpoint = Option.of(nextBatch.getValue().getLeft()); + return writeClient.bulkInsert(nextBatch.getRight().getRight(), instantTime.get()); + } + } + + public JavaRDD compact(Option instantTime) throws Exception { + if (cfg.useDeltaStreamer) { + return deltaStreamerWrapper.compact(); + } else { + if (!instantTime.isPresent()) { + Option> compactionPlanPair = Option + .fromJavaOptional(hoodieReadClient.getPendingCompactions() + .stream().findFirst()); + if (compactionPlanPair.isPresent()) { + instantTime = Option.of(compactionPlanPair.get().getLeft()); + } + } + if (instantTime.isPresent()) { + HoodieWriteMetadata> compactionMetadata = writeClient.compact(instantTime.get()); + return compactionMetadata.getWriteStatuses(); + } else { + return null; + } + } + } + + public void inlineClustering() { + if (!cfg.useDeltaStreamer) { + Option clusteringInstantOpt = writeClient.scheduleClustering(Option.empty()); + clusteringInstantOpt.ifPresent(clusteringInstant -> { + // inline cluster should auto commit as the user is never given control + log.warn("Clustering instant :: " + clusteringInstant); + writeClient.cluster(clusteringInstant, true); + }); + } else { + // TODO: fix clustering to be done async https://issues.apache.org/jira/browse/HUDI-1590 + throw new IllegalArgumentException("Clustering cannot be triggered with deltastreamer"); + } + } + + public Option scheduleCompaction(Option> previousCommitExtraMetadata) throws + Exception { + if (cfg.useDeltaStreamer) { + deltaStreamerWrapper.scheduleCompact(); + return Option.empty(); + } else { + return writeClient.scheduleCompaction(previousCommitExtraMetadata); + } + } + + public void commit(JavaRDD records, JavaRDD generatedDataStats, + Option instantTime) { + if (!cfg.useDeltaStreamer) { + Map extraMetadata = new HashMap<>(); + /** Store the checkpoint in the commit metadata just like + * {@link HoodieDeltaStreamer#commit(SparkRDDWriteClient, JavaRDD, Option)} **/ + extraMetadata.put(HoodieDeltaStreamerWrapper.CHECKPOINT_KEY, lastCheckpoint.get()); + if (generatedDataStats != null && generatedDataStats.count() > 1) { + // Just stores the path where this batch of data is generated to + extraMetadata.put(GENERATED_DATA_PATH, generatedDataStats.map(s -> s.getFilePath()).collect().get(0)); + } + writeClient.commit(instantTime.get(), records, Option.of(extraMetadata)); + } + } + + public void commitCompaction(JavaRDD records, JavaRDD generatedDataStats, + Option instantTime) throws IOException { + if (!cfg.useDeltaStreamer) { + Map extraMetadata = new HashMap<>(); + /** Store the checkpoint in the commit metadata just like + * {@link HoodieDeltaStreamer#commit(SparkRDDWriteClient, JavaRDD, Option)} **/ + extraMetadata.put(HoodieDeltaStreamerWrapper.CHECKPOINT_KEY, lastCheckpoint.get()); + if (generatedDataStats != null && generatedDataStats.count() > 1) { + // Just stores the path where this batch of data is generated to + extraMetadata.put(GENERATED_DATA_PATH, generatedDataStats.map(s -> s.getFilePath()).collect().get(0)); + } + HoodieSparkTable table = HoodieSparkTable.create(writeClient.getConfig(), writeClient.getEngineContext()); + HoodieCommitMetadata metadata = CompactHelpers.getInstance().createCompactionMetadata(table, instantTime.get(), HoodieJavaRDD.of(records), writeClient.getConfig().getSchema()); + writeClient.commitCompaction(instantTime.get(), metadata, Option.of(extraMetadata)); + } + } +} diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieTestSuiteJob.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieTestSuiteJob.java index 2d9f841ae351c..5e2f9812ba529 100644 --- a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieTestSuiteJob.java +++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieTestSuiteJob.java @@ -190,11 +190,12 @@ public WorkflowDag createWorkflowDag() throws IOException { } public void runTestSuite() { + WriterContext writerContext = null; try { WorkflowDag workflowDag = createWorkflowDag(); log.info("Workflow Dag => " + DagUtils.convertDagToYaml(workflowDag)); long startTime = System.currentTimeMillis(); - WriterContext writerContext = new WriterContext(jsc, props, cfg, keyGenerator, sparkSession); + writerContext = new WriterContext(jsc, props, cfg, keyGenerator, sparkSession); writerContext.initContext(jsc); startOtherServicesIfNeeded(writerContext); if (this.cfg.saferSchemaEvolution) { @@ -217,6 +218,9 @@ public void runTestSuite() { log.error("Failed to run Test Suite ", e); throw new HoodieException("Failed to run Test Suite ", e); } finally { + if (writerContext != null) { + writerContext.shutdownResources(); + } if (stopJsc) { stopQuietly(); } @@ -310,5 +314,8 @@ public static class HoodieTestSuiteConfig extends HoodieDeltaStreamer.Config { @Parameter(names = {"--use-hudi-data-to-generate-updates"}, description = "Use data from hudi to generate updates for new batches ") public Boolean useHudiToGenerateUpdates = false; + + @Parameter(names = {"--test-continuous-mode"}, description = "Tests continuous mode in deltastreamer.") + public Boolean testContinousMode = false; } } diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieTestSuiteWriter.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieTestSuiteWriter.java index a98c7f2aec3f0..7a9e122e86c15 100644 --- a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieTestSuiteWriter.java +++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieTestSuiteWriter.java @@ -18,37 +18,25 @@ package org.apache.hudi.integ.testsuite; -import org.apache.hudi.avro.model.HoodieCompactionPlan; import org.apache.hudi.client.HoodieReadClient; import org.apache.hudi.client.SparkRDDWriteClient; import org.apache.hudi.client.WriteStatus; import org.apache.hudi.client.common.HoodieSparkEngineContext; -import org.apache.hudi.common.model.HoodieAvroRecord; -import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; -import org.apache.hudi.common.model.WriteOperationType; -import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.config.HoodieCompactionConfig; import org.apache.hudi.config.HoodieIndexConfig; import org.apache.hudi.config.HoodiePayloadConfig; import org.apache.hudi.config.HoodieWriteConfig; -import org.apache.hudi.data.HoodieJavaRDD; import org.apache.hudi.index.HoodieIndex; -import org.apache.hudi.integ.testsuite.HoodieTestSuiteJob.HoodieTestSuiteConfig; import org.apache.hudi.integ.testsuite.dag.nodes.CleanNode; import org.apache.hudi.integ.testsuite.dag.nodes.DagNode; import org.apache.hudi.integ.testsuite.dag.nodes.RollbackNode; import org.apache.hudi.integ.testsuite.dag.nodes.ScheduleCompactNode; import org.apache.hudi.integ.testsuite.writer.DeltaWriteStats; -import org.apache.hudi.table.HoodieSparkTable; -import org.apache.hudi.table.action.HoodieWriteMetadata; -import org.apache.hudi.table.action.compact.CompactHelpers; import org.apache.hudi.utilities.schema.SchemaProvider; -import org.apache.avro.Schema; import org.apache.avro.generic.GenericRecord; import org.apache.hadoop.conf.Configuration; import org.apache.spark.api.java.JavaRDD; @@ -57,38 +45,31 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.IOException; import java.io.Serializable; import java.util.Arrays; -import java.util.HashMap; import java.util.HashSet; import java.util.Map; import java.util.Properties; import java.util.Set; -/** - * A writer abstraction for the Hudi test suite. This class wraps different implementations of writers used to perform write operations into the target hudi dataset. Current supported writers are - * {@link HoodieDeltaStreamerWrapper} and {@link SparkRDDWriteClient}. - */ -public class HoodieTestSuiteWriter implements Serializable { +public abstract class HoodieTestSuiteWriter implements Serializable { private static Logger log = LoggerFactory.getLogger(HoodieTestSuiteWriter.class); - private HoodieDeltaStreamerWrapper deltaStreamerWrapper; - private HoodieWriteConfig writeConfig; - private SparkRDDWriteClient writeClient; - protected HoodieTestSuiteConfig cfg; - private Option lastCheckpoint; - private HoodieReadClient hoodieReadClient; - private Properties props; - private String schema; - private transient Configuration configuration; - private transient JavaSparkContext sparkContext; - private static Set VALID_DAG_NODES_TO_ALLOW_WRITE_CLIENT_IN_DELTASTREAMER_MODE = new HashSet<>( + protected HoodieDeltaStreamerWrapper deltaStreamerWrapper; + protected HoodieWriteConfig writeConfig; + protected SparkRDDWriteClient writeClient; + protected HoodieTestSuiteJob.HoodieTestSuiteConfig cfg; + protected Option lastCheckpoint; + protected HoodieReadClient hoodieReadClient; + protected Properties props; + protected String schema; + protected transient Configuration configuration; + protected transient JavaSparkContext sparkContext; + protected static Set VALID_DAG_NODES_TO_ALLOW_WRITE_CLIENT_IN_DELTASTREAMER_MODE = new HashSet<>( Arrays.asList(RollbackNode.class.getName(), CleanNode.class.getName(), ScheduleCompactNode.class.getName())); - private static final String GENERATED_DATA_PATH = "generated.data.path"; - public HoodieTestSuiteWriter(JavaSparkContext jsc, Properties props, HoodieTestSuiteConfig cfg, String schema) throws Exception { + public HoodieTestSuiteWriter(JavaSparkContext jsc, Properties props, HoodieTestSuiteJob.HoodieTestSuiteConfig cfg, String schema) throws Exception { // We ensure that only 1 instance of HoodieWriteClient is instantiated for a HoodieTestSuiteWriter // This does not instantiate a HoodieWriteClient until a // {@link HoodieDeltaStreamer#commit(HoodieWriteClient, JavaRDD, Option)} is invoked. @@ -110,7 +91,7 @@ public HoodieWriteConfig getWriteConfig() { return this.writeConfig; } - private HoodieWriteConfig getHoodieClientConfig(HoodieTestSuiteConfig cfg, Properties props, String schema) { + private HoodieWriteConfig getHoodieClientConfig(HoodieTestSuiteJob.HoodieTestSuiteConfig cfg, Properties props, String schema) { HoodieWriteConfig.Builder builder = HoodieWriteConfig.newBuilder().combineInput(true, true).withPath(cfg.targetBasePath) .withAutoCommit(false) @@ -131,159 +112,35 @@ private boolean allowWriteClientAccess(DagNode dagNode) { return false; } - public RDD getNextBatch() throws Exception { - Pair>> nextBatch = fetchSource(); - lastCheckpoint = Option.of(nextBatch.getValue().getLeft()); - JavaRDD inputRDD = nextBatch.getRight().getRight(); - return inputRDD.map(r -> (GenericRecord) ((HoodieAvroRecord) r).getData() - .getInsertValue(new Schema.Parser().parse(schema)).get()).rdd(); - } + public abstract void shutdownResources(); - public void getNextBatchForDeletes() throws Exception { - Pair>> nextBatch = fetchSource(); - lastCheckpoint = Option.of(nextBatch.getValue().getLeft()); - JavaRDD inputRDD = nextBatch.getRight().getRight(); - inputRDD.collect(); - } + public abstract RDD getNextBatch() throws Exception; - public Pair>> fetchSource() throws Exception { - return this.deltaStreamerWrapper.fetchSource(); - } + public abstract Pair>> fetchSource() throws Exception ; - public Option startCommit() { - if (cfg.useDeltaStreamer) { - return Option.of(HoodieActiveTimeline.createNewInstantTime()); - } else { - return Option.of(writeClient.startCommit()); - } - } + public abstract Option startCommit(); - public JavaRDD upsert(Option instantTime) throws Exception { - if (cfg.useDeltaStreamer) { - return deltaStreamerWrapper.upsert(WriteOperationType.UPSERT); - } else { - Pair>> nextBatch = fetchSource(); - lastCheckpoint = Option.of(nextBatch.getValue().getLeft()); - return writeClient.upsert(nextBatch.getRight().getRight(), instantTime.get()); - } - } + public abstract JavaRDD upsert(Option instantTime) throws Exception; - public JavaRDD insert(Option instantTime) throws Exception { - if (cfg.useDeltaStreamer) { - return deltaStreamerWrapper.insert(); - } else { - Pair>> nextBatch = fetchSource(); - lastCheckpoint = Option.of(nextBatch.getValue().getLeft()); - return writeClient.insert(nextBatch.getRight().getRight(), instantTime.get()); - } - } + public abstract JavaRDD insert(Option instantTime) throws Exception; - public JavaRDD insertOverwrite(Option instantTime) throws Exception { - if (cfg.useDeltaStreamer) { - return deltaStreamerWrapper.insertOverwrite(); - } else { - Pair>> nextBatch = fetchSource(); - lastCheckpoint = Option.of(nextBatch.getValue().getLeft()); - return writeClient.insertOverwrite(nextBatch.getRight().getRight(), instantTime.get()).getWriteStatuses(); - } - } + public abstract JavaRDD insertOverwrite(Option instantTime) throws Exception; - public JavaRDD insertOverwriteTable(Option instantTime) throws Exception { - if (cfg.useDeltaStreamer) { - return deltaStreamerWrapper.insertOverwriteTable(); - } else { - Pair>> nextBatch = fetchSource(); - lastCheckpoint = Option.of(nextBatch.getValue().getLeft()); - return writeClient.insertOverwriteTable(nextBatch.getRight().getRight(), instantTime.get()).getWriteStatuses(); - } - } + public abstract JavaRDD insertOverwriteTable(Option instantTime) throws Exception; - public JavaRDD bulkInsert(Option instantTime) throws Exception { - if (cfg.useDeltaStreamer) { - return deltaStreamerWrapper.bulkInsert(); - } else { - Pair>> nextBatch = fetchSource(); - lastCheckpoint = Option.of(nextBatch.getValue().getLeft()); - return writeClient.bulkInsert(nextBatch.getRight().getRight(), instantTime.get()); - } - } + public abstract JavaRDD bulkInsert(Option instantTime) throws Exception; - public JavaRDD compact(Option instantTime) throws Exception { - if (cfg.useDeltaStreamer) { - return deltaStreamerWrapper.compact(); - } else { - if (!instantTime.isPresent()) { - Option> compactionPlanPair = Option - .fromJavaOptional(hoodieReadClient.getPendingCompactions() - .stream().findFirst()); - if (compactionPlanPair.isPresent()) { - instantTime = Option.of(compactionPlanPair.get().getLeft()); - } - } - if (instantTime.isPresent()) { - HoodieWriteMetadata> compactionMetadata = writeClient.compact(instantTime.get()); - return compactionMetadata.getWriteStatuses(); - } else { - return null; - } - } - } + public abstract JavaRDD compact(Option instantTime) throws Exception; - public void inlineClustering() { - if (!cfg.useDeltaStreamer) { - Option clusteringInstantOpt = writeClient.scheduleClustering(Option.empty()); - clusteringInstantOpt.ifPresent(clusteringInstant -> { - // inline cluster should auto commit as the user is never given control - log.warn("Clustering instant :: " + clusteringInstant); - writeClient.cluster(clusteringInstant, true); - }); - } else { - // TODO: fix clustering to be done async https://issues.apache.org/jira/browse/HUDI-1590 - throw new IllegalArgumentException("Clustering cannot be triggered with deltastreamer"); - } - } + public abstract void inlineClustering() throws Exception ; - public Option scheduleCompaction(Option> previousCommitExtraMetadata) throws - Exception { - if (cfg.useDeltaStreamer) { - deltaStreamerWrapper.scheduleCompact(); - return Option.empty(); - } else { - return writeClient.scheduleCompaction(previousCommitExtraMetadata); - } - } + public abstract Option scheduleCompaction(Option> previousCommitExtraMetadata) throws Exception; - public void commit(JavaRDD records, JavaRDD generatedDataStats, - Option instantTime) { - if (!cfg.useDeltaStreamer) { - Map extraMetadata = new HashMap<>(); - /** Store the checkpoint in the commit metadata just like - * {@link HoodieDeltaStreamer#commit(SparkRDDWriteClient, JavaRDD, Option)} **/ - extraMetadata.put(HoodieDeltaStreamerWrapper.CHECKPOINT_KEY, lastCheckpoint.get()); - if (generatedDataStats != null && generatedDataStats.count() > 1) { - // Just stores the path where this batch of data is generated to - extraMetadata.put(GENERATED_DATA_PATH, generatedDataStats.map(s -> s.getFilePath()).collect().get(0)); - } - writeClient.commit(instantTime.get(), records, Option.of(extraMetadata)); - } - } + public abstract void commit(JavaRDD records, JavaRDD generatedDataStats, + Option instantTime); - public void commitCompaction(JavaRDD records, JavaRDD generatedDataStats, - Option instantTime) throws IOException { - if (!cfg.useDeltaStreamer) { - Map extraMetadata = new HashMap<>(); - /** Store the checkpoint in the commit metadata just like - * {@link HoodieDeltaStreamer#commit(SparkRDDWriteClient, JavaRDD, Option)} **/ - extraMetadata.put(HoodieDeltaStreamerWrapper.CHECKPOINT_KEY, lastCheckpoint.get()); - if (generatedDataStats != null && generatedDataStats.count() > 1) { - // Just stores the path where this batch of data is generated to - extraMetadata.put(GENERATED_DATA_PATH, generatedDataStats.map(s -> s.getFilePath()).collect().get(0)); - } - HoodieSparkTable table = HoodieSparkTable.create(writeClient.getConfig(), writeClient.getEngineContext()); - HoodieCommitMetadata metadata = CompactHelpers.getInstance().createCompactionMetadata(table, instantTime.get(), HoodieJavaRDD.of(records), writeClient.getConfig().getSchema()); - writeClient.commitCompaction(instantTime.get(), metadata, Option.of(extraMetadata)); - } - } + public abstract void commitCompaction(JavaRDD records, JavaRDD generatedDataStats, + Option instantTime) throws Exception; public SparkRDDWriteClient getWriteClient(DagNode dagNode) throws IllegalAccessException { if (cfg.useDeltaStreamer & !allowWriteClientAccess(dagNode)) { @@ -301,7 +158,7 @@ public HoodieDeltaStreamerWrapper getDeltaStreamerWrapper() { return deltaStreamerWrapper; } - public HoodieTestSuiteConfig getCfg() { + public HoodieTestSuiteJob.HoodieTestSuiteConfig getCfg() { return cfg; } @@ -325,3 +182,4 @@ public String getSchema() { return schema; } } + diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/configuration/DeltaConfig.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/configuration/DeltaConfig.java index 1578e86be47b6..a781d19cb78c5 100644 --- a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/configuration/DeltaConfig.java +++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/configuration/DeltaConfig.java @@ -103,6 +103,7 @@ public static class Config { private static String DELETE_INPUT_DATA_EXCEPT_LATEST = "delete_input_data_except_latest"; private static String PARTITIONS_TO_DELETE = "partitions_to_delete"; private static String INPUT_PARTITIONS_TO_SKIP_VALIDATE = "input_partitions_to_skip_validate"; + private static String MAX_WAIT_TIME_FOR_DELTASTREAMER_TO_CATCH_UP_MS = "max_wait_time_for_deltastreamer_catch_up_ms"; // Spark SQL Create Table private static String TABLE_TYPE = "table_type"; @@ -253,6 +254,10 @@ public boolean enableRowWriting() { return Boolean.valueOf(configsMap.getOrDefault(ENABLE_ROW_WRITING, false).toString()); } + public long maxWaitTimeForDeltastreamerToCatchupMs() { + return Long.valueOf(configsMap.getOrDefault(MAX_WAIT_TIME_FOR_DELTASTREAMER_TO_CATCH_UP_MS, 5 * 60 * 1000).toString()); + } + public Option getTableType() { return !configsMap.containsKey(TABLE_TYPE) ? Option.empty() : Option.of(configsMap.get(TABLE_TYPE).toString()); diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/WriterContext.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/WriterContext.java index d31ef195ecdd5..83b5751c8646b 100644 --- a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/WriterContext.java +++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/WriterContext.java @@ -21,7 +21,9 @@ import org.apache.hudi.common.config.SerializableConfiguration; import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.exception.HoodieException; +import org.apache.hudi.integ.testsuite.HoodieContinousTestSuiteWriter; import org.apache.hudi.integ.testsuite.HoodieTestSuiteJob.HoodieTestSuiteConfig; +import org.apache.hudi.integ.testsuite.HoodieInlineTestSuiteWriter; import org.apache.hudi.integ.testsuite.HoodieTestSuiteWriter; import org.apache.hudi.integ.testsuite.configuration.DFSDeltaConfig; import org.apache.hudi.integ.testsuite.generator.DeltaGenerator; @@ -37,6 +39,8 @@ import org.apache.spark.sql.SparkSession; import java.util.Map; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; /** * WriterContext wraps the delta writer/data generator related configuration needed to init/reinit. @@ -53,6 +57,7 @@ public class WriterContext { private BuiltinKeyGenerator keyGenerator; private transient SparkSession sparkSession; private transient JavaSparkContext jsc; + private ExecutorService executorService; public WriterContext(JavaSparkContext jsc, TypedProperties props, HoodieTestSuiteConfig cfg, BuiltinKeyGenerator keyGenerator, SparkSession sparkSession) { @@ -67,7 +72,8 @@ public void initContext(JavaSparkContext jsc) throws HoodieException { try { this.schemaProvider = UtilHelpers.createSchemaProvider(cfg.schemaProviderClassName, props, jsc); String schemaStr = schemaProvider.getSourceSchema().toString(); - this.hoodieTestSuiteWriter = new HoodieTestSuiteWriter(jsc, props, cfg, schemaStr); + this.hoodieTestSuiteWriter = (cfg.testContinousMode && cfg.useDeltaStreamer) ? new HoodieContinousTestSuiteWriter(jsc, props, cfg, schemaStr) + : new HoodieInlineTestSuiteWriter(jsc, props, cfg, schemaStr); int inputParallelism = cfg.inputParallelism > 0 ? cfg.inputParallelism : jsc.defaultParallelism(); this.deltaGenerator = new DeltaGenerator( new DFSDeltaConfig(DeltaOutputMode.valueOf(cfg.outputTypeName), DeltaInputType.valueOf(cfg.inputFormatName), @@ -75,6 +81,10 @@ public void initContext(JavaSparkContext jsc) throws HoodieException { schemaStr, cfg.limitFileSize, inputParallelism, cfg.deleteOldInput, cfg.useHudiToGenerateUpdates), jsc, sparkSession, schemaStr, keyGenerator); log.info(String.format("Initialized writerContext with: %s", schemaStr)); + if (cfg.testContinousMode) { + executorService = Executors.newFixedThreadPool(1); + executorService.execute(new TestSuiteWriterRunnable(hoodieTestSuiteWriter)); + } } catch (Exception e) { throw new HoodieException("Failed to reinitialize writerContext", e); } @@ -113,4 +123,35 @@ public String toString() { public SparkSession getSparkSession() { return sparkSession; } + + public void shutdownResources() { + this.hoodieTestSuiteWriter.shutdownResources(); + if (executorService != null) { + executorService.shutdownNow(); + } + } + + /** + * TestSuiteWriterRunnable to spin up a thread to execute deltastreamer with async table services. + */ + class TestSuiteWriterRunnable implements Runnable { + private HoodieTestSuiteWriter hoodieTestSuiteWriter; + + TestSuiteWriterRunnable(HoodieTestSuiteWriter hoodieTestSuiteWriter) { + this.hoodieTestSuiteWriter = hoodieTestSuiteWriter; + } + + @Override + public void run() { + try { + Thread.sleep(20000); + log.info("Starting continuous sync with deltastreamer "); + hoodieTestSuiteWriter.getDeltaStreamerWrapper().sync(); + log.info("Completed continuous sync with deltastreamer "); + } catch (Exception e) { + log.error("Deltastreamer failed in continuous mode " + e.getMessage()); + throw new HoodieException("Shutting down deltastreamer in continuous mode failed ", e); + } + } + } } diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/BaseValidateDatasetNode.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/BaseValidateDatasetNode.java index a0ebdc5754716..15c209e4752b8 100644 --- a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/BaseValidateDatasetNode.java +++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/BaseValidateDatasetNode.java @@ -20,10 +20,17 @@ package org.apache.hudi.integ.testsuite.dag.nodes; import org.apache.hudi.DataSourceWriteOptions; +import org.apache.hudi.common.model.HoodieCommitMetadata; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.StringUtils; +import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.integ.testsuite.configuration.DeltaConfig; import org.apache.hudi.integ.testsuite.dag.ExecutionContext; import org.apache.hudi.integ.testsuite.schema.SchemaUtils; +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -40,6 +47,9 @@ import org.apache.spark.sql.types.StructType; import org.slf4j.Logger; +import java.io.IOException; +import java.util.Arrays; +import java.util.Comparator; import java.util.List; import java.util.stream.Collectors; @@ -47,6 +57,8 @@ import scala.collection.JavaConversions; import scala.collection.JavaConverters; +import static org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer.CHECKPOINT_KEY; + /** * This nodes validates contents from input path are in tact with Hudi. By default no configs are required for this node. But there is an * optional config "delete_input_data" that you can set for this node. If set, once validation completes, contents from inputPath are deleted. This will come in handy for long running test suites. @@ -78,6 +90,12 @@ public void execute(ExecutionContext context, int curItrCount) throws Exception int itrCountToExecute = config.getIterationCountToExecute(); if ((itrCountToExecute != -1 && itrCountToExecute == curItrCount) || (itrCountToExecute == -1 && ((curItrCount % validateOnceEveryItr) == 0))) { + FileSystem fs = new Path(context.getHoodieTestSuiteWriter().getCfg().inputBasePath) + .getFileSystem(context.getHoodieTestSuiteWriter().getConfiguration()); + if (context.getHoodieTestSuiteWriter().getCfg().testContinousMode) { + awaitUntilDeltaStreamerCaughtUp(context, context.getHoodieTestSuiteWriter().getCfg().targetBasePath, fs, + context.getHoodieTestSuiteWriter().getCfg().inputBasePath); + } SparkSession session = SparkSession.builder().sparkContext(context.getJsc().sc()).getOrCreate(); // todo: Fix partitioning schemes. For now, assumes data based partitioning. String inputPath = context.getHoodieTestSuiteWriter().getCfg().inputBasePath + "/*/*"; @@ -85,8 +103,6 @@ public void execute(ExecutionContext context, int curItrCount) throws Exception // listing batches to be validated String inputPathStr = context.getHoodieTestSuiteWriter().getCfg().inputBasePath; if (log.isDebugEnabled()) { - FileSystem fs = new Path(inputPathStr) - .getFileSystem(context.getHoodieTestSuiteWriter().getConfiguration()); FileStatus[] fileStatuses = fs.listStatus(new Path(inputPathStr)); log.info("fileStatuses length: " + fileStatuses.length); for (FileStatus fileStatus : fileStatuses) { @@ -145,8 +161,6 @@ public void execute(ExecutionContext context, int curItrCount) throws Exception if (config.isDeleteInputData()) { // clean up input data for current group of writes. inputPathStr = context.getHoodieTestSuiteWriter().getCfg().inputBasePath; - FileSystem fs = new Path(inputPathStr) - .getFileSystem(context.getHoodieTestSuiteWriter().getConfiguration()); FileStatus[] fileStatuses = fs.listStatus(new Path(inputPathStr)); for (FileStatus fileStatus : fileStatuses) { log.debug("Micro batch to be deleted " + fileStatus.getPath().toString()); @@ -157,6 +171,50 @@ public void execute(ExecutionContext context, int curItrCount) throws Exception } } + private void awaitUntilDeltaStreamerCaughtUp(ExecutionContext context, String hudiTablePath, FileSystem fs, String inputPath) throws IOException, InterruptedException { + HoodieTableMetaClient meta = HoodieTableMetaClient.builder().setConf(new Configuration(fs.getConf())).setBasePath(hudiTablePath).build(); + HoodieTimeline commitTimeline = meta.getActiveTimeline().getCommitsTimeline().filterCompletedInstants(); + Option latestCheckpoint = getLatestCheckpoint(commitTimeline); + FileStatus[] subDirs = fs.listStatus(new Path(inputPath)); + List subDirList = Arrays.asList(subDirs); + subDirList.sort(Comparator.comparingLong(entry -> Long.parseLong(entry.getPath().getName()))); + String latestSubDir = subDirList.get(subDirList.size() -1).getPath().getName(); + log.info("Latest sub directory in input path " + latestSubDir + ", latest checkpoint from deltastreamer " + + (latestCheckpoint.isPresent() ? latestCheckpoint.get() : "none")); + long maxWaitTime = config.maxWaitTimeForDeltastreamerToCatchupMs(); + long waitedSoFar = 0; + while (!(latestCheckpoint.isPresent() && latestCheckpoint.get().equals(latestSubDir))) { + log.warn("Sleeping for 20 secs awaiting for deltastreamer to catch up with ingested data"); + Thread.sleep(20000); + meta.reloadActiveTimeline(); + commitTimeline = meta.getActiveTimeline().getCommitsTimeline().filterCompletedInstants(); + latestCheckpoint = getLatestCheckpoint(commitTimeline); + waitedSoFar += 20000; + if (waitedSoFar >= maxWaitTime) { + throw new AssertionError("DeltaStreamer has not caught up after 5 mins of wait time. Last known checkpoint " + + (latestCheckpoint.isPresent() ? latestCheckpoint.get() : "none") + ", expected checkpoint to have caugth up " + latestSubDir); + } + log.info("Latest sub directory in input path " + latestSubDir + ", latest checkpoint from deltastreamer " + + (latestCheckpoint.isPresent() ? latestCheckpoint.get() : "none")); + } + } + + private Option getLatestCheckpoint(HoodieTimeline timeline) { + return (Option) timeline.getReverseOrderedInstants().map(instant -> { + try { + HoodieCommitMetadata commitMetadata = HoodieCommitMetadata + .fromBytes(timeline.getInstantDetails(instant).get(), HoodieCommitMetadata.class); + if (!StringUtils.isNullOrEmpty(commitMetadata.getMetadata(CHECKPOINT_KEY))) { + return Option.of(commitMetadata.getMetadata(CHECKPOINT_KEY)); + } else { + return Option.empty(); + } + } catch (IOException e) { + throw new HoodieIOException("Failed to parse HoodieCommitMetadata for " + instant.toString(), e); + } + }).filter(Option::isPresent).findFirst().orElse(Option.empty()); + } + private Dataset getInputDf(ExecutionContext context, SparkSession session, String inputPath) { String recordKeyField = context.getWriterContext().getProps().getString(DataSourceWriteOptions.RECORDKEY_FIELD().key()); String partitionPathField = context.getWriterContext().getProps().getString(DataSourceWriteOptions.PARTITIONPATH_FIELD().key()); diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/generator/DeltaGenerator.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/generator/DeltaGenerator.java index c30be2a2a5d2c..20e12e9030854 100644 --- a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/generator/DeltaGenerator.java +++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/generator/DeltaGenerator.java @@ -216,15 +216,22 @@ public JavaRDD generateDeletes(Config config) throws IOException adjustedRDD = deltaInputReader.read(config.getNumRecordsDelete()); adjustedRDD = adjustRDDToGenerateExactNumUpdates(adjustedRDD, jsc, config.getNumRecordsDelete()); } else { - deltaInputReader = - new DFSHoodieDatasetInputReader(jsc, ((DFSDeltaConfig) deltaOutputConfig).getDatasetOutputPath(), - schemaStr); - if (config.getFractionUpsertPerFile() > 0) { - adjustedRDD = deltaInputReader.read(config.getNumDeletePartitions(), config.getNumUpsertFiles(), - config.getFractionUpsertPerFile()); + if (((DFSDeltaConfig) deltaOutputConfig).shouldUseHudiToGenerateUpdates()) { + deltaInputReader = + new DFSHoodieDatasetInputReader(jsc, ((DFSDeltaConfig) deltaOutputConfig).getDatasetOutputPath(), + schemaStr); + if (config.getFractionUpsertPerFile() > 0) { + adjustedRDD = deltaInputReader.read(config.getNumDeletePartitions(), config.getNumUpsertFiles(), + config.getFractionUpsertPerFile()); + } else { + adjustedRDD = deltaInputReader.read(config.getNumDeletePartitions(), config.getNumUpsertFiles(), config + .getNumRecordsDelete()); + } } else { - adjustedRDD = deltaInputReader.read(config.getNumDeletePartitions(), config.getNumUpsertFiles(), config - .getNumRecordsDelete()); + deltaInputReader = new DFSAvroDeltaInputReader(sparkSession, schemaStr, + ((DFSDeltaConfig) deltaOutputConfig).getDeltaBasePath(), Option.empty(), Option.empty()); + adjustedRDD = deltaInputReader.read(config.getNumRecordsDelete()); + adjustedRDD = adjustRDDToGenerateExactNumUpdates(adjustedRDD, jsc, config.getNumRecordsDelete()); } } From 676d5cefe0294f495db04ec6476afdc00cbf0dd2 Mon Sep 17 00:00:00 2001 From: Danny Chan Date: Tue, 24 May 2022 13:07:55 +0800 Subject: [PATCH 09/44] [HUDI-4138] Fix the concurrency modification of hoodie table config for flink (#5660) * Remove the metadata cleaning strategy for flink, that means the multi-modal index may be affected * Improve the HoodieTable#clearMetadataTablePartitionsConfig to only update table config when necessary * Remove the modification of read code path in HoodieTableConfig --- .../org/apache/hudi/table/HoodieTable.java | 14 ++++++-------- .../hudi/client/HoodieFlinkWriteClient.java | 4 +--- .../apache/hudi/table/HoodieFlinkTable.java | 4 ---- .../hudi/common/table/HoodieTableConfig.java | 19 ++++++------------- 4 files changed, 13 insertions(+), 28 deletions(-) diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTable.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTable.java index 807865dae2416..56526d23db006 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTable.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTable.java @@ -885,24 +885,22 @@ private boolean shouldExecuteMetadataTableDeletion() { // partitions are ready to use return !HoodieTableMetadata.isMetadataTable(metaClient.getBasePath()) && !config.isMetadataTableEnabled() - && (!metaClient.getTableConfig().contains(TABLE_METADATA_PARTITIONS) - || !metaClient.getTableConfig().getMetadataPartitions().isEmpty()); + && !metaClient.getTableConfig().getMetadataPartitions().isEmpty(); } /** * Clears hoodie.table.metadata.partitions in hoodie.properties */ private void clearMetadataTablePartitionsConfig(Option partitionType, boolean clearAll) { - if (clearAll) { + Set partitions = getCompletedMetadataPartitions(metaClient.getTableConfig()); + if (clearAll && partitions.size() > 0) { LOG.info("Clear hoodie.table.metadata.partitions in hoodie.properties"); metaClient.getTableConfig().setValue(TABLE_METADATA_PARTITIONS.key(), EMPTY_STRING); HoodieTableConfig.update(metaClient.getFs(), new Path(metaClient.getMetaPath()), metaClient.getTableConfig().getProps()); - return; + } else if (partitions.remove(partitionType.get().getPartitionPath())) { + metaClient.getTableConfig().setValue(HoodieTableConfig.TABLE_METADATA_PARTITIONS.key(), String.join(",", partitions)); + HoodieTableConfig.update(metaClient.getFs(), new Path(metaClient.getMetaPath()), metaClient.getTableConfig().getProps()); } - Set completedPartitions = getCompletedMetadataPartitions(metaClient.getTableConfig()); - completedPartitions.remove(partitionType.get().getPartitionPath()); - metaClient.getTableConfig().setValue(HoodieTableConfig.TABLE_METADATA_PARTITIONS.key(), String.join(",", completedPartitions)); - HoodieTableConfig.update(metaClient.getFs(), new Path(metaClient.getMetaPath()), metaClient.getTableConfig().getProps()); } public HoodieTableMetadata getMetadataTable() { diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/HoodieFlinkWriteClient.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/HoodieFlinkWriteClient.java index 2d23c3afb7f14..49fa2ec246cf9 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/HoodieFlinkWriteClient.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/HoodieFlinkWriteClient.java @@ -53,7 +53,6 @@ import org.apache.hudi.io.MiniBatchHandle; import org.apache.hudi.metadata.FlinkHoodieBackedTableMetadataWriter; import org.apache.hudi.metadata.HoodieBackedTableMetadataWriter; -import org.apache.hudi.metadata.HoodieTableMetadataWriter; import org.apache.hudi.table.BulkInsertPartitioner; import org.apache.hudi.table.HoodieFlinkTable; import org.apache.hudi.table.HoodieTable; @@ -365,8 +364,7 @@ public void completeCompaction( // commit to data table after committing to metadata table. // Do not do any conflict resolution here as we do with regular writes. We take the lock here to ensure all writes to metadata table happens within a // single lock (single writer). Because more than one write to metadata table will result in conflicts since all of them updates the same partition. - table.getMetadataWriter(compactionInstant.getTimestamp()).ifPresent( - w -> ((HoodieTableMetadataWriter) w).update(metadata, compactionInstant.getTimestamp(), table.isTableServiceAction(compactionInstant.getAction()))); + writeTableMetadata(table, compactionCommitTime, compactionInstant.getAction(), metadata); LOG.info("Committing Compaction {} finished with result {}.", compactionCommitTime, metadata); CompactHelpers.getInstance().completeInflightCompaction(table, compactionCommitTime, metadata); } finally { diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkTable.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkTable.java index f1e43b9d30d42..6eae15e7e1aff 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkTable.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkTable.java @@ -105,13 +105,9 @@ protected HoodieIndex getIndex(HoodieWriteConfig config, HoodieEngineContext con public Option getMetadataWriter(String triggeringInstantTimestamp, Option actionMetadata) { if (config.isMetadataTableEnabled()) { - // even with metadata enabled, some index could have been disabled - // delete metadata partitions corresponding to such indexes - deleteMetadataIndexIfNecessary(); return Option.of(FlinkHoodieBackedTableMetadataWriter.create(context.getHadoopConf().get(), config, context, actionMetadata, Option.of(triggeringInstantTimestamp))); } else { - maybeDeleteMetadataTable(); return Option.empty(); } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableConfig.java b/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableConfig.java index edc6caa5bcbdf..886911466b95f 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableConfig.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableConfig.java @@ -272,8 +272,8 @@ private static Properties getOrderedPropertiesWithTableChecksum(Properties props * @throws IOException */ private static String storeProperties(Properties props, FSDataOutputStream outputStream) throws IOException { - String checksum; - if (props.containsKey(TABLE_CHECKSUM.key()) && validateChecksum(props)) { + final String checksum; + if (isValidChecksum(props)) { checksum = props.getProperty(TABLE_CHECKSUM.key()); props.store(outputStream, "Updated at " + Instant.now()); } else { @@ -285,8 +285,8 @@ private static String storeProperties(Properties props, FSDataOutputStream outpu return checksum; } - private boolean isValidChecksum() { - return contains(TABLE_CHECKSUM) && validateChecksum(props); + private static boolean isValidChecksum(Properties props) { + return props.containsKey(TABLE_CHECKSUM.key()) && validateChecksum(props); } /** @@ -298,20 +298,13 @@ public HoodieTableConfig() { private void fetchConfigs(FileSystem fs, String metaPath) throws IOException { Path cfgPath = new Path(metaPath, HOODIE_PROPERTIES_FILE); - Path backupCfgPath = new Path(metaPath, HOODIE_PROPERTIES_FILE_BACKUP); try (FSDataInputStream is = fs.open(cfgPath)) { props.load(is); - // validate checksum for latest table version - if (getTableVersion().versionCode() >= HoodieTableVersion.FOUR.versionCode() && !isValidChecksum()) { - LOG.warn("Checksum validation failed. Falling back to backed up configs."); - try (FSDataInputStream fsDataInputStream = fs.open(backupCfgPath)) { - props.load(fsDataInputStream); - } - } } catch (IOException ioe) { if (!fs.exists(cfgPath)) { LOG.warn("Run `table recover-configs` if config update/delete failed midway. Falling back to backed up configs."); // try the backup. this way no query ever fails if update fails midway. + Path backupCfgPath = new Path(metaPath, HOODIE_PROPERTIES_FILE_BACKUP); try (FSDataInputStream is = fs.open(backupCfgPath)) { props.load(is); } @@ -631,7 +624,7 @@ public List getMetadataPartitions() { CONFIG_VALUES_DELIMITER ); } - + /** * Returns the format to use for partition meta files. */ From c05ebf2417f4b170c396dfb60c738ca55bd52309 Mon Sep 17 00:00:00 2001 From: Sivabalan Narayanan Date: Tue, 24 May 2022 03:33:21 -0400 Subject: [PATCH 10/44] [HUDI-2473] Fixing compaction write operation in commit metadata (#5203) --- .../hudi/table/action/compact/RunCompactionActionExecutor.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/RunCompactionActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/RunCompactionActionExecutor.java index 5c184e77dfaa2..fc4ae986e6d55 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/RunCompactionActionExecutor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/RunCompactionActionExecutor.java @@ -27,6 +27,7 @@ import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.HoodieWriteStat; +import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.util.CompactionUtils; import org.apache.hudi.common.util.Option; @@ -99,6 +100,7 @@ public HoodieWriteMetadata> execute() { metadata.addMetadata(SerDeHelper.LATEST_SCHEMA, schemaPair.getLeft().get()); metadata.addMetadata(HoodieCommitMetadata.SCHEMA_KEY, schemaPair.getRight().get()); } + metadata.setOperationType(WriteOperationType.COMPACT); compactionMetadata.setWriteStatuses(statuses); compactionMetadata.setCommitted(false); compactionMetadata.setCommitMetadata(Option.of(metadata)); From eb219010d2491165dbdc00a6b9547a9072e012ac Mon Sep 17 00:00:00 2001 From: Danny Chan Date: Tue, 24 May 2022 17:33:30 +0800 Subject: [PATCH 11/44] [HUDI-4145] Archives the metadata file in HoodieInstant.State sequence (#5669) --- .../apache/hudi/client/HoodieTimelineArchiver.java | 11 ++++++++--- 1 file changed, 8 insertions(+), 3 deletions(-) diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/HoodieTimelineArchiver.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/HoodieTimelineArchiver.java index 2974cc2ef6d6f..f111bb70ef007 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/HoodieTimelineArchiver.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/HoodieTimelineArchiver.java @@ -72,7 +72,6 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; -import java.util.Collections; import java.util.Comparator; import java.util.HashMap; import java.util.List; @@ -507,10 +506,16 @@ private Stream getInstantsToArchive() { List instantsToStream = groupByTsAction.get(Pair.of(hoodieInstant.getTimestamp(), HoodieInstant.getComparableAction(hoodieInstant.getAction()))); if (instantsToStream != null) { - return instantsToStream.stream(); + // sorts the instants in natural order to make sure the metadata files be removed + // in HoodieInstant.State sequence: requested -> inflight -> completed, + // this is important because when a COMPLETED metadata file is removed first, + // other monitors on the timeline(such as the compaction or clustering services) would + // mistakenly recognize the pending file as a pending operation, + // then all kinds of weird bugs occur. + return instantsToStream.stream().sorted(); } else { // if a concurrent writer archived the instant - return Collections.EMPTY_LIST.stream(); + return Stream.empty(); } }); } From 0caa55ecb42cdecbc027521535200540a2467832 Mon Sep 17 00:00:00 2001 From: liujinhui <965147871@qq.com> Date: Tue, 24 May 2022 18:56:28 +0800 Subject: [PATCH 12/44] [HUDI-4135] remove netty and netty-all (#5663) --- hudi-spark-datasource/hudi-spark2/pom.xml | 13 ------------- 1 file changed, 13 deletions(-) diff --git a/hudi-spark-datasource/hudi-spark2/pom.xml b/hudi-spark-datasource/hudi-spark2/pom.xml index 679579ae9a5e6..3d7f61c290f46 100644 --- a/hudi-spark-datasource/hudi-spark2/pom.xml +++ b/hudi-spark-datasource/hudi-spark2/pom.xml @@ -203,19 +203,6 @@ true - - io.netty - netty - 3.9.9.Final - true - - - io.netty - netty-all - 4.1.17.Final - true - - org.apache.hudi From c20db99a7b79b9545e6fae0d762250ceb55a8b79 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=96=BB=E5=85=86=E9=9D=96?= Date: Sat, 21 May 2022 21:25:15 +0800 Subject: [PATCH 13/44] [HUDI-2207] Support independent flink hudi clustering function --- .../hudi/client/BaseHoodieWriteClient.java | 2 +- .../hudi/config/HoodieClusteringConfig.java | 2 + .../hudi/table/BulkInsertPartitioner.java | 16 +- .../strategy/ClusteringPlanStrategy.java | 11 + .../hudi/client/HoodieFlinkWriteClient.java | 68 ++++ ...FlinkRecentDaysClusteringPlanStrategy.java | 65 ++++ ...ectedPartitionsClusteringPlanStrategy.java | 67 ++++ .../FlinkSizeBasedClusteringPlanStrategy.java | 129 +++++++ .../table/HoodieFlinkCopyOnWriteTable.java | 3 +- .../JavaCustomColumnsSortPartitioner.java | 2 +- .../bulkinsert/JavaGlobalSortPartitioner.java | 2 +- .../hudi/configuration/FlinkOptions.java | 67 ++++ .../hudi/sink/bulk/sort/SortOperatorGen.java | 2 +- .../clustering/ClusteringCommitEvent.java | 77 +++++ .../sink/clustering/ClusteringCommitSink.java | 174 ++++++++++ .../sink/clustering/ClusteringOperator.java | 318 ++++++++++++++++++ .../sink/clustering/ClusteringPlanEvent.java | 73 ++++ .../ClusteringPlanSourceFunction.java | 91 +++++ .../clustering/FlinkClusteringConfig.java | 148 ++++++++ .../clustering/HoodieFlinkClusteringJob.java | 191 +++++++++++ .../hudi/streamer/FlinkStreamerConfig.java | 2 +- .../org/apache/hudi/util/StreamerUtil.java | 20 +- .../cluster/ITTestHoodieFlinkClustering.java | 184 ++++++++++ 23 files changed, 1700 insertions(+), 14 deletions(-) create mode 100644 hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/clustering/plan/strategy/FlinkRecentDaysClusteringPlanStrategy.java create mode 100644 hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/clustering/plan/strategy/FlinkSelectedPartitionsClusteringPlanStrategy.java create mode 100644 hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/clustering/plan/strategy/FlinkSizeBasedClusteringPlanStrategy.java create mode 100644 hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/ClusteringCommitEvent.java create mode 100644 hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/ClusteringCommitSink.java create mode 100644 hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/ClusteringOperator.java create mode 100644 hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/ClusteringPlanEvent.java create mode 100644 hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/ClusteringPlanSourceFunction.java create mode 100644 hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/FlinkClusteringConfig.java create mode 100644 hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/HoodieFlinkClusteringJob.java create mode 100644 hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/cluster/ITTestHoodieFlinkClustering.java diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieWriteClient.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieWriteClient.java index 270027df18053..251ff97799ffa 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieWriteClient.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieWriteClient.java @@ -1379,7 +1379,7 @@ protected Option inlineScheduleClustering(Option> ex return scheduleClustering(extraMetadata); } - protected void rollbackInflightClustering(HoodieInstant inflightInstant, HoodieTable table) { + public void rollbackInflightClustering(HoodieInstant inflightInstant, HoodieTable table) { Option pendingRollbackInstantInfo = getPendingRollbackInfo(table.getMetaClient(), inflightInstant.getTimestamp(), false); String commitTime = pendingRollbackInstantInfo.map(entry -> entry.getRollbackInstant().getTimestamp()).orElse(HoodieActiveTimeline.createNewInstantTime()); table.scheduleRollback(context, commitTime, inflightInstant, false, config.shouldRollbackUsingMarkers()); 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 eee6f4f4927e0..1180845a6ed8a 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 @@ -51,6 +51,8 @@ public class HoodieClusteringConfig extends HoodieConfig { public static final String CLUSTERING_STRATEGY_PARAM_PREFIX = "hoodie.clustering.plan.strategy."; 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 = + "org.apache.hudi.client.clustering.plan.strategy.FlinkSizeBasedClusteringPlanStrategy"; public static final String JAVA_SIZED_BASED_CLUSTERING_PLAN_STRATEGY = "org.apache.hudi.client.clustering.plan.strategy.JavaSizeBasedClusteringPlanStrategy"; public static final String SPARK_SORT_AND_SIZE_EXECUTION_STRATEGY = diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/BulkInsertPartitioner.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/BulkInsertPartitioner.java index 63b502531a896..89360c247403d 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/BulkInsertPartitioner.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/BulkInsertPartitioner.java @@ -25,20 +25,20 @@ import java.io.Serializable; /** - * Repartition input records into at least expected number of output spark partitions. It should give below guarantees - - * Output spark partition will have records from only one hoodie partition. - Average records per output spark - * partitions should be almost equal to (#inputRecords / #outputSparkPartitions) to avoid possible skews. + * Repartition input records into at least expected number of output partitions. It should give below guarantees - + * Output partition will have records from only one hoodie partition. - Average records per output + * partitions should be almost equal to (#inputRecords / #outputPartitions) to avoid possible skews. */ public interface BulkInsertPartitioner extends Serializable { /** - * Repartitions the input records into at least expected number of output spark partitions. + * Repartitions the input records into at least expected number of output partitions. * - * @param records Input Hoodie records - * @param outputSparkPartitions Expected number of output partitions + * @param records Input Hoodie records + * @param outputPartitions Expected number of output partitions * @return */ - I repartitionRecords(I records, int outputSparkPartitions); + I repartitionRecords(I records, int outputPartitions); /** * @return {@code true} if the records within a partition are sorted; {@code false} otherwise. @@ -48,6 +48,7 @@ public interface BulkInsertPartitioner extends Serializable { /** * Return file group id prefix for the given data partition. * By defauult, return a new file group id prefix, so that incoming records will route to a fresh new file group + * * @param partitionId data partition * @return */ @@ -57,6 +58,7 @@ default String getFileIdPfx(int partitionId) { /** * Return write handle factory for the given partition. + * * @param partitionId data partition * @return */ 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 479f63932c5b3..a96ff73947cdb 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 @@ -70,6 +70,9 @@ public static String checkAndGetClusteringPlanStrategy(HoodieWriteConfig config) String sparkSizeBasedClassName = HoodieClusteringConfig.SPARK_SIZED_BASED_CLUSTERING_PLAN_STRATEGY; String sparkSelectedPartitionsClassName = "org.apache.hudi.client.clustering.plan.strategy.SparkSelectedPartitionsClusteringPlanStrategy"; String sparkRecentDaysClassName = "org.apache.hudi.client.clustering.plan.strategy.SparkRecentDaysClusteringPlanStrategy"; + String flinkSizeBasedClassName = HoodieClusteringConfig.FLINK_SIZED_BASED_CLUSTERING_PLAN_STRATEGY; + String flinkSelectedPartitionsClassName = "org.apache.hudi.client.clustering.plan.strategy.FlinkSelectedPartitionsClusteringPlanStrategy"; + String flinkRecentDaysClassName = "org.apache.hudi.client.clustering.plan.strategy.FlinkRecentDaysClusteringPlanStrategy"; String javaSelectedPartitionClassName = "org.apache.hudi.client.clustering.plan.strategy.JavaRecentDaysClusteringPlanStrategy"; String javaSizeBasedClassName = HoodieClusteringConfig.JAVA_SIZED_BASED_CLUSTERING_PLAN_STRATEGY; @@ -82,6 +85,14 @@ public static String checkAndGetClusteringPlanStrategy(HoodieWriteConfig config) config.setValue(HoodieClusteringConfig.PLAN_PARTITION_FILTER_MODE_NAME, ClusteringPlanPartitionFilterMode.SELECTED_PARTITIONS.name()); LOG.warn(String.format(logStr, className, sparkSizeBasedClassName, HoodieClusteringConfig.PLAN_PARTITION_FILTER_MODE_NAME.key(), ClusteringPlanPartitionFilterMode.SELECTED_PARTITIONS.name())); return sparkSizeBasedClassName; + } else if (flinkRecentDaysClassName.equals(className)) { + config.setValue(HoodieClusteringConfig.PLAN_PARTITION_FILTER_MODE_NAME, ClusteringPlanPartitionFilterMode.RECENT_DAYS.name()); + LOG.warn(String.format(logStr, className, sparkSizeBasedClassName, HoodieClusteringConfig.PLAN_PARTITION_FILTER_MODE_NAME.key(), ClusteringPlanPartitionFilterMode.RECENT_DAYS.name())); + return flinkSizeBasedClassName; + } else if (flinkSelectedPartitionsClassName.equals(className)) { + config.setValue(HoodieClusteringConfig.PLAN_PARTITION_FILTER_MODE_NAME, ClusteringPlanPartitionFilterMode.SELECTED_PARTITIONS.name()); + LOG.warn(String.format(logStr, className, sparkSizeBasedClassName, HoodieClusteringConfig.PLAN_PARTITION_FILTER_MODE_NAME.key(), ClusteringPlanPartitionFilterMode.SELECTED_PARTITIONS.name())); + return flinkSizeBasedClassName; } else if (javaSelectedPartitionClassName.equals(className)) { config.setValue(HoodieClusteringConfig.PLAN_PARTITION_FILTER_MODE_NAME, ClusteringPlanPartitionFilterMode.RECENT_DAYS.name()); LOG.warn(String.format(logStr, className, javaSizeBasedClassName, HoodieClusteringConfig.PLAN_PARTITION_FILTER_MODE_NAME.key(), ClusteringPlanPartitionFilterMode.SELECTED_PARTITIONS.name())); diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/HoodieFlinkWriteClient.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/HoodieFlinkWriteClient.java index 49fa2ec246cf9..ddfbabaf36ae9 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/HoodieFlinkWriteClient.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/HoodieFlinkWriteClient.java @@ -29,8 +29,10 @@ 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.model.HoodieReplaceCommitMetadata; import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.model.HoodieWriteStat; +import org.apache.hudi.common.model.TableServiceType; import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.HoodieTableVersion; @@ -39,6 +41,7 @@ import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.exception.HoodieClusteringException; import org.apache.hudi.exception.HoodieCommitException; import org.apache.hudi.exception.HoodieNotSupportedException; import org.apache.hudi.index.FlinkHoodieIndexFactory; @@ -68,6 +71,8 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.io.IOException; +import java.nio.charset.StandardCharsets; import java.text.ParseException; import java.util.HashMap; import java.util.Iterator; @@ -399,6 +404,52 @@ public HoodieWriteMetadata> cluster(final String clusteringIns throw new HoodieNotSupportedException("Clustering is not supported yet"); } + private void completeClustering( + HoodieReplaceCommitMetadata metadata, + HoodieTable>, List, List> table, + String clusteringCommitTime) { + this.context.setJobStatus(this.getClass().getSimpleName(), "Collect clustering write status and commit clustering"); + HoodieInstant clusteringInstant = new HoodieInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.REPLACE_COMMIT_ACTION, clusteringCommitTime); + List writeStats = metadata.getPartitionToWriteStats().entrySet().stream().flatMap(e -> + e.getValue().stream()).collect(Collectors.toList()); + if (writeStats.stream().mapToLong(HoodieWriteStat::getTotalWriteErrors).sum() > 0) { + throw new HoodieClusteringException("Clustering failed to write to files:" + + writeStats.stream().filter(s -> s.getTotalWriteErrors() > 0L).map(HoodieWriteStat::getFileId).collect(Collectors.joining(","))); + } + + try { + this.txnManager.beginTransaction(Option.of(clusteringInstant), Option.empty()); + finalizeWrite(table, clusteringCommitTime, writeStats); + // commit to data table after committing to metadata table. + // Do not do any conflict resolution here as we do with regular writes. We take the lock here to ensure all writes to metadata table happens within a + // single lock (single writer). Because more than one write to metadata table will result in conflicts since all of them updates the same partition. + writeTableMetadata(table, clusteringCommitTime, clusteringInstant.getAction(), metadata); + LOG.info("Committing Clustering {} finished with result {}.", clusteringCommitTime, metadata); + table.getActiveTimeline().transitionReplaceInflightToComplete( + HoodieTimeline.getReplaceCommitInflightInstant(clusteringCommitTime), + Option.of(metadata.toJsonString().getBytes(StandardCharsets.UTF_8))); + } catch (IOException e) { + throw new HoodieClusteringException( + "Failed to commit " + table.getMetaClient().getBasePath() + " at time " + clusteringCommitTime, e); + } finally { + this.txnManager.endTransaction(Option.of(clusteringInstant)); + } + + WriteMarkersFactory.get(config.getMarkersType(), table, clusteringCommitTime) + .quietDeleteMarkerDir(context, config.getMarkersDeleteParallelism()); + if (clusteringTimer != null) { + long durationInMs = metrics.getDurationInMs(clusteringTimer.stop()); + try { + metrics.updateCommitMetrics(HoodieActiveTimeline.parseDateFromInstantTime(clusteringCommitTime).getTime(), + durationInMs, metadata, HoodieActiveTimeline.REPLACE_COMMIT_ACTION); + } catch (ParseException e) { + throw new HoodieCommitException("Commit time is not of valid format. Failed to commit compaction " + + config.getBasePath() + " at time " + clusteringCommitTime, e); + } + } + LOG.info("Clustering successfully on commit " + clusteringCommitTime); + } + @Override protected HoodieTable doInitTable(HoodieTableMetaClient metaClient, Option instantTime, boolean initialMetadataTableIfNecessary) { // Create a Hoodie table which encapsulated the commits and files visible @@ -412,6 +463,23 @@ protected void tryUpgrade(HoodieTableMetaClient metaClient, Option insta // no need to execute the upgrade/downgrade on each write in streaming. } + public void completeTableService( + TableServiceType tableServiceType, + HoodieCommitMetadata metadata, + HoodieTable>, List, List> table, + String commitInstant) { + switch (tableServiceType) { + case CLUSTER: + completeClustering((HoodieReplaceCommitMetadata) metadata, table, commitInstant); + break; + case COMPACT: + completeCompaction(metadata, table, commitInstant); + break; + default: + throw new IllegalArgumentException("This table service is not valid " + tableServiceType); + } + } + /** * Upgrade downgrade the Hoodie table. * diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/clustering/plan/strategy/FlinkRecentDaysClusteringPlanStrategy.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/clustering/plan/strategy/FlinkRecentDaysClusteringPlanStrategy.java new file mode 100644 index 0000000000000..0109aaa60ffb9 --- /dev/null +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/clustering/plan/strategy/FlinkRecentDaysClusteringPlanStrategy.java @@ -0,0 +1,65 @@ +/* + * 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.hudi.client.common.HoodieFlinkEngineContext; +import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.table.HoodieFlinkCopyOnWriteTable; +import org.apache.hudi.table.HoodieFlinkMergeOnReadTable; + +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; + +import java.util.Comparator; +import java.util.List; +import java.util.stream.Collectors; + +/** + * Clustering Strategy based on following. + * 1) Only looks at latest 'daybased.lookback.partitions' partitions. + * 2) Excludes files that are greater than 'small.file.limit' from clustering plan. + */ +public class FlinkRecentDaysClusteringPlanStrategy> + extends FlinkSizeBasedClusteringPlanStrategy { + private static final Logger LOG = LogManager.getLogger(FlinkRecentDaysClusteringPlanStrategy.class); + + public FlinkRecentDaysClusteringPlanStrategy(HoodieFlinkCopyOnWriteTable table, + HoodieFlinkEngineContext engineContext, + HoodieWriteConfig writeConfig) { + super(table, engineContext, writeConfig); + } + + public FlinkRecentDaysClusteringPlanStrategy(HoodieFlinkMergeOnReadTable table, + HoodieFlinkEngineContext engineContext, + HoodieWriteConfig writeConfig) { + super(table, engineContext, writeConfig); + } + + @Override + protected List filterPartitionPaths(List partitionPaths) { + int targetPartitionsForClustering = getWriteConfig().getTargetPartitionsForClustering(); + int skipPartitionsFromLatestForClustering = getWriteConfig().getSkipPartitionsFromLatestForClustering(); + return partitionPaths.stream() + .sorted(Comparator.reverseOrder()) + .skip(Math.max(skipPartitionsFromLatestForClustering, 0)) + .limit(targetPartitionsForClustering > 0 ? targetPartitionsForClustering : partitionPaths.size()) + .collect(Collectors.toList()); + } +} diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/clustering/plan/strategy/FlinkSelectedPartitionsClusteringPlanStrategy.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/clustering/plan/strategy/FlinkSelectedPartitionsClusteringPlanStrategy.java new file mode 100644 index 0000000000000..ae5726bb4a46e --- /dev/null +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/clustering/plan/strategy/FlinkSelectedPartitionsClusteringPlanStrategy.java @@ -0,0 +1,67 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.client.clustering.plan.strategy; + +import org.apache.hudi.client.common.HoodieFlinkEngineContext; +import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.table.HoodieFlinkCopyOnWriteTable; +import org.apache.hudi.table.HoodieFlinkMergeOnReadTable; + +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; + +import java.util.List; +import java.util.stream.Collectors; + +import static org.apache.hudi.config.HoodieClusteringConfig.CLUSTERING_STRATEGY_PARAM_PREFIX; + +/** + * Clustering Strategy to filter just specified partitions from [begin, end]. Note both begin and end are inclusive. + */ +public class FlinkSelectedPartitionsClusteringPlanStrategy> + extends FlinkSizeBasedClusteringPlanStrategy { + private static final Logger LOG = LogManager.getLogger(FlinkSelectedPartitionsClusteringPlanStrategy.class); + + public static final String CONF_BEGIN_PARTITION = CLUSTERING_STRATEGY_PARAM_PREFIX + "cluster.begin.partition"; + public static final String CONF_END_PARTITION = CLUSTERING_STRATEGY_PARAM_PREFIX + "cluster.end.partition"; + + public FlinkSelectedPartitionsClusteringPlanStrategy(HoodieFlinkCopyOnWriteTable table, + HoodieFlinkEngineContext engineContext, + HoodieWriteConfig writeConfig) { + super(table, engineContext, writeConfig); + } + + public FlinkSelectedPartitionsClusteringPlanStrategy(HoodieFlinkMergeOnReadTable table, + HoodieFlinkEngineContext engineContext, + HoodieWriteConfig writeConfig) { + super(table, engineContext, writeConfig); + } + + @Override + protected List filterPartitionPaths(List partitionPaths) { + String beginPartition = getWriteConfig().getProps().getProperty(CONF_BEGIN_PARTITION); + String endPartition = getWriteConfig().getProps().getProperty(CONF_END_PARTITION); + List filteredPartitions = partitionPaths.stream() + .filter(path -> path.compareTo(beginPartition) >= 0 && path.compareTo(endPartition) <= 0) + .collect(Collectors.toList()); + LOG.info("Filtered to the following partitions: " + filteredPartitions); + return filteredPartitions; + } +} diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/clustering/plan/strategy/FlinkSizeBasedClusteringPlanStrategy.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/clustering/plan/strategy/FlinkSizeBasedClusteringPlanStrategy.java new file mode 100644 index 0000000000000..8347da6014af8 --- /dev/null +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/clustering/plan/strategy/FlinkSizeBasedClusteringPlanStrategy.java @@ -0,0 +1,129 @@ +/* + * 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.hudi.avro.model.HoodieClusteringGroup; +import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.client.common.HoodieFlinkEngineContext; +import org.apache.hudi.common.model.FileSlice; +import org.apache.hudi.common.model.HoodieBaseFile; +import org.apache.hudi.common.model.HoodieKey; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecordPayload; +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.HoodieFlinkCopyOnWriteTable; +import org.apache.hudi.table.HoodieFlinkMergeOnReadTable; +import org.apache.hudi.table.action.cluster.strategy.PartitionAwareClusteringPlanStrategy; + +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.stream.Stream; + +import static org.apache.hudi.config.HoodieClusteringConfig.PLAN_STRATEGY_SORT_COLUMNS; + +/** + * Clustering Strategy based on following. + * 1) Creates clustering groups based on max size allowed per group. + * 2) Excludes files that are greater than 'small.file.limit' from clustering plan. + */ +public class FlinkSizeBasedClusteringPlanStrategy> + extends PartitionAwareClusteringPlanStrategy>, List, List> { + private static final Logger LOG = LogManager.getLogger(FlinkSizeBasedClusteringPlanStrategy.class); + + public FlinkSizeBasedClusteringPlanStrategy(HoodieFlinkCopyOnWriteTable table, + HoodieFlinkEngineContext engineContext, + HoodieWriteConfig writeConfig) { + super(table, engineContext, writeConfig); + } + + public FlinkSizeBasedClusteringPlanStrategy(HoodieFlinkMergeOnReadTable table, + HoodieFlinkEngineContext engineContext, + HoodieWriteConfig writeConfig) { + super(table, engineContext, writeConfig); + } + + @Override + protected Stream buildClusteringGroupsForPartition(String partitionPath, List fileSlices) { + HoodieWriteConfig writeConfig = getWriteConfig(); + + List, Integer>> fileSliceGroups = new ArrayList<>(); + List currentGroup = new ArrayList<>(); + long totalSizeSoFar = 0; + + for (FileSlice currentSlice : fileSlices) { + // check if max size is reached and create new group, if needed. + // in now, every clustering group out put is 1 file group. + if (totalSizeSoFar >= writeConfig.getClusteringTargetFileMaxBytes() && !currentGroup.isEmpty()) { + LOG.info("Adding one clustering group " + totalSizeSoFar + " max bytes: " + + writeConfig.getClusteringMaxBytesInGroup() + " num input slices: " + currentGroup.size()); + fileSliceGroups.add(Pair.of(currentGroup, 1)); + currentGroup = new ArrayList<>(); + totalSizeSoFar = 0; + } + + // Add to the current file-group + currentGroup.add(currentSlice); + // assume each file group size is ~= parquet.max.file.size + totalSizeSoFar += currentSlice.getBaseFile().isPresent() ? currentSlice.getBaseFile().get().getFileSize() : writeConfig.getParquetMaxFileSize(); + } + + if (!currentGroup.isEmpty()) { + fileSliceGroups.add(Pair.of(currentGroup, 1)); + } + + return fileSliceGroups.stream().map(fileSliceGroup -> + HoodieClusteringGroup.newBuilder() + .setSlices(getFileSliceInfo(fileSliceGroup.getLeft())) + .setNumOutputFileGroups(fileSliceGroup.getRight()) + .setMetrics(buildMetrics(fileSliceGroup.getLeft())) + .build()); + } + + @Override + protected Map getStrategyParams() { + Map params = new HashMap<>(); + if (!StringUtils.isNullOrEmpty(getWriteConfig().getClusteringSortColumns())) { + params.put(PLAN_STRATEGY_SORT_COLUMNS.key(), getWriteConfig().getClusteringSortColumns()); + } + return params; + } + + @Override + protected List filterPartitionPaths(List partitionPaths) { + return partitionPaths; + } + + @Override + protected Stream getFileSlicesEligibleForClustering(final String partition) { + return super.getFileSlicesEligibleForClustering(partition) + // Only files that have basefile size smaller than small file size are eligible. + .filter(slice -> slice.getBaseFile().map(HoodieBaseFile::getFileSize).orElse(0L) < getWriteConfig().getClusteringSmallFileLimit()); + } + + private int getNumberOfOutputFileGroups(long groupSize, long targetFileSize) { + return (int) Math.ceil(groupSize / (double) targetFileSize); + } +} diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkCopyOnWriteTable.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkCopyOnWriteTable.java index 9ab633f9e3b37..0e5f1c26e32f4 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkCopyOnWriteTable.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkCopyOnWriteTable.java @@ -55,6 +55,7 @@ import org.apache.hudi.table.action.bootstrap.HoodieBootstrapWriteMetadata; import org.apache.hudi.table.action.clean.CleanActionExecutor; import org.apache.hudi.table.action.clean.CleanPlanActionExecutor; +import org.apache.hudi.table.action.cluster.ClusteringPlanActionExecutor; import org.apache.hudi.table.action.commit.FlinkDeleteCommitActionExecutor; import org.apache.hudi.table.action.commit.FlinkInsertCommitActionExecutor; import org.apache.hudi.table.action.commit.FlinkInsertOverwriteCommitActionExecutor; @@ -286,7 +287,7 @@ public HoodieWriteMetadata> compact( @Override public Option scheduleClustering(final HoodieEngineContext context, final String instantTime, final Option> extraMetadata) { - throw new HoodieNotSupportedException("Clustering is not supported on a Flink CopyOnWrite table"); + return new ClusteringPlanActionExecutor<>(context, config,this, instantTime, extraMetadata).execute(); } @Override diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/execution/bulkinsert/JavaCustomColumnsSortPartitioner.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/execution/bulkinsert/JavaCustomColumnsSortPartitioner.java index eb3d4ef312e99..b9e466485f209 100644 --- a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/execution/bulkinsert/JavaCustomColumnsSortPartitioner.java +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/execution/bulkinsert/JavaCustomColumnsSortPartitioner.java @@ -49,7 +49,7 @@ public JavaCustomColumnsSortPartitioner(String[] columnNames, Schema schema, boo @Override public List> repartitionRecords( - List> records, int outputSparkPartitions) { + List> records, int outputPartitions) { return records.stream().sorted((o1, o2) -> { Object values1 = HoodieAvroUtils.getRecordColumnValues(o1, sortColumnNames, schema, consistentLogicalTimestampEnabled); Object values2 = HoodieAvroUtils.getRecordColumnValues(o2, sortColumnNames, schema, consistentLogicalTimestampEnabled); diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/execution/bulkinsert/JavaGlobalSortPartitioner.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/execution/bulkinsert/JavaGlobalSortPartitioner.java index fded0ffab51bd..d272849a19f28 100644 --- a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/execution/bulkinsert/JavaGlobalSortPartitioner.java +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/execution/bulkinsert/JavaGlobalSortPartitioner.java @@ -37,7 +37,7 @@ public class JavaGlobalSortPartitioner @Override public List> repartitionRecords(List> records, - int outputSparkPartitions) { + int outputPartitions) { // Now, sort the records and line them up nicely for loading. records.sort(new Comparator() { @Override diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/configuration/FlinkOptions.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/configuration/FlinkOptions.java index 729f0147b5940..3de4bd4f757b8 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/configuration/FlinkOptions.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/configuration/FlinkOptions.java @@ -18,6 +18,7 @@ package org.apache.hudi.configuration; +import org.apache.hudi.client.clustering.plan.strategy.FlinkRecentDaysClusteringPlanStrategy; import org.apache.hudi.common.config.ConfigClassProperty; import org.apache.hudi.common.config.ConfigGroups; import org.apache.hudi.common.config.HoodieConfig; @@ -583,6 +584,72 @@ private FlinkOptions() { .defaultValue(40)// default min 40 commits .withDescription("Min number of commits to keep before archiving older commits into a sequential log, default 40"); + // ------------------------------------------------------------------------ + // Clustering Options + // ------------------------------------------------------------------------ + + public static final ConfigOption CLUSTERING_SCHEDULE_ENABLED = ConfigOptions + .key("clustering.schedule.enabled") + .booleanType() + .defaultValue(false) // default false for pipeline + .withDescription("Schedule the cluster plan, default false"); + + public static final ConfigOption CLUSTERING_DELTA_COMMITS = ConfigOptions + .key("clustering.delta_commits") + .intType() + .defaultValue(4) + .withDescription("Max delta commits needed to trigger clustering, default 4 commits"); + + public static final ConfigOption CLUSTERING_TASKS = ConfigOptions + .key("clustering.tasks") + .intType() + .defaultValue(4) + .withDescription("Parallelism of tasks that do actual clustering, default is 4"); + + public static final ConfigOption CLUSTERING_TARGET_PARTITIONS = ConfigOptions + .key("clustering.plan.strategy.daybased.lookback.partitions") + .intType() + .defaultValue(2) + .withDescription("Number of partitions to list to create ClusteringPlan, default is 2"); + + public static final ConfigOption CLUSTERING_PLAN_STRATEGY_CLASS = ConfigOptions + .key("clustering.plan.strategy.class") + .stringType() + .defaultValue(FlinkRecentDaysClusteringPlanStrategy.class.getName()) + .withDescription("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 last N (determined by " + + CLUSTERING_TARGET_PARTITIONS.key() + ") day based partitions picks the small file slices within those partitions."); + + public static final ConfigOption CLUSTERING_PLAN_STRATEGY_TARGET_FILE_MAX_BYTES = ConfigOptions + .key("clustering.plan.strategy.target.file.max.bytes") + .intType() + .defaultValue(1024 * 1024 * 1024) // default 1 GB + .withDescription("Each group can produce 'N' (CLUSTERING_MAX_GROUP_SIZE/CLUSTERING_TARGET_FILE_SIZE) output file groups, default 1 GB"); + + public static final ConfigOption CLUSTERING_PLAN_STRATEGY_SMALL_FILE_LIMIT = ConfigOptions + .key("clustering.plan.strategy.small.file.limit") + .intType() + .defaultValue(600) // default 600 MB + .withDescription("Files smaller than the size specified here are candidates for clustering, default 600 MB"); + + public static final ConfigOption CLUSTERING_PLAN_STRATEGY_SKIP_PARTITIONS_FROM_LATEST = ConfigOptions + .key("clustering.plan.strategy.daybased.skipfromlatest.partitions") + .intType() + .defaultValue(0) + .withDescription("Number of partitions to skip from latest when choosing partitions to create ClusteringPlan"); + + public static final ConfigOption CLUSTERING_SORT_COLUMNS = ConfigOptions + .key("clustering.plan.strategy.sort.columns") + .stringType() + .noDefaultValue() + .withDescription("Columns to sort the data by when clustering"); + + public static final ConfigOption CLUSTERING_MAX_NUM_GROUPS = ConfigOptions + .key("clustering.plan.strategy.max.num.groups") + .intType() + .defaultValue(30) + .withDescription("Maximum number of groups to create as part of ClusteringPlan. Increasing groups will increase parallelism, default is 30"); + // ------------------------------------------------------------------------ // Hive Sync Options // ------------------------------------------------------------------------ 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 4d3fc08efe197..b5599886a9d0b 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 @@ -48,7 +48,7 @@ public OneInputStreamOperator createSortOperator() { codeGen.generateRecordComparator("SortComparator")); } - private SortCodeGenerator createSortCodeGenerator() { + public SortCodeGenerator createSortCodeGenerator() { SortSpec.SortSpecBuilder builder = SortSpec.builder(); IntStream.range(0, sortIndices.length).forEach(i -> builder.addField(i, true, true)); return new SortCodeGenerator(tableConfig, rowType, builder.build()); diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/ClusteringCommitEvent.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/ClusteringCommitEvent.java new file mode 100644 index 0000000000000..30a8fbed3fafd --- /dev/null +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/ClusteringCommitEvent.java @@ -0,0 +1,77 @@ +/* + * 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.client.WriteStatus; + +import java.io.Serializable; +import java.util.List; + +/** + * Represents a commit event from the clustering task {@link ClusteringFunction}. + */ +public class ClusteringCommitEvent implements Serializable { + private static final long serialVersionUID = 1L; + + /** + * The clustering commit instant time. + */ + private String instant; + /** + * The write statuses. + */ + private List writeStatuses; + /** + * The clustering task identifier. + */ + private int taskID; + + public ClusteringCommitEvent() { + } + + public ClusteringCommitEvent(String instant, List writeStatuses, int taskID) { + this.instant = instant; + this.writeStatuses = writeStatuses; + this.taskID = taskID; + } + + public void setInstant(String instant) { + this.instant = instant; + } + + public void setWriteStatuses(List writeStatuses) { + this.writeStatuses = writeStatuses; + } + + public void setTaskID(int taskID) { + this.taskID = taskID; + } + + public String getInstant() { + return instant; + } + + public List getWriteStatuses() { + return writeStatuses; + } + + public int getTaskID() { + return taskID; + } +} 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 new file mode 100644 index 0000000000000..bc87270a49f1b --- /dev/null +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/ClusteringCommitSink.java @@ -0,0 +1,174 @@ +/* + * 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.HoodieClusteringGroup; +import org.apache.hudi.avro.model.HoodieClusteringPlan; +import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.common.model.HoodieCommitMetadata; +import org.apache.hudi.common.model.HoodieFileGroupId; +import org.apache.hudi.common.model.TableServiceType; +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.ClusteringUtils; +import org.apache.hudi.common.util.CommitUtils; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.exception.HoodieClusteringException; +import org.apache.hudi.sink.CleanFunction; +import org.apache.hudi.table.HoodieFlinkTable; +import org.apache.hudi.table.action.HoodieWriteMetadata; +import org.apache.hudi.util.StreamerUtil; + +import org.apache.flink.configuration.Configuration; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; + +/** + * Function to check and commit the clustering action. + * + *

Each time after receiving a clustering commit event {@link ClusteringCommitEvent}, + * it loads and checks the clustering plan {@link org.apache.hudi.avro.model.HoodieClusteringPlan}, + * if all the clustering operations {@link org.apache.hudi.common.model.ClusteringOperation} + * of the plan are finished, tries to commit the clustering action. + * + *

It also inherits the {@link CleanFunction} cleaning ability. This is needed because + * the SQL API does not allow multiple sinks in one table sink provider. + */ +public class ClusteringCommitSink extends CleanFunction { + private static final Logger LOG = LoggerFactory.getLogger(ClusteringCommitSink.class); + + /** + * Config options. + */ + private final Configuration conf; + + private transient HoodieFlinkTable table; + + /** + * Buffer to collect the event from each clustering task {@code ClusteringFunction}. + * The key is the instant time. + */ + private transient Map> commitBuffer; + + public ClusteringCommitSink(Configuration conf) { + super(conf); + this.conf = conf; + } + + @Override + public void open(Configuration parameters) throws Exception { + super.open(parameters); + if (writeClient == null) { + this.writeClient = StreamerUtil.createWriteClient(conf, getRuntimeContext()); + } + this.commitBuffer = new HashMap<>(); + this.table = writeClient.getHoodieTable(); + } + + @Override + public void invoke(ClusteringCommitEvent event, Context context) throws Exception { + final String instant = event.getInstant(); + commitBuffer.computeIfAbsent(instant, k -> new ArrayList<>()) + .add(event); + commitIfNecessary(instant, commitBuffer.get(instant)); + } + + /** + * Condition to commit: the commit buffer has equal size with the clustering plan operations + * and all the clustering commit event {@link ClusteringCommitEvent} has the same clustering instant time. + * + * @param instant Clustering commit instant time + * @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(); + boolean isReady = clusteringPlan.getInputGroups().size() == events.size(); + if (!isReady) { + return; + } + List statuses = events.stream() + .map(ClusteringCommitEvent::getWriteStatuses) + .flatMap(Collection::stream) + .collect(Collectors.toList()); + + HoodieWriteMetadata> writeMetadata = new HoodieWriteMetadata<>(); + writeMetadata.setWriteStatuses(statuses); + writeMetadata.setWriteStats(statuses.stream().map(WriteStatus::getStat).collect(Collectors.toList())); + writeMetadata.setPartitionToReplaceFileIds(getPartitionToReplacedFileIds(clusteringPlan, writeMetadata)); + validateWriteResult(clusteringPlan, instant, writeMetadata); + if (!writeMetadata.getCommitMetadata().isPresent()) { + HoodieCommitMetadata commitMetadata = CommitUtils.buildMetadata( + writeMetadata.getWriteStats().get(), + writeMetadata.getPartitionToReplaceFileIds(), + Option.empty(), + WriteOperationType.CLUSTER, + this.writeClient.getConfig().getSchema(), + HoodieTimeline.REPLACE_COMMIT_ACTION); + writeMetadata.setCommitMetadata(Option.of(commitMetadata)); + } + // commit the clustering + this.table.getMetaClient().reloadActiveTimeline(); + this.writeClient.completeTableService( + TableServiceType.CLUSTER, writeMetadata.getCommitMetadata().get(), table, instant); + + // reset the status + reset(instant); + } + + private void reset(String instant) { + this.commitBuffer.remove(instant); + } + + /** + * Validate actions taken by clustering. In the first implementation, we validate at least one new file is written. + * But we can extend this to add more validation. E.g. number of records read = number of records written etc. + * We can also make these validations in BaseCommitActionExecutor to reuse pre-commit hooks for multiple actions. + */ + private static void validateWriteResult(HoodieClusteringPlan clusteringPlan, String instantTime, HoodieWriteMetadata> writeMetadata) { + if (writeMetadata.getWriteStatuses().isEmpty()) { + throw new HoodieClusteringException("Clustering plan produced 0 WriteStatus for " + instantTime + + " #groups: " + clusteringPlan.getInputGroups().size() + " expected at least " + + clusteringPlan.getInputGroups().stream().mapToInt(HoodieClusteringGroup::getNumOutputFileGroups).sum() + + " write statuses"); + } + } + + private static Map> getPartitionToReplacedFileIds( + HoodieClusteringPlan clusteringPlan, + HoodieWriteMetadata> writeMetadata) { + Set newFilesWritten = writeMetadata.getWriteStats().get().stream() + .map(s -> new HoodieFileGroupId(s.getPartitionPath(), s.getFileId())).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/ClusteringOperator.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/ClusteringOperator.java new file mode 100644 index 0000000000000..a415ac9d46165 --- /dev/null +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/ClusteringOperator.java @@ -0,0 +1,318 @@ +/* + * 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.HoodieAvroUtils; +import org.apache.hudi.client.FlinkTaskContextSupplier; +import org.apache.hudi.client.HoodieFlinkWriteClient; +import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.client.utils.ConcatenatingIterator; +import org.apache.hudi.common.model.ClusteringGroupInfo; +import org.apache.hudi.common.model.ClusteringOperation; +import org.apache.hudi.common.table.HoodieTableConfig; +import org.apache.hudi.common.table.log.HoodieFileSliceReader; +import org.apache.hudi.common.table.log.HoodieMergedLogRecordScanner; +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.configuration.FlinkOptions; +import org.apache.hudi.exception.HoodieClusteringException; +import org.apache.hudi.exception.HoodieIOException; +import org.apache.hudi.io.IOUtils; +import org.apache.hudi.io.storage.HoodieFileReader; +import org.apache.hudi.io.storage.HoodieFileReaderFactory; +import org.apache.hudi.sink.bulk.BulkInsertWriterHelper; +import org.apache.hudi.sink.bulk.sort.SortOperatorGen; +import org.apache.hudi.table.HoodieFlinkTable; +import org.apache.hudi.util.AvroToRowDataConverters; +import org.apache.hudi.util.StreamerUtil; + +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.configuration.Configuration; +import org.apache.flink.metrics.Gauge; +import org.apache.flink.runtime.memory.MemoryManager; +import org.apache.flink.streaming.api.operators.BoundedOneInput; +import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.binary.BinaryRowData; +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.TableStreamOperator; +import org.apache.flink.table.runtime.operators.sort.BinaryExternalSorter; +import org.apache.flink.table.runtime.typeutils.AbstractRowDataSerializer; +import org.apache.flink.table.runtime.typeutils.BinaryRowDataSerializer; +import org.apache.flink.table.runtime.typeutils.RowDataSerializer; +import org.apache.flink.table.runtime.util.StreamRecordCollector; +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.Iterator; +import java.util.List; +import java.util.Spliterator; +import java.util.Spliterators; +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}. + */ +public class ClusteringOperator extends TableStreamOperator implements + OneInputStreamOperator, BoundedOneInput { + private static final Logger LOG = LoggerFactory.getLogger(ClusteringOperator.class); + + private final Configuration conf; + private final RowType rowType; + private int taskID; + private transient HoodieWriteConfig writeConfig; + private transient HoodieFlinkTable table; + private transient Schema schema; + private transient Schema readerSchema; + private transient int[] requiredPos; + private transient AvroToRowDataConverters.AvroToRowDataConverter avroToRowDataConverter; + private transient HoodieFlinkWriteClient writeClient; + private transient BulkInsertWriterHelper writerHelper; + private transient String instantTime; + + private transient BinaryExternalSorter sorter; + private transient StreamRecordCollector collector; + private transient BinaryRowDataSerializer binarySerializer; + + public ClusteringOperator(Configuration conf, RowType rowType) { + this.conf = conf; + this.rowType = rowType; + } + + @Override + public void open() throws Exception { + super.open(); + + this.taskID = getRuntimeContext().getIndexOfThisSubtask(); + this.writeConfig = StreamerUtil.getHoodieClientConfig(this.conf); + this.writeClient = StreamerUtil.createWriteClient(conf, getRuntimeContext()); + this.table = writeClient.getHoodieTable(); + + this.schema = StreamerUtil.getTableAvroSchema(table.getMetaClient(), false); + this.readerSchema = HoodieAvroUtils.addMetadataFields(this.schema); + this.requiredPos = getRequiredPositions(); + + this.avroToRowDataConverter = AvroToRowDataConverters.createRowConverter(rowType); + + ClassLoader cl = getContainingTask().getUserCodeClassLoader(); + + AbstractRowDataSerializer inputSerializer = new BinaryRowDataSerializer(rowType.getFieldCount()); + this.binarySerializer = new BinaryRowDataSerializer(inputSerializer.getArity()); + + NormalizedKeyComputer computer = createSortCodeGenerator().generateNormalizedKeyComputer("SortComputer").newInstance(cl); + RecordComparator comparator = createSortCodeGenerator().generateRecordComparator("SortComparator").newInstance(cl); + + MemoryManager memManager = getContainingTask().getEnvironment().getMemoryManager(); + this.sorter = + new BinaryExternalSorter( + this.getContainingTask(), + memManager, + computeMemorySize(), + this.getContainingTask().getEnvironment().getIOManager(), + inputSerializer, + binarySerializer, + computer, + comparator, + getContainingTask().getJobConfiguration()); + this.sorter.startThreads(); + + collector = new StreamRecordCollector<>(output); + + // register the metrics. + getMetricGroup().gauge("memoryUsedSizeInBytes", (Gauge) sorter::getUsedMemoryInBytes); + getMetricGroup().gauge("numSpillFiles", (Gauge) sorter::getNumSpillFiles); + getMetricGroup().gauge("spillInBytes", (Gauge) sorter::getSpillInBytes); + } + + @Override + public void processElement(StreamRecord element) throws Exception { + ClusteringPlanEvent event = element.getValue(); + final String instantTime = event.getClusteringInstantTime(); + final ClusteringGroupInfo clusteringGroupInfo = event.getClusteringGroupInfo(); + + initWriterHelper(instantTime); + + List clusteringOps = clusteringGroupInfo.getOperations(); + boolean hasLogFiles = clusteringOps.stream().anyMatch(op -> op.getDeltaFilePaths().size() > 0); + + 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); + } else { + // We want to optimize reading records for case there are no log files. + iterator = readRecordsForGroupBaseFiles(clusteringOps); + } + + RowDataSerializer rowDataSerializer = new RowDataSerializer(rowType); + while (iterator.hasNext()) { + RowData rowData = iterator.next(); + BinaryRowData binaryRowData = rowDataSerializer.toBinaryRow(rowData).copy(); + this.sorter.write(binaryRowData); + } + + BinaryRowData row = binarySerializer.createInstance(); + while ((row = sorter.getIterator().next(row)) != null) { + this.writerHelper.write(row); + } + } + + @Override + public void close() { + if (this.writeClient != null) { + this.writeClient.cleanHandlesGracefully(); + this.writeClient.close(); + } + } + + /** + * End input action for batch source. + */ + public void endInput() { + List writeStatuses = this.writerHelper.getWriteStatuses(this.taskID); + collector.collect(new ClusteringCommitEvent(instantTime, writeStatuses, this.taskID)); + } + + // ------------------------------------------------------------------------- + // Utilities + // ------------------------------------------------------------------------- + + private void initWriterHelper(String clusteringInstantTime) { + if (this.writerHelper == null) { + this.writerHelper = new BulkInsertWriterHelper(this.conf, this.table, this.writeConfig, + clusteringInstantTime, this.taskID, getRuntimeContext().getNumberOfParallelSubtasks(), getRuntimeContext().getAttemptNumber(), + this.rowType); + this.instantTime = clusteringInstantTime; + } + } + + /** + * Read records from baseFiles, apply updates and convert to Iterator. + */ + @SuppressWarnings("unchecked") + private Iterator readRecordsForGroupWithLogs(List clusteringOps, String instantTime) { + List> recordIterators = new ArrayList<>(); + + long maxMemoryPerCompaction = IOUtils.getMaxMemoryPerCompaction(new FlinkTaskContextSupplier(null), writeConfig); + LOG.info("MaxMemoryPerCompaction run as part of clustering => " + maxMemoryPerCompaction); + + for (ClusteringOperation clusteringOp : clusteringOps) { + try { + Option baseFileReader = StringUtils.isNullOrEmpty(clusteringOp.getDataFilePath()) + ? Option.empty() + : Option.of(HoodieFileReaderFactory.getFileReader(table.getHadoopConf(), new Path(clusteringOp.getDataFilePath()))); + HoodieMergedLogRecordScanner scanner = HoodieMergedLogRecordScanner.newBuilder() + .withFileSystem(table.getMetaClient().getFs()) + .withBasePath(table.getMetaClient().getBasePath()) + .withLogFilePaths(clusteringOp.getDeltaFilePaths()) + .withReaderSchema(readerSchema) + .withLatestInstantTime(instantTime) + .withMaxMemorySizeInBytes(maxMemoryPerCompaction) + .withReadBlocksLazily(writeConfig.getCompactionLazyBlockReadEnabled()) + .withReverseReader(writeConfig.getCompactionReverseLogReadEnabled()) + .withBufferSize(writeConfig.getMaxDFSStreamBufferSize()) + .withSpillableMapBasePath(writeConfig.getSpillableMapBasePath()) + .build(); + + HoodieTableConfig tableConfig = table.getMetaClient().getTableConfig(); + HoodieFileSliceReader hoodieFileSliceReader = HoodieFileSliceReader.getFileSliceReader(baseFileReader, scanner, readerSchema, + tableConfig.getPayloadClass(), + tableConfig.getPreCombineField(), + tableConfig.populateMetaFields() ? Option.empty() : Option.of(Pair.of(tableConfig.getRecordKeyFieldProp(), + tableConfig.getPartitionFieldProp()))); + + recordIterators.add(StreamSupport.stream(Spliterators.spliteratorUnknownSize(hoodieFileSliceReader, Spliterator.NONNULL), false).map(hoodieRecord -> { + try { + return this.transform((IndexedRecord) hoodieRecord.getData().getInsertValue(readerSchema).get()); + } catch (IOException e) { + throw new HoodieIOException("Failed to read next record", e); + } + }).iterator()); + } catch (IOException e) { + throw new HoodieClusteringException("Error reading input data for " + clusteringOp.getDataFilePath() + + " and " + clusteringOp.getDeltaFilePaths(), e); + } + } + + return new ConcatenatingIterator<>(recordIterators); + } + + /** + * Read records from baseFiles and get iterator. + */ + private Iterator readRecordsForGroupBaseFiles(List clusteringOps) { + List> iteratorsForPartition = clusteringOps.stream().map(clusteringOp -> { + Iterable indexedRecords = () -> { + try { + return HoodieFileReaderFactory.getFileReader(table.getHadoopConf(), new Path(clusteringOp.getDataFilePath())).getRecordIterator(readerSchema); + } catch (IOException e) { + throw new HoodieClusteringException("Error reading input data for " + clusteringOp.getDataFilePath() + + " and " + clusteringOp.getDeltaFilePaths(), e); + } + }; + + return StreamSupport.stream(indexedRecords.spliterator(), false).map(this::transform).iterator(); + }).collect(Collectors.toList()); + + return new ConcatenatingIterator<>(iteratorsForPartition); + } + + /** + * Transform IndexedRecord into HoodieRecord. + */ + private RowData transform(IndexedRecord indexedRecord) { + GenericRecord record = buildAvroRecordBySchema(indexedRecord, schema, requiredPos, new GenericRecordBuilder(schema)); + return (RowData) avroToRowDataConverter.convert(record); + } + + 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(); + } + + private SortCodeGenerator createSortCodeGenerator() { + SortOperatorGen sortOperatorGen = new SortOperatorGen(rowType, + conf.getString(FlinkOptions.CLUSTERING_SORT_COLUMNS).split(",")); + return sortOperatorGen.createSortCodeGenerator(); + } + + @Override + public void setKeyContextElement(StreamRecord record) throws Exception { + OneInputStreamOperator.super.setKeyContextElement(record); + } +} diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/ClusteringPlanEvent.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/ClusteringPlanEvent.java new file mode 100644 index 0000000000000..c82075877bcf3 --- /dev/null +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/ClusteringPlanEvent.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.clustering; + +import org.apache.hudi.common.model.ClusteringGroupInfo; + +import java.io.Serializable; +import java.util.Map; + +/** + * Represents a cluster command from the clustering plan task {@link ClusteringPlanSourceFunction}. + */ +public class ClusteringPlanEvent implements Serializable { + private static final long serialVersionUID = 1L; + + private String clusteringInstantTime; + + private ClusteringGroupInfo clusteringGroupInfo; + + private Map strategyParams; + + public ClusteringPlanEvent() { + } + + public ClusteringPlanEvent( + String instantTime, + ClusteringGroupInfo clusteringGroupInfo, + Map strategyParams) { + this.clusteringInstantTime = instantTime; + this.clusteringGroupInfo = clusteringGroupInfo; + this.strategyParams = strategyParams; + } + + public void setClusteringInstantTime(String clusteringInstantTime) { + this.clusteringInstantTime = clusteringInstantTime; + } + + public void setClusteringGroupInfo(ClusteringGroupInfo clusteringGroupInfo) { + this.clusteringGroupInfo = clusteringGroupInfo; + } + + public void setStrategyParams(Map strategyParams) { + this.strategyParams = strategyParams; + } + + public String getClusteringInstantTime() { + return clusteringInstantTime; + } + + public ClusteringGroupInfo getClusteringGroupInfo() { + return clusteringGroupInfo; + } + + public Map getStrategyParams() { + return strategyParams; + } +} diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/ClusteringPlanSourceFunction.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/ClusteringPlanSourceFunction.java new file mode 100644 index 0000000000000..a3db2d41c8371 --- /dev/null +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/ClusteringPlanSourceFunction.java @@ -0,0 +1,91 @@ +/* + * 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.HoodieClusteringGroup; +import org.apache.hudi.avro.model.HoodieClusteringPlan; +import org.apache.hudi.common.model.ClusteringGroupInfo; +import org.apache.hudi.common.model.ClusteringOperation; +import org.apache.hudi.common.table.timeline.HoodieInstant; + +import org.apache.flink.api.common.functions.AbstractRichFunction; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.streaming.api.functions.source.SourceFunction; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Flink hudi clustering source function. + * + *

This function read the clustering plan as {@link ClusteringOperation}s then assign the clustering task + * event {@link ClusteringPlanEvent} to downstream operators. + * + *

The clustering instant time is specified explicitly with strategies: + * + *

    + *
  • If the timeline has no inflight instants, + * use {@link org.apache.hudi.common.table.timeline.HoodieActiveTimeline#createNewInstantTime()} + * as the instant time;
  • + *
  • If the timeline has inflight instants, + * use the median instant time between [last complete instant time, earliest inflight instant time] + * as the instant time.
  • + *
+ */ +public class ClusteringPlanSourceFunction extends AbstractRichFunction implements SourceFunction { + + protected static final Logger LOG = LoggerFactory.getLogger(ClusteringPlanSourceFunction.class); + + /** + * The clustering plan. + */ + private final HoodieClusteringPlan clusteringPlan; + + /** + * Hoodie instant. + */ + private final HoodieInstant instant; + + public ClusteringPlanSourceFunction(HoodieInstant instant, HoodieClusteringPlan clusteringPlan) { + this.instant = instant; + this.clusteringPlan = clusteringPlan; + } + + @Override + public void open(Configuration parameters) throws Exception { + // no operation + } + + @Override + public void run(SourceContext sourceContext) throws Exception { + for (HoodieClusteringGroup clusteringGroup : clusteringPlan.getInputGroups()) { + LOG.info("ClusteringPlanSourceFunction cluster " + clusteringGroup + " files"); + sourceContext.collect(new ClusteringPlanEvent(this.instant.getTimestamp(), ClusteringGroupInfo.create(clusteringGroup), clusteringPlan.getStrategy().getStrategyParams())); + } + } + + @Override + public void close() throws Exception { + // no operation + } + + @Override + public void cancel() { + // no operation + } +} diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/FlinkClusteringConfig.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/FlinkClusteringConfig.java new file mode 100644 index 0000000000000..e87a7d6752b6e --- /dev/null +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/FlinkClusteringConfig.java @@ -0,0 +1,148 @@ +/* + * 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.configuration.FlinkOptions; + +import com.beust.jcommander.Parameter; +import org.apache.flink.configuration.Configuration; + +/** + * Configurations for Hoodie Flink clustering. + */ +public class FlinkClusteringConfig extends Configuration { + + @Parameter(names = {"--help", "-h"}, help = true) + public Boolean help = false; + + // ------------------------------------------------------------------------ + // Hudi Write Options + // ------------------------------------------------------------------------ + + @Parameter(names = {"--path"}, description = "Base path for the target hoodie table.", required = true) + public String path; + + // ------------------------------------------------------------------------ + // Clustering Options + // ------------------------------------------------------------------------ + @Parameter(names = {"--clustering-delta-commits"}, description = "Max delta commits needed to trigger clustering, default 4 commits", required = false) + public Integer clusteringDeltaCommits = 1; + + @Parameter(names = {"--clustering-tasks"}, description = "Parallelism of tasks that do actual clustering, default is -1", required = false) + public Integer clusteringTasks = -1; + + @Parameter(names = {"--compaction-max-memory"}, description = "Max memory in MB for compaction spillable map, default 100MB.", required = false) + public Integer compactionMaxMemory = 100; + + @Parameter(names = {"--clean-retain-commits"}, + description = "Number of commits to retain. So data will be retained for num_of_commits * time_between_commits (scheduled).\n" + + "This also directly translates into how much you can incrementally pull on this table, default 10", + required = false) + public Integer cleanRetainCommits = 10; + + @Parameter(names = {"--archive-min-commits"}, + description = "Min number of commits to keep before archiving older commits into a sequential log, default 20.", + required = false) + public Integer archiveMinCommits = 20; + + @Parameter(names = {"--archive-max-commits"}, + description = "Max number of commits to keep before archiving older commits into a sequential log, default 30.", + required = false) + public Integer archiveMaxCommits = 30; + + @Parameter(names = {"--schedule", "-sc"}, description = "Not recommended. Schedule the clustering plan in this job.\n" + + "There is a risk of losing data when scheduling clustering outside the writer job.\n" + + "Scheduling clustering in the writer job and only let this job do the clustering execution is recommended.\n" + + "Default is true", required = false) + public Boolean schedule = true; + + @Parameter(names = {"--clean-async-enabled"}, description = "Whether to cleanup the old commits immediately on new commits, enabled by default", required = false) + public Boolean cleanAsyncEnable = false; + + @Parameter(names = {"--plan-strategy-class"}, description = "Config to provide a strategy class to generator clustering plan", required = false) + public String planStrategyClass = "org.apache.hudi.client.clustering.plan.strategy.FlinkRecentDaysClusteringPlanStrategy"; + + @Parameter(names = {"--target-file-max-bytes"}, description = "Each group can produce 'N' (CLUSTERING_MAX_GROUP_SIZE/CLUSTERING_TARGET_FILE_SIZE) output file groups, default 1 GB", required = false) + public Integer targetFileMaxBytes = 1024 * 1024 * 1024; + + @Parameter(names = {"--small-file-limit"}, description = "Files smaller than the size specified here are candidates for clustering, default 600 MB", required = false) + public Integer smallFileLimit = 600; + + @Parameter(names = {"--skip-from-latest-partitions"}, description = "Number of partitions to skip from latest when choosing partitions to create ClusteringPlan, default 0", required = false) + public Integer skipFromLatestPartitions = 0; + + @Parameter(names = {"--sort-columns"}, description = "Columns to sort the data by when clustering.", required = false) + public String sortColumns = ""; + + @Parameter(names = {"--max-num-groups"}, description = "Maximum number of groups to create as part of ClusteringPlan. Increasing groups will increase parallelism. default 30", required = false) + public Integer maxNumGroups = 30; + + @Parameter(names = {"--target-partitions"}, description = "Number of partitions to list to create ClusteringPlan, default 2", required = false) + public Integer targetPartitions = 2; + + public static final String SEQ_FIFO = "FIFO"; + public static final String SEQ_LIFO = "LIFO"; + @Parameter(names = {"--seq"}, description = "Clustering plan execution sequence, two options are supported:\n" + + "1). FIFO: execute the oldest plan first;\n" + + "2). LIFO: execute the latest plan first, by default LIFO", required = false) + public String clusteringSeq = SEQ_LIFO; + + @Parameter(names = {"--write-partition-url-encode"}, description = "Whether to encode the partition path url, default false") + public Boolean writePartitionUrlEncode = false; + + @Parameter(names = {"--hive-style-partitioning"}, description = "Whether to use Hive style partitioning.\n" + + "If set true, the names of partition folders follow = format.\n" + + "By default false (the names of partition folders are only partition values)") + public Boolean hiveStylePartitioning = false; + + /** + * Transforms a {@code FlinkClusteringConfig.config} into {@code Configuration}. + * The latter is more suitable for the table APIs. It reads all the properties + * in the properties file (set by `--props` option) and cmd line options + * (set by `--hoodie-conf` option). + */ + public static Configuration toFlinkConfig(FlinkClusteringConfig config) { + Configuration conf = new Configuration(); + + conf.setString(FlinkOptions.PATH, config.path); + conf.setInteger(FlinkOptions.ARCHIVE_MAX_COMMITS, config.archiveMaxCommits); + conf.setInteger(FlinkOptions.ARCHIVE_MIN_COMMITS, config.archiveMinCommits); + conf.setInteger(FlinkOptions.CLEAN_RETAIN_COMMITS, config.cleanRetainCommits); + conf.setInteger(FlinkOptions.COMPACTION_MAX_MEMORY, config.compactionMaxMemory); + conf.setInteger(FlinkOptions.CLUSTERING_DELTA_COMMITS, config.clusteringDeltaCommits); + conf.setInteger(FlinkOptions.CLUSTERING_TASKS, config.clusteringTasks); + conf.setString(FlinkOptions.CLUSTERING_PLAN_STRATEGY_CLASS, config.planStrategyClass); + conf.setInteger(FlinkOptions.CLUSTERING_PLAN_STRATEGY_TARGET_FILE_MAX_BYTES, config.targetFileMaxBytes); + conf.setInteger(FlinkOptions.CLUSTERING_PLAN_STRATEGY_SMALL_FILE_LIMIT, config.smallFileLimit); + conf.setInteger(FlinkOptions.CLUSTERING_PLAN_STRATEGY_SKIP_PARTITIONS_FROM_LATEST, config.skipFromLatestPartitions); + conf.setString(FlinkOptions.CLUSTERING_SORT_COLUMNS, config.sortColumns); + conf.setInteger(FlinkOptions.CLUSTERING_MAX_NUM_GROUPS, config.maxNumGroups); + conf.setInteger(FlinkOptions.CLUSTERING_TARGET_PARTITIONS, config.targetPartitions); + conf.setBoolean(FlinkOptions.CLEAN_ASYNC_ENABLED, config.cleanAsyncEnable); + + // use synchronous clustering always + conf.setBoolean(FlinkOptions.CLUSTERING_SCHEDULE_ENABLED, config.schedule); + + // bulk insert conf + conf.setBoolean(FlinkOptions.URL_ENCODE_PARTITIONING, config.writePartitionUrlEncode); + conf.setBoolean(FlinkOptions.HIVE_STYLE_PARTITIONING, config.hiveStylePartitioning); + + return conf; + } +} 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 new file mode 100644 index 0000000000000..f7c361533a0d9 --- /dev/null +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/HoodieFlinkClusteringJob.java @@ -0,0 +1,191 @@ +/* + * 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.HoodieFlinkWriteClient; +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.util.ClusteringUtils; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.configuration.FlinkOptions; +import org.apache.hudi.table.HoodieFlinkTable; +import org.apache.hudi.util.AvroSchemaConverter; +import org.apache.hudi.util.CompactionUtil; +import org.apache.hudi.util.StreamerUtil; + +import com.beust.jcommander.JCommander; +import org.apache.avro.Schema; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.planner.plan.nodes.exec.utils.ExecNodeUtil; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.RowType; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Flink hudi clustering program that can be executed manually. + */ +public class HoodieFlinkClusteringJob { + + protected static final Logger LOG = LoggerFactory.getLogger(HoodieFlinkClusteringJob.class); + + public static void main(String[] args) throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + FlinkClusteringConfig cfg = new FlinkClusteringConfig(); + JCommander cmd = new JCommander(cfg, null, args); + if (cfg.help || args.length == 0) { + cmd.usage(); + System.exit(1); + } + + Configuration conf = FlinkClusteringConfig.toFlinkConfig(cfg); + + // create metaClient + HoodieTableMetaClient metaClient = StreamerUtil.createMetaClient(conf); + + // set table name + conf.setString(FlinkOptions.TABLE_NAME, metaClient.getTableConfig().getTableName()); + + // set table type + conf.setString(FlinkOptions.TABLE_TYPE, metaClient.getTableConfig().getTableType().name()); + + // set record key field + conf.setString(FlinkOptions.RECORD_KEY_FIELD, metaClient.getTableConfig().getRecordKeyFieldProp()); + + // set partition field + conf.setString(FlinkOptions.PARTITION_PATH_FIELD, metaClient.getTableConfig().getPartitionFieldProp()); + + // set table schema + CompactionUtil.setAvroSchema(conf, metaClient); + + HoodieFlinkWriteClient writeClient = StreamerUtil.createWriteClient(conf); + HoodieFlinkTable table = writeClient.getHoodieTable(); + + // judge whether have operation + // to compute the clustering instant time and do cluster. + if (cfg.schedule) { + String clusteringInstantTime = HoodieActiveTimeline.createNewInstantTime(); + boolean scheduled = writeClient.scheduleClusteringAtInstant(clusteringInstantTime, Option.empty()); + if (!scheduled) { + // do nothing. + LOG.info("No clustering plan for this job "); + return; + } + } + + table.getMetaClient().reloadActiveTimeline(); + + // fetch the instant based on the configured execution sequence + HoodieTimeline timeline = table.getActiveTimeline().filterPendingReplaceTimeline() + .filter(instant -> instant.getState() == HoodieInstant.State.REQUESTED); + Option requested = CompactionUtil.isLIFO(cfg.clusteringSeq) ? timeline.lastInstant() : timeline.firstInstant(); + if (!requested.isPresent()) { + // do nothing. + LOG.info("No clustering plan scheduled, turns on the clustering plan schedule with --schedule option"); + return; + } + + HoodieInstant clusteringInstant = requested.get(); + + HoodieInstant inflightInstant = HoodieTimeline.getReplaceCommitInflightInstant(clusteringInstant.getTimestamp()); + if (timeline.containsInstant(inflightInstant)) { + LOG.info("Rollback inflight clustering instant: [" + clusteringInstant + "]"); + writeClient.rollbackInflightClustering(inflightInstant, table); + table.getMetaClient().reloadActiveTimeline(); + } + + // generate clustering plan + // should support configurable commit metadata + Option> clusteringPlanOption = ClusteringUtils.getClusteringPlan( + table.getMetaClient(), clusteringInstant); + + if (!clusteringPlanOption.isPresent()) { + // do nothing. + LOG.info("No clustering plan scheduled, turns on the clustering plan schedule with --schedule option"); + return; + } + + HoodieClusteringPlan clusteringPlan = clusteringPlanOption.get().getRight(); + + if (clusteringPlan == null || (clusteringPlan.getInputGroups() == null) + || (clusteringPlan.getInputGroups().isEmpty())) { + // No clustering plan, do nothing and return. + LOG.info("No clustering plan for instant " + clusteringInstant.getTimestamp()); + return; + } + + HoodieInstant instant = HoodieTimeline.getReplaceCommitRequestedInstant(clusteringInstant.getTimestamp()); + HoodieTimeline pendingClusteringTimeline = table.getActiveTimeline().filterPendingReplaceTimeline(); + if (!pendingClusteringTimeline.containsInstant(instant)) { + // this means that the clustering plan was written to auxiliary path(.tmp) + // but not the meta path(.hoodie), this usually happens when the job crush + // exceptionally. + + // clean the clustering plan in auxiliary path and cancels the clustering. + + LOG.warn("The clustering plan was fetched through the auxiliary path(.tmp) but not the meta path(.hoodie).\n" + + "Clean the clustering plan in auxiliary path and cancels the clustering"); + CompactionUtil.cleanInstant(table.getMetaClient(), instant); + return; + } + + // get clusteringParallelism. + int clusteringParallelism = conf.getInteger(FlinkOptions.CLUSTERING_TASKS) == -1 + ? clusteringPlan.getInputGroups().size() : conf.getInteger(FlinkOptions.CLUSTERING_TASKS); + + // Mark instant as clustering inflight + table.getActiveTimeline().transitionReplaceRequestedToInflight(instant, Option.empty()); + + final Schema tableAvroSchema = StreamerUtil.getTableAvroSchema(table.getMetaClient(), false); + final DataType rowDataType = AvroSchemaConverter.convertToDataType(tableAvroSchema); + final RowType rowType = (RowType) rowDataType.getLogicalType(); + + // setup configuration + long ckpTimeout = env.getCheckpointConfig().getCheckpointTimeout(); + conf.setLong(FlinkOptions.WRITE_COMMIT_ACK_TIMEOUT, ckpTimeout); + + DataStream dataStream = env.addSource(new ClusteringPlanSourceFunction(timeline.lastInstant().get(), clusteringPlan)) + .name("clustering_source") + .uid("uid_clustering_source") + .rebalance() + .transform("clustering_task", + TypeInformation.of(ClusteringCommitEvent.class), + new ClusteringOperator(conf, rowType)) + .setParallelism(clusteringPlan.getInputGroups().size()); + + ExecNodeUtil.setManagedMemoryWeight(dataStream.getTransformation(), + conf.getInteger(FlinkOptions.WRITE_SORT_MEMORY) * 1024L * 1024L); + + dataStream + .addSink(new ClusteringCommitSink(conf)) + .name("clustering_commit") + .uid("uid_clustering_commit") + .setParallelism(1); + + env.execute("flink_hudi_clustering"); + } +} diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/streamer/FlinkStreamerConfig.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/streamer/FlinkStreamerConfig.java index f82712bca2c2a..e9574dd52bedd 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/streamer/FlinkStreamerConfig.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/streamer/FlinkStreamerConfig.java @@ -192,7 +192,7 @@ public class FlinkStreamerConfig extends Configuration { public Boolean indexGlobalEnabled = true; @Parameter(names = {"--index-partition-regex"}, - description = "Whether to load partitions in state if partition path matching, default *") + description = "Whether to load partitions in state if partition path matching, default *") public String indexPartitionRegex = ".*"; @Parameter(names = {"--source-avro-schema-path"}, description = "Source avro schema file path, the parsed schema is used for deserialization") diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/StreamerUtil.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/StreamerUtil.java index b977dfd7c5343..fcffbed54b48f 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/StreamerUtil.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/StreamerUtil.java @@ -29,6 +29,7 @@ import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodieCleaningPolicy; import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.TableSchemaResolver; import org.apache.hudi.common.table.log.HoodieLogFormat; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; @@ -37,6 +38,7 @@ import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.ReflectionUtils; import org.apache.hudi.common.util.ValidationUtils; +import org.apache.hudi.config.HoodieClusteringConfig; import org.apache.hudi.config.HoodieCompactionConfig; import org.apache.hudi.config.HoodieMemoryConfig; import org.apache.hudi.config.HoodiePayloadConfig; @@ -162,6 +164,17 @@ public static HoodieWriteConfig getHoodieClientConfig( .withPath(conf.getString(FlinkOptions.PATH)) .combineInput(conf.getBoolean(FlinkOptions.PRE_COMBINE), true) .withMergeAllowDuplicateOnInserts(OptionsResolver.insertClustering(conf)) + .withClusteringConfig( + HoodieClusteringConfig.newBuilder() + .withAsyncClustering(conf.getBoolean(FlinkOptions.CLUSTERING_SCHEDULE_ENABLED)) + .withClusteringPlanStrategyClass(conf.getString(FlinkOptions.CLUSTERING_PLAN_STRATEGY_CLASS)) + .withClusteringTargetPartitions(conf.getInteger(FlinkOptions.CLUSTERING_TARGET_PARTITIONS)) + .withClusteringMaxNumGroups(conf.getInteger(FlinkOptions.CLUSTERING_MAX_NUM_GROUPS)) + .withClusteringTargetFileMaxBytes(conf.getInteger(FlinkOptions.CLUSTERING_PLAN_STRATEGY_TARGET_FILE_MAX_BYTES)) + .withClusteringPlanSmallFileLimit(conf.getInteger(FlinkOptions.CLUSTERING_PLAN_STRATEGY_SMALL_FILE_LIMIT) * 1024 * 1024L) + .withClusteringSkipPartitionsFromLatest(conf.getInteger(FlinkOptions.CLUSTERING_PLAN_STRATEGY_SKIP_PARTITIONS_FROM_LATEST)) + .withAsyncClusteringMaxCommits(conf.getInteger(FlinkOptions.CLUSTERING_DELTA_COMMITS)) + .build()) .withCompactionConfig( HoodieCompactionConfig.newBuilder() .withPayloadClass(conf.getString(FlinkOptions.PAYLOAD_CLASS_NAME)) @@ -505,6 +518,11 @@ public static boolean haveSuccessfulCommits(HoodieTableMetaClient metaClient) { * Returns the max compaction memory in bytes with given conf. */ public static long getMaxCompactionMemoryInBytes(Configuration conf) { - return conf.getInteger(FlinkOptions.COMPACTION_MAX_MEMORY) * 1024 * 1024; + return (long) conf.getInteger(FlinkOptions.COMPACTION_MAX_MEMORY) * 1024 * 1024; + } + + public static Schema getTableAvroSchema(HoodieTableMetaClient metaClient, boolean includeMetadataFields) throws Exception { + TableSchemaResolver schemaUtil = new TableSchemaResolver(metaClient); + return schemaUtil.getTableAvroSchema(includeMetadataFields); } } diff --git a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/cluster/ITTestHoodieFlinkClustering.java b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/cluster/ITTestHoodieFlinkClustering.java new file mode 100644 index 0000000000000..ac2ee0be374ea --- /dev/null +++ b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/cluster/ITTestHoodieFlinkClustering.java @@ -0,0 +1,184 @@ +/* + * 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.cluster; + +import org.apache.hudi.avro.model.HoodieClusteringPlan; +import org.apache.hudi.client.HoodieFlinkWriteClient; +import org.apache.hudi.common.model.WriteOperationType; +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.util.ClusteringUtils; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.configuration.FlinkOptions; +import org.apache.hudi.sink.clustering.ClusteringCommitEvent; +import org.apache.hudi.sink.clustering.ClusteringCommitSink; +import org.apache.hudi.sink.clustering.ClusteringOperator; +import org.apache.hudi.sink.clustering.ClusteringPlanSourceFunction; +import org.apache.hudi.sink.clustering.FlinkClusteringConfig; +import org.apache.hudi.sink.clustering.HoodieFlinkClusteringJob; +import org.apache.hudi.table.HoodieFlinkTable; +import org.apache.hudi.util.AvroSchemaConverter; +import org.apache.hudi.util.CompactionUtil; +import org.apache.hudi.util.StreamerUtil; +import org.apache.hudi.utils.TestConfigurations; +import org.apache.hudi.utils.TestData; +import org.apache.hudi.utils.TestSQL; + +import org.apache.avro.Schema; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.EnvironmentSettings; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.api.config.ExecutionConfigOptions; +import org.apache.flink.table.api.internal.TableEnvironmentImpl; +import org.apache.flink.table.planner.plan.nodes.exec.utils.ExecNodeUtil; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.RowType; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; + +import java.io.File; +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.TimeUnit; + +import static org.junit.jupiter.api.Assertions.assertTrue; + +/** + * IT cases for {@link HoodieFlinkClusteringJob}. + */ +public class ITTestHoodieFlinkClustering { + + private static final Map EXPECTED = new HashMap<>(); + + static { + EXPECTED.put("par1", "[id1,par1,id1,Danny,23,1000,par1, id2,par1,id2,Stephen,33,2000,par1]"); + EXPECTED.put("par2", "[id3,par2,id3,Julian,53,3000,par2, id4,par2,id4,Fabian,31,4000,par2]"); + EXPECTED.put("par3", "[id5,par3,id5,Sophia,18,5000,par3, id6,par3,id6,Emma,20,6000,par3]"); + EXPECTED.put("par4", "[id7,par4,id7,Bob,44,7000,par4, id8,par4,id8,Han,56,8000,par4]"); + } + + @TempDir + File tempFile; + + @Test + public void testHoodieFlinkClustering() throws Exception { + // Create hoodie table and insert into data. + EnvironmentSettings settings = EnvironmentSettings.newInstance().inBatchMode().build(); + TableEnvironment tableEnv = TableEnvironmentImpl.create(settings); + tableEnv.getConfig().getConfiguration() + .setInteger(ExecutionConfigOptions.TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM, 1); + Map options = new HashMap<>(); + options.put(FlinkOptions.PATH.key(), tempFile.getAbsolutePath()); + + // use append mode + options.put(FlinkOptions.OPERATION.key(), WriteOperationType.INSERT.value()); + options.put(FlinkOptions.INSERT_CLUSTER.key(), "false"); + + String hoodieTableDDL = TestConfigurations.getCreateHoodieTableDDL("t1", options); + tableEnv.executeSql(hoodieTableDDL); + tableEnv.executeSql(TestSQL.INSERT_T1).await(); + + // wait for the asynchronous commit to finish + TimeUnit.SECONDS.sleep(3); + + // Make configuration and setAvroSchema. + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + FlinkClusteringConfig cfg = new FlinkClusteringConfig(); + cfg.path = tempFile.getAbsolutePath(); + cfg.targetPartitions = 4; + Configuration conf = FlinkClusteringConfig.toFlinkConfig(cfg); + + // create metaClient + HoodieTableMetaClient metaClient = StreamerUtil.createMetaClient(conf); + + // set the table name + conf.setString(FlinkOptions.TABLE_NAME, metaClient.getTableConfig().getTableName()); + conf.setString(FlinkOptions.TABLE_TYPE, metaClient.getTableConfig().getTableType().name()); + + // set record key field + conf.setString(FlinkOptions.RECORD_KEY_FIELD, metaClient.getTableConfig().getRecordKeyFieldProp()); + // set partition field + conf.setString(FlinkOptions.PARTITION_PATH_FIELD, metaClient.getTableConfig().getPartitionFieldProp()); + + long ckpTimeout = env.getCheckpointConfig().getCheckpointTimeout(); + conf.setLong(FlinkOptions.WRITE_COMMIT_ACK_TIMEOUT, ckpTimeout); + conf.setString(FlinkOptions.PARTITION_PATH_FIELD, "partition"); + + // set table schema + CompactionUtil.setAvroSchema(conf, metaClient); + + // judge whether have operation + // To compute the clustering instant time and do clustering. + String clusteringInstantTime = HoodieActiveTimeline.createNewInstantTime(); + + HoodieFlinkWriteClient writeClient = StreamerUtil.createWriteClient(conf, null); + HoodieFlinkTable table = writeClient.getHoodieTable(); + + boolean scheduled = writeClient.scheduleClusteringAtInstant(clusteringInstantTime, Option.empty()); + + assertTrue(scheduled, "The clustering plan should be scheduled"); + + // fetch the instant based on the configured execution sequence + table.getMetaClient().reloadActiveTimeline(); + HoodieTimeline timeline = table.getActiveTimeline().filterPendingReplaceTimeline() + .filter(instant -> instant.getState() == HoodieInstant.State.REQUESTED); + + // generate clustering plan + // should support configurable commit metadata + Option> clusteringPlanOption = ClusteringUtils.getClusteringPlan( + table.getMetaClient(), timeline.lastInstant().get()); + + HoodieClusteringPlan clusteringPlan = clusteringPlanOption.get().getRight(); + + // Mark instant as clustering inflight + HoodieInstant instant = HoodieTimeline.getReplaceCommitRequestedInstant(clusteringInstantTime); + table.getActiveTimeline().transitionReplaceRequestedToInflight(instant, Option.empty()); + + final Schema tableAvroSchema = StreamerUtil.getTableAvroSchema(table.getMetaClient(), false); + final DataType rowDataType = AvroSchemaConverter.convertToDataType(tableAvroSchema); + final RowType rowType = (RowType) rowDataType.getLogicalType(); + + DataStream dataStream = env.addSource(new ClusteringPlanSourceFunction(timeline.lastInstant().get(), clusteringPlan)) + .name("clustering_source") + .uid("uid_clustering_source") + .rebalance() + .transform("clustering_task", + TypeInformation.of(ClusteringCommitEvent.class), + new ClusteringOperator(conf, rowType)) + .setParallelism(clusteringPlan.getInputGroups().size()); + + ExecNodeUtil.setManagedMemoryWeight(dataStream.getTransformation(), + conf.getInteger(FlinkOptions.WRITE_SORT_MEMORY) * 1024L * 1024L); + + dataStream + .addSink(new ClusteringCommitSink(conf)) + .name("clustering_commit") + .uid("uid_clustering_commit") + .setParallelism(1); + + env.execute("flink_hudi_clustering"); + TestData.checkWrittenData(tempFile, EXPECTED, 4); + } +} From 10363c1412b8f9b5b16b3e2e075b895e0cc9a293 Mon Sep 17 00:00:00 2001 From: Sivabalan Narayanan Date: Tue, 24 May 2022 08:17:15 -0400 Subject: [PATCH 14/44] [HUDI-4132] Fixing determining target table schema for delta sync with empty batch (#5648) --- .../hudi/utilities/deltastreamer/DeltaSync.java | 11 +++++++++-- 1 file changed, 9 insertions(+), 2 deletions(-) diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/DeltaSync.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/DeltaSync.java index a1a804b9ed123..a4a7e10abc004 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/DeltaSync.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/DeltaSync.java @@ -840,8 +840,15 @@ private Schema getSchemaForWriteConfig(Schema targetSchema) { && SchemaCompatibility.checkReaderWriterCompatibility(InputBatch.NULL_SCHEMA, targetSchema).getType() == SchemaCompatibility.SchemaCompatibilityType.COMPATIBLE) { // target schema is null. fetch schema from commit metadata and use it HoodieTableMetaClient meta = HoodieTableMetaClient.builder().setConf(new Configuration(fs.getConf())).setBasePath(cfg.targetBasePath).setPayloadClassName(cfg.payloadClassName).build(); - TableSchemaResolver schemaResolver = new TableSchemaResolver(meta); - newWriteSchema = schemaResolver.getTableAvroSchema(false); + int totalCompleted = meta.getActiveTimeline().getCommitsTimeline().filterCompletedInstants().countInstants(); + if (totalCompleted > 0) { + try { + TableSchemaResolver schemaResolver = new TableSchemaResolver(meta); + newWriteSchema = schemaResolver.getTableAvroSchema(false); + } catch (IllegalArgumentException e) { + LOG.warn("Could not fetch schema from table. Falling back to using target schema from schema provider"); + } + } } } return newWriteSchema; From f30b3aef3e5e12e9119d6b5294c94a0282719e00 Mon Sep 17 00:00:00 2001 From: luoyajun Date: Wed, 25 May 2022 02:13:18 +0800 Subject: [PATCH 15/44] [MINOR] Fix a potential NPE and some finer points of hudi cli (#5656) --- .../hudi/cli/commands/MetadataCommand.java | 2 +- .../apache/hudi/cli/commands/SparkMain.java | 2 +- .../commands/UpgradeOrDowngradeCommand.java | 4 ++-- .../org/apache/hudi/cli/utils/SparkUtil.java | 19 +++++++++++-------- .../hudi/cli/testutils/SparkUtilTest.java | 11 +++++++++++ 5 files changed, 26 insertions(+), 12 deletions(-) diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/MetadataCommand.java b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/MetadataCommand.java index 8ab6c0ca4f992..637f1393f51ad 100644 --- a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/MetadataCommand.java +++ b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/MetadataCommand.java @@ -364,7 +364,7 @@ private HoodieWriteConfig getWriteConfig() { private void initJavaSparkContext(Option userDefinedMaster) { if (jsc == null) { - jsc = SparkUtil.initJavaSparkConf(SparkUtil.getDefaultConf("HoodieCLI", userDefinedMaster)); + jsc = SparkUtil.initJavaSparkContext(SparkUtil.getDefaultConf("HoodieCLI", userDefinedMaster)); } } } \ No newline at end of file diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/SparkMain.java b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/SparkMain.java index 323c7bb5c36e0..9fe83f1995c2b 100644 --- a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/SparkMain.java +++ b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/SparkMain.java @@ -95,7 +95,7 @@ public static void main(String[] args) throws Exception { LOG.info("Invoking SparkMain: " + commandString); final SparkCommand cmd = SparkCommand.valueOf(commandString); - JavaSparkContext jsc = SparkUtil.initJavaSparkConf("hoodie-cli-" + commandString, + JavaSparkContext jsc = SparkUtil.initJavaSparkContext("hoodie-cli-" + commandString, Option.of(args[1]), Option.of(args[2])); int returnCode = 0; diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/UpgradeOrDowngradeCommand.java b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/UpgradeOrDowngradeCommand.java index a5e513c614939..2ddb88792c0fd 100644 --- a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/UpgradeOrDowngradeCommand.java +++ b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/UpgradeOrDowngradeCommand.java @@ -56,7 +56,7 @@ public String upgradeHoodieTable( if (exitCode != 0) { return String.format("Failed: Could not Upgrade/Downgrade Hoodie table to \"%s\".", toVersion); } - return String.format("Hoodie table upgraded/downgraded to ", toVersion); + return String.format("Hoodie table upgraded/downgraded to %s", toVersion); } @CliCommand(value = "downgrade table", help = "Downgrades a table") @@ -78,6 +78,6 @@ public String downgradeHoodieTable( if (exitCode != 0) { return String.format("Failed: Could not Upgrade/Downgrade Hoodie table to \"%s\".", toVersion); } - return String.format("Hoodie table upgraded/downgraded to ", toVersion); + return String.format("Hoodie table upgraded/downgraded to %s", toVersion); } } diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/utils/SparkUtil.java b/hudi-cli/src/main/java/org/apache/hudi/cli/utils/SparkUtil.java index ae99b0b8240d7..bcccb66b3716c 100644 --- a/hudi-cli/src/main/java/org/apache/hudi/cli/utils/SparkUtil.java +++ b/hudi-cli/src/main/java/org/apache/hudi/cli/utils/SparkUtil.java @@ -32,8 +32,8 @@ import java.io.File; import java.net.URISyntaxException; +import java.util.Arrays; import java.util.Map; -import java.util.Objects; import java.util.Properties; /** @@ -56,9 +56,12 @@ public static SparkLauncher initLauncher(String propertiesFile) throws URISyntax if (!StringUtils.isNullOrEmpty(propertiesFile)) { sparkLauncher.setPropertiesFile(propertiesFile); } + File libDirectory = new File(new File(currentJar).getParent(), "lib"); - for (String library : Objects.requireNonNull(libDirectory.list())) { - sparkLauncher.addJar(new File(libDirectory, library).getAbsolutePath()); + // This lib directory may be not required, such as providing libraries through a bundle jar + if (libDirectory.exists()) { + Arrays.stream(libDirectory.list()).forEach(library -> + sparkLauncher.addJar(new File(libDirectory, library).getAbsolutePath())); } return sparkLauncher; } @@ -99,20 +102,20 @@ public static SparkConf getDefaultConf(final String appName, final Option master, Option executorMemory) { + public static JavaSparkContext initJavaSparkContext(String name, Option master, Option executorMemory) { SparkConf sparkConf = getDefaultConf(name, master); if (executorMemory.isPresent()) { sparkConf.set(HoodieCliSparkConfig.CLI_EXECUTOR_MEMORY, executorMemory.get()); } - return initJavaSparkConf(sparkConf); + return initJavaSparkContext(sparkConf); } - public static JavaSparkContext initJavaSparkConf(SparkConf sparkConf) { + public static JavaSparkContext initJavaSparkContext(SparkConf sparkConf) { SparkRDDWriteClient.registerClasses(sparkConf); JavaSparkContext jsc = new JavaSparkContext(sparkConf); jsc.hadoopConfiguration().setBoolean(HoodieCliSparkConfig.CLI_PARQUET_ENABLE_SUMMARY_METADATA, false); diff --git a/hudi-cli/src/test/java/org/apache/hudi/cli/testutils/SparkUtilTest.java b/hudi-cli/src/test/java/org/apache/hudi/cli/testutils/SparkUtilTest.java index 4966438292949..a470ee1c2a37d 100644 --- a/hudi-cli/src/test/java/org/apache/hudi/cli/testutils/SparkUtilTest.java +++ b/hudi-cli/src/test/java/org/apache/hudi/cli/testutils/SparkUtilTest.java @@ -22,11 +22,22 @@ import org.apache.hudi.cli.utils.SparkUtil; import org.apache.spark.SparkConf; +import org.apache.spark.launcher.SparkLauncher; import org.junit.jupiter.api.Test; +import java.net.URISyntaxException; + import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; public class SparkUtilTest { + + @Test + public void testInitSparkLauncher() throws URISyntaxException { + SparkLauncher sparkLauncher = SparkUtil.initLauncher(null); + assertNotNull(sparkLauncher); + } + @Test public void testGetDefaultSparkConf() { SparkConf sparkConf = SparkUtil.getDefaultConf("test-spark-app", Option.of("")); From a6bc9e8e818be42f26995f927c406ab1c3ddb7d6 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=86=AF=E5=81=A5?= Date: Wed, 25 May 2022 20:31:39 +0800 Subject: [PATCH 16/44] [HUDI-4146] Claim RFC-55 for Improve Hive/Meta sync class design and hierachies (#5682) --- rfc/README.md | 1 + 1 file changed, 1 insertion(+) diff --git a/rfc/README.md b/rfc/README.md index 5040860423014..2bfd028e9fda4 100644 --- a/rfc/README.md +++ b/rfc/README.md @@ -90,3 +90,4 @@ The list of all RFCs can be found here. | 52 | [Introduce Secondary Index to Improve HUDI Query Performance](./rfc-52/rfc-52.md) | `UNDER REVIEW` | | 53 | [Use Lock-Free Message Queue Improving Hoodie Writing Efficiency](./rfc-53/rfc-53.md) | `UNDER REVIEW` | | 54 | [New Table APIs and Streamline Hudi Configs](./rfc-54/rfc-54.md) | `UNDER REVIEW` | +| 55 | [Improve Hive/Meta sync class design and hierachies](./rfc-55/rfc-55.md) | `UNDER REVIEW` | From cf837b49008fd351a3f89beb5e4e5c17c30b9a3c Mon Sep 17 00:00:00 2001 From: Sagar Sumit Date: Wed, 25 May 2022 19:38:56 +0530 Subject: [PATCH 17/44] [HUDI-3193] Decouple hudi-aws from hudi-client-common (#5666) Move HoodieMetricsCloudWatchConfig to hudi-client-common --- .../aws/cloudwatch/CloudWatchReporter.java | 2 +- hudi-client/hudi-client-common/pom.xml | 1 + .../apache/hudi/config/HoodieWriteConfig.java | 1 + .../HoodieMetricsCloudWatchConfig.java | 47 ++++++++++--------- .../config/metrics/HoodieMetricsConfig.java | 1 - hudi-flink-datasource/hudi-flink/pom.xml | 6 +++ 6 files changed, 33 insertions(+), 25 deletions(-) rename {hudi-aws/src/main/java/org/apache/hudi/config => hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/metrics}/HoodieMetricsCloudWatchConfig.java (65%) diff --git a/hudi-aws/src/main/java/org/apache/hudi/aws/cloudwatch/CloudWatchReporter.java b/hudi-aws/src/main/java/org/apache/hudi/aws/cloudwatch/CloudWatchReporter.java index e4bc598ce2938..b208ec92260fd 100644 --- a/hudi-aws/src/main/java/org/apache/hudi/aws/cloudwatch/CloudWatchReporter.java +++ b/hudi-aws/src/main/java/org/apache/hudi/aws/cloudwatch/CloudWatchReporter.java @@ -74,7 +74,7 @@ public static Builder forRegistry(MetricRegistry registry) { } public static class Builder { - private MetricRegistry registry; + private final MetricRegistry registry; private Clock clock; private String prefix; private TimeUnit rateUnit; diff --git a/hudi-client/hudi-client-common/pom.xml b/hudi-client/hudi-client-common/pom.xml index a348a63921179..ddfd4a2fd584d 100644 --- a/hudi-client/hudi-client-common/pom.xml +++ b/hudi-client/hudi-client-common/pom.xml @@ -40,6 +40,7 @@ org.apache.hudi hudi-aws ${project.version} + provided
org.apache.hudi 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 dd5c0bfd6ded3..42208a0734aa7 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 @@ -48,6 +48,7 @@ import org.apache.hudi.common.util.ReflectionUtils; import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.common.util.ValidationUtils; +import org.apache.hudi.config.metrics.HoodieMetricsCloudWatchConfig; import org.apache.hudi.config.metrics.HoodieMetricsConfig; import org.apache.hudi.config.metrics.HoodieMetricsDatadogConfig; import org.apache.hudi.config.metrics.HoodieMetricsGraphiteConfig; diff --git a/hudi-aws/src/main/java/org/apache/hudi/config/HoodieMetricsCloudWatchConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/metrics/HoodieMetricsCloudWatchConfig.java similarity index 65% rename from hudi-aws/src/main/java/org/apache/hudi/config/HoodieMetricsCloudWatchConfig.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/metrics/HoodieMetricsCloudWatchConfig.java index e4e46d5a1f7b3..3c4b860e69230 100644 --- a/hudi-aws/src/main/java/org/apache/hudi/config/HoodieMetricsCloudWatchConfig.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/metrics/HoodieMetricsCloudWatchConfig.java @@ -7,16 +7,17 @@ * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. */ -package org.apache.hudi.config; +package org.apache.hudi.config.metrics; import org.apache.hudi.common.config.ConfigClassProperty; import org.apache.hudi.common.config.ConfigGroups; @@ -36,22 +37,22 @@ public class HoodieMetricsCloudWatchConfig extends HoodieConfig { public static final String CLOUDWATCH_PREFIX = "hoodie.metrics.cloudwatch"; public static final ConfigProperty REPORT_PERIOD_SECONDS = ConfigProperty - .key(CLOUDWATCH_PREFIX + ".report.period.seconds") - .defaultValue(60) - .sinceVersion("0.10.0") - .withDocumentation("Reporting interval in seconds"); + .key(CLOUDWATCH_PREFIX + ".report.period.seconds") + .defaultValue(60) + .sinceVersion("0.10.0") + .withDocumentation("Reporting interval in seconds"); public static final ConfigProperty METRIC_PREFIX = ConfigProperty - .key(CLOUDWATCH_PREFIX + ".metric.prefix") - .defaultValue("") - .sinceVersion("0.10.0") - .withDocumentation("Metric prefix of reporter"); + .key(CLOUDWATCH_PREFIX + ".metric.prefix") + .defaultValue("") + .sinceVersion("0.10.0") + .withDocumentation("Metric prefix of reporter"); public static final ConfigProperty METRIC_NAMESPACE = ConfigProperty - .key(CLOUDWATCH_PREFIX + ".namespace") - .defaultValue("Hudi") - .sinceVersion("0.10.0") - .withDocumentation("Namespace of reporter"); + .key(CLOUDWATCH_PREFIX + ".namespace") + .defaultValue("Hudi") + .sinceVersion("0.10.0") + .withDocumentation("Namespace of reporter"); /* Amazon CloudWatch allows a maximum of 20 metrics per request. Choosing this as the default maximum. Reference: https://docs.aws.amazon.com/AmazonCloudWatch/latest/APIReference/API_PutMetricData.html @@ -66,13 +67,13 @@ public HoodieMetricsCloudWatchConfig() { super(); } - public static HoodieMetricsCloudWatchConfig.Builder newBuilder() { - return new HoodieMetricsCloudWatchConfig.Builder(); + static Builder newBuilder() { + return new Builder(); } - public static class Builder { + static class Builder { - private HoodieMetricsCloudWatchConfig hoodieMetricsCloudWatchConfig = new HoodieMetricsCloudWatchConfig(); + private final HoodieMetricsCloudWatchConfig hoodieMetricsCloudWatchConfig = new HoodieMetricsCloudWatchConfig(); public HoodieMetricsCloudWatchConfig.Builder fromProperties(Properties props) { this.hoodieMetricsCloudWatchConfig.getProps().putAll(props); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/metrics/HoodieMetricsConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/metrics/HoodieMetricsConfig.java index 85f98935fd3c3..a515eb702b8cc 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/metrics/HoodieMetricsConfig.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/metrics/HoodieMetricsConfig.java @@ -24,7 +24,6 @@ import org.apache.hudi.common.config.HoodieConfig; import org.apache.hudi.common.table.HoodieTableConfig; import org.apache.hudi.common.util.Option; -import org.apache.hudi.config.HoodieMetricsCloudWatchConfig; import org.apache.hudi.metrics.MetricsReporterType; import javax.annotation.concurrent.Immutable; diff --git a/hudi-flink-datasource/hudi-flink/pom.xml b/hudi-flink-datasource/hudi-flink/pom.xml index 97288d19cd35c..04d45358b5b5e 100644 --- a/hudi-flink-datasource/hudi-flink/pom.xml +++ b/hudi-flink-datasource/hudi-flink/pom.xml @@ -89,6 +89,12 @@ hudi-client-common ${project.version} + + org.apache.hudi + hudi-aws + ${project.version} + provided + org.apache.hudi hudi-flink-client From 4e42ed5eae36f706ec35e5d09d8dc206b7fea130 Mon Sep 17 00:00:00 2001 From: Danny Chan Date: Thu, 26 May 2022 11:21:39 +0800 Subject: [PATCH 18/44] [HUDI-4145] Archives the metadata file in HoodieInstant.State sequence (part2) (#5676) --- .../hudi/client/HoodieTimelineArchiver.java | 46 ++++++++++++------- 1 file changed, 30 insertions(+), 16 deletions(-) diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/HoodieTimelineArchiver.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/HoodieTimelineArchiver.java index f111bb70ef007..c53554d8e04d2 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/HoodieTimelineArchiver.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/HoodieTimelineArchiver.java @@ -506,13 +506,7 @@ private Stream getInstantsToArchive() { List instantsToStream = groupByTsAction.get(Pair.of(hoodieInstant.getTimestamp(), HoodieInstant.getComparableAction(hoodieInstant.getAction()))); if (instantsToStream != null) { - // sorts the instants in natural order to make sure the metadata files be removed - // in HoodieInstant.State sequence: requested -> inflight -> completed, - // this is important because when a COMPLETED metadata file is removed first, - // other monitors on the timeline(such as the compaction or clustering services) would - // mistakenly recognize the pending file as a pending operation, - // then all kinds of weird bugs occur. - return instantsToStream.stream().sorted(); + return instantsToStream.stream(); } else { // if a concurrent writer archived the instant return Stream.empty(); @@ -522,19 +516,29 @@ private Stream getInstantsToArchive() { private boolean deleteArchivedInstants(List archivedInstants, HoodieEngineContext context) throws IOException { LOG.info("Deleting instants " + archivedInstants); - boolean success = true; - List instantFiles = archivedInstants.stream().map(archivedInstant -> - new Path(metaClient.getMetaPath(), archivedInstant.getFileName()) - ).map(Path::toString).collect(Collectors.toList()); - context.setJobStatus(this.getClass().getSimpleName(), "Delete archived instants: " + config.getTableName()); - Map resultDeleteInstantFiles = deleteFilesParallelize(metaClient, instantFiles, context, false); + List pendingInstantFiles = new ArrayList<>(); + List completedInstantFiles = new ArrayList<>(); - for (Map.Entry result : resultDeleteInstantFiles.entrySet()) { - LOG.info("Archived and deleted instant file " + result.getKey() + " : " + result.getValue()); - success &= result.getValue(); + for (HoodieInstant instant : archivedInstants) { + String filePath = new Path(metaClient.getMetaPath(), instant.getFileName()).toString(); + if (instant.isCompleted()) { + completedInstantFiles.add(filePath); + } else { + pendingInstantFiles.add(filePath); + } } + context.setJobStatus(this.getClass().getSimpleName(), "Delete archived instants: " + config.getTableName()); + // Delete the metadata files + // in HoodieInstant.State sequence: requested -> inflight -> completed, + // this is important because when a COMPLETED metadata file is removed first, + // other monitors on the timeline(such as the compaction or clustering services) would + // mistakenly recognize the pending file as a pending operation, + // then all kinds of weird bugs occur. + boolean success = deleteArchivedInstantFiles(context, true, pendingInstantFiles); + success &= deleteArchivedInstantFiles(context, success, completedInstantFiles); + // Remove older meta-data from auxiliary path too Option latestCommitted = Option.fromJavaOptional(archivedInstants.stream().filter(i -> i.isCompleted() && (i.getAction().equals(HoodieTimeline.COMMIT_ACTION) || (i.getAction().equals(HoodieTimeline.DELTA_COMMIT_ACTION)))).max(Comparator.comparing(HoodieInstant::getTimestamp))); @@ -545,6 +549,16 @@ private boolean deleteArchivedInstants(List archivedInstants, Hoo return success; } + private boolean deleteArchivedInstantFiles(HoodieEngineContext context, boolean success, List files) { + Map resultDeleteInstantFiles = deleteFilesParallelize(metaClient, files, context, false); + + for (Map.Entry result : resultDeleteInstantFiles.entrySet()) { + LOG.info("Archived and deleted instant file " + result.getKey() + " : " + result.getValue()); + success &= result.getValue(); + } + return success; + } + /** * Remove older instants from auxiliary meta folder. * From 98c5c6c6543177ff4320b73b13bc153938300fe4 Mon Sep 17 00:00:00 2001 From: RexAn Date: Thu, 26 May 2022 13:09:04 +0800 Subject: [PATCH 19/44] [HUDI-4040] Bulk insert Support CustomColumnsSortPartitioner with Row (#5502) * Along the lines of RDDCustomColumnsSortPartitioner but for Row --- .../RDDCustomColumnsSortPartitioner.java | 5 +- .../RowCustomColumnsSortPartitioner.java | 60 +++++++++++++++++++ ...tBulkInsertInternalPartitionerForRows.java | 55 ++++++++++++++--- 3 files changed, 112 insertions(+), 8 deletions(-) create mode 100644 hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/RowCustomColumnsSortPartitioner.java diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/RDDCustomColumnsSortPartitioner.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/RDDCustomColumnsSortPartitioner.java index 2fe6fe969c482..b1cbe47a6b72e 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/RDDCustomColumnsSortPartitioner.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/RDDCustomColumnsSortPartitioner.java @@ -29,6 +29,8 @@ import org.apache.avro.Schema; import org.apache.spark.api.java.JavaRDD; +import java.util.Arrays; + /** * A partitioner that does sorting based on specified column values for each RDD partition. * @@ -78,6 +80,7 @@ public boolean arePartitionRecordsSorted() { } private String[] getSortColumnName(HoodieWriteConfig config) { - return config.getUserDefinedBulkInsertPartitionerSortColumns().split(","); + return Arrays.stream(config.getUserDefinedBulkInsertPartitionerSortColumns().split(",")) + .map(String::trim).toArray(String[]::new); } } 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 new file mode 100644 index 0000000000000..ceeb2b3fe8f00 --- /dev/null +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/RowCustomColumnsSortPartitioner.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.execution.bulkinsert; + +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.table.BulkInsertPartitioner; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; + +import java.util.Arrays; + +/** + * A partitioner that does sorting based on specified column values for each spark partitions. + */ +public class RowCustomColumnsSortPartitioner implements BulkInsertPartitioner> { + + private final String[] sortColumnNames; + + public RowCustomColumnsSortPartitioner(HoodieWriteConfig config) { + this.sortColumnNames = getSortColumnName(config); + } + + public RowCustomColumnsSortPartitioner(String[] columnNames) { + this.sortColumnNames = columnNames; + } + + @Override + public Dataset repartitionRecords(Dataset records, int outputSparkPartitions) { + final String[] sortColumns = this.sortColumnNames; + return records.coalesce(outputSparkPartitions) + .sortWithinPartitions(HoodieRecord.PARTITION_PATH_METADATA_FIELD, sortColumns); + } + + @Override + public boolean arePartitionRecordsSorted() { + return true; + } + + private String[] getSortColumnName(HoodieWriteConfig config) { + return Arrays.stream(config.getUserDefinedBulkInsertPartitionerSortColumns().split(",")) + .map(String::trim).toArray(String[]::new); + } +} diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/bulkinsert/TestBulkInsertInternalPartitionerForRows.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/bulkinsert/TestBulkInsertInternalPartitionerForRows.java index 276ad5b43ab3a..52cf67228a583 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/bulkinsert/TestBulkInsertInternalPartitionerForRows.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/bulkinsert/TestBulkInsertInternalPartitionerForRows.java @@ -20,6 +20,8 @@ import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.testutils.HoodieTestDataGenerator; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.table.BulkInsertPartitioner; import org.apache.hudi.testutils.HoodieClientTestHarness; import org.apache.hudi.testutils.SparkDatasetTestUtils; @@ -29,6 +31,7 @@ import org.apache.spark.sql.Row; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.Arguments; import org.junit.jupiter.params.provider.MethodSource; @@ -48,6 +51,8 @@ */ public class TestBulkInsertInternalPartitionerForRows extends HoodieClientTestHarness { + private static final Comparator KEY_COMPARATOR = + Comparator.comparing(o -> (o.getAs(HoodieRecord.PARTITION_PATH_METADATA_FIELD) + "+" + o.getAs(HoodieRecord.RECORD_KEY_METADATA_FIELD))); @BeforeEach public void setUp() throws Exception { initSparkContexts("TestBulkInsertInternalPartitionerForRows"); @@ -77,21 +82,47 @@ public void testBulkInsertInternalPartitioner(BulkInsertSortMode sortMode, Dataset records1 = generateTestRecords(); Dataset records2 = generateTestRecords(); testBulkInsertInternalPartitioner(BulkInsertInternalPartitionerWithRowsFactory.get(sortMode), - records1, isGloballySorted, isLocallySorted, generateExpectedPartitionNumRecords(records1)); + records1, isGloballySorted, isLocallySorted, generateExpectedPartitionNumRecords(records1), Option.empty()); testBulkInsertInternalPartitioner(BulkInsertInternalPartitionerWithRowsFactory.get(sortMode), - records2, isGloballySorted, isLocallySorted, generateExpectedPartitionNumRecords(records2)); + records2, isGloballySorted, isLocallySorted, generateExpectedPartitionNumRecords(records2), Option.empty()); + } + + @Test + public void testCustomColumnSortPartitionerWithRows() { + Dataset records1 = generateTestRecords(); + Dataset records2 = generateTestRecords(); + String sortColumnString = records1.columns()[5]; + String[] sortColumns = sortColumnString.split(","); + Comparator comparator = getCustomColumnComparator(sortColumns); + + testBulkInsertInternalPartitioner(new RowCustomColumnsSortPartitioner(sortColumns), + records1, false, true, generateExpectedPartitionNumRecords(records1), Option.of(comparator)); + testBulkInsertInternalPartitioner(new RowCustomColumnsSortPartitioner(sortColumns), + records2, false, true, generateExpectedPartitionNumRecords(records2), Option.of(comparator)); + + HoodieWriteConfig config = HoodieWriteConfig + .newBuilder() + .withPath("/") + .withUserDefinedBulkInsertPartitionerClass(RowCustomColumnsSortPartitioner.class.getName()) + .withUserDefinedBulkInsertPartitionerSortColumns(sortColumnString) + .build(); + testBulkInsertInternalPartitioner(new RowCustomColumnsSortPartitioner(config), + records1, false, true, generateExpectedPartitionNumRecords(records1), Option.of(comparator)); + testBulkInsertInternalPartitioner(new RowCustomColumnsSortPartitioner(config), + records2, false, true, generateExpectedPartitionNumRecords(records2), Option.of(comparator)); } private void testBulkInsertInternalPartitioner(BulkInsertPartitioner partitioner, Dataset rows, boolean isGloballySorted, boolean isLocallySorted, - Map expectedPartitionNumRecords) { + Map expectedPartitionNumRecords, + Option> comparator) { int numPartitions = 2; Dataset actualRecords = (Dataset) partitioner.repartitionRecords(rows, numPartitions); List collectedActualRecords = actualRecords.collectAsList(); if (isGloballySorted) { // Verify global order - verifyRowsAscendingOrder(collectedActualRecords); + verifyRowsAscendingOrder(collectedActualRecords, comparator); } else if (isLocallySorted) { // Verify local order actualRecords.mapPartitions((MapPartitionsFunction) input -> { @@ -99,7 +130,7 @@ private void testBulkInsertInternalPartitioner(BulkInsertPartitioner partitioner while (input.hasNext()) { partitionRows.add(input.next()); } - verifyRowsAscendingOrder(partitionRows); + verifyRowsAscendingOrder(partitionRows, comparator); return Collections.emptyList().iterator(); }, SparkDatasetTestUtils.ENCODER); } @@ -130,10 +161,20 @@ public Dataset generateTestRecords() { return rowsPart1.union(rowsPart2); } - private void verifyRowsAscendingOrder(List records) { + private void verifyRowsAscendingOrder(List records, Option> comparator) { List expectedRecords = new ArrayList<>(records); - Collections.sort(expectedRecords, Comparator.comparing(o -> (o.getAs(HoodieRecord.PARTITION_PATH_METADATA_FIELD) + "+" + o.getAs(HoodieRecord.RECORD_KEY_METADATA_FIELD)))); + Collections.sort(expectedRecords,comparator.orElse(KEY_COMPARATOR)); assertEquals(expectedRecords, records); } + private Comparator getCustomColumnComparator(String[] sortColumns) { + Comparator comparator = Comparator.comparing(row -> { + StringBuilder sb = new StringBuilder(); + for (String col : sortColumns) { + sb.append(row.getAs(col).toString()); + } + return sb.toString(); + }); + return comparator; + } } From 31e13db1f0e12e107cc02c60dec3e52a8914a5b2 Mon Sep 17 00:00:00 2001 From: Sagar Sumit Date: Thu, 26 May 2022 11:28:49 +0530 Subject: [PATCH 20/44] [HUDI-4023] Decouple hudi-spark from hudi-utilities-slim-bundle (#5641) --- .../utilities/deltastreamer/DeltaSync.java | 2 - .../hudi-utilities-slim-bundle/README.md | 89 ++++++++++- packaging/hudi-utilities-slim-bundle/pom.xml | 143 ++---------------- pom.xml | 7 + 4 files changed, 109 insertions(+), 132 deletions(-) diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/DeltaSync.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/DeltaSync.java index a4a7e10abc004..0ae72f94b82e0 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/DeltaSync.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/DeltaSync.java @@ -605,8 +605,6 @@ private Pair, JavaRDD> writeToSink(JavaRDD 0; - long hiveSyncTimeMs = 0; - long metaSyncTimeMs = 0; if (!hasErrors || cfg.commitOnErrors) { HashMap checkpointCommitMetadata = new HashMap<>(); if (checkpointStr != null) { diff --git a/packaging/hudi-utilities-slim-bundle/README.md b/packaging/hudi-utilities-slim-bundle/README.md index 58353c403d325..60ee739153fdd 100644 --- a/packaging/hudi-utilities-slim-bundle/README.md +++ b/packaging/hudi-utilities-slim-bundle/README.md @@ -17,6 +17,89 @@ # Usage of hudi-utilities-slim-bundle -Starting from versions 0.11, Hudi provides hudi-utilities-slim-bundle which excludes hudi-spark-datasource modules. -This new bundle is intended to be used with Hudi Spark bundle together, if using hudi-utilities-bundle solely -introduces problems for a specific Spark version. \ No newline at end of file +Starting from versions 0.11, Hudi provides hudi-utilities-slim-bundle which excludes hudi-spark-datasource modules. This new bundle is intended to be used with Hudi Spark bundle together, if using +hudi-utilities-bundle solely introduces problems for a specific Spark version. + +## Example with Spark 2.4.7 + +* Build Hudi: `mvn clean install -DskipTests` +* Run deltastreamer + +``` +bin/spark-submit \ + --driver-memory 4g --executor-memory 2g --num-executors 3 --executor-cores 1 \ + --conf spark.serializer=org.apache.spark.serializer.KryoSerializer \ + --conf spark.sql.catalogImplementation=hive \ + --conf spark.driver.maxResultSize=1g \ + --conf spark.ui.port=6679 \ + --packages org.apache.spark:spark-avro_2.11:2.4.7 \ + --jars /path/to/hudi/packaging/hudi-spark-bundle/target/hudi-spark-bundle_2.11-0.12.0-SNAPSHOT.jar \ + --class org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer `ls /path/to/hudi/packaging/hudi-utilities-slim-bundle/target/hudi-utilities-slim-bundle_2.11-0.12.0-SNAPSHOT.jar` \ + --props `ls /path/to/hudi/dfs-source.properties` \ + --source-class org.apache.hudi.utilities.sources.ParquetDFSSource \ + --schemaprovider-class org.apache.hudi.utilities.schema.FilebasedSchemaProvider \ + --source-ordering-field tpep_dropoff_datetime \ + --table-type COPY_ON_WRITE \ + --target-base-path file:\/\/\/tmp/hudi-ny-taxi-spark24/ \ + --target-table ny_hudi_tbl \ + --op UPSERT \ + --continuous \ + --source-limit 5000000 \ + --min-sync-interval-seconds 60 +``` + +## Example with Spark 3.1.2 + +* Build Hudi: `mvn clean install -DskipTests -Dspark3.1 -Dscala-2.12` +* Run deltastreamer + +``` +bin/spark-submit \ + --driver-memory 4g --executor-memory 2g --num-executors 3 --executor-cores 1 \ + --conf spark.serializer=org.apache.spark.serializer.KryoSerializer \ + --conf spark.sql.catalogImplementation=hive \ + --conf spark.driver.maxResultSize=1g \ + --conf spark.ui.port=6679 \ + --packages org.apache.spark:spark-avro_2.12:3.1.2 \ + --jars /path/to/hudi/packaging/hudi-spark-bundle/target/hudi-spark3.1-bundle_2.12-0.12.0-SNAPSHOT.jar \ + --class org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer `ls /path/to/hudi/packaging/hudi-utilities-slim-bundle/target/hudi-utilities-slim-bundle_2.12-0.12.0-SNAPSHOT.jar` \ + --props `ls /path/to/hudi/dfs-source.properties` \ + --source-class org.apache.hudi.utilities.sources.ParquetDFSSource \ + --schemaprovider-class org.apache.hudi.utilities.schema.FilebasedSchemaProvider \ + --source-ordering-field tpep_dropoff_datetime \ + --table-type COPY_ON_WRITE \ + --target-base-path file:\/\/\/tmp/hudi-ny-taxi-spark31/ \ + --target-table ny_hudi_tbl \ + --op UPSERT \ + --continuous \ + --source-limit 5000000 \ + --min-sync-interval-seconds 60 +``` + +## Example with Spark 3.2.0 + +* Build Hudi: `mvn clean install -DskipTests -Dspark3.2 -Dscala-2.12` +* Run deltastreamer + +``` +bin/spark-submit \ + --driver-memory 4g --executor-memory 2g --num-executors 3 --executor-cores 1 \ + --conf spark.serializer=org.apache.spark.serializer.KryoSerializer \ + --conf spark.sql.catalogImplementation=hive \ + --conf spark.driver.maxResultSize=1g \ + --conf spark.ui.port=6679 \ + --packages org.apache.spark:spark-avro_2.12:3.2.0 \ + --jars /path/to/hudi/packaging/hudi-spark-bundle/target/hudi-spark3.2-bundle_2.12-0.12.0-SNAPSHOT.jar \ + --class org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer `ls /path/to/hudi/packaging/hudi-utilities-slim-bundle/target/hudi-utilities-slim-bundle_2.12-0.12.0-SNAPSHOT.jar` \ + --props `ls /path/to/hudi/dfs-source.properties` \ + --source-class org.apache.hudi.utilities.sources.ParquetDFSSource \ + --schemaprovider-class org.apache.hudi.utilities.schema.FilebasedSchemaProvider \ + --source-ordering-field tpep_dropoff_datetime \ + --table-type COPY_ON_WRITE \ + --target-base-path file:\/\/\/tmp/hudi-ny-taxi-spark32/ \ + --target-table ny_hudi_tbl \ + --op UPSERT \ + --continuous \ + --source-limit 5000000 \ + --min-sync-interval-seconds 60 +``` diff --git a/packaging/hudi-utilities-slim-bundle/pom.xml b/packaging/hudi-utilities-slim-bundle/pom.xml index 60f0af9d64f07..993e2ad7fd912 100644 --- a/packaging/hudi-utilities-slim-bundle/pom.xml +++ b/packaging/hudi-utilities-slim-bundle/pom.xml @@ -77,7 +77,7 @@ - true + true META-INF/LICENSE @@ -92,10 +92,7 @@ org.apache.hudi:hudi-common org.apache.hudi:hudi-client-common - org.apache.hudi:hudi-spark-client org.apache.hudi:hudi-utilities_${scala.binary.version} - org.apache.hudi:hudi-hive-sync - org.apache.hudi:hudi-sync-common org.apache.hudi:hudi-hadoop-mr org.apache.hudi:hudi-timeline-service org.apache.hudi:hudi-aws @@ -136,13 +133,6 @@ org.apache.kafka:kafka_${scala.binary.version} com.101tec:zkclient org.apache.kafka:kafka-clients - - org.apache.hive:hive-common - org.apache.hive:hive-service - org.apache.hive:hive-service-rpc - org.apache.hive:hive-metastore - org.apache.hive:hive-jdbc - org.apache.hbase:hbase-client org.apache.hbase:hbase-common org.apache.hbase:hbase-hadoop-compat @@ -178,10 +168,6 @@ com.beust.jcommander. org.apache.hudi.com.beust.jcommander. - - org.apache.hive.jdbc. - ${utilities.bundle.hive.shade.prefix}org.apache.hive.jdbc. - org.apache.commons.io. org.apache.hudi.org.apache.commons.io. @@ -205,10 +191,6 @@ org.apache.hadoop.hive.metastore. ${utilities.bundle.hive.shade.prefix}org.apache.hadoop.hive.metastore. - - org.apache.hive.common. - ${utilities.bundle.hive.shade.prefix}org.apache.hive.common. - org.apache.hadoop.hive.common. ${utilities.bundle.hive.shade.prefix}org.apache.hadoop.hive.common. @@ -217,10 +199,6 @@ org.apache.hadoop.hive.conf. ${utilities.bundle.hive.shade.prefix}org.apache.hadoop.hive.conf. - - org.apache.hive.service. - ${utilities.bundle.hive.shade.prefix}org.apache.hive.service. - org.apache.hadoop.hive.service. ${utilities.bundle.hive.shade.prefix}org.apache.hadoop.hive.service. @@ -344,116 +322,27 @@ org.apache.hudi - hudi-client-common - ${project.version} - - - org.apache.hudi - hudi-spark-client - ${project.version} - - - org.apache.hudi - hudi-hive-sync + hudi-utilities_${scala.binary.version} ${project.version} - javax.servlet - servlet-api + org.apache.hudi + hudi-spark-common_${scala.binary.version} + + + org.apache.hudi + hudi-spark_${scala.binary.version} + + + org.apache.hudi + ${hudi.spark.module}_${scala.binary.version} + + + org.apache.hudi + ${hudi.spark.common.module} - - org.apache.hudi - hudi-spark-common_${scala.binary.version} - ${project.version} - provided - - - org.apache.hudi - hudi-spark_${scala.binary.version} - ${project.version} - provided - - - org.apache.hudi - ${hudi.spark.module}_${scala.binary.version} - ${project.version} - provided - - - org.apache.hudi - ${hudi.spark.common.module} - ${project.version} - provided - - - org.apache.hudi - hudi-utilities_${scala.binary.version} - ${project.version} - - - - - ${hive.groupid} - hive-service - ${hive.version} - ${utilities.bundle.hive.scope} - - - - ${hive.groupid} - hive-service-rpc - ${hive.version} - ${utilities.bundle.hive.scope} - - - - ${hive.groupid} - hive-jdbc - ${hive.version} - ${utilities.bundle.hive.scope} - - - - ${hive.groupid} - hive-metastore - ${hive.version} - ${utilities.bundle.hive.scope} - - - - ${hive.groupid} - hive-common - ${hive.version} - ${utilities.bundle.hive.scope} - - - - org.apache.htrace - htrace-core - ${htrace.version} - compile - - - - - org.apache.curator - curator-framework - ${zk-curator.version} - - - - org.apache.curator - curator-client - ${zk-curator.version} - - - - org.apache.curator - curator-recipes - ${zk-curator.version} - diff --git a/pom.xml b/pom.xml index d898d34d35e43..1188ec620aa39 100644 --- a/pom.xml +++ b/pom.xml @@ -99,6 +99,7 @@ 2.8.1 5.3.4 2.17 + 3.0.1-b12 1.10.1 5.7.0-M1 5.7.0-M1 @@ -556,6 +557,12 @@ jersey-container-servlet-core ${glassfish.version} + + org.glassfish + javax.el + ${glassfish.el.version} + provided + From 8d2f00904882b9946331bfb325c0033f72aa29fe Mon Sep 17 00:00:00 2001 From: komao Date: Thu, 26 May 2022 20:21:28 +0800 Subject: [PATCH 21/44] [HUDI-4124] Add valid check in Spark Datasource configs (#5637) Co-authored-by: wangzixuan.wzxuan --- .../apache/hudi/config/HoodieIndexConfig.java | 11 ++++++ .../hudi/common/config/ConfigProperty.java | 35 ++++++++++++++----- .../hudi/common/config/HoodieConfig.java | 1 + .../org/apache/hudi/DataSourceOptions.scala | 23 +++++++++++- 4 files changed, 61 insertions(+), 9 deletions(-) diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieIndexConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieIndexConfig.java index dbd45b9738285..0cef5550af8b7 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieIndexConfig.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieIndexConfig.java @@ -44,6 +44,13 @@ import static org.apache.hudi.config.HoodieHBaseIndexConfig.TABLENAME; import static org.apache.hudi.config.HoodieHBaseIndexConfig.ZKPORT; import static org.apache.hudi.config.HoodieHBaseIndexConfig.ZKQUORUM; +import static org.apache.hudi.index.HoodieIndex.IndexType.BLOOM; +import static org.apache.hudi.index.HoodieIndex.IndexType.BUCKET; +import static org.apache.hudi.index.HoodieIndex.IndexType.GLOBAL_BLOOM; +import static org.apache.hudi.index.HoodieIndex.IndexType.GLOBAL_SIMPLE; +import static org.apache.hudi.index.HoodieIndex.IndexType.HBASE; +import static org.apache.hudi.index.HoodieIndex.IndexType.INMEMORY; +import static org.apache.hudi.index.HoodieIndex.IndexType.SIMPLE; /** * Indexing related config. @@ -57,7 +64,10 @@ public class HoodieIndexConfig extends HoodieConfig { public static final ConfigProperty INDEX_TYPE = ConfigProperty .key("hoodie.index.type") + // Builder#getDefaultIndexType has already set it according to engine type .noDefaultValue() + .withValidValues(HBASE.name(), INMEMORY.name(), BLOOM.name(), GLOBAL_BLOOM.name(), + SIMPLE.name(), GLOBAL_SIMPLE.name(), BUCKET.name()) .withDocumentation("Type of index to use. Default is Bloom filter. " + "Possible options are [BLOOM | GLOBAL_BLOOM |SIMPLE | GLOBAL_SIMPLE | INMEMORY | HBASE | BUCKET]. " + "Bloom filters removes the dependency on a external system " @@ -141,6 +151,7 @@ public class HoodieIndexConfig extends HoodieConfig { public static final ConfigProperty BLOOM_FILTER_TYPE = ConfigProperty .key("hoodie.bloom.index.filter.type") .defaultValue(BloomFilterTypeCode.DYNAMIC_V0.name()) + .withValidValues(BloomFilterTypeCode.SIMPLE.name(), BloomFilterTypeCode.DYNAMIC_V0.name()) .withDocumentation("Filter type used. Default is BloomFilterTypeCode.DYNAMIC_V0. " + "Available values are [BloomFilterTypeCode.SIMPLE , BloomFilterTypeCode.DYNAMIC_V0]. " + "Dynamic bloom filters auto size themselves based on number of keys."); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/config/ConfigProperty.java b/hudi-common/src/main/java/org/apache/hudi/common/config/ConfigProperty.java index 9612914303588..934803d8d315e 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/config/ConfigProperty.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/config/ConfigProperty.java @@ -24,7 +24,10 @@ import java.io.Serializable; import java.util.Arrays; +import java.util.Collections; +import java.util.HashSet; import java.util.List; +import java.util.Set; import java.util.function.Function; import java.util.Objects; @@ -48,19 +51,22 @@ public class ConfigProperty implements Serializable { private final Option deprecatedVersion; + private final Set validValues; + private final String[] alternatives; // provide the ability to infer config value based on other configs private final Option>> inferFunction; ConfigProperty(String key, T defaultValue, String doc, Option sinceVersion, - Option deprecatedVersion, Option>> inferFunc, String... alternatives) { + Option deprecatedVersion, Option>> inferFunc, Set validValues, String... alternatives) { this.key = Objects.requireNonNull(key); this.defaultValue = defaultValue; this.doc = doc; this.sinceVersion = sinceVersion; this.deprecatedVersion = deprecatedVersion; this.inferFunction = inferFunc; + this.validValues = validValues; this.alternatives = alternatives; } @@ -95,33 +101,46 @@ Option>> getInferFunc() { return inferFunction; } + public void checkValues(String value) { + if (validValues != null && !validValues.isEmpty() && !validValues.contains(value)) { + throw new IllegalArgumentException( + "The value of " + key + " should be one of " + + String.join(",", validValues) + ", but was " + value); + } + } + public List getAlternatives() { return Arrays.asList(alternatives); } public ConfigProperty withDocumentation(String doc) { Objects.requireNonNull(doc); - return new ConfigProperty<>(key, defaultValue, doc, sinceVersion, deprecatedVersion, inferFunction, alternatives); + return new ConfigProperty<>(key, defaultValue, doc, sinceVersion, deprecatedVersion, inferFunction, validValues, alternatives); + } + + public ConfigProperty withValidValues(String... validValues) { + Objects.requireNonNull(validValues); + return new ConfigProperty<>(key, defaultValue, doc, sinceVersion, deprecatedVersion, inferFunction, new HashSet<>(Arrays.asList(validValues)), alternatives); } public ConfigProperty withAlternatives(String... alternatives) { Objects.requireNonNull(alternatives); - return new ConfigProperty<>(key, defaultValue, doc, sinceVersion, deprecatedVersion, inferFunction, alternatives); + return new ConfigProperty<>(key, defaultValue, doc, sinceVersion, deprecatedVersion, inferFunction, validValues, alternatives); } public ConfigProperty sinceVersion(String sinceVersion) { Objects.requireNonNull(sinceVersion); - return new ConfigProperty<>(key, defaultValue, doc, Option.of(sinceVersion), deprecatedVersion, inferFunction, alternatives); + return new ConfigProperty<>(key, defaultValue, doc, Option.of(sinceVersion), deprecatedVersion, inferFunction, validValues, alternatives); } public ConfigProperty deprecatedAfter(String deprecatedVersion) { Objects.requireNonNull(deprecatedVersion); - return new ConfigProperty<>(key, defaultValue, doc, sinceVersion, Option.of(deprecatedVersion), inferFunction, alternatives); + return new ConfigProperty<>(key, defaultValue, doc, sinceVersion, Option.of(deprecatedVersion), inferFunction, validValues, alternatives); } public ConfigProperty withInferFunction(Function> inferFunction) { Objects.requireNonNull(inferFunction); - return new ConfigProperty<>(key, defaultValue, doc, sinceVersion, deprecatedVersion, Option.of(inferFunction), alternatives); + return new ConfigProperty<>(key, defaultValue, doc, sinceVersion, deprecatedVersion, Option.of(inferFunction), validValues, alternatives); } /** @@ -156,13 +175,13 @@ public static final class PropertyBuilder { public ConfigProperty defaultValue(T value) { Objects.requireNonNull(value); - ConfigProperty configProperty = new ConfigProperty<>(key, value, "", Option.empty(), Option.empty(), Option.empty()); + ConfigProperty configProperty = new ConfigProperty<>(key, value, "", Option.empty(), Option.empty(), Option.empty(), Collections.emptySet()); return configProperty; } public ConfigProperty noDefaultValue() { ConfigProperty configProperty = new ConfigProperty<>(key, null, "", Option.empty(), - Option.empty(), Option.empty()); + Option.empty(), Option.empty(), Collections.emptySet()); return configProperty; } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieConfig.java b/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieConfig.java index c77e292b4775f..1aa0cfba5bc13 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieConfig.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieConfig.java @@ -57,6 +57,7 @@ public HoodieConfig(Properties props) { } public void setValue(ConfigProperty cfg, String val) { + cfg.checkValues(val); props.setProperty(cfg.key(), val); } diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DataSourceOptions.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DataSourceOptions.scala index 36dd07f28a180..ac4d0e5794392 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DataSourceOptions.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DataSourceOptions.scala @@ -56,6 +56,7 @@ object DataSourceReadOptions { .key("hoodie.datasource.query.type") .defaultValue(QUERY_TYPE_SNAPSHOT_OPT_VAL) .withAlternatives("hoodie.datasource.view.type") + .withValidValues(QUERY_TYPE_SNAPSHOT_OPT_VAL, QUERY_TYPE_READ_OPTIMIZED_OPT_VAL, QUERY_TYPE_INCREMENTAL_OPT_VAL) .withDocumentation("Whether data needs to be read, in incremental mode (new data since an instantTime) " + "(or) Read Optimized mode (obtain latest view, based on base files) (or) Snapshot mode " + "(obtain latest view, by merging base and (if any) log files)") @@ -65,6 +66,7 @@ object DataSourceReadOptions { val REALTIME_MERGE: ConfigProperty[String] = ConfigProperty .key("hoodie.datasource.merge.type") .defaultValue(REALTIME_PAYLOAD_COMBINE_OPT_VAL) + .withValidValues(REALTIME_SKIP_MERGE_OPT_VAL, REALTIME_PAYLOAD_COMBINE_OPT_VAL) .withDocumentation("For Snapshot query on merge on read table, control whether we invoke the record " + s"payload implementation to merge (${REALTIME_PAYLOAD_COMBINE_OPT_VAL}) or skip merging altogether" + s"${REALTIME_SKIP_MERGE_OPT_VAL}") @@ -210,6 +212,23 @@ object DataSourceWriteOptions { val OPERATION: ConfigProperty[String] = ConfigProperty .key("hoodie.datasource.write.operation") .defaultValue(UPSERT_OPERATION_OPT_VAL) + .withValidValues( + WriteOperationType.INSERT.value, + WriteOperationType.INSERT_PREPPED.value, + WriteOperationType.UPSERT.value, + WriteOperationType.UPSERT_PREPPED.value, + WriteOperationType.BULK_INSERT.value, + WriteOperationType.BULK_INSERT_PREPPED.value, + WriteOperationType.DELETE.value, + WriteOperationType.BOOTSTRAP.value, + WriteOperationType.INSERT_OVERWRITE.value, + WriteOperationType.CLUSTER.value, + WriteOperationType.DELETE_PARTITION.value, + WriteOperationType.INSERT_OVERWRITE_TABLE.value, + WriteOperationType.COMPACT.value, + WriteOperationType.INSERT.value, + WriteOperationType.ALTER_SCHEMA.value + ) .withDocumentation("Whether to do upsert, insert or bulkinsert for the write operation. " + "Use bulkinsert 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.") @@ -220,6 +239,7 @@ object DataSourceWriteOptions { val TABLE_TYPE: ConfigProperty[String] = ConfigProperty .key("hoodie.datasource.write.table.type") .defaultValue(COW_TABLE_TYPE_OPT_VAL) + .withValidValues(COW_TABLE_TYPE_OPT_VAL, MOR_TABLE_TYPE_OPT_VAL) .withAlternatives("hoodie.datasource.write.storage.type") .withDocumentation("The table type for the underlying data, for this write. This can’t change between writes.") @@ -308,7 +328,8 @@ object DataSourceWriteOptions { Option.of(classOf[NonpartitionedKeyGenerator].getName) } else { val numOfPartFields = p.getString(PARTITIONPATH_FIELD).split(",").length - if (numOfPartFields == 1) { + val numOfRecordKeyFields = p.getString(RECORDKEY_FIELD).split(",").length + if (numOfPartFields == 1 && numOfRecordKeyFields == 1) { Option.of(classOf[SimpleKeyGenerator].getName) } else { Option.of(classOf[ComplexKeyGenerator].getName) From 85962ee55d098d457a598bc19d5597ff1b361ed5 Mon Sep 17 00:00:00 2001 From: YueZhang <69956021+zhangyue19921010@users.noreply.github.com> Date: Thu, 26 May 2022 23:03:09 +0800 Subject: [PATCH 22/44] [HUDI-3963][RFC-53] Use Lock-Free Message Queue Disruptor Improving Hoodie Writing Efficiency (#5567) Co-authored-by: yuezhang --- rfc/rfc-53/DisruptorExecutor.png | Bin 0 -> 208870 bytes rfc/rfc-53/rfc-53.md | 160 +++++++++++++++++++++++++++++++ 2 files changed, 160 insertions(+) create mode 100644 rfc/rfc-53/DisruptorExecutor.png create mode 100644 rfc/rfc-53/rfc-53.md diff --git a/rfc/rfc-53/DisruptorExecutor.png b/rfc/rfc-53/DisruptorExecutor.png new file mode 100644 index 0000000000000000000000000000000000000000..6cf3323b8957effe84c19686238b63c6430dcc2f GIT binary patch literal 208870 zcmeFZ2Ut_vwl<8|LBxWhfPfVc0qG(oHb6i?q)Q1(uc3DmJJLjYCrXtry%Q7xsgd3Z zHIW)1KoUs#gSgeR@80LQ&pqz@J^z8{Av0yInVC7~c*i^5G1l_Att3lF!$dpIs>Dyn@}=GU*^mb-qP{kA>G#N5i5it46MnC5OR)rO<- zI>9erexiPTb8qF|Q>r!Q0Z_yG_r=mL5`fLx>C2DDvHYPGO(`c zX~|wH&(q4MTDCuU^GQdkDTH%D;<;1u#VgNyIBio7p*2Pv&)Uou%RM)|?sG`F7W9ylnTSXKNfBQaNqKJ~)5Ur{?$H?V3ui zj>3zQmal?&MI8=sW2PAk4+Px`u8CWOgd-seHZMG@z3HsfUJIR43Ib$~#?5jC(cF4k zq8iJ3BJqvz2gy@adGl1S7+y=9QZWx?!Wda;0jeMLK84fyJn20)cF|7BF;pHJf0AbW z%05ln%X*>pau0f`#?De}n^iQLWH`#lHZtCgIcc1)wuaF)hXJ4F$S%#^2=^st-L6(y zG-TTO{)y3Zt^+!ge!&SRyC3vwho4|e%R8>lZX`?Jdvnsx7njBU;_&mD$6*95@H3$c zO2)ugZsw1S1Lqm+64Y@9@fS~C(`K6$3LIbqH18}we<1(RZr2@ob}HLT7j~X}-8h&V zL2Y-x!Z%emn2{wl)=%|}Snhd`!D}xMXMfVUGRxZ(sM%5*vNSTLwlsb`xT1H5ajfgr zqb`n_0<4~ApHo{MppLw4$F$@$rbhq+|mQ!(DWdhP%fN0^C$j6M1JG5ztY z+i-_ZZF@=AN;}VXFCwc`j*+oz+vZ&8Xe|=dnD7r|Q;-kN@`oO~=Fk4@;GErQ2`X$&;)w7b5zRqn z?D3D|<&vXfufrCe*oJ(PDG}FJ<2-R_^4OczeKpGeG2zWh;EGCJ zw3HTTb>!o1@+@0A+}ms7wu04O=RM-yaUXaM0<=yT+~;`fJ}cmS>)~$Ao_tc5@CZ*f zAl~ZZ={0Xac4KNnut0=G#Qq4r#$1Va%M!J9hRQ+BFMS1`y4^ZTxV$Hlg&*#AsiUj5 z>j+hCu?&9Az5X-#8N0t9!s#h`+X4!^vLX!bk$zVRc0D*qg2QI40&9AnNKv5_TCe+<(8NS(BUOIn0|;N|%V#tBaQL!}Q2PPqit-`u@) z6z_kZ^GKJp;%#Qf9n%4MpKc0mi+gU8pxQ?5c(ROzN0z&-(u~LbQ00^18v~y-%|v%y zbn<@~WvCP>T32`ft=fg{A@?2L-p|~Z@xkKlE?jJmp|uvr>Ryg9$52Q)| zlJlLw=(6aj=#0COchjS_vTkSHvJULGE3~xW>vV6pYh; zp_0Mcm!}pQe5E0wIW<((g42|fo)g{K)y324)mfkJ*NIAt$Y8q-$RTMF5374mUolqY zxINi9*{RogZdz|zoGHNMI&DI7`;u9cxq5+WfouUQUD>nzU9yjJ_%#!+C2EXE*I6p& zr4*i1a0oVeXcC1y*}@$ich`uAnWyNersWHbr`;-919yk|9p{4j`TI-zn+mXIz)#?` z>+QL1d%H$wE_C)~JIJHBP9*BI_c^k9ZS&eq+*hd=@y0Tvsja1*r)?;8AYmXmH4&8^ zktC{@a}6D-&%E8R_u?ggmO7fct93kere24=!mF-8(!c>EVPfE}nthM*}xT|r25SKrFr-iQ^$D92@}VBD#4&Wru@Sukc%sk7fkl&$GS!L z8QWN~-s2TTm<-<^w!CK=9YV-Mp-6xKkK3h?oYa&cOJ}; ziDh1FsjWF%Z|vPf47z+;Ht+&V-^Hb@jK|#Y!uKoBm|J+7t+RD12bf)CT^_ERV8vM- z1Nm9~B%>v(+}+*eB`qbyD>5tED@5FfR=ZbTtmq@XhP_7)&kFXWwxL3DQbFT+x$*Mk*82I>NiiFynsGbA#o$a|gV66SI;wZP&&En3zWD=$Re78>ooG2qj|p&w}^ z#AVu^+YlX68bY6ZusyhAHOXtV&X&7$XexF&U`kIy1aty@1n?a83~_RSF_cE%2xl5< zQqjB&*Ny8txsM@yueh{T+pe~l%;%Xr+HCoTc`nA-$up=~xO$UfljQmCgQkAU%KUKx zUh)|qRPLNHkC_&Cn(O2>PjXD%7o-^4g_?Q~U2C!z$DhKF6CMzJ3GwD~9s3n~;(6o2 zZqI7`lLhq|o*W@a9ELW;^hq>%b`!J%P4i7aF^`+&pdAuD_Ve#c=P>V4bMeYiP)Ty} zwTSoa@3FMOj6xTLv@d~1JETYb(*1;dlcb9o*LjBXb^uS^ZO`kpo?1&r_~L6B6?Ekr z<*P1?^OWN{K0+HI(>n3F6~<3dzAfB7+_l`GEE?Uya<%e;SIbk={27)#7ke0T++fa2 z{j|pO#yZ_9-3>j3VX|R8ImLHj_QVLf_jID%kGNMsLc=AFHV!*4cF(cg~a54T! zmsw#&nIZZ~r6%_2Y(|bLuDvt;)2h=tfiI-`2S<8rCp1w&7vX484VDh^qOnVpxcB2;95hOfqH@ufl}+M3 z`z1>8q%^x!n^;zlvdLQ$Q5e&j*7(cymtDpx$oM+iD#jQ3%T-z~4VXJ~#(_<;U<_{V zrk|<8+>spa6-}>fk%AW%l3<>N{)U;8J@Da?!4gNoHItdF4>~S3$|Xb2-VoH}P;5D9 zeiAWL2|#Zfy}Eck%iB26pG}=)LtZC|z>5q6ud@QdhsYBo)!`0JxDmXnfX`FOLx1)C ztaO(`Pej0pdnYu>$H=nl+zlDStqFia@*u8%eq|K)h*UNxoSi`oSXM*~7^C+g=G@Vw zs#5Wy^)~N8`bsuGk9{7_kR_7iyu%WTv9hD;WmPDokleq(vo!FyYOqRbuV&*M`AD_F zbwL3t*K%5_yEIget~QDOn(;f=cdEn#*AL>9wcPY)8I`Fl3{r6crlQ+ysYGJjUW)Ib zs^#;%cHqj;;I0^XqvJN&^1SCI_8xR!TN{p|x?({!l2MS8g^qjcZTTUgUs2+dy z$h9Y?&$6!4ICeY$(C<4qiC^|+@m{@h@S*szell%RZ=i<3aJoGu{O~o_k~2|Mq&iP| zzK3eZHYTc_lxN#0|5DqSzde`P#!0pP%lp(+RNm%PJH8&HL}@qvUQ+%y&e?3Y2S20Q zO?h>Q^6&bJ`nRL^%D>wF+w<+`D4$VXRlP1JM`=|J?Tw9X9LzwDlWZ&QlsER;-qd!W zqB?qZTv>de*Z(TM7S@YgA0^K*}b+xwL=!Z(e^)ltDwXx$pc2{dF z8;8rTlBd2LahdXb<88iE>|YLXw3IxhrFfhDI>_FbU4-`{@5NJ6H0}nv+8use)#o73BHY={=-yk&hwXdDHAP4Bf&Rk$Ls^Z|Qg+$K-?k9S1LtN-dA`m3iO>O?WGbpFRB|`2s=02P9g+-Yl?y&MO5MjO zSA>T6S%}aFE8d43JFV7UuV%Hg@PV1C3g`{JOg!9=_9_vFhmo;8OAs;<4IzW@IS`_y z+Zd#kgn^O?E0nj^A^RC?$und!2||WpAvc$Fbp2(65!+1{&%<`MpKd@u%y$h#P)`L- zV5_w&8bJ@2Z!UR^T+p8^zT$ocFKK%;XGw=$GMK=6$xjo{^O98#OdngDK{$Fd@CzF#2HtFB(4%Ud4Ws~Ow-AKP)cQFX+FAs(G{EVFn7~%HAQi#P$>{L|5i&OZh>vY0{8+3Rs-VgYPe^W~*$0i@JGro+IUFAC1m(~vaceU5sz3&)4Oy#Ay z+4&MWb}g3eZ)y=QDO*5mB+ZZdknlum$pO^vzo`{Hv$e#3&r<&vE^$L(PEHOK69^+s zVKA7Sz`#Htc{lka(Dv@zhBw#N&-8tv7R!0KJ z$Ont5o^!}w?&rD}}4JUs3)ox}P!6#)MPCj;4T$w7aQ^0%eoBR+s6+&<% zCk%lf{o<-IQin$%A`Rv~`+s*)*6%slyRAIn-;@vB${_qI`RDijZplHvis=2m-!0;| zCF=g(`ub@^y+-wy%k}A$wr;TV_uOb_3s3$&JlVSE$uGKe z{`c2cZe17otBBt3``sdbzb9zF>h*siP%n+o%*>z&<2jHu0)a3yK0e+*@S)^*nZk(+F8iMm^&?iMTfJxBj{U;pR1J&Do7g_=W7KWE99u8dHLLiqEZD<3vGw{t;Z%-(m=;bt&3 z?yQ|yEl%u=+%`gRN$hVyEsEOtBNsuIFT?Z{(JSb*j1C+W)a@T}ksH~4P#;!K& zH3gKKO2Y@mjhh>f8=AQ1MNN{Wi(D=Qa|BRP?_q!N<>3MUcrB2bqdq{}^nK)0OntLE zt$&|w9Dn}kM1gieHop#Uw2+R~MU$dIvt}!O2@ySe5>dP8B-T{`X&j+v;yh?4>(=ky zQ0bK@75`wg$EJV88>+#MMLn$G!zgEn1@8)a`rj-2WcW<9uS8>Z{XZ(RgOrA#d*+4u>Mh}0|4Q$;sXFazst{| z(TzIKp%Kz2bPp4=%dTUySd37e{|Q=T zq__}L_XbG1S4Wr*6h!(0S|U9(+8K^qnn?OSWMpLZ`v!i@8=ktB{W>pV>LuP(*`3Ft z{V3lwYxy4U?YuVf2lI(DPbN6w@OlL*gn~Py%LD>%KPUJu7i36xHBqGza;wQ~etQgM zrf({#ZQp(SX9%7{7kZ4@qh6IZIe$2#6o)d7cG31orZk zQC_h8&<)%Rlafy4zzN@<$q|Ym@WR~cV!ej^#ZYDD4)ylYiEF!qIDXsZRMeJP)iSho z&8&W?C=OoUY@tFkx2dF-+2?FvF6vU0!u+BUP=<)z zofjbF>w9f2*rRHKReAL}J_4+pVL77NbrbReSf=enM@VOyf-fqFW@NX8A$pS|^cw+l z*^L;Mucq;3#|7^sx%ON}zSj{EVI02eIZ*THJ2p^J%TWgo&F;tawtz%+vRy#!?hL0l zVdJ-hlJ?q4r#;Y|-#a0a?SlX%2p~+XGjs`4*mdw+h0C@5T3xqU(q|A9BI~ zbbm4W5Qrd}g-$P2w@&&T`fX$l&jR}y1Js)b*D3ZuNSDp%b3g(GMaz$ZH4^$xgIfk? z@8Yz5OagJj4*8Y=zKAeLG(QK~zCYGQlJ3rG@51xnemwhwE4&ZuttH{J>yd6I3A4|4X5jX2F))nU``Yk9K-OnSe7us?gbU};F8_wG{;3R1g|o|gmOlw zvk7LQOyM~dtwhF*joEY-RyluRI<*uligm@M(HmQz#N>p_gjFlos_8rd@&ufJ2=z@X z+5@}Gq)rVuqH5hXy}DjAC+RR&tBP#4G=G^cdGpM7R5|eR%-bFjOf#wcR^)DLiK8ci z5KtMPM>}yl-4f&o2{+zR6CO#lodK&P7;FPyjW;_4<5YP|1!Gqh@&Sf&YCQnHB1PX+HfkAjM5D#Y`P{Ana5{CN-rvGRq(N z=7lITYINB3ozh|C*}%Aq;}5}HU`YQ9W&q*N4tsb6WiqNVvk`1&ESnSA!BT5WhvAz& z+%(2`m8{xfFu9fD-7+>6&hiOf+DvMNRd|&vV*^-*xh`t`hn4CC( zF=@FtLC1IJkJh5$SACS(KPF;c3NB%{D1lzbtgHTmU;3k6zNo6|M!Sf~d-E8VmR9j} z!*y_>Pge3pu2)msPm#Tor^-=w=9`;VBWhd1RB1rk0JSGTXhPdz0#oja=KCDww$FRP z0ljP7qdlo+%Sj5b0rXT<4&l+jtnlUAnNPlN$?EfmUJfi)j_o~cGXC^&M8p_2b*_4` zhEJLu>)9X~KdTV20dFDwUv__=4B^IW{3D=)xb|THp@7G*(0b zLLY8_^Aln~a5kZJZLUCUHI~bveyP;{NZLC-AL=JR;@h4*%>gj7O@To(p^5b{QjQ*4 zS;hc^>}S#VYuW)ak1gQN3D(|H2^H#Fg=w2VXd%1GEK{21jQtmrqh*mW_Qj_@N$0A` z*(GzlPI<4o>N6mnzkomVPzlH0@aN z*H$BB-~)UCGdvef;u`n`VK{xWVON)d58@}k;b117x_%phZo9HQd zw}&qY0~o^E-&QqAtx$EFN2%nEi+3{{8(V7J%xb`cHLN9xN&_G zE2Qb~6+FN*o~1cD!Z_h`Zqe`1PTB?9r5hnQ>8}Rgpg!d?WG#sS0(8u?E3kUj#TO{m=v{2prnO&@xogU%8Ig{U+8O}|+ z^PT#n3sSfPsS1b_5UiZ9>ea4jDCu*`u*=6hu5VeD4q>4E0fHR#MwVhso0B1t{ES1F z471l2T*y@SJgu55R`&{;z-#M1aduf9U0g?**f{eLGH>L|68>y?XaU@9vj<_61GBym z1xerd`oY~ISl7IUjA=gsdRco;ED6a<51Y6?omoxzXIk!J`hqxqNe%;uaCFG1%<|Kj zlC)baaeH5bjM~k|g{BMGdq0Y`S`CmnvkBJB%FeZdldG z7$cn4Jgqs8U_`Mh@23XOrJMre%y;7OUBA)tKcJDHHyGr3+fWC#-VL|pkZ|DAgP+m+ z8s=P@;q}eR5JuOUTWP6$QKs>C38RfFQCE)d`Ua{7N_#(srL=NpkFAAdq-x@w0V@YF z%`miZ)TEa^6!KAqB8kp^njnGib=Ds~iv;GE-v7Z2zR&aT7o>b?W6YkHRQCK7V=%H_ zDmP8$XOo!Icj#l=&d#Iicl1S+R}s{7b=W4p@2zX=54@D{N72NH=lC9iehR4Ew}Y587Ef?jE@{=!JmjqfqjE14@c4cHdHlol>cr7krgdQjnFi5TH{ zOLTI7EMyRf;)?10%NwjKraq}#8_XReflGuFgoQaw$9-W!?qX}rSZHe|IiB#k)xpz`H>S}|OAo)+&#P^u=cVL2HEUez9 z_7kZExx+9I4JJ%BTjoVgL^QT3e7kep#dY+1>s4xRKZYMrJYa`XiEASZL}fTbVPk7o ztZQRh-pdis35${6FiT?5^|D?PcGWpHBA>@_Q?b+O3mGA8wTm5MUFxHn9EpD8sSfEG z?{k!780){W*#GObdFc^MJ2cpmeKHcvD62G^nM8odGwi&D~>*WVYoA{soF0(LKDmALUuMcEgBo<_vyClr1pVYrjYf#_#tM)f0AnrJ^{*@AxJvEEDJ@vY;O}Kw;R04bts#Uas2-LEtu)x_3g=e_y z#@>rvDVwY^L(*}22j&mr+3ytOE=({dN(6dI5YEWO%yUeTycjB%-=Q&Rq3rdF^9Mv= z&#z7vZH-K9)iMv~>CvM*zq$(QwQQeVdALgFA)(vtp9qd z(nWHo+VC|(#f52h@c)$A-Yk~S7sb1EG;lZH${{jx>hLj%g24SrZVL(gD!Fu}FvN+K#Sc3HbF&M-mjVBP zZ+wxh^ftmcJh6$B{1`t&(5K@gQMjc?F{VFUfbYn-$9t_ zp*s5)_dTMl5zS5NB%uidR+nw0iV%eUtl4Ql-+sD1XSzc%#nrs~lVrRg4j|~P1O{Y4 za|Tw;7iDFzld$VjFmNQ|mdIOb@OG94z6FJI%v#%E+g?mC;?9Q9Wq#V2`|Y!j`Rq5H zNg>6VTs=ONvFS`4>-p7c3A9rm9B{P(*N)(xI*y4E#$;a85UyQbWC7bH-j;4gJD4g* zspiDe(7kS)BfN4$T6-u*<448q-qa=R`9_^6KWT|3latg)e#n+ZQNZq%tyIGYZQipG zKI}zb%Ei#5<%QpE&i#{Zu;_+Y`^Nt)650o2I%a$fa8^z6I9T{%qwSX%djFoEo%?1d zSUyauwYFIUOnx}c$4wh12#Qz5TSb?iR(iY#^k?XDWGc7^A6OV&>%>N ze$$fu7?CL*Vu}-1iP75))AlnTBhMVArK!Lt$8XTSNKi&1OH&Aj{ z?{44($)3Ki{^8PS zk7LnbG3!Q1V-I2dxGq30So=W=KLG|aS~CE$LSg#>Fh7z0FQn;o@H(HyzY|;DgGG%g zd63rt+|5u@_dGC=ZP6cZlA2Z5HE{Lh4}@fofA|-SKMgKP=#sI{eOH2S2EawfvE+51 zQB0u7V-$%v|0a+_16LTuk1On`p$Al6nTM;onM#qDNmcR$&+}`?!0z0@vtX$sUi$!* zVL5pl)9$&3t*0OiJB6-BQ4oeo)%yHqO!=(A0^|hOf(OltyYU^tb~=;*W|2z zN;};A2brGlikb$IWa|MZ4EQv4&8*R2hn?TE(H;AVt27Qn zz-UCgM~*czN>sur7y&4j80^9nhhJ}d(|+h87E4PRw=F-pUy+Hh&_Odnl6Z+VCYi4g zf0qC<+r-A3^W?Yxjux8rMfwJ<57Vr;SG}i~G_7)*I67X%z`x^%`HrlihHn|MI+#!3TB_U1QE=^O!_VoozfMq0PZr@^!GIuo!s?iAah9aby zYrn<9Uy?x7vSQeWEtJW=3{jzFx$-ljpcagtV3sU~UPEezTsG+0n@{E)wCB5`Nb?;; z9*XhUzNaYTIKLDobj^5Cyy>dj$PpoV;iD5+EYIgg^*KCfz|M#`h=BW3LNGYa=Ne`w z4C$PAZWB)8DPXd+;zS^H510wy%tNOw%P0~%Spo644=2fw+LpDf@)X&=7nUd3_31&e1+y$ zcBBED=|8oy|7dQQFe;`QWyp)Ts|V06fjb{YzT&WNs=UXj|3mKTZ3>MiX_m{OF{}mj z&ZQGf+dQUNe-|^c3B?AnxkWx8#@y~)ZOB6P@BRq;0H1wv8edd;b6dYz&)n|KeollI(nu_}$ zv=0<@)BIG_UwtOT(<15Z(A%Et@( zd*&??GFMONNi#nHifa-6pqvp?NG=+Te5_Ue1v5)4gl=2*ze2<70|qt3gR5eoK)2 zYg=y{nG}Ub7LyI63tBbbnvF){mlR5)L*dVk{Xl9gL#nx9MS~YAQ+v!J5adj}i8qWH zRFIt+x;friK@=?Eljpmf(zr|?Ht^ktgV5AwK$fT12>@(GBzhm1DS3^FFl{Y#oFF8> zBD_%yq7K<7L;9fjjlq-r)G0~Re;I4vcnlTyXpD8ARHF>nwmSQ;30d!)C)fQS1CO^y z_r}^ThKi9AOi)L@hDb=%k*|Su!5E=U<9e`lIYH7wd%=Q zKX}tC26IQXG;|fGtH;vsD24|d^^eM7?Ip-lk|!G?fDb9B8kX5KvBHw=BE8v z%jvlSH)DZh7DWgT=Ff_OzQJnSX1*tA5Rwb0FaYT^gS^T>rM! zcZ{)SbnZ!2@A=fN-m=2326k%Vl6~pA$3;H}t4QYlV>45+DVBh+hNsv0_6T7rbJeK$(1>A2(Rxxp402qK3hIsVf5syhj@-b zOqdJ&iNf>4+*X4goW}12HyCZ_-WYTz)%pmnQo*-r{8thNn=}kdvP~}iTsQTOPQBtT zB|j-P?KbGFd{uhyk8qn@{TiWvkCKvHo5MN}Sj4-n3!V#Uky-T2C_B7iZ1dB8@zY>4 zDC$e{I8`!u#i<$Q&aAi#5OBwf4*Um@`(^Goi}08zMR?G^T!fb_f*Z@ZAd1bL?Ly8R zT1N}r+us{7FyDpGv|dDmvE|70jO?ZP)rpCuKJfZn%M0;&Gge&nZg}}oDFyb7j#ia^!LQq7+ zM4!CY6SI@{XaTEW4;3ob;RPNKhuDY=SP5c$H7ic_y!9VHcnMvBY1TH4Q#aW!HgO=I zv^L|BK2J-W(Kjg`a#qlOdUwZx&o@VA7;jP1QHn6)@J6*>fMyceMG%X2 zp=1}t{5SYx^Uw3gje?b%|5m}u;;tM@8RUMVQvSd_7W^#UWSytJa{Q3)`VRetM~smm zuNj@T6A^W>2d(gaY#5YH17VL!` zD)wB5kydg{+7hHQELCH%UT}|L!{L=|mnh10G@T-1m$|bZK7@BeaR{1UG z`)cUDK3_+^t=Z3AUp4xSxmiQUT76w=qo0;UHsWUiNHc$ZZ9X2(3` z^L%afJb0I;_I}Cb+nB{b$e;w0KadL&*l9A5w76Od_p)g}z=r=Q+Z;2zoRZ{(yteqp zGrVOdcafdr&W##DO31~{WIdv?&U4XERI$9pWzz{zH{n5vQ(Zq675T7#Xf(bNqS35mkMBqvrk(-32mS zrSH3o25fZZ$0#&w- zn;nTN7R#nDbSIUhpc=-BbOCqFF|YbL3*YTxjtmg#=e+zth*-JSmGh#zrSOpo{z8nU zjTQRy-ipsnO)ZtGYux(%l9wiYLQK$e$i9g5dKGa5)9`1CNIUMbA`M@tGSmNxIbyz} zOvqoGS8;s|m9$J2n@cwb@iGiLWw;?1Om97t|CnV=U8G`xX%XcEK}9Rp?G-9Pp5x3& z0BIbs##I{J)Dgd0K$@A!fenA08yD#_3Yp+Oy|VlGYmSV3mFW&BslaV$Mql(A(~wQ$ z8zII)ryjk@uq-3=WTT=zZIz04NkXTW>?4`9<>OQ337uKR!xj_i=Cq8wLiBDQuea~c z%2crqEyIoZwX}+(bsMtvlw~LLt>*5Sf;692dq7IsbsY`Y#Oj9wV=sD3aLs8TzVgn+ z4c?hqOApAR$bH-0fJmqY;`f5c_sZ$bf`%Iu#qa!=km)ZeexkVW5Y5O8P~0N&+Plrn zY4B>Yw3&px1CYI&(-^z3fq`o+D)y6U`EaR$6tzGK)Wv9*+9&tfH}R;i+z}EMHV)_2 zyDI{mv{DP=Pv$a+bLw8SQIJmz)orVr@4=Oj!o%%7V3SP|5yg%f`YIMq!t!#4pC0X` zZT|3N9&`0YOMvLGSvL^kmEUI)<8~ruM`6p(N`3gCC#Gd`Rlep{<0%J~Y)3({s8n+y zBMwx0Ue-)@dA9puLyYPG=h+s&C5O}X!p04y00Cu@lJGyB}nz}yenx6PUpmIE8vHA=VDGEFOrgGsutw*3+v!Q$W zY@R|6?phU=@_s2y$xc0Em@73_;3b<@sp~yV`CO3>U}XkKE*s2Fm0C%wOgNtAG7%)Z z%1Xx7W9!5EJCww;fPBhEY>a&RgE>K{JUti%V%D6B5nzuTvrC*UAKsJPS{>p$xNS6h zbMlp?C~P3YkU`UTnm9@i`79TgnNM6Dh&WptF+}9|mEkIno?aY6gI&amtPJhArA$wV zox6WmsCcCc-vZh{0l}6dR`e-;ebrZo7>4GnmnmuyeW90&dwRp%0;|nxEZO%W;p)5a zl|XOFc;W}5^0UC6KnP*n9*{oXC{;=EH*8w!$be?(bZl_xI+|h&mE!XyhOdguK5=37 z>hKiW(6>IV3F`-<2N0yyD{~8eP%;5UEOkJ|xrCdG&F29G^z8~`_3iws?R7HWW$3S@ z593i#V7GksMb}@ulw$#YIyFCxo1>Y&wofl|IfsiYDE46z$VlQ3{oLzhg-y2oq?1uJ0 z=yW&0i$Ot9o<0-&zF_Hf?Qj9Me$Q}xQPeSNE^7PvPZP1%Th;GG48nlSl+3a>2JzGR z!Xx0c`;S-j@C`6`HNH}1iHU&new7s^4Vul`;tZC|XUb6!y3u(c-9J{uey?FMV}=3v zV$7GHKJ{#{s~^e-lFW<4%7k^;lU3xS6DOLC@TXhooP*Bo@sV=~(M#HUok`595}n&` zrLLQs4)pkV(t8=&H+_C_Ml?ahfeRW1ob7%|IAB(n;%Qi_Xp8FnLDzvO7Y#U-2v z=sH(Bx{`#K3x^PUAS-KNTn!s9CISd96$RCYU|&AF&=Q=XGHng=(6NnK9dv-}+f9Z% zRFscH=j91KoU6nR7ov?J$c%tH-9V|BbcJckrW+8Fm%}7U$}C~jbJCFh#@=G! z7|XEc0&^Mq0z6B$PgcsO!K)It2^=xnz91+F_*NRDU@DS=s;A0N+UxHjqxmnJDcI}>N6+` znn1l)`De0<8L$RoOmg-4Gkh!Zqih$fZVwx$0)bfm` z(4NrZ>R{bAH6vEif_KaY(_)}Ij_iYji=HU9u^2fIyP0A>c}sORLjec|I=U1yTP{iv zV%2!LR5;Qf3nvrcDP{~@AjFw|DgR0m$I8Lp7QM)b$%J;q3b94 zCPY+Mk3Jt;2a;ESCOvvC!qOpOT-i2;RZC$xp)(aTdT0Sz7P}jGT^oKgkR3-ok`{=2 z(uezsoCMs~GMw5p9TvKzn|IFMVD+24^3ACr`*6gY`DQeQI^>zPUL>!A$Qo4;J7dAq#Xg}rtqoap-k=;Uq2r+# zCFJ{V9gtr3_&VUY1Huc-qbh)Ae^n#uTlJfu9#@;j0a*} zpKPiEW}USbtQA6$J~A`Xv8R~vo-e)%uJTPuSrUnmPJGRk{ZW+_Aa_^Q4gC4Fcl3sr zTq@9=K$TFctK6DO#->eVY#h0nqy#3T@SQZSpm4?3<0)8otZfI>9k3RPLjTr@q7b3v6DelV}>k2iBbVmh|8<8fVCj5=mw)Br=g6} ziR$=1PJqHvpN_6qc5}me<6?N}S*S}FG%jz1MD|$809sGv%S);Hn(!T7zhL>rU#S;| zq4o$Jab4+&h#6q=R9t{6&qT@2pi2fFG00_iib^0($w2&*dqvD;BPq2+b1yn9faMuJ zdHpVqN35y9a|VllaBGONLS@dH>X)|Y>bg+T9o&6-;(RPDV0q1J7r~`ydd_SHFxbHEW>#djU z$v1O%5Zvgw6au|v6Pow<1SRa1DME;x=q(X)yQvy{;froWQHBgvCliG{`iNmE2h5K# z+wi_LIbnXT{X(t`S8hmMHl66;TR^NtgvoyGbc~FClWvfS06$Om%X6lWhk)P~kGv8lOKS|m3mauz`HnJ%2ILM{TMK@KGh2Hi_gTWud~(!OqGyH)@EU0 zzB(&W8~}X7o2)mcF+(=uZG&{{;>S?w_BvRh>({VOQ*vk79zNBPwJMHIckJhBik!~m(loV zHB|w@2nZCG*T!>E3Y*_I&@04m5GSn4KH2SY>b8*kM^XLCgtk}o_d!_o%9UY?Ub2*@ zWZ{=#w{LJYtE6AW%`baQVUcvKO9$P+GyE)~Q>V0HtxG4y3n>Ua&YI_`r#qTDsMNQ+ zoB`O1Ev__k%^lTmGg7ImedVt627Glf$xNTi(w94Ixn@y0M zSV^J(L}Raz@w42G$MT5MiOOOxh=6sHbj@QrI*CS}$ysnyR*6yCaP~BXP0n5Mnw6gI z_OQR@&6pcx{igM-j;he;N->BM=o*(y5ehJJ$Y{NZBJZVPY1J2$4m3dd8*y8Bsl|nQ z5`|ApqZ0?ITZ{ic_TDae1b5E51X&gmk946|^mfXe~HI>{ThUYK+P66uGSPjn<{6 zvB9zDBEUAe$^P{hBjioxZK4! zR1buAs(s0pTn~p43uL~I^M_2$$~$N0Pp2F%9ly*I-Ghnls|h%cnPj3aYy_nu7wRao z5*+5VroC<}F}xRMqrtyhRM9S7dzeJjb*!f1ehvMTPkyj{PgOA;X&VhSgnqAZ8&7mS zU*J2rYNXEp*ZW65mwkUGcrR5wY-->RzXiPey@UP_ed3=Rt$&*U{<-UCal!B05-6sl z;qL_aPbHFcEIn7WFgH;-}wgD&ua;zQRh;D6O*Vmljak0=$8+ zacUWStDF!}bAHT*{JpR~dV`97cBKJ;TG@{rP>Fwd|2~~ImTCsAnStk~9)X4*0iSu@ z7>Kg0SvldIXCNKMs4^mP5FO3oar2ggbT!c<&yz4#?9sS6S_wXrXP*mU zCZZ**T-whI3adViQS88&+_R51Exe_zgIbC+s;5+e;(hGUE#PIElC?dtaoHTL3M`U! z8kGrRg&IUR30=+D@Bf}@3j8Z}xs97MA%D_QnY*XSNhE9x6s7MOIKFXs+*`N2lQt=9 zGEx*|F%X_$MN{xVIh5yOUq9^lO&XMSV8lI2Je$sPel+ra*IbTNT}4)iG65`5Lvd(G zGPP1q>)PESZ2(vy8V$p~UhA##upaO6=*F!-Y4*Bb4?We^Y|bUjJR4=X@Y{PrPc_){ zY+c6!THH!w$lvaI^K-`M5CmEmO4_dIA6jKj;<^$BsW4d=6`KIC{f8mAw|S6-h~`#@GJ4QkPqwN*pZ%HN?3-f)nXnLK7$ zHZw0e+yi{?47jFRGL(EVV&yp%tdAZt+4zEL+~dTnz#*&!qv|Gh{q!=SF~RF*&>Z%N z0WB`Ep)yswVpX~vcHA~)S4i~BG^iDiIp7cL!pr%3ArP8`9-`;0bFo0V3*Bq7Z_3)E>{q2 zG|3vKs_kqjhfigZ&suJB^Vu$V+dnp{e%*JEW{Z7P9Ad%W$$Bn<3dekx%l|cy_$<@Y zbb=DCPRn_Kr54v8pvB0ce<}p<55-^J&$WS8isJx<|F}7fL+!Lu5yNp-JO`{HS_v#X z1ncHwOPD}&v=`G{d^4)WXoR9Nd{G!HKOLt$a?h$bo^m~|PmgAbhN*qihQVA!v#CNL zgBB%-B>)0+Bi4x)i0RjJR#ar=nn8F}aDn^WA*RXB<0H7G|9t2UUrWKv*k@yR! z=2qlJFr9<}q}L3Ib?y`+7+;(f$Rj!&zZhuh*m_MA{MZVsB^ugcSBmFJFA|Do??xyR z%O(NHzQ*_;C9QRmX-XXzgh_IPbhYiuUV5!c9*hRrKa(TZ8g?#~A`8yd^vE-ha)q3& z^3d!Mh{KHQR#hhT`C)lQZ9a8KtOHtm+83OPk-l=^`1A~?E4EBkmXkd_>cu=PA4xOR37wCTE3lswgu*&MObrUZf72{(eyvRKm8{wxHlVQhF`x< zzXq6=d7XKk2kYf;bJ!#W0j^t>FT927G(wG7HSc3BxB-Aiw_Zk6u08RgT*f`a!yD4@ zDrRtRv7DYJ1bBSBhpTA(Tv$BoC0;d{#OGmCK&jeNygsmb z4irMYNrc|OWbzln+%XW0PrEo3KpxG_^Iu}+kyT8fb#9PGyUdn*R^0^G5eQyF8#_9GJXwa!O6PT?0w=f2Q7W@n1Ky|5z-%T=3GVt>}%x_N)yO z%jPGOJA;c8qcJHRwW_~7(|Oj(5SG&kc*e!$1bca#a;RU-T1*`YK)!U^0r`KN!107F zq961=()cHeZL=8Dv;iSRB7?@u)|(R7E99#}?ypw~??2v%iT0UL2YoP1;MId^lL(j^L`kcD8uv*C~ zl2Ua$9M--Yap)ET!I-wFnC!5KXhEK4?_{`}f7#=4#G(dMwiOz_2?hNe$>xp8co%yy z_4LQ}XP&s)POqlL@f8f(Cqc@!wKK;2`>|`|F+DKn)~UKZ`$AVOfn=ah*TvGav7LEA z_=st+2i` zcCbL>vP$4^K(X`lrE}zQu_{sNp-m^a1~pDBJ7YiPY`SY;{Qa24BDxnloDEuUx&ruy zFuZLHHTmsNL!ZnI{mY)P`h{jB6zN7N$3oSkq+ZLR;%mSH|LzBl6;tuVLAOL9MEg!e-;$YJkO_vfdZ2%jz^_O8ak+|1Jf5epwl#`!XLag* zPML56P({Y{1D#_ct1-}BOl38z`J*Br9;rH?O&!En-t!m2pbfm93>uqOwn32mOfCb(BZ+@Zv0UzC*&EE=V>Qm zK?aq;`Ark~e7}u*#NIs&a5e+!8qebC#*nm|36isM^438{nWdTwX_c`3QR8F_|Mm8$ zFAMvvR}Y)t{-WhSxdBVGaqy1(7F81&0)NG z%2IwUSJg05pW%A5_}mpBs7IAMG?>Sn4xQROGxL%cHZM-Dj_apM#PQ`9_aQCU0NdGn zJrRC3E0$f|Q%tp=(rv8f+3NgG5an+PjMUb2SlF49p1#06#e_9e@tZw15)XS8Arj-O zzf&^$hXm%=eo6i4G~;=>9;ijf&R-BiXD$YbT`6?>vBaFz1;$suj$HycNw)wUvJrVM z4-wJ;rd=s`0@_w9B#K=_g8Iw4kvs)c({fxgYcm?1NCfjV=;a*e!O2OhK%HYJLj@<{ zQRm{#^vR{>VYx(K!c(2MQX(N1mLHD8?^BPz!8C(Lc=;2gZVa0ERg79Ty&(@AXfIbH zcu!#zAGG1J_i|EJtyP-{HpcmM9jyO-$H9HeHaQn9%5J@O);-&m;ha(GkXEG+Nujn| z`_RFzuTsydl~U(GHNMG!{xo8mSln4|bopkkMOR=UhhcCt{FKz5U8NTMA&+wdgGakq z_Auwg^MbA?YUu&OmgmOPQd6o0NQxG+kuS^cX3E%pU3(3UaF29<-z^6eaCoDYMBR&2`np9;%t&sJ#oUgmOgrDC*<1dc;GB9 z=bt?x7jP|qDyIT~oMA06sifba!;hJx8eeiBV6&dk+4Hw^4~k{4@gnh6jN5D!wH{Sy z;eW~ravEdSK1jtqR0IfJ62#D4&OnPnjic!+M==#F~EUkBJn-XM-g%0#~!qCy@(XczN`~WQ|hV%Y9GDv+0F_solQMn=SLTA zegC+r^sn}Sm}Rm4J>l3Vl=aIVb!ur!y2ldv&2Wv~Oh-~W=OCS>PHfYKp!qdrw;Y3V zwVxCm+eBn8^Qg5S3CvljZlPT2Fj}q7h(0?qw^@n6ZR4ZOV(1!n*v3b|LN`6xPt0En zTu)~GAGrls4qpOXGu!7)o5RO7Z7&@z*V5daEv<$P%oca~ww>kFm~`gOqt^ywa^`)L z#xu}e7a6n5DiU(*#LGrgTwH}t=W55~A?}4DXk1NI4?YIds48&Hy(j;2epPo|+fzlU zv$h*?{6U{7gw4S0}GlkkPJhy+u-cZ0GoNjbv z1R%w?9vT<=?MK9XX=-=hG}BoxG_xukvh(m;iu@8*7$CBgq*Q$Dd&=yFJA zMrC_MPI4-7onu)(rM;M4pwOZHr0u(NP0|FoU~ESZAmvrziuK2i3-~*F+$N1$Sa?XhNE-;f% zUi7egC=-O|MX|1+GSTcy4n=XXU(^*17ch;GbyUN@1UWGPCw-pQ^M^TSwe z&F%HRn#v?q68R6YPm*cSgpzdnG`qUR@hxBkX}fn2m#^+Jp!qgR1`&OAN;<{mc%VG< zua*u23ocp=$O9wY2O$@^gTR+nq($Q z?=Dt2Y9V)&<5uWDZ^oU=IDwlC4hcCl3u1DW?!{BMQuDk#>`o&-1nuwm3rt8$OCOjm zeTz(en!@Y&WvhCi=1-Qr#D!~T%?#Kw zy&sjUABAa6ehH{2tBJP}N_U;8?b#qap zY@ha{e@lUVa5WIi&fQpV5Tz9FWlw2;Pdk6(MSN z>s=wPxJ^6%xjv&@rf3@UdjKu`>ld%kf{TF<0u4hO|G5$^kNrs@5Qs8vulEwR1^%ry zXQB%uhB)5%p9*rJPPYnjc6F~G{yPhPwBXm!J}w2{J0YMhF9jT3kI=o|SK~j|CUW4k z5h{P}p0a3ktl0Km)%N z+yG2rnkDV{P(#OE`(-#{5&2exY>mc8f3WGiJ*`?I(LcJ`jeDsGAD8GE92==qA}I3= zysS|d^1ar5k66pScYZwiCH?Kn?g)DDa;zt?`;Xx9G|>sIzEU9zuKt58ufNU&NXbh_ zf;y>@$r3B)=OBNBLfa+o`Hd3Of40LX`*{=>JL4o(kI0z*Pow34V+l%p8a+h}fjm`g zx4t6yFN%8f|BTzb9Z4j#;aCFpqW;+1xs@;bzY~A7e%D=>^+}<_i{kILN))_|mPuH_ z&AMFIA^$~_1E>C@{9j6U+>WuMRT0?UXrcXQS{sl7dzpu;k^Q_;9(jZ3sc`XR(1p;tI0Ugu-^jEu4h-rQ5s4x59pn69( zf*#<*2qpgRCO*b~?QkWOrt~m+fBK)e`X5L9Q>EsoCrtm;F_`~Q!%Y33PX=R}ABiFW zL5JdRtCPT2?;o)8Tz^uay$?*q|I2p9-+96nqxDk#&*M~S4EV}^3%l~kRYz{u&T|p=fdQb9Ck}-GTQ{lS53%rH~ezp1k z0wF_Ivh03q0f-X)*T&-iqUGO7r~=D%|35{`|AmylPd5G+QvR`_{J%-pKd3(cZ_@QA z_3NKGYTa!U`u~K?PPaBNUT)MLL796bGc?n?Yv%aoucjDpo}RUOp?y)t ztTih?*j}h+4EiSZ_lYR|k5Yv-0;WXdM_;0ixvniZ?*8j1WodN-s5(ZJIaueQifLH3 zy+S*7=VI*ZUp>;x&a58_;$#81CJNq)yi0&_|tZ3d;2o+O7 zM4m033JHT5<1x`ZtI#Z}zis(a6Q4GX&pyJ!e@SnCkUBOZTdCu3@uk$caac+O1A(dY>;1bGzR%!gd6Fc+n-!-v| zxSkwdp{SK(F>)~)la`Ug_Ed?>&WJAvLt~yxVTjBxpZ`46?TtQ z#&YoY06S(q(oj#Tz-7$l{k8dp%R-n~()6*5o?f4H$NrqcT5FK?mRTw-`*Gd(?7c%% z-Fb#zx}UmizjAJeHqQTn+E*7F8LH|{TX>?xDkRCG*o$PZ@Vw2O1Auryys4<*YnJl_e{AqNZKeHPs=TH zPO|WEZG69PF*Qh=@)kn^>EV0xXJ=3L!$U&4j0yA~vU}(d)cfIL1`E9U{Pk||-A<~w z;KyPZ4?y`Uz;hj_mseK_>uqx~ja`h*a&mIbTwIOhgyd9*^Q-OGJ-2hL2A$8iD4DT` z8&QTSgC0MI6Ba~5y8HC=5z@z+>&Ge3DTxo~I())oi{Iqp!?d$^`pMI^7%xgsSE3#W z{EL?l=ct8gLOedJf7G{kpEz(q#X^GeJ9=gF&rBPs73_44Lj%4`;@(f`%yyC zP4Z7LxD21uQ1n@UUJ1|%Xd=|3acMS|#KIO0*4|wpM9%1=cta)*D)83!C98j6z#+435k+Pnd2&aA=y3&ziEZ%eyz*@bu(MX1gwJW4ZE$r%TjC-bn~Q&G?d; z!V5(Q9^_;!aaxCDn-78=L2tw0mH07aPaBc$i*&d&;J#=e@JXJ!JlHZqkLpiDDi{e5 zOw1xYig+f~q$vFo=P4+>1IW}jZ`s+YH|t(R5v&sLZZKVkf^Ym67Q7Wz#N zJ7_lKThBAvK_cH|TAW4#-{et9UEaFuyNsTl9#UGQ$j3OMR?X*u6c%zBm z`WCrPMYDG_DV{-Ih*7>$YERFmkDv|60KcP|%8@TB9|gB$S9?kPpE}k`{0812Y>o7Eer)7`BTCII6naOnGL|!*ak2P?2MA$^qmtn#r?@W>r3#vCFEz{4 zzc21VNln;K1>s1HZtRnRWItqbibVF%KNst5yo+wT|A$>ah5teXZIX`5 z!hg?a|K#C&F1wg1P%(j*3|&Rl{FW}Y_XF=|7zX9lY3HtEdUQG@&cZ$M^i#;*0$nH- z;Z1=q{t6vzzKYp;JwOL*-GQ(p-Nl9rY{fwK!*Ss!ih|NeHfzItd_AdlwO#~ zuohY$NG%1_lrja%Qq!f-!frcBp9h&*m0Q(pj#$S+&cs5;u3awX8WWqN!I1AAj1=sI zr7xo(lE3#E`!UuRa1_m0>Nk#l;16QZy`ycDF?UHA3hAgX7Ktv3Zr8pAR{#(MYkynw zy6u0?nk_Eyw2YkD0J@re6l`Y>G?3k&1N^SYOo@z398IAFFA=|K)oUJ)kVhmrGrWW1 zLmJfeGjV;N=BBX;v6mDr(Q79sZSUWb9LOGGasZgag z<%{Jm)$=s@Z^k*p$ZEaxAM9}?sO8@hs=Cg%AVd+EHHN+&yylzP?>J!r>$nTdW`1v8 zQ2}0s{*|HDo8^2U__D^xoV^GboM+4kdOFZmR;gP;0T&U9{OEQ1M6sEP^3x*mog2 zR|29gOzU(Mse85PQcW?qrp-G82OwKyKcpSWnQudw3@Q)k!_7Hx}`uf%3 z{-ju8n5*QLrx`)+H@TB1l)8tWk50ZkzY6YCot8Vvwq>qX2402LCGRLM>)GY*uzdhyA_C;B1ayuUYVta^5W^Yqp3{>Gw zsxx_zLHkzFbE#EShFfI{hXs2ZO3lLuht2HWzgdkIuxOIb|7A|%$MHa*F-yr2%OmF{ za76)e^D<-Z@ zL%1)W&t~>Wm_3@!6cw+RK3Qey8xuJ(>Fs60^f(JHJ$me+Zglgv+^&*zhjLv&eu z>ohv_j%jR(pt%JnbmI&Zyf_9uyMp(t@Q-D%!_)In)m(!3D}EMo9w~iq*Gt91V(qnpqz46w%bK6huQI2xpPh?O27w)zGz1)ek5v(^Nwt z2e3#rgB3AuagM&DR80N~lEi%Xk-Arp*bgj(T+=lqhnLCxX%M+Q!i)2VjM{d8`1V_ez=)d2PWb zZn9R)W{-=~s!1Wd7x{sH#)eFS3D<(|@iqmQpy(>LF*Qg}?u@Dvrx`VvwyUHc3%01( zw-2+ivtV~h^eC-3&=uLnG1sm@)v~A!)a_W_RrKVJDs_uydqIowtCAyy{~0*9dw|$L zta1}L1sG>5j!uE9NNql=) z0qL^3<{9^P?MSLbv3q+R9?!PXb4g+^C#J1=Qqptp70tA*zNq^kP9G^s)>BNr-wA#I zjq zPwtKwSGp@(gVHVY)O>x~mrtWmSnk{fBg}u++8M-<90~u1;vHrT4R&g=bz$d0j+{Hi zT4)c5kAL_Z{pM=*hAXM=9zJlRC+*NMG0ZPG; zHPrcq26S-YG5z<@gcr|=#Cw;>hSRj(Q&z}FCcX|z5O!TQ!2rV~K+4y-Go|)2mX93D z%XH1!{9kZ;XZp2Rb4O)hIV{CQVSue&$1goLOsu}Pe8NQr`*MW0k;b(*O6e6AP(7-W z9$n?`bF#C;@D9N0QyBv@2z?t?1C4xmf6mO;>!V2}_FZ%S4Oj8&J;dH=_b266W8O0E zWzWqYVI_+i1y{^0k~9i5mFc+7S#=LsBqCU1b7OGzr6fGl5Z6^uxYwAxZ}m>(VGW0 zI75^9g42wXn%fpg100rg$&W!{(+DxGK}tC&Y4bcEz_8!Ph;+}P|&*^bm7TnD3V@=tr}ALKuCy6`Oj@PH&4#Py4kPVjm-rSdJy{TO-8^RDPaRQGyRda_Hovwu&&Q*%IP`fy-B)Nwm2@Z0}}pzel?HaVEb!j*EK3x5CZM?#h(H zC;Gj39&Ew|kcEgch?EzpVU~cuU&T}WO`rbQlq=p~`PX>OdKm3a_}MRTnhDD$F<YdWgZYSSY}YmEz7TSSnd#)>*#NKV!`uXkU$E2= zh+diT=%hP0khC9h5~hubI_a0!N93bpK}3BKd{cQd%N7rhJ#C;l+JovbMJJ?TxrzjG zuI&U|E|A)z(b!<@oh0la@C8bANWx`<(WUvffsE@hYM8kQ`?@%Y^W9bw`2sCA>~d{? zoD_%1S&MPQ+(Ka@)_jzALK_#kuFok?$QQycA1{eyFKWi(HgIJup1~a6NovIe2?+6# z6s&y(}niL7#mE=LbD;hCVz(U*8 zJVBG|y(V_i)w`HQtN}STJJt7A%(h#*Ze z>EA%q0k4ThFm>ogy*080vH}L5W9(sT?e^^I7K0`LNVv^r_6Kk`f0IM};Z{^7h(H<& zX*uc84iS0V%xQ>7gCmhh$ocM4Jh@?~iOT@RgmaQzloUu)XWxurdtsTleAnRA(srFLSeUhh?WXs{ma!RB^IAM_6a%@$5kr@#(%yQGL(LC4(vH-G=ZqlLq3O#3zJkoc| zBbmn}XqTb&A8LDIXP5Z4Fo$mv+)K`GOn4L}eMc>Xh z&t*1Ed&I)dxf*4nhPx*O^fMIZ!?Ri5JFp17BX7Q^L6gQZ{B5KgE0E)+YxZ2?Trr22^4uhi#jn87gWTjt(@46apk_ zIK12G8J|o61X>&tG4qs=SXk~Dy^=Q@!AoZGN~NF%$lMBXRqe%oY}x7Ra(n5Q&`gvT zp`zKen;N8Gp@ar5l05FcM}inoFiI5UD~jgAR;4vE))zd4elfFAp0HgkTfN3%#Q?Pq;S5%sk-J%7B5To(J5>CK7FmXbAI8i z7FgdKrfh%?ck9ca#gH%XpJ0DEHySQhb!4vdjIrp7fXkgX1S+)deB62|FzBWxQ9L%@ z-T2%#5_1v%A?(*X?;~%l7R|de*`_k%*rIi?#!`teBktm9r6y}Z8>JLshFy3E z+O0L-1fc)olwXk@*Sq+XcKRl>lhpRP>?zJ!31ps(e`uOMIQvak=^kRvoQ#t|F7id^ ztiZ-+Zv36xW!n2OQeq%`hn)9e(=p8rdD4CK1Te_2>V%cotL`%qiBmYrI@z)~%Q}!X z-)l2{75R^Koe`rvKx{K)Kpb|FvD)fDF`as*4sgbPp|i2%ax*6lLQRSaHXq(a=!I#e z6E1WZR3u+d!iHUB$6E>@`f2gcxCZ5NlF7z6Ei{%TZPGikxG`1-vsJ${*hNSwI6ixu zJErlNqPADjJCgH>+}CBt*&|e~Tw#!Moh;6y{LBNdc;UM~6S6+MmM4;|*m8!}2Ie@3 zM*0%+F8t3oirr?ff?DI4pBE$8+9tlKp>fw6e#Kf59DxaCP9a#YeR%-mly2@1e4X>5 z?CFA4%h1*2NX@46OP~m!)eTCYFZqH96TZFuFE`=4COv_gu5fzsN;T|*L)A|IQkp72 zFf4-<cpg z#*q=jHu-7mwWh7Dw4Jj#<)Dd|^va}&d74!I2JaO27Im4pdwcsrh~kg*hL>`G4343Y zz~*hawB{ILgIqwp@Dpic9^SP*CIP4$0#cjnTycpS%@dRTUb=k5`bbQKz$)q{?DE{JQx+mCr*y+ z+c=#$;-ul5n!ATk!mwy(kx*yh^oGDFjZaAieR2UjfbryAXCk{s{+hJ|hNN?wjHy)m z6P`vP3?swcohr#-d9YATf9U5hAIQ3DjC|ms+^bAP23d{?c0^%*4(@lBFWeX_+fvql zEU=>Wv@pk0s?wm~;z%xLP2cw~*LNt7ZxKe*N)$&=@zCdZ){mkkS0_luXVVm@$XzG> zT>Kl8z>uRw3tNrVw~VR$*Oq||>${DX`d(A1SrZ3bi-Nb_4KxCwzZ!w|0HgG`MxbgO z36JEZRY3i!y*KnW?{2`SCf((*hCXL&Qng8K;R?Ura8-M#4YhOKdpkJrw$cC%!?8pv zBX?2QSMVPC46B=`eI`mbMVxy3_Ov?F=q-XABG-?(sGU5SH~xtp`gVqeE{{PG5&``i z&5F!~0?*j*Qd4`x5=E2&LAGJj z|EfeA_KDg@D@MyV*O8wje#pYVXEPJaGc+f*{qFDVZs@@;e_!{kx*q(an_WO55L)kk z;u>KoSNJGh^A`(;3YXxwP3~CTOSW4P$Kyu1%oG)((H{!om2`EeJo3+pYM>qF5Zf5( zHE9(C8#Z~@GrSgwjPX$3NcTZVjm&vLx&AAh`rJVz>&Nm*kiuq0r_<9dN0P;Mbt-+U zs7QA_S(e!v8un#^K{%Q6Qem^zMP?w!0(aZ0>{%DU#D{zfqgi44*$3~VrgP0>y^AA( z-QG-{6#UA~2n$heqY)F4_?v+pw3LJEL_`~-!D)tcn6hT_jDQ*klM+(~k9aDl66`!- zyF!+tWZQ{^{3=hfzV*AvQ;f%(rePoAp(Opi6iK8hN>FSu3O}14_6U>2y!91Cj5Y%< zM!FcwZ6>rr>#@S<&Phm0Re3)0fvdJ8ZypG(u`gszgY=0GNw9Z#Y6%zmnpY8YDvR#D z!rhq|!Agn^6z+v6JENM`*m@fF@dt+;>f4x-!?e57$`T7prJj!-=np8t$Mn_;JTqV=kIEul*>@?uW}zLj|^^UOW*|vHr)JYsM3=3 zAi3gip7vAOdSXO#Wdy_ltvM?fB8J6Z(VO_LM^YK^yI@>WMaGh6DyG%F7_}zX}X7xOrPqUWbgg zh;j?o23==}X!Yfj0R+N`pB=ZgrFLHOp_?9@;f;a$U+0u50>u|RLBMS!F(tTbW0w#A zfF8cX@ELA=^Js6R)6$MUr`_5Uiel!huM&jzS6Su%Su=#Dgz_6(Hu>1=JU3FHib)iUK^gok;iBa zVsmNe7!Vm_7Q8TO>j-U=F_q0?3}G5sl97d!0~hTIGE0JB%J9_k@MtrSSN7pa?;Zik-i6>F zk-Zvn!GEnAJ!du4GgtxCyGAz{nEML@1ySKu_^*~eos0_sW$+JiIyO(Wv2->XsLM() zK{6~LE~-<8fXL~T5z)b}(;Grp1=hMf_D?c1q+H}zGlImA5{r*!8|gSg=Pa1khWV?$ z_K9IcXR(d09qn?hu_-9l5Z7A#WYLiYYKb-JIev~vkO`^fQr~q5r~S}CC9}Giu}rnQ z(M!EMV*5zf+~os(z}{=qqsDbdI!4GwOKBfU=5$cE=u$JsY{W^%`eWf&W=$M!s0`Wy zmMg&@-oJmL<0eGWK!f66Igy<{O?OOh_$#1bkj2Q$ zH$$6`8+yD)v2eDyLy=LeW)nR}%)sDg|EdDFB5(Z#3m2G(FtMdyu_O?b_eIWL)#hp{ zFv}t*hOnWW4k3I=$nKEPt243u#el=rD2{+Ncb!{-;1-0YtDB--t0Q3Q$V1cd6yCY0M!NSSq-0ulMcJ7U~|O)KBXi6Au8?iurU zira?hbPIyq1vx}zH;hrJ?B=`b7`TVp~V5kLmi-AYVLRXeCaT zA&aE_)IJ`}r%~`#nI}1{E)*5%EdIU7<~|Iz9NqIfbm52mI+?>XUV$Hfgf8`Nxqy22 z9zXNL`&BtIQ77@VIHRE3e9fNf#=>fNp0<#S&FjdM_*Lyvm#c56RTMsuplB)n(c zmtXEpTYIB8ZaE}W$ngK{J^_GR`4PVESd&=CtqFg)Y5k1oWG1K`KeeUV^r;NMd{+Y3 zhNYkFThGI?W_|o$Acy0sIJ748Rx2G6OJTGup=-Ou#cRKV4z6+}guZ>m4=!0oshqXr zVhsF2k!%~Div_bI7z>c5669!NSy|^3w1_Ja^J%Q_PPro#4(Vrj00Rq*SaT5~x$;$7 zC{k-Ks2`lHS|>Nu>w}5F~6k<**TJUPYYzG zKg(;yTe8JkRWDgMi3-f5A=0_e7>`TfQa-n)#i3_*ZRrETP99&RB4*RJt7Vk)xSJ#N8k7bYsKMi;!Cfp%ufcuX0^4pB zn1>swZx3HrjQV&nvtH}=7t$a_#j?clE-6xY%9ti!ON$-i+{*SQMrA@}@-`h8C2_Z^nv+pN74lm0dmb+nulm zmsMQf?P{T(c{BdlbhmLcP05u4skmQC3MU=ut%S)7b!@2^vQc)!=fn?DEk*1K{F@}X z>>P=OqzpbbX;2&#>Apgvk5C`8lQqFTaTotivvcMf$YTF0+z>O}P%MD&hqZY?K)~ z)Hu}M+pd{J=y>?__w1yXALR!!#ca8$WLSe)T?V_u{mm2qnhCbS6rieCv<41rSP3|= zoqp;cEAz1Eqo2nG_8e#H5iD8paa|&9N8r~I@`#+>E zacy{JDSXjUzRp3TEu9zxN=mGy7g^dyQ8EaNqP5lex0^Ieu|*nrKFA_u%Nb#}77D)c zYqBp3_^L*_2!ndP0(*|U(C?K@MzKpQiG7$lN*Dn)6*}x~;k8*{#15jucBG-^y(#}I zxAt&#^`})%COk}&FS1NcJUfhU!a`*=tYpRvtO=``+n=VuTu2=GNRk%8GkW4rs5?y zF+ZH;^)@F$43A8*^8G-sv#M@O|HwXsYMLlrq}qiKq5&jP~6LL74fq3T7?ps_lR`F+MBcr_j{IK_8fpls<-^-CydmYb99Q&e{hcenrQG5^v3)} zqFF{QBmP06L2^&rZFNEcs(w;H@POeW(Oy@tqK$lFE6XpNKIhc~}czS5YmneA@sXPUQ!vO0bk860MmVd4WNG_KfLyAD4O6Toe{~mrISlnaPqi*-S|2g;oa~Q+y(O z#Iv>s7V=&RbwCzz-`}D4UeGh%Bs>>aG}0fesQ!&#?o0^HePP7N+%!Q=3u=O>+4#i; z8;7S_FI)5cko`u3tJV2}*Na*#^I3R1S*|8M=F8$Ai^bz=aUNoL)I~~3klivrZtKkg zjnBsGgV4Z2Ol@$pvFcm;&~wVNHu2vXVXohA!s7GaLFm`8c`GY`{WO=^<=|tnS>L`1&HtnSU&ivll*KH%yj@XOxbuFhw;=q}>m!(wrR5|@hP8`DS6Mn!2#GiG^-_n^i z{-WCbvjvdwdBX&H$93yXPAxD$0JOZD$QQ^CCw8gBJDSnWj})r;NMK{+(@B)-=p4wf zA^-R{`lCJ){414W{J(Iy) z&D`hH$s>%!X~Fr7+v~EQx#_jJSqK*n^3rqI|Hao^xJA82UBil?fJ&E2GsMse(%l_H zhop3OgOtF~Fmy=_rAVj1Al)q}-Q6J#-;Z;S=Q+=JUC;Xua1Ar}z4zK{t$i<5?=bV1 zzHYd`ZJP)y>Tp|JY&`R)9${w{+W^Y|2tw?q9FDK(tybr_|kzYd6IC? zbCmvj)#5+AJ>;bAnC~}nDgO(`AJUZpKNzXJ_T)F<%DK&_eQ((r)u%~f(Z9PknQmi)aZuM4CI>jo7@rY;~PEe(Z$U8BH}$wz$VA|5wZVfzz#Q*_cuf zPAB4u2+3+DE0Bz$wCjrthY<=mZh#A>bhJxDf*@%+t#!?80sa7ID!><9N z5Jw+p6?=MSlD@_i|J%^05schnVC zig|1IfyrOWTU2i4G8d)IqjAo){q0}!^gE{uYjYUv@hB6qv08lG0^!f#d>|%*pIl@6 zj?2(J^_@@bvAC5^?gsDC$3SyqI5LOT;3&%zmCN_(ki!9*#h@oiCDZV{rRni+_`-gn z>1F-=2gM&Vb{PHvi3ZeYfA3W2Wpy7v`k&l%neP|WYS(A2MV%9HVPh8PkAp_Mc{bD_$vid{d~D!(Vq| z?(d)^BK|3DE8%;7qXJ-m)KbBG)B-s*?KWZR^?VPclbRNzxl!qh#yVwg+@}%fD6eQ} zISs9?8A^DEXI-#jC!9c|J?aNkEyocFwHGdPb#W1qOAF1(DVs{f%@r>MQv-6LZ5rfU z--c~-UDWovzz3-_wmUZAEwaRz&jt+0;~%GknwJj6d3k#-EL3FKY&V`!k;*RJW~0fO zSnRzgw{3fuo3|wU9xDs(3HC8i)DiqTe}%O3)16ye1ZFh4n4`kR2P|{< zzg7Ksk8Gzl(eh9JfSux%*rJ=m1yK7qfZV!&N*f6hj^Af}{Rykv+O1$nmQVM`5QVhu z>F^uA`A z_Fk-IOg%;yqjMp}xKJ`fx)34VltTwRq&vLVbfms(>&X#OEriawVKeT!qy=BegMuMC z-O5`HQ3S+fRBl=27NG9guGjl7)yP-T{EW+WhGhJ%>^*Cpdqc9n?}iS#v~Kptg^ws# ziGA!*gN`+6JxvyW3lCP8xy$ln@%t1F5?j83Z5N#3ARA>{FAu?8;6lG7M8G(*;<>L_ z`E_t)s>Ly=@aWOtc(1YayBg_26AnIIg#396K)BcC(9E+uUtXE=&=qdqoX(k4L)X#U zg@2h_=cOzd5oOyse&Q*fHvveaHa2DnRbJKU>h;bI7^MSfy(^*Y@bNxw^c6^rdxou} z&(yG-2v`Qe$(htX^hBCY+^EGey&nowcpX1z7&R(QWBkHd{L+3IzjhhAJ4rzt(X#5C zgd9|@8$tZ%axPF}0edmMLP>v{2g3g{4;&`t{w-h|*kH$;>-bz&>K!B0GCZTx_Z=n? z`I{Pp*Q)iGtGR9s<+;->ijtp8wWI!6N9~nSiHVsPMCaL(0)|%07Om_KW2Qx1e> ze)*a|Iv1%&R$HMrk=3f0C6pVT1k5%7JzsQoA{n`pE&0Jjwnvp%ba@?E&m21FFYk)& z?d%rs4HSwe#7KIFJ-6{=*{Pu{w;E*#FrpHn!7&uN44fY76+CjLf`Z_{Q$%Yq{e&@Q zF2m?7+>UQjGq=8XZ>X<0fzByub2I1KRwD2Vw#(P9;eF%*p!`J_UzTA8#Ys)_MTJaT zqo>Yyy;*%G*Iz^PdgaMq!zguA3Atk1r5E2h&N`AOVN#`8QIFv9f+Az-%u;_BoH#m* z+>%*+f(3sK3x-g~;z{Rjcx8@(rZ#7V_c1P8*WFxEjI(2oAWVrJ8? z`5GfTHHjYHv3oXPj@G4SyC&m$>JN*V3Jf_Orpjoh@{#I}{Q6iTOg5;j-7G29$SGS) zPoSb!$-1pe6Yb6()~@@QLjUR48Q({lejPR&p(icNro5;40h;onCi;JniyGEFk>3fWrwq%#gWp?;1>7@}!{MUFqR*G{yfbyaqAnvM zC1D>^B-lNDyyZ_*3(*99kRk>2EG_<~h5!1o#mV)%DPkj*XEwlm75lz?Uf0qxiAfUV zJN|+jYC#+l;_SYJH>~jL7n3z1ukjTr6p$z|!<>F9LC5kOURL(J#|g<_XU!O-6wlId zG>laMtR>&SlSqNI!SdQ4gCI|^Frwql3opm3850fPL3*&E?cB~jC0-@Q zncH@C-C^(a$ZRu)K(zkm@za;Iul}GTcDg^}kHUYs&642x+=63(47e2tn+6?nHf6>- z{xgP_AcJmQeTRPF+;JiDI~1$?^$D!NB^K3oYER(+X^kPsBbRweL!exl_JJnkZ!pwS zOs*9EdHf&$%I5}!k}wH1{DJ)$z*`S!`^tb(9QmbBGAd|7RrG~FpulmYmA)O13(2ICpj9y)Y3{lBbCxnXsGXwpgZE~y@tU}vr%mqCj>tXm-M{W56Kk0kvz3uYy-Xcm z!K4t{$B;17=4bDjinrfOtsn2Be)XCN*JcKq{MSL?`!m&SZc)6wlUGydYQ8+7FDJB8#|ogSEYFLMwgW$P^D^jH*lTy5c635@uQn z(EZ-ApI%a2qWmsbi47xc3h*7- z$P0+!YNaPf7=vO(purH^ClhaYW#IM~^ZM7Yo+z5NYgW=gPRyz;c=uWjIP^G+c)&L@ zIvnCFmxT? zFW?2AF1q5Ra7&ZFh0e{1s+vI(no_pacsu2R7$*dohD+6_zU9_&U3A3Dvlo9`c2vUl2e&@$`P;=0Wo(0|#Vdxl^UQZV{a?=mq(%B}Yg#`^H$pJAy> zJCkhV-`!iQMYppi3n=29dvu=Lpbehi6#|ex+CYD#4=9?{wN3f4Wtbo1gyXF?({a+V z6;jnytz|G|ER1sQa(@#(X}NLCYiqwaI7`6y{H-&`uYV2XH)!7b^?c3eq1;|Oei!nX z+Sk-w=u}LTspdr7zMBH~u%K2Cw;FC8;Z5KxLC1tpq&j5#OOZNiVxYD}#(RhrW@frZ z<8add@cLRquw}U20;l6Ami1wR7=&imm1{O730W)?i5txX9i|;U+WJchRb2nI?wVV@ z{R{Zow}#v>FEOkWJ-c zK3=#tLs*W?zPE#kSs{MY@KgGAD+U|Y(Y=u;w`2uEjPNZ6_`2~EdIGsf1q=gkOVt;z zDi2?5yS^^gs~ zhb=X|0IiokBr=XwRb>4@5{u>8NE#s??`~@Tc%5$5c#u2GaR_=Sq5HI*Eu){3BZJ)R zH4c)ib?fdV0x50Mrr`C+m)_L?=D6p7Bo=1MKMmr|e`~?pdpPL%9{$G{U;+SnogJUg z%V6EgUl)Bwp9v=a{{0m*!3$J^J&bxL_*VMvxleZG)%l*6Eag@F<55NR=fNS$Vpz)1 z=cJ{(JqVwzYnZP)_MZ@I>!;@Yzd?dBpyPb9W)7?u!wXU557Za%AsdV!4VgBRWR)9~ zTxm95R<){TWaP@Rehw81AxYcV7B7kwV`Rp`s&>Drv+?281xoQ5X;>qdI@MS6Cnl$#Qd`%3#Z~m8K93#s-95RVkMaZNlLPi0TfJb36$>F^f;EZ zwS~4-)z0+WJLDZj$1|Wz$I(9F$qvF#G(@A8lO4>52_-IosrtaGHL&Gl?R$k4RMK-4U+|6(%_MNkco1dmwvfYT z`&4?{gba}Q`9l^l`>_kQNLUj-T3$O0do;m%%UJN*Yu3n9EqpjKzq7T$@uDrfqSEt< zWl0Jh8iFYmdXlAMHKC&it8wDiTeDGVUmG6*u1bYLh%gt)4>+e@p{Dwu%0KS^huDhz zFR_)n3b6kC6@Th|(wm222&Uh@Oo!1O+JpLRynft>#8KLk-zD1FcBqKd4k$wdtwjA5mZl#5F*brg@$$KkUT zL@R3Du-@$a*vz!C-z+CxvVQhhifENW7>gT!%q3w(sCw9P+*lSv2UshlP$ntZOpBCX zp6QlOf5`>CUtpiQU!qATPH-fD^t!H5!86tXs+-RxP8OXj$MfOvFfv(5xABd+9dFog z1xIEoLYB{j0$E8Jd8tTs`BVW9d`Poom&%>Sh=xpK+*T5ty{e=3SEmMy`9+Ifn9&+_ z7uaY{)}yi1AHIcF;uVvIKXRMjdgDz3$N+5#syjacFsA~5xz8;|?!dk%@O*{wKXn>r+11}Eylj6!!Aq;s@NT+Z0ks99a(4Gty!Q8ea4N@*gQDS+QUF#O~RKdXX70nIJwqAB2G;aq4O- z1nez2*PbbRHmOr*WLBojM8lQeqW6qXO>|FdKFGVa6HwEqNo#!;mJdjD^TR*2o<*g9 z-x&Wy+1!WE2Fr}rZQ}!R95=+uxfoe{17xSGWBf}=w@GF8V45()o{rw$ZS~RJwggPhmlB z3_iXL9)DaJ_M3l=WWe(b*p&a3N=+|Pq7mG{4#agzj^5RuLK`HMi*w!;QgQx0d#jor zXu|u-mB)nDsntvTp8?m-;=b9zj?yXfi~agv!MrtN@yh?hgE}DHJN(VazRVRFDn^r# zdDJ_ci37vfi*}&J>I2(Umj7$=76IQVJ6Y_17fB)xOEfipP0Z)Bo@8&LW)FPF?2hOsxrxqdzam`-C* z)RMC&wL>3{4&!!OJ-wwWajQAw>Qj%EBksIWX>q{X&L1!)+GTHgj+}4Sg{|0CI0TN= zuL!6UlFGXAzjyuFPEEl_^T1`qq-98od@D+MYDZo&N?+ZZKxU=73HCgSc(@ zoV&lDnvOqPS!0o#|ztev`38ojbK6a^q4`C?&eyXAR)8&AT3koR0 zM`_7UbKo#njREF)oHQ1i|2eDBXEBnric{+V&$Z;5A#wYk2N~YV8hC#Vb@R=j{I14q z82wz|v$E?zSzR#*kVqdJ{}zsaIWB?F`#cS7hfDZha{M*w7LCD5Vh_WcaLSJq{;D+MNSW>u*wXboq+%kw zKO1eIxLP;h_`r-x>x+WL7%+a?9Ob5_epf;1=T90kVipMhe#pkhB-FIpPa6(2tM~_uZcVOcJ_UnFGQaI#{JhoX$O{5cRHc-{71EO!F`&OJCIH>GCv=EAAl(vzFLV@2z}5IblB-9ujcY>f5-9XL6#k` zxJFP1sC`vaWx6R7?RNFXRW&;A$g*V5EwJ?&{~n9(jd~y@a@vCWb6`)9KQv3=a-NIR z6T?G?5GQ3<3dAG*MOjG2o|q(qb^)_Lpaa68b9$zeub z!bnX8A+)chEG~``g5T=$gy)SGJqXKdc+LGw5ktq;aZo0kn@?<$NpVDzSdv1gz4L2{ z`(b2h=f_MY)bq4L8-ll=KGyVQ|Cw|CUa+uR`S0U1w?_C14l`_2xdz+spoo!Ld8;Q1plN^Fx1<9OoS#>=hG z4nW>Pi9JJ}ClfG@F^p*aQ?^bCe^TULJ~^BBEYKhow!Hb=9Ml)_SYoek8N+*1R$s)8 z^X9VIC?z>r$jj?nMKHqPJ_Hsz`lG&Azvk-qZmv)Y$$d_e#x6`^^Hu8?eBEKjwll*;R6O@p*nW-@_L#UTD>Q|5g?5F{eIO(8#k^L)0gc>o;m;BCD-%I!91=$> zI>JGwlMwYu%{W}*R18aV`#BD5J{v0@~(Cx3^_5ytR z9MCwPEjUz8C5ch`XWi_W^_m<+ucW1=X`IB}9t+;#+PSyAA{hM62c$(SGJC7*Tx>`R zLNR?dM4yV^9kJX6)Hbd%J$?f#`6^FHwg3rmu2+HCtQ|(N^6EbfsASl%9FlESl&4^uokXsS34N_yNcjeZp zR~{!_eC6Jk|LpTwlg@Y90EGft_dCC_EG+q5FbN2soeX5s)=~In&*dL&vez^BUmWrEM zVe;8x?y4AzvQGiNt-szPC_9 zN}OF8C#1DFWYq$#cil$EN(AiczSIwDFhwzf;)0#0|ntKaideD(XfKoE!YSKMJ&WLE-k z{^=)a4ADyMiXwPPltPBI>E453zkXrJ$=M3=P2_~;iewvB)n@n$YRa@};O55Bi(|7f z;g=3?;>SXTnfg%%49tp=;2N^=ylOnL;(FLpD8tipK-S%FgR9QIk%>`hN<1>+m7gk^ z$NEaK)`blIaaS6oo7U*@mBw6-m)yhx^<`_z`^n?zh9BKY0&ue&4(mX#mikfUB^=T$jHc3g|=y~j#mZy^ektyZe+HIo8L#05z*1cmX_$(qGQ^X?u@FQ*c!|= zN>G*?|0kpmzG)RPrKbQEfP`>VIK4%(4vXM1aLKuS9aT?}`+f9#ueGA9?q{|p?ZX$? zGDX<&Hc24=Bf`Q|rD7b^aQN-y-KBnSKBnJ!8`QnHLf0yzLp8lP(rHfJ__T*3d(-(P z6BCp64~=)72#|sDkYllHt_T>zxr}FS)0sIhn{6sNCFMm2HZ(<*M}~>bafX)-1ZlvS!e;U}II%86zjz+^)W?i`k|IQMQQ z3XAB@=}Imis^I!{YO$H4YLTOt>E{S-(*4-ZA&n!YZ0d^1Emi&2&FJc5)6W_QPWbe; zM9U-ysR%y&YJ=^Js^H@a%o|`nxm?#CV5Gah%pD7qQkD@YNL#CX$f;{9)6=zi{;Up2 z4X!j{NeTAsBd{bqK%`ODYXFN|$(dUzgwnCE5<|ow1CPwZo{e}y*Y*h;D#|tK@z>eC z!CdI1x`?yjxmDDl$};b}A==5tY0!U{3vU^J5bNHIdDg0zG)^wfo%k5JA2fg6s z+Y`k*y!nekH-eh-hC8)VPyD~oW>?bF=7V6Cg-owK+t_Gt-u5)8VDROsajp> ziSKo|2bPDmepj`1I~9Gfp=`Q4o*aJ;~mamdi+R8%_W(W-0j zW6-~l>A2W-^Idntv+eDK3cAgC(Eg0wy4m*=oIAX(odQv` z8F8{aIAM(ARluDqjegg0ehGX|KQF^O%S!~%oNlg)6n)SIfv%z6O^Rn3|X5X zuRp2Vd^iw18WX448%W138>whb8SDE2l4UXiJl>$Q&F%P`qFlAa;+*lpJ4Z^&Sk=TJ zgH7T$VSLuGRz@7lt~Hlwhl`1I!*`Ps_h$3b@{kY$cG=>>*asJaV2jNq7>(omCF0*? zUPzKXCYJO(kGD+fe4^sM!NTLVN6wk%zx{C69t$}}B#C?%9#rymjN5AKx5nhxq@@Kb z5;K9~pa55tk%!k+Y%>IVQrR)l@~C=gjbXo~+RHY(d?r6)Hryw&k6nl3&vGw~ z!QQPX62GUY$p0p2Qm*#>`J#unqZ8;k>0wL9f6e3o7+`{f?*JJ0C@SpW2*8vWKuUUP z+}EhdzNcN#+so3sm-^RBgoH*y#GLh;`Dw6Zj4qGfzIa7}qIPdhxub(NOa_rph>J9Y zGV~&3LHCx{?})Y+bFbceGrAF|>SXk*2@Z5oB{M!}v(Vw7Wuxm{k0>e!9)7*dv&MCQ zV^iXFFpGC2-|xP|Qu#Cv3ZiFA^LID%K5=z-Gg!B z!Ck$*E?XVD0N6xaeM5xbPk0()N;{))ny-eolF_7UUn7_O8w=6k3Rwar_AAXxMA((< z$wd0gKoB^=YF)43j}t;Njon-Ceqf~Xq5j%qQ>@EX(|g#;P}*4GSXnU`z#SFrdz?0p z+%;dP$R%)@2K;~SmzxC%L_}TSc>wNaxmj0F>NiJ!Rio=zPB6~5rcsWHb!_8O=yP}* zCEPH(7hK~=yf3SNiKF+sB55)Db|V@X?Ytid-X2C1UPdf;5OX#IvnF8_xb#5tw%<(P z5y6|IdanGgLZ_oi&8jK72Npm^Z!{PQthc|tt%Egtu-+StBE;8I>}*}kxbA|XaM+3~ zRl*`eAXomO_kjNaZ>KVV6?=>TJTRCN2a1!Tiz>(go1 z`Lq_{jId(WJr@3`($!c=Qe_%D-I{cz;&heNU`?{S^R5KT;@!InB`BueSNX;a+V})c z3dUj2)6e=P5uB^-vbP68Ay;22E1j~%N(BjAVF_yTLNu0?uws`7R`Q^MiFc)8UsHYq z?}b;7o9)e)@+Gs6?OW|8>k5PeX5s6j&8(4af%Z3SsI`w5a#c-T9apt%4G$%mw_E$2 zYbg_D;J+flxp`ROF`B(($zyoFj2fkC(;hEVhgM^=t4t2f(L2(g8~QQSoa}cd@P2K( z+#5i>6oCf9)Yf61TZ(`oi>e1qC8=mQpDpRIF%AeMc6bVw_oG#<~wX=>+8^LXp9zgqqu82#Ek+dak}NJxwK;BCp$Z? zl_w#yj1vp)m@KPYcu$MD2s-=JG~NXlWO~YI>0cJSS*!FoUPFmhUG*hiUgYY@77c-& zztfUZt5@{M=y*?T2WIhf_$C8riEq6}!u9fqdzX?3f#$es2|F{(@*(I?N$T}ULWgU{ zI0It=6Y4nZ5&fG&c?;(=h1iPn0ji7M~E!oy)pV$ zs_OC%2Y%AP1Tf(*-CW1?gze?CH-vhe!1+leG}rKVafzm zv}OczUW@B1u(<<-7B%^I8P8_*xujrS?{|D?@5_lkFJFY;0NK0I5HiI&4@yl_9fCEw z;FA#iWj^WqL{k`AF}5W57IP}==2sTD7%;Gx$a)}k$ZDaQNYKGmh5Q77tb2y|_cfy{ z`m|+N1S|=-S-oh2-)s{)BrQdXEUyrJ*$@R8zjN*R>orb>_}Nn5)6Y=07fFYuI09F{ zhUmYQhrduf=u^;~u@zuB!;DrVusy6$)h(DL>lHpS`+*x1mWm3uBx%>im9BIF{Ki)K zKI-~*Os6SVXAGD$11DG7JRikr0IM-h(oR=2>E&b7W!!_1S=8&h9Urt9-5(x%f@4?C zL-{GOL7*ys0xWGIB-v1LB9cgN zJ??q#+I_#FyV-raC|3W%Ikv%{_RX0I4gpZcA3p{R&yyMP4`?;1*Pm5aXF=$>Pfsr# zf}bH58P>SHA#B?*(G;g&i5^YgJhoVkOEcdLNEpYn%Km_9sWl@h9Z&3UsBR@9)%@%f z^K^9NY%agaZZB^vqYE5M`o0xU8APyD=I#4-Zdp3HmEqog+qyfjXz%3&t59X-YNv-3 zobzzY`YS=F)u!I&;Eg)AvHv72XctQ4Ni`jr-6s14f2sj77Tr9$j`wbXUj%$0Cr3p^ zdK>Uz_Be#gv+<*1re715;}GHX%;8~gb;r=&na~7PW2f<#qdp3m?;*MRQz7~rBa&ol z1EaoZ@pLFC@-#zuU%=B(bq%X;EQ1i3Hdj`Qk>suiwS47fn>h# zJ9pq~6t*d0G?N!%XYjbl#OB`9%Ef%=`=lN#*wyd$+!J(2jX!dH-$&NRRqYgdCW1#8 zvrrX{YaGRwaSQNjkRtFR?-5okN)GM;wdEbvkz5W;{!TgMx42QnqNtNB`J}&lNcrCW zJ{c1_HMpMvbUq~jl%3-0kVj82=AZ8nMg1=|gXJ zg1?m9EUA{J`Q=BkSbsT{Z#Y5dT_mI%b4Si!mN_*}Ic~*NsNg>IdDQXzS!b+HfLNia zm}m(aXNi#*n1m98Q(TH6RXRTNo|pDqOEFVT!L1rZSlpmBUkRNlBvwaGIA^k$P zHeeH;z%EvhiU~n}{RFKdPJ0W&c3w{Qk#3Gdh{yAU&Oh6ZW@h;9#g>`i)eGS_xILh? zWIKipWBHtMyW2H8JqU}@aVzVCa#>oUsr}}wuW|An%`#yS);0Dq{j6o6Pq16$;=X@f z$d&RKL8x3jbIAF;_2Ofb zbNi<+2G}F9cS@^@|-X5u%j|jlzptD@x z^<-W5&^Zi%Wc2l7qK~EcSF^GRSXC zE?JhGjUuI)%jd>GFlTlnRO~!zWo0sONOOoKW-F0JUjUMP7tkJLg^dgbvE;y>>jjH} zOX~6!==@|Pgm%{BvWt5Ha_H3J+bAPbQISucSK;O0C9k9s=k+jlghE)6DPExkq687N zdry3%Giq5f4cgZ>9cQm=Y5#a=mJb~0U{2lf=WXcyrXW^0wjyB>k+KYKTua_Z2c2Z{ zTpp)2x$0Mj=-JSqyfAKql)$pwp`tez(bBNdE2s|=VWa=2=en@5-3nzGQrmSfq2BrM z=7-7N1co2JOtqFTul2-=9J}V(3)4vns=fzaAg9NY51{>SviMLO5$@MtALhQ4bv3iD z8a3&j?9%;vMZ9G~Y5LN>a7E$U>k8Sg?R1*kr`biz9p;<{U=Ni8h8Of*0o6fSv!@Jp z_Bq$8}6CAuMo>d!I;!aa|*Oz8Wc z+JP<8Tp_XWfNbRHGI-G%%*RpS!3y?ukEtUp!c5bKR-Qut(ZDw&ijsF8Pnc2==`UMf zqg9$c+5_>?2&QX*T9(b`vXe0Qa@RcQ2@9~|;V(7c^{$^CdcV&fCyIkRatjW15V>S# zDc(F{)6}AGX)i=}sUlMW6$!ESgjcvO7&Qkz`{`#G3JK)a{tjvx*xoC+s4QLM-mBH# zskRCurUqIV(8=t%sSM7pDeT(423}|h9*A`F>mN9Ebumm52!5M?q!oSeGP=B%C4?@R z!#pea0txY3Hdzde6@6gDKYBlefFa5t9Oe1)#}5PWAdCbmS^4wG^a@HCZ&cu7;26sV zGO%3eEfUf=0D)!gnlJKunI&MC_cWA95MjK5swSCU;*-}VZ6BX+!97;IE_TtwAO|6ncXq!j#BG*Sb*n9ZY;aubYA?#-!sXt)(#s3MtYA%Pt^$}Fy5IuGuFo?^N^zd2 zKQw+EgRj%43ZOLd9ZI3$p%=#*tXMn^1!D`|#+5OG% z4j`{((kwGjeJwNjh>{4^P%2+4sM)uSOjF2XK#tsBc(~afs;YrdkdTQ0|Uu3^-@sV85WXrSsf`PE~`<|1QhL8+}D1`Pj_u&Bjg~?Ur zjj;dg2N3i)4n4Uy=n-$q%46^8t6@NFrLK+?tiu*@#qAXbIG}>KWa5F9=Qj2!oN{ak zNDd75;_<1K1l!bOs;50+bS86rv6vmy1$mJ+jcNy3cNba3q0Rnn561d^k2S~Y9yQa* zlCBET1v(%F5_bAIIaPYNi;IEd$xYQQMdtSCk*vOeju{M@hlyEv12MVHNmg<@vHn*@$9_JR^>Ne6T?x{ z8QQDYZS4INN186{ty}an~dBXC1E(O=@k8ouqMz19lU-(?0!M z$i+A_H`wpudRdwC#bU=kqC1W#F7mLL+Fo0Q@8h(04d6ml?FTZp$ zf?f3|#>HOLhuIs*j+vWkmtvt%C`6Bn;v(re(&ghVQ*sMdIc}+e$1D=C)&K#KJz){$ zSLh9--jt1`n!rLv8An}J_WQ(+8bW26J`u|6mw9BdC-Ou)&V{Pn)IrmavU^mi9VunL zx4f$|k1ZBCn=NYAv8@;H@c2maB=D&;t`qxM{Q*}zUA51Zt6Fne#IzfWICx-KV#hED zQ*2ZsS^cq7v3`8B!LTqzANzSzC}cGX;Y+Ud&6P=btAcg&BKvuwZ3Z z@}xT+DZ@N1Es{|onsV`IiVoUH!Owad!maeJ&#qUNMFQWpH3(=j)_PRupB!~xWI9!C zpRRAfCo|zE_UQ?Elz36d+|M*k#BvtBNvPwMx9VBLdmv6)T2M4_HihM?l5xq{V4;$+ zno6RPgiNM1q~1DeH0%qdbg+*yMk#031fz~(_&Rg$TBpUz8HxlUBLDKI%pJ4%gYe6G z-Neg}o*T1cA?EwvNt~~5+(KPE!9SH07e6k9HsDD)C&>G_EXCS;owBo!_u9b*m1Q7p z9)j`6DWHxZ;h^c`D>jpm>2Fm4lbtxUs#Q%d|A-$AilSq2Q#^1Ii(jkE5D#bFr~daE z7>E5c=X8qwzvi5jm8U8tQdiV+Ie1gyQUu+Pl=E2qv5rgZe8Ly{gy`2eN4h+);h(hO zXjJaMMMch=Pq!yO+f+Sd86~6{i$+00zZL=@#MGqm@1`;VdDB+kbA7BA;#Vfni{1MJ zc;&OkmZdC~4qT?=OKGJdxok2XzBF8AOTf;Ae5YN6R(G?vfBThQZ_s`5gJWMCCp5-E zp;=*84;T$xZ>|7ZlkT-#(rZC%39NIA998O@N#8k{P)JuaS;76_oh!y(SwcRv+02U# zhhCgkMXO((Vu|2PzKn}{Z=CsL`hedOQT|G~m*9TXV2x5jj#m}uOWf3W7ZusZz{D>n zZ|WLDAq_SPVqhQK37a>9F4qPswm;Zes|_tX@t9p7arW?H3SO7Rz3HfL4COYKh9O;j ztFBga`2}=>nz8l{?>3}oK2ntgZ0AW#I{6~#w-;G=^>6a}J!MGzZiJAQ$&TUohr4Uk zn(P^a`~wC)1;3EX@d3{HFU_8jAp#Ntw4DE(@c+BaZ@*TG3qY4265?zc+~QGeI>Fsi z)cd4qm7Eh98>{qQZG#vFhlJ4~Hc?HAxM2e*9{wy}$DOsc1r0_yM5HwcK4Nk_zYI|F zDsoQ*$bRwb*V=shJPj5dyH)nTjo`&98XEe+q|*5DBoF(O-t!2X@hFadoy^Q_E5H~v z$O*Y#u3%m!H>wT=@mysFfD(+Ui{8hz?Ya!EcPL@hc7clQ>$0Y%6%VGf*1O51 zG`@h;eu~7vA791e?aL~w)J5*io?|iT)Ine7GKpX@BFi>E{kz7i;y?Vhj_GjX{yc#1 z&XZy#ljXdAc{#!ApZe||*`>LfBnm_7L18(iN1 zk4*S#nZX0bjOa^yEZ6z`|eulQ=Pw$sj^glodb&+RAy{N z-4d!w2H3Op_vo1B2sv&sa43P07CKtq%NV@{geZ^u0r~f=6*1Tp0+2{wn>9SdpHLxOq-s7o<-}c;QSU_GLR-AV?HDo0V z3n0VX^@rk(KE^N8x12sk@CM_j(h2!(W9djjZQC8iZDM?L7XF-+00mTFbD`Oq!H%INyf#zpKB1TG4y-?g z0xrh{Z&XY)Dci#6P;$|pd4gtU1OfJ|*ycF;LLKQD6;Yj(UVFMFHhh?-YCw*4r5bxn z3`TF-lKePaeVE?#fDi|6-Kqa7s$(j5J%mOkJ9;W6Xwi3T94z?+{h1vfXu%)e)bvhx z`7J4pZw0>xHkPP-C?sJ}k5hECX7BKgv&|HsU8qCuVgA#2j%5L@=pM}Vi`&c`&gdoy zMMPCXf(wQU6@zp~D%F=_m3Jbsl(N)RMVA+2#KQx6pi*FN z;9=i5t(QBp2MlSxsdGHxwvBu0+T|M(sS`884-ti%auXgr>bsq6a*03SkC%L$7lxUq zoDRf!WKgg8~sx9C$9x5DbG-x7a(7L&KJ#!Avg`Fra9@K5bFjsCw6xC{*1wag9- zU+~aJ0l41c{w>{=HNR!eb9#J&g}74NEnGI~sM50T%i-mCfq}`F6|!f(EQG!GojRHGK`b`VOM zh*VR&RnIG5N$cnO0enNSz}Vau{~v2#9aZJqcBx1lx{h>rcXxLl5R`5S1tg?V zx~03jLnNdHBt$x;ySp3aLGgX(`{wFxH z%EYvAuZ+I17S*H>%}<=bJ9NxZ3*Ea{Oc%~7oo-Z*P5jVPkERH5BxMuPe77gep|34= z6fGDc7=5x0i0gfY#+5b@@{ik3d|NXmgb9U@QFdqU<@pZp(B;;n!JjT;>%rZSRo@JC zIS4OtM}XGG%_TY|?MLV+b?NJOmX@%x!R@3^1KMQJvnxx6PzNBB?~on zhtlT9_2I*zHIw8fv~@&se!4x6kSE6Lv1mx**7ccvXO3Zno!lGk-X4Cr7K}Irl^^01 z-#`K(Dm`9&=vlrb9;@{FnFAhs_SuSnysOC7<{C#O?QuOlpErM*I*-d3f*ft~v<(bS z4Zc)7@iMzB({0!wC%z2qCj%jVJjiZO;ziFTAv-9)zben}GRXe&B^pf$b(8sSiCJo_ zG8kxluWzFS^`ZjWViFJ9>dwmDFMsM&`EH00d-6uFm=I}((ws5_uErC%4+n5X1t`;I zI$*)~X68cRLB+nAdQ}~@=!Y1+E zBC%9^Z1>; zcq3uA%(qr8JY(By>2Cx(_a@4~7)>iU?;ku|HzyYYLX{)pX^EKHZ8i< z&w979^_)dx+liTtlyZY1-_%+<4o*gt0sCn($VT?&AV8R6pSf1x9WZyqnQ!r{=ks{&X)g!*KG@F-mB{X%4FH@#R z<3D8`^`&{SiS(YG;_WsOv-m`w$Jm|?<>{nuOC^g1`(E^60tdaUv6_P=GOQLbq)KXI=+cKWMt2Eh2crQ4qG3 zC(KJcUlJy{*)XR*F#dKAMM)x|?R9Sku!s7B>?|;w=+sr|{1u}l0o0F;3E}4%t$j|5bPZJT!zyJMu%%l#{?Z3*Nc8EHWD_KYe~k0Q1mKzl zO>sJ>#vaD#p38bBuqY-8d{g_4Foz5zZ$6BF^x*{s9yAeq4!08^zQ`vfDX+PU4E+-Pi>ST^8lLt0A*1&WU)lN!E zk}}>kN~gj$#NWI)T&5#f=;ns>uGz2m(~>_wB?l8?OjW$!)e=kNBKy3b_&QCsGfzkl zAJPF1rK6NvbSIJCgl0oOpi=;NVm-kp@!>8pUPLJ_6N|`T+#8_wVi=unnvm!Og#ilM zpM7qO|Es+lP8P|=Y$Hb;ZfhTqwSN9sUy%|);v|)O!RIrJBs*Q-^>eV=LcjW2Th`8VGm2KV17j&bR%+HZZ^=XQ448Q-BF^n5pU}&49^3bgae$?9y!a44 zM~#MVN*AKrYe(h~puGR(H*(=3%87+EII|{|gpK?KK_?02{n;SgCekY5C)ybpw#6p_ z3;0`#s@iY(JWf6-7pb@p$t-U`*)Fmk=>qGJ-&>Ii$!~j?;sk)H z^y+FZCU1$lU418FF@>Fuyo=!-CKBVJv!WZ01$VDQ)N1oC0S)NFVqyGK%6{R_~QKmVHXKh_vwUwIW#(n zNCu|fg><)IbOFpN9nH(loLlBqj9lBBRHwPZ8?RqnhaKm@BCZOq>!~)yLY3>OC7;XD zN`W$}2h^q<=I1}V5EFtb_^+aUa?%T;^Hm?vTuAOjJ|liLnsF3sQ2da0XARJUtMIGp z^ockcm~=dFheP^Deli$RI~0tYB9LRi896GnK(C<3ZH#zK8-X-P=E5}cE4JhtjtCA7lkgALaXnE zrs6{4m+|(z-PY&+GRieoY^;z>dM%_rXbNXU;?rbJb%I}{see5+5i{iV`QxIu17Jp` zZ^J(NW(h|>qrh@^X!cBOZ_~tov>j(70D&<8Y$vO=`&@Ws|fn@W_PXL+>JVF>$Wo79F9usl%V8N|#VHTd*= zLSuNgR-gbp;q9d&Dtk%yXS9)k)cqf!q)PlhLdkHTN%sV(mUyBuEi`<@7y-O}+*=`R zKdoAPak-E$=dZ?VL55Q)82yMXQJ$@h3G?3tu;j!7HL~DE_)v2wcDfL+WTpVygFG`W z8`c?cDrH`cjz3Aj+e79c{+p&L0=|K;dey;jvl4U5O{ME1@qC{-9UpP}!52Y5@a?fN z**ia)PTUd+oPz3g^hOwOM-?tARDo1D!s3f`nMhyqUoVxJLM;SBW zBLj5H>ipgA{MMV|yY1?nwm^~WSUmg#^0xFq)_uSg&?o9ZO7p%|-<(Q^A4eVyS@<|{ zfJT_$j1(%o*%MIZ(BR>g^)@D+y@Gcn#fwZ_7;)Pw%$HgpiX@kox-A9tjDq5nODJL+ z8ji}cC&UK;1-0Q4t&x}bw&zKE_657K<}5_5$@&hLRF0RF>KfXMRpd!0`mb37_hmD| z4|neMfU8F;OeR@vi%(liYa_|3OqJUq-xZi9CM={*K`O2RD1=!sh**O~FN$gK!uSY_ z&Pa8%ot$dxgRCGCOh}ZBI4ECL+b_h^4+@bnbgc!1hlH!(j?&E$jkK-Ek~H}beBk}v zsG?P+Qc#N59aA)$iHa>hwT$qJ_O>~H3-o!zC}U(bUXnO?Xlt3jgD(vIT?I2sJ@-pr zUqn@@TO*Qg^4W`qme|qPPPaX6IelN1|es{x@=1pA+8nVGD_Vo zTttNc8_82SkNjdQCT2R_R>VYtwGyu%kX-`*^1|pK?N^dqLl;@MmjnTr@}@Mr36%4; zfo8P>*~yx-q9JQc#!>3UQ5+Mit7>Q!k|gm1!zxA*bAld$dO`q^pLVe>!BO$nehz){i~MUBFEzVdG0{T!vfsPo%I}ANI7D4kQBvw~lVG9f1ghyWiEL}kh}M=ihvX?YEG?Zv z$d-$8zQ~gP-sD4j{ltCL?fP(C?oR*H&hkeRDopBZgHth@i%OZO3_lH@s6`i-sPhx= zCo%`~;Nx~+i|IS!BL{XDE^oe~tFEr5oyY2??!cHr86%G{M7Y&B$R&{=xa%-tU7c?G zdGF_2Y~&a|;d!xU%P77IVm|RXfo;+Itp3{*)68^%9sMKcNbq?L%-aro)@~ucmuq&s zd~RnYM5b&sq_0eK1}d-AR*?(=U{#!39LAmn^1*o@vXF$!iPyD8RLz}`ErOT!SPE%FX} zCr$YKkQ%{IB*A}96x1Kg#80WS)Gw>M(J}p=1WKuBi+OKhgMaoy8^RGLuKl1+&tcTA zNtmnzY?A_yia+f5xaYf~i9TU+30ms!Cg~(B9cXC4m%vxq{6wMx`$td@-CTM(+85y< zkl~xUY`PIQ7=!w}@cjq+!hyZ3Sic{r$Xnn^R)u!(YrhoWv4rE1*Dcbo>p!Xa!V`xv zBA0wpLkY{S|Hy*v*S>ct9Y~d`z=3PIKIkeG%r=x`e8p;MQW-OlZb(@ufGIc}v!Se( zAtd&*tln$9ygqtuKuf|?Uql8nMu=6oO9u7TnoU88Z^R|)ebX5Mq4R5Q$}c`(Yb1y` zLoaj=TEMhbc7MMgI9z;)pWP zsG0B&P*cY95(H8$a__a=1QopBH0J{K1)%vUDk>HQspL&nSwusky_Y5~kJqKeTgM>8 zIgH*P@&9~}Sr*B(N1vC}@147ls6j9(i9{hSSH>`*Xa>@sMv6Pq@wp(GbkkV`o*#hy zjG45=bdY~*zr=W>w3Xz2Jv{66N+axm59%6amqOn>A!#~`z-~ZzQBVS>S*Kt6uvJ4N zPOBi0T4l?NPFcE(RS&$Kr9jsT9ggI;m0fkS#p&>KtB)f)OqR%tr1ry7pdKbl8izqI zx=QADLV`o)zQFFUdk#XL#$?zi5Af0;LgI`N*q32)%$mML6;}T%_`yBT-l6{v!(v< z$GqoPK@oyz{DJrxT<8^UcWBb#6Xret8R7ps?7M*jAMTD%9^CMph`t5S=`{v_L{o2` zY>s4Kr+v7)L@iVt(-O`R^!|wXC+?{g9Osi$%w~kH)4xk0EXo}LN)5;BVV`y~Ip7(G z#iUcPB%{1oCN}}ljf^LMG!GWap1UnGnK<3aj%S_!3;l8{C4S~Du`j~CD_oUavwMTy z@#S`uqtuIRC{D~&ilX-38UiXC-qPJbhZKJV?# zaeS_k-m^*YK>$K=~x(WiK7*|NZRyuG9sOX9S6JF*a;mY}_c& zOjLaQB#BtXu&f*OKa8pX`f)NDG7Yrof$G)~J@+o0SErUg3zrw$`uBXT%9E3mF0$HOZ+?jY=Y~Eg1CXH3q{ST_nhNo; z|8!3LxBm0sRsfnFi{zgf5UXVR4>(-!XP1E}ZSuhIkeba<4WV#WJGFj(UJYz8mvlm} zs=k^c94$rs>Byi6y@#zeYXF;7-c>zmupi3MNgR=u{VZPnt$#y7sUz&=mIvq9dyC#x zuP5CQUBUHD;Ox;x;q2@v`(U?8o&e@mz4;ZA@ANhysq71Q6dlYvKD6AIs#`rhK@f8d zrsQm|Ip|ZTE{oNr@IhU`Ukl8T!|F~xjnz*05{Pgv-s6}r{42yt>Hp`=v-lAVfCao1 zsKlw2QZm?I_aDg1?N&K#LXNH_2`}x^Fx4{m|FE{sx&Mcd<*-}iQk~Aw3+L;?rDwZ) z1Z(l({U9z-gh00)m?koC4#JrhCaMdkWPOUWcNlS;7`}?dH*-`=4Frn{Cg*@SH3| z#r3-_3VD7GYJWQ1$G;73CGtNrtO)nN+2r@fn^OMD;8A`?2h3GQmCAqBkOVSNU?!DB zp$aG;x1*|7&4KHb`>_W{sw~7)wShG=Y8PJONs_9I*-7fIYx?^gx2Utmy40~L zHt<_`RThi|miG!0B^cQ67y==^rx26ExjoSy%5C$>8~lyAFjQ6PZ3To2zfc}8HF+h* zhtKvcN9u8)`ut<3dBj0#h))}?bAJu!efjoS1ZvpfUB$)p`=e0t9>CQaD2#fIzjA%; z$3NW9a}~z{iu^kJPrW~3dkVN^NerwCJnq#le~kQqH!m!`WqYbv`a=!DGZ#4|wFy|8 z=4NVN=VrK$jJbU}Yr18-UFx{cN?}&)+f_x7wv3049QXoJ26jDn;4xFEs+}VIjWhfY zVQaeY06(-9ySl;d0JO#8OR|W?!MN8AbJ@G)*4Y!`JDCc)-bQ_f8gU5fiq&s{upnUS z{gC~xHad_a;k(eVUK2iYDn9`T0DdoW-;B$5VptSv|M`*zY$PPxaK`L%fN2Z2%zOU( z0`)d}>>`u&5hg^z9pe!$zd)&lb1eC+IF> zxnL_I$XlxLB`r!P`U$-1ZtWbn*54THjj?2h(5tlR{kx)=lD``Hn7g9bT5eK6fH;of35cOIa=a)z z6w`@)W-&>l_T>Vi%q@ouV*z%f!*eptPm0;b4D>~!pbW|9(kbMwEi|s(r+O(0i`Vt! z7fECw)xWmzpdm>dq@a($;ce56?m}+(iJ74hn@HxAeE&|C=-i8>;s^%LI6&VcgzjiA z;~9JH<1n4nuvs&vT~J+^+735PRcI#$`%39iOi=-D1sxz>m9#C$U1y|LDVhZyh95^w ziD2?bU(n>!P@??1pUY>B@c5hg*pHdd-u5T+o3Z~x$#)uBV8C-GMatw@X7lO!Cn@?A z&(2#;1PH-<=7yF0HU0Yf7|s7dZ2d}UqzX3-Yz-PW;bB~}P+^MB zz7PqH^1%h9q=-EoH)dU@Q%i&^an8xLDMAm6NeQxu9*WI_AfFP3Q%t^hXCS~;iqeiv zDvSxHU%U#VR7G(1Do7xZQ&nCki3Vd23f6jt#xZM&d#3A`wFIT8*jfM1QYzAiI4o@( z;#vL^rNRD#MG~|?zh;>`JkOF~9+eNqN9E&>Hvrm;)*JA6ip3N;&^6#j)i{R7XIhOk zTb|cOUG0?Vze=dWJ=fx+idBL*=~LgjUmH6d{3-V`%Zl<;V<0C$%W_jjyHX)a zu9H1xLw+PQj@JRPcyh+JuMdnYUDuRluI5Hg#YpEbk|rB`_?@K5-i0T9#9a%Bm7b%{=W{+Us>q?A~?LT5@EoQ zI$3Lg+c?~v&zGo4PnyD`0edB>msZ*35UNDs^{+pdrQ=M<4s3}WFK>*t*7C%!=5mtx zRX&BsWT6-Y0&We&!`aH#oSzbXAk0mNOu-y5FET&)>SlzO7fWxE z{5J9uM+$`E^Ct%0kH95ztkx;w3h4_6^#449ffBOnT|Z*hkSWU~_Kgybv$+RYY7N zqeoLBb_p$^gl=q}U1GD<5SY=h^-k^L9NBMc5L(yA%ziu;4jT0T&NdUU@;FNr7@_~{ zwELj`t4r!JKHlHJqK^xw*ne0!xiJzvlT2uCMLg+g#kQFKM66mPB-;0u6@iu4rw)rV z_UO{EC-0R@KDE-FdqUZ(ir|`(4hU3sQI(p4fV`QVpoq z)Btf6Kkx$KMfUAHfUELS4jK2uR?*?uk8k~6o810ER?$CX1_F~94Tta6$4sB*ruL%4v#SSyZ$ zl?UFAgDQ0Jem?hU2$T5^JN0IPW-QErbBrgy`-uEQZ z;)LeS!8uaEG(pd6bDa1~^E{6)nRJ}L;xq0L;p+{5@YRRFhuXhp%HczS3k1%C|NSvD zXn$xneqML<|7RG=jWlcX;d=UHBRhObgVj~1dRkNdk4u1;kC$plDoz%1xI)rYb^prs z{lIce_Zcvi6al-W4m@ss2-R)tf@#5Xy3N%-u$-L1P)6{$~_u!6rMj8S6Is*eW!{3QE>augx4CUzF4C) z3%L1UNZIoUGq__19&5~6pK6_!MTyO1|1JL?O2Maev!1CGnWX!L9bjxiBvcFY|Vg_+G@ z$!C2I*IWA8)%08tv;8;Z}&J=PfZNxr9ctm@DCDEx93Q}*rO-iq>fnE4*soj6yw)3(H= zeMp@O@Tl}uK}sAxo9_Q}Mu+CDdQQwE*HZ^{wuf!U#KrXQx|?kH<4XINof|m!_s4qj zr(=&Jrt;kN(3h*56pj^ZYQNCjJsX4k*@bH<=D-Wkqz|YLe1mbld@`EjDOGnxQSI=Z zr!js$TEy!>-;+vK7u_o3EF(Q?lcR}0A_E-FFA^2(@692MM=E^2-{eJFAMpc{Hi`cf z1(7Q`JQ3??#DGqb8g40O?bh0XHX_H4z|JOvT2wT6m=jUs;jC>=O&spZ`65yjv{zx^ z6HKJbDW3V`r*G$vp?6)5C7Y!ZX@245k?0FslU`6EAU`K!?`xq?eKxvalgG#{EW*vy z5Nm`SHlwE+n^f>F@)+T_EQ0%3ofE%4#b=p>pE_a*R6YUpOo6S@OH)I0T%=< z+P?g;X;*jTZglY`-Qx@wi{T}Ud#5O`_2ml_6*-}JDuEB_W;7Z(^&3D|tWf4kfajkpyK}YEPm!|%6J|?|OpY*2B7zJOF#D!;=88|@7(PoK zXIq;OO$uR?6hur*rY9-J3f+)pUVs-;96#qRY0u<6+nLkYh!@Nr3j?)ij^>79(;i|Y z=vDJg_N%*w9|L7)Ci)M^kS&tnF6pm@&QeYH=ZLSFy}EXvS!iSJw#Ggt(jd{(x^vXI z4cs!yU!E)kVs-mpZb`U>+Gk^TKiC~VohJ9ULTw%Zu|S1BAy?4@b$jR^UAcVKBWdtd zjZU^|2?&B%2EM~1|MvJ7RGQ5DH;*l67( zcx#f~!jhz>;bt4Xs{X9V3L0G?s#Jy;noMpK$oJ-D!=Yoz?GExuXT9o+eg8f~KVDQ+ zTi5m&g`B}@OtA`n|Kl~CP%EZ8vQADVGF_LMtywG`qTjzb*ZbqlHJr@}(nPcC+Y_Yx zfGH31ru(8z>S4P{RN#`h_a%Bwt_9KqGQeXK)#J<|>7MAizlh#R`_gT7_-NcB^RBp7c9M@%bdDk~povm7{yJ4XieE0s_}NUlLf2{lCT@|7`$o5_%^M zksQ<_n8YGUgeK3P>UrMtUbK^{kBzmjDz#gNW5Ux)3X8+E^EZ*n)#Sj_wpy~~vm0K2 zw+m%bToIG=UBpPsQIPP52+V^%e^0~^UgA(3D2V(5ohTo{=q(676R>WN`TWIpB-aT7 zDi0!P3$b0!9!8TPk5tIWUFI2gzaXgDy$4%p6U^ah>Vt%go1O(7;)i`78k>@zTg7|# zGq}OFzxQ~ds==IyC9W)XJo;-hkayoeIL{xllu8bHB26I%4<&Qve9|)lKYekp=S;Zn zGUH?yz44w)Cw%F;LrkQx_3%#n^WeH2u2<$`$qxpa?LG9!m9?pT!>kxQBUssOP~PWF zbA)N-oQiV6u=ZJrPZ-I>?krHS(jqu>)ME)OO|8CPx{^Mdd*hxGadqN1PQGknll+Kn z3H*PHiT>({nAlRY4RGcA-6_8SB3Btw#mP5Z0_QXKUl;xH?Xa>RJA?c@OX8Q-pKE4# zhY|$TLAHkhPMWG4O#=}eVbWxRFnTsm^z`8f`bik#e8uAM;#efhK}&+7jkHK8&!}V| zMVn;8vnm?qRqy$IxJA#SJklw%dIZ6$G@IzoTv zk*8Sx<#=Gb8a3D3XNt5Q1J0UfYNQs(mlyg20+~afIs?xm@&oq6%vgR{fDSdFK z?dcI{kBjZAPY)7N~gUI&`A=cn4M@ zQVD&BuT{Y5_)z^170HdQP2m&cLO#B<2+Cp4fOi1T-|^}2t2ii2w81-6UkvH66LIUm zr(WyhA=W4ka~N}h;3~cPSEqDarBzcQ)r`c@E6vOQfM_tGU7q}MJTr_3`(&5!sC);%Bvq`@pa#G2KRVlA&$@KK}n{M>> zxC1wTA2#iKDAzRt_fcF0sZCx_Dwm$JM+wv&YUBC;cys+?EB}CMjBtY_oMXVK{Zn;d zyJ+TNzqc?8%Hh`#v5mx_ZZx0iF~6-XEayhuifM9*1UZa4(EukHIx=)+GK^5top9s+ z7{9M)OVcbaI}T?OYK5tB+a7pOMo6{qiMr(Xh5Tlvg#vy=`qoatv%AF7aXqC?&0|TS zMGP~xdlp zaXyOG&+Pk<(m3R7Sp9+=T;$i1kKU?6=(lE1mqU}xh?Jg>*<8f+RY$O5GHb=r(g9wr z&;)PDHnh!=Qf0NIt2%zB%sKNNzu=53P^BUuP&p*>CCi%P;i6ZZ&{tt zXp+=m16 zuNjON?k`EaG~ZL$jmO%sL9K>j zmFzjRCn7eJdXa+FP|6E^VAqP_YID5ShakHJI6A&dV(Z2nC)(A$V%nayvp_A~A_ZX) z&X)eFc+04FXYib%A6$grSn-lkCK@{d$K@kD^w!WAC(5ENauhAXx*x~RCp~o+ZB;dU z3?X7eZ z&c?HdFB>iuC8C~=a$sx@Sr5G}0eOQrsAcV?1=dsoRXsnDN9Jq=k;EPxn`4S|(|)?9gc~bg?~VdT79tX;i6vMF5Sl=}daE*xfkahn>$$ zjfY%pLZm*P&&$=FzC4GB0JtH0M9#G|J&}f!Zyyt}05vU}Ue71pR@ ztH|))kq%BjX=#C9l;Ly<{!Ev7P9TwRbg8AfY85a-n1SZ;uH!fcYmD?0*l$OU6dXuG z1l`H4(7{N**G7R*_xj6JHI26^3zVskND?U20+0DRe6P)lSSWxHj=NoZ`{U1T60+qD z*I39}a18IVAl(wFw>q+`+jdMHSz;BveNac#$3CJgw6s|ceS-W}4}#AI(67~K`jRzo zxI--xzAc{jD)^%66kk41I*;UXM)rwtQmQoVkph8QHRp4Cm=C8rtXR z`>q#Uy(Ginhk2So_>ju$`}QlP5=;}ZE{M1hLmNlFMXq&s1W)AW@^iiQ$FpM&^rH6< z*FW5?U*vzdqNcpozv<9l*uTHq|FNv}vhIDI&SmhA#K*RxJG&0ua#;cqpZtd5hi&a$YY4yF>?5FYbd{*9+PukD0oHJ0{d#F_9S`;_X&REg?!}pnAeQa zv~Mek3=)klL-iJc^Mnn48bdtpXG#yV=%-m5l z{EjM*w-z_M^sE=zmo^BWviKO1&{@DxFHkT&ZBq$>g?B~!>jJ#bY@s5D=nH;$9Mr>s z8e>+OR3koij5nbq074JVR1>)FCJ$f83?PCO*&X+BF9X+o`SqDS!^{QC#OO;Qa(c+I z3(4n3nePe&aSPuG3jJz?O~Zq8=iDQ08UPU)@+p1~^hItpK)w;|1F6WAzEqFES6ES` zgRO0dZab_Ve9gUc#)6&)kr<3P)}ohzM5+)(rtz#P=mp|4-Lg?}C=R6@v34yoMpfJ11$_9@rZGc>2_=+1nu6*18WxyW;x#TQDxYe)ee5@~NlSR@utmr#<( zB??-=E*>E?qM?CS!L$sYv8T_FY!b;HH__mi(_y84psifHIX!wUz zD;h`v9y>vPNwg zDIbYo=mQZ9gJ?r15Kw&vD-Zk-IhA}@@;5*1 z5YrqVr0sFv%#Aa+WfFN*MG2RovTGEIuAd3PZIe ztFtA3>-R}tUOm?|ybnq(AXbCP|5@fcyFE}~7j;U38O}@dUE_IuLl@dRxd;ljIfFMf zEH2@jB94xTE)knqw%V+e3hBWGq|v^fL$(A>y6Ecs__lo=`rqqbGy08`h1J%OBCg?w ziu@pt$cu0y-QlZx&fIv5Q}11h06)C>4$Vup*>3V(df+tJF6Mw}OnzAi@02u>#2Xw* z$+*T7neBa0&$oE$IPAWZe@IMP4)JD?wtzo> z0XyfmFZeDfKiw0rcJ_U;olum%)`SxUfO%}*AxD$9HqWFhMolDwB58%Dj}jMl@eq(WH$%Tyf}NJTe3HNd(%LZ&fcE-y#_HtfJ6Nc+;>( zFl1_Byrabkq7b{ISt2!9EJ3s#ldLsTxOFDeu~>dCU9F>J0^I}n(7bmxcgZkTEZ@vE zp;%}my>KKtIxy>Zb4R3MTcL@@&{GaQHs`=}RYS{JV#S862qJLgmB!(qXSE%;Oo8ok zMsU>ydTHmfyBlJrQpma^f?i_mQoH>IUD)&fA#3F%%IaJ_1IvvH^<sl~N{NuWa=tu(hjj)P1-#y3V9W^&Gc!H?)+UD{cvH#U12p~$PwZap6 z+JA99{U0i-wa=z1hx}Pp#?8S4YQQ}X2j&Mu6grJEw6(TDhi3fCpI_{B=%qxYAHT=@ zS{+2$SCd*ed|ynE0AnTobAUM-Zq$3&-9jvIZF0-Z=Ex_FU=UW?&Pk7h2xqm}Es!Jdq1%jXs( zBDz@X{&vMs3XNuIrCKh6TFE6Im*i77?uuft*j`YC4v{vpb>oRrZ?cP!&Zp3>A}^QA z6d*YL5+b3Q&Sd_jZrpT4%TqU93&l>YdsWE7ofZNitf^awUvNX}(GWCgO}cG!`~|vr zV+Y;5p(z`?cvGJ`3+RHazl8LQOVlU~{vvX9M;+x#RA`-lE=C4Q;a)eIsam2xs4EU~ z?%3CT_HN1kH*BT*EebqHS1o^8f%H*xQGkFd>@@0VM8B}&Z<5vV<$ge?UAK1 zA6e=x{R2y_iSt*=oPW5|aB(q!SYx#AdavPddj3vBgY&2#QzOl>t_q%;3&X{3xz^cV z)?$!;rPYAva5qN@7M`j^Ou_LOijOmhqm+zb*+m7^w z_G$cx@?5S`Mo~R7$`KEFAePS$i1$KdUWAU9_13j2zL}x`T&=Dns--*fj-p(%#l;T< z36fN$O#ZCwPSl3Hq8U(C-3X%1#dqZ;MuEGo^-# zO>EolFsKFIBduxI9D}W;7x67LS2|io>oApx1RH%R&c!yiq+Tq;zr$HhrfxN~M>z%M zv;P_3>SzyAY~v8S4))L^(G6emc!BDYg;YaaK$`j?>CJs~GHS5#Z=!qsY~2JJmJDc6 z&xRK4r9@C3xmiHX&XuI1b=fJ?!xSIk*}ayCnvyf2eHa-TEYSU=QKR$duYys5&mJND zTZ4((7fhtKXQVJSwR*b&b`?18Wxp7&ZT2%@38a*dqshFWQc8JQ5&WiX$C+qb)JuR z>gY;+qzT|aS^nj6V1IK5B#147_x*6w-@=bQp(`=LLMw;+%mU z0xN=YzRSJ7!wyqF++u1Gi53Ne8{1!X^QU0wh4=J)S`>?*)}wGqlg}z`%)ut3H7K4BhBpy3?8 zOS2-SrFY{%N^N^KdUd+}Zolc=2)Ht0MKWKDR9@F*H5R6@(9uJhOxgMdtBd6oTKDsW zSH3HJtAf7B}MrXHM z`Ipy|+ZArT2kQJm3ljV65;_XBIa2uOIWuz3O?qT9l4LI=D(vV0uCtcIrU6qoBo9p+ zUo1_^tp!u3+CiQY+TZ4r(!%filEm*-^xE?KBa#Ufet!YYL;iTAv>T+~gMHQa?_h`f z?}HunTJ*(>*rP7ZTwsD&g)?{}Vlm)sokeT{(D&{P1bmxH_)Dq82fl4Oo}rUkDJrse z9W^n~?)c>IRF`tT|Imoj-X9lh5)-k)mVl}xx3TJhrIa7cngLY0b}NO7r2 zkhVlPh2Wj4eNx|rKDT{7F)(a1j2@scK=Y<9tGQATJfwaO(waCqIT^3^#5x1O>^r6B z-EsOd@T!SQ|M6w+<5$|UN$7=}kQ=}ZJ~3MNVtm=O)-s5{t=`znBW^F-T$zAxwQn2sqQ5*9@+hu%Y7cKp& zsSPA{&Zh3p4kTe@_-4KqccA%F8+v)x7E`rn&w&ixi3IOmc5N+@=g}1w=Hh7-*CKv! zrw#!MUNVsrbz}4Uud_f_bQoYk)ADlK2|eEnBOAmOjs5aezwWNZxHE^Bz>}HU$UXY^ zH181+*}L`OcJ<#TeAMr6BOA-NNR?w)lg@n7(uih&$9GH(|9qEKg_xz=+Dxsqk)O{W z)TN2IRo7TLs76nw-CJ%zb6e^jA>%q4b!BgE{vLdZDMb`#3@+};!hQMzx`se;k$W4H zXONYxlB7#c!!I5a*Q*hYX4Nlgdbhv$Q%YeKI@%n!lpT`kFOP#(xaUzZ5sdvG&c39t zzUWmsI&ytUIr1>t^{M0vImCye`CO+C4&$5-hp&}siOh<&1tq+3B=;FPN`^`HIc3@X z7BIhu@|~P;0BRFuI2RZ;w9S>5;HmoMr#4ezU*`hMbp>JK34J?$Dh}mUG#$x())Em~ z@(2aWp}8?{>^V;)epDN;AJ+aIX zkOSCrBJw1h1rrBdFPE^wDHrxZr086WaX`t}N9aG|^De7^7>^ z#P>a-V@B7lBqB#&V<@)(DfDXULAwh_`Sz*@?;*O|Ye4#e9sq3TOW_a@p%wx8{5r{g ze)eVS{`K4s<6Bd|eMRMT?L-ZfGfYb$^qJU7$Q=vc$x%Pd0)~mI9@8z4!j2VuL=v)k{cr! ze(B3B@pA0BW;Qj(lQ+>r<3jNQ4Rz16vPzl3c?0ru*#d$9I5twGJC}+ki7i#2OB7@F zn>ncscsF$OGxl&^hdWI+>ea`KKRJyc3FLG;4!+U{syIf z=Ww1`H)sHnKC)6PQ?+;?(iWko?3lSv9&*_E`pQ_?VzFzu5bI_&wx*4Eg&OCn z581)F&G*B_r@WQwr9XY=OB`^j%u2Ax4cm1N2) zrj1|W>ZW}YQNGc0+vS|G9}qORuxJIIX88;p0B6u^d|65x!$H0XEKpB56;R+Q=2g(p z{BI=KKE1u>TCA%jlGGkx5iu^QilYWDqS@z$s0m^OSb6 zWEvthJM=XLbJNoM%`QIxbt!)lGIVJ-Cq#{wJNzBqN=!3p&Pee3a8f5cwqN6kGyf+0 zlVnD@#ZzcexqfSs;@pJftVJ9?C1{q<7O>V0TSUDHECG$L;FXr8d6-0LOfO0Fy}r3MNH>+hM{3;462qKK^!h31xb?rUyZ=Gm3 zPruvrC2@WQprQP*X9lMy(AJxOxAmei-X@F(MMLh5W7!vUG=NcabY3L}K>(&!&03Yu z>DS^+N=4c7!>X-iX6@rN(8Tv3=Wg!N{GKy zM$E&4Dr#Hn&4vJGEpQe00`N8sg3#TC@#wXVXT=OIF0}f5J?`h1?|+Q0=bz|y@HRny zNB6BunbYS7z2e`|Jv?4om%VY-)see%()eY&iF6pB0_^@){k}E0MPIDfu6A}bVNcl} z3lL$ij*)-mawSpedDyG`I7Q=fKnY7GkruqWSeEnO|8ReOF%<6UAZ7L^joB5_o)S@~ zc!>yz5^1|?9X@cc?K|Xyv0-9y^*RUm1}A?u5BL=w-?qc2Lo!gUdmy|<<99!wI~k7u z58~d!tIEII9#uLdHXYJkQX<`(+H`kIw}3QAw@7z`bb}x*9nv8oAf3{kzh~?DzVA6> z+;PS^cieG*|A06?dp&E-HP>8g7C0L<)*z1MICeA7ili?SvS5|$i9IG=*o zAM^<<^8LA8Cf&)c)R^hgwnMkRY3SlMxr0jI@m3t95ER)nF)y}yMf7=n_=hiSe4m1< zz^JrFDj6pMOspkRx?4rRjZL+R zEf8> z6RP9sJ~-bit2`qvbFQ?Lt46l)YNKD*{t;6D2^)_*0d4Wjv`UU!!JCbE zs0U|hP@Z`-%I#HzA3IGd)kH-7DlUy{d#_?CXA9D28{Zw)m z{gUBt7Sl@0Z0v$5t-A9A^Np=xq;TsU&V4L8&(eEJ+0=*|+}C{Sz>!K&7#0Nw2!3mf zhpf@yM*a1Vm?AcBxr8BP9r%@me?>Omq4cum3}jj-i#~tu685BDy{jbw$;{^MoxhG( z6x_c8c+6ka{6>{y02L`0?!-@yz(aa?e}$?wnKK{6W&yTLy@j((pZn1qIR zL)FJCi{}5EPUZTL%mltn|LD!C@X?wh8$4`3hMK_NFi+6Cnpt2gX%J8@Ga=>+Mo)iZ z7jR79;C5udlc^BQ>^ddSExLdE5u6326WdD+G3aF37P zct}&F@}}u_fU(vwCESe0xBQ%{RihB@Cf0YrO3oN@6MuQLdBM{^1r492>E#dB7QpIN z(^W_*Onp#}LL*WeyH?W}#rd!clRzZF(*ZZnO1N}y2|SByUzoC|L6Ag~kse~}poo;4 zMX;rS3oc2NOJH2xnih8j=C!7_=WNQ5%Frl)5U2oGZ2n87ReKEueDhGi_axNG%Y+I5 zzOwSmtpz+XC?Q~44$QQA!6nMXqK3DwI443>CKSN}x9W)A3U7390|>WSSm0n{U$@`V zKk%CRhmM$Z?RllrtCQPiWP(H(pbaO@5C4WYa@H)0(~w!EqL2aUb5w*t6z#puWJ3+` zQ@8LvEUEji=bJx~SWy6QC){12g5c$Sy`AgOHUZmKI# zj+#J<`BNACc6lckz8{FIy;(Klj1Xw;EJHq!<6oS=Ukj9X__c(_N@W-GO>;l**ef{n zE6xUeQ7=mNQiM(@iBOsPcmd`|x^djdZbe}br5J7TcW(?(Tv;QX9Cbt!R>+yGrXR@# zSM4X6^6WN*#LTY;nMJOCNW|3 zP4;pg&vzyH;U7{0VU{T1TEgkMHU)rqrJXaQ)cuPEp_m|u$|GBYNg2ok|4=i`P%!eR zTn=|&IF1g9L1$iNZC}6iN&!9QB*DG~)q&rOnE>eE+~ezP(Pt&;uHUgb3~^R1Zbye0 zt}0<+RRXk9w4RSb78JeG+lj;kfY)%g-bjJ^CL-Z;eGsPkovt5lETd}G2~*)qD0;yG zeqbtW-^SYG;~E|DPc#r@j>rjg0Mt+HDKq;-g{&9??x<^k&-GWBeyFqg{bI+1ZcPqv zgO}bwzEEjmTmh52jCR zRvXP?(M%_0c{*xGiO^7Bh6g+S+Idy3bC$h>($J-O6rhjXk^Ge2K`!&v?u}f-L{`%? zMnKUN*Lx42(EK3s8tJzyL8X4w<)aPHhY;lf%j*ZIr6bg;MZ$(^9wQ_`8BaKWk!oX$ z37`W?vmHHe$f1*%sU*GlNuY(j8z8uIkd)MV|Da--nL@8lZ{o3)OhPcW(ogDq8kAC( z$Oa=I1Zg_ax`-1OdEd&e%;=`N-=H*HA+fv6Md=IJAXXj!cJR}E?mAy<<_@?Xj!#dk zseOvJPhn={31JnG&(h>OoB-?~`o+C%U-J?>;&m5>}R{T!wR0#5JkMVR*We1*)l zg^#F)9dzb)H1wpJvBSWEq-n8t&Ksxnjt%EruQ_?V&6~>fBBVb1-@MCg8St7;NV8_P zFGo-=7~i5hRna8q(dfD+!tcAukfvR`Zmq!P_Y6Q%m2UIKST(CG<9cQ@6wRvd%(^9~ zE$tkDZKn_N{!&DeeY9V2?B8}Ax?X%T=X6G+>iBzd5Xk@dzw$N#crlT~t@KSUfkVkO zA&jxmUKo>&Q@Iq(r~BNRXu|iH?CFFIYW}objM9F@4s)p+z^HKyHjl>byWf6&Oqi-- z)3jl9e1}D)goSi`O&@MFc0aJZwOp!1?RmP7^+-};Q6iJFatJ_tV*J&a zMn$YZ?=}APdwPIiBa3fODGEn#ZJ9UFGSS;nUjxnm!)y!ub7kYysa+g}dXiq^eS@Cn z3J29WLAutNU=WogY%Mj2G608qs_~c)DHxPBNXyqd z@a0SQd?s7urTcAcOj$m+b!Z-xtuiw$dQFzLutbCL)BcywzSP5yi}%p4P7Yc$8ovP* zZ%uoQW23Lf$CL2jg>U^R1kZ$WH7BGZ3DCVD`1$`A*hx zoQ<078tLXR;lmLEeC8>GFlY(RsuO zgSKr_AAOWYF+W0bAyK124HiJeV4$}MgYEW`{;{&X@<}20v$4t5jJf6=% zYxVOLY{=#lY(Sy=cD_KQH?9@v$QFPRRDJz?EiG9ZBYdUTCGxKTM@Vl75#?B?uqAsG^t+sZGqE-lWY6kYNTTa^JJ3?e!~P zGs`MptaTCkLqy0W3Xg#rGg#IdGANr*j8u0dY z_K!+!ZmEK*m}mNfDywUr)ZVa!h_W3&f57 zK)TNILrfguz8t_ggE{@+5^v?jCMT-)`%PBUqvbY7zD%t)PcHk`6G?H8jX2_t)~xT1 z7*QHRAoDVAoX!ORt0gulb{{};)5J`)!{L<5#d(5cR0*a;+>nxjP*KhGKBT|8LlJmo z*s+~tvfH=G0h9RQ13431_w&}4RovFLRhpCoWOwTfj}g=v1^gtHxwaGVm-`(@;>$|} z5Cw`_(lhFxlG<|S8B^e_ti9G9w28A@+tstaMocsC_SUZ>n zbgQY5po=;kj_oel#6%>u)Ow*KbrixfR=W?}Om`nFg0muu%7oUE-c<|?6<8c0nJE`<69Q_rgvWoteN^R&((i9T z0wzToh>q!MAG;_nWz;?aOVULU-5`Isx3AF&KE(D7P1+*0J!Xi15}oZbS^cp?GmP5reO>${|x2N zKNck#{}Ciq`GyR7S6B83l&E2mJicjdzw~JB@IY&IRk$wI_we{g$kN&)X+@b8)FBu| zq={Xr`<@?fr2SKuIj>}ybkZ;&ObWydqcedjsmlS4^ir+{FugN!*23u;$tf?=pKTFn za<>&E3ePx42Nj}+iiLqlWH8-oLsNHnr>ys@3noI{Uq^7qe8~%KUq7XQAQOf+AwO2) z@GTxV^5Id~rGay=n8SLc5kQr2AjnIh7&Oj4O;f#mVkeYge8mFH{I3$;=6(r|J|%Jk z?FrW}-CrT8yJQxSv9rcn1{={xYve^u1;Ae>@>XYwA;x+W5r)6&X&`6U-jO;1OHh~j zlNKd-1>A#$@tTXgBb;wRaLmNUOkt|;nLJvtvRyJy9BC$d1XoWOlbtg9XmTKVZJ)(&Y+6~srPgt#_pk?sjhp`~^_ z*0x3z5Qz)^FFbAyYuZ%-3CHcEZ^O z@E{9&4a#rSZVF9=2_Sy|H9rJ10JCLDsSo|36S`p+OmxrykTk5as>t3UgXV|!L|I{A z3_|f)G>sHi9F|KPj!Fp!&fOaa{{5?Zv484yXV%|(4GbS&T0@<+BmY=2*M?Vn0j@sN zRx!XF6!fC^PxrTB)p-KQio)N*Na>=lK{Z9g%NNBqm)|y>=nZ6UId{F|AXupsX(Bm- z0Su<*>mO|nft{pqFdltYGE*kSuQdYyJSy_m5fU50DEt)<7=Cp7fGW|Tl{%(^OuZql zrnV+`Ig-7BvPu3>T_9EqRMXZa?SPz4>Q<0i7DZv=VZ6fE#(22-D=n>dCP?m7hqSGIYX0U7!Y z2GNWm%;G$4O7)B_3MWMhbs7+EwvW{sp=dp1T#qJ9L*`cQt#GhhtU~sh0WBa9-%t+I za#6%AMu@9OaK0@(uGmTD87|1=!*9TE&12TyeI29hv)kiAcZB^H>;iRBO#CP}Ow>*j7iE8s0nt2y#o9Jy_{B9o8#RL|GoEKMN^(%^fEc2y3Cr~$k@imu%lm+mUKP@nFh%%0pQ>}c zdxxdxHWB;ZspcF5=rMIxh6>64C=AHuWBhad)0hK5^e8}Y6amAJuagU{MDJK$zcu<3 ztuH)*E+TkA-)$)vxDPHicq;s~{S?IJi7o`}mN9i7{+Y);yoe91S>OM@kw*bFWrRD?+X zQK7Z*1h@6#gzVRG>MDhUbHI!EeS-Ci)vhhRYx*Q2T1tpM z1k6C}(V**mP0hD{gIzcYejO2Hy2G>DU+?4$%$W?u#DBlAU0e)zUDS2@l|t6G%44oj z-j`gN^6`d~DD0kBTH1yBKwN`lzOJz4BrINZ~O zV`eTqLRiO<;N+>1$wX*rp@f$naqeF*ikI_A zw5lL!f%~xFm_x6rITDY33*EbFEe5oAU2yJV>^qxTSSjsL^Fjn64A+{Ith5ky6&o8> z&97H6U15k+9>OYRW>9oDJx(aZ5}6GGIP6J{s_5Epo5^VpM%9n=Hvc8Vn6YmtO6E*M zA)+-)-)$1de~J=BXp|WKFQY_7{CB56EOZ7@wqX?6UWuK;}C!G6Co|s$fx@m(s#lI4x-Q6rme|k?K*_#6L(B$ZCVRRueoov87vED zsq--0jqba!fO7$pzeF}UJB&HGPmiIr|JW&&%WL<3f1GIinq<4I3x1^G7z=y%cQ*@< z47h|R(?W)C%~&)6i?r|+2+VM5eMPjn&)AutkkYcY!BzSI#z-rrecRo?9V>meCFVO%_IR z%OT9v9_NRkqV{p$0HmamDL(}(7WtdED`Ou$GN`B7@u@P zorruvx6%J8>WF$we?Ke%e=3}whTa6i5(MLoL-?GX^bg}@n!>*+@Y8b>0;PW+Ia7AE z-b#FR?+!yDDjS0|ILNC@XjN+!iU#0GED8O14D-`-<#KV`xIRkc^Lj+~&*mJJEgSEd zQyPM}Wg=(D_3S06Fco_oe_h)PT!_JljLQSpq441Bxmv(G$&Q|)6e{@90O`UEi$V1Vt9gL{+eBVE z-vlferJ(0H#;_7dowhrj2Sx8i?WCbeA%)vDG=g^>YOOV1;n!n z<9-B$vo?KKbD<9;DIcp>0A*E~mE+o-kGG6MA8V z0kK)$z5it(a23(?VsD1VRCt`|?-1a>p$;Z~*y!xj-G&r-4d^;j=MAMM39lMJ zXharlwAEMwto2QccEjk6ymv^x=sCg{ z3LFE}WQTeINGko$Z2Xxlgs6(t5bNaRR&$AoC9juu{_)45o|y7|IBR%UgbyI2yoAS; zWY2Ku5;zD$^3=B=#6g1;(0q8^J~SKa0UIt%tQy zCWKb{+I{_o2Mof5=vRu-qH6 z&bhb||FoZ%lWw8D6Rn+tmq5I^4*=BNi+GRc zI>xEsxwgwgjb7dFkJ!)^8kN>w%inGJm|F20zo~x`i#O4IEIg~~0emb)?bO97D0x_+ zBZi<4Vmjn&18Ff?{kIiIptWe|hOn7RrZC zCB5qnjE*_B9d`sD{8?AApr0ufY1Hf;-XaF1X&pC*ZxI9KSs82Dfn@$4B1lZ3qid_I zYhnu!p_l@{K{vtgP8~G_Xr`Ue5hgZ%GA%Zsgf1Z@qyA6&0(;Xb9klkl$UmlR93+Clj?ernXLR zt6sE|FvIN`yp!eu%-1P3^!x@%nv9kU9r*=ury2FsB|CYLkSOJ-iTW&BeykT>Wq}=9 zBL|qk334#>&&ahat3@HfqT z?(JBM4a$=#L^XOHezb1M^YL&Nla_m=XDIzhO57D2WE_HymNMfnd?6)5p%iX%r&!CE z0YQ=oxOTYDqZ5BnCgCut{slsfW$`p5UKQ)hq^_L=$3-wdL=w`0@(N$|rR&-`O* z2sWM2w_6{wJg#xyw9>p4bb(tL6IkrTvnl5+fX3xLh71t9R|sURC#U`VHinVl!RSZ( z<5%l^p28+J!`=m7B3)y7x!jE+K*yMBmM9Lj`QbG0cBWYY%%D+|)w*|gBj zo=11IQ3mK1@xM8tzPKuY6A}eDp^|!>eQor=oKQ9qzzN|)dp@DWe{n*B2q6g803nn^ z4`tU;4)iR8DFH@sxMBR0|I!X{tl?azIFZvg-*`u_!&mUI?aIJ*@eA6_%#8c(ZjlpU z)P)C^AA1Xxzl;Gg(`wwfuiQ8}(Q72`2N)e9uSrDoopHYdPTeooH!=p}wQ!!I zG#u#_8>vGIH=L_vQ;@JOxaY4bOFAMx`L01WsJ@DYAepAn(51>HD6C5bm3&{u{%*!u zs6Du!7%UW?SG=L9Qal8X$STqbi;z2=G|v|h%Nkb|pmM>NVoVgt7}6NRGOTxbdyad7 z^$u~5xTqG}hc8tzQMR_{ST2sa@GwYJ!$BfPvIC%OL7#A`B@Z9++1C7Hmm8gwpQ}HkYP}vkdZrsC(0grE zWBx`iQrV}*j~ET07-XVKQo|CtmtwY>PwH271u`T2VUq`u2#}wP4(yyZnqH)>82yMH z`&Clcy;?ifq-MdDI>4e9zZhtSO{6TYeEg-+)wt!DF_9VuQT@AVow1}H8s6+3CX&R- zNzMD0;gTH71=A9xT@+i9Zz;noXP;$kG5&PAXydV22oS<>fky&d5yNk(auAh=A_q;A zO>Yh)EP31U$bLV;`J&V$7ZCVsM8dkNV;gbyj?XN9tVU{++MtSw?~*)zWc2rt{ zk4m@WHWcRQ57VpjoqSAWz>Uw zL9i8G1E^+nu~oG}j94WT-@pvfAIizu|00Me;Ec>{J%H_ojPCr2CU1(6?vs{wn& z&6sk?5-=Ie`*TkQ5bFHx5|mQj@lhGM0H}|kEm*7Loft4>_!*QLtY$6Ce!T|{1B_iIGLIpzsVzRHo|q{os7+PS-RGr2pgmsK}U6<1!N%lMAN zhTzy~gu;TT#qhpodcJC}HleZhb}h}aIUf%lhMsy^ji!}=i!f+C{Whi5cwwidC*xkK zU#)a4I1`8@_v%!IGqe>lQp8Im6BHRJ`1~bCzY*m-_4xkiU3Ggc%hn^?-+f7y&%#P6 zn}~x+0_Y{0FY2$lUb?N7M%sUqK-K8nE}>NpMqw(q92F!&iV{*DGI0X=TY@ycw`6F2 zWw~})x#L-$5(}c<^9SGNqL*TZ3|QdA#?VI<*u2(%mI zP1CfoaJR;X=Ok^JJ{vTgr9KwM^Yh7F>4#nXi-zJDASA&oWG#U^$m`}_EiAsCa3ytk zRmOnses0s@*b$^Y?DGTOrgy8GVIYs;&399CVH8Qym`>G1Z=8wx745r1VD~5FN$V$R ziTk*|_vcy3Dk&E985M;7)SF+e{~G)8?qu-r$FG=lcC-7NlnEXg25!k9HqO|xpKthe z78=)`OYa6nY_hKye%}JCr1jmkD%6Aju97X*|2L|n4k4drrCvDTe5U8W*Dw6Y4z)HL z>YROw`lm$BV-RK2BZXlKRFKdmU`nOGuL?}p>$tWSfDB}th)}n77YwY+Ik*wb77!1Q zwc!nqwS=_3{ZxcXD~OXX(hjB58S5n?!bL&H=I>8Yc*LUBQeN7@|oxFIwONCSH1lp!+4N?#M{=ZBJbLbzt|@AZQI6H{Qv4Ws6?Z&*0d`0dUg2RlG;zvt z(%tBC*)H&->F>~cgY3G|DXAL}9~}A8*__urQk3+A+Ync}`?}L(sbkC|xAkJIE= zHE|keF)(`zSJ&U>qow1ei)pNu(-j5ZQF7uA9uEdua zKf_$etga^Z_&b<W4 zilQ;(geim;P!%hN*}9jQz+Bv_O^B;mV1o56uq@=msOj@(S?Ir?!nOeQnQym2iOJ$# z(t77-odrhwNwmYSwyR}TBTpzmTeNE&6nVeEDY@tOFK!;*BV0x$EVEndVZovL>H59< ze#UWJa*7FwiSz4J%w^+%vHv8NPJ)7BbC&O`1x4Qv7*`IW3zHwp@K=yo`D#A4D*v{h zkL2+gYoT}fmB-6%GI7nI)v~F5;2}wiI9PEz4;uIvsl#dUFI7jd#Az~aA# zsSW4>q}u=B|Ed2gnT?9wrTu(Tw(05d&guHt_w8();Sd9CAJ|AY<4AeV`R${?V16Xsp-{fIyDD{MVFy6p6f9&orf z2&lSc&ft!emI;4^Twh!TMR)tpM z5fx2A4uQ>kFI;oQ7ChOa%;*cSC*Pb0jA{C4e{UV0=~%8?&F(LRb4+%*C>m6s6+Sys zH4j>odEOuS-^}{ZPg8Jn?kE$;MG+hGKT7LtccmWom1S3INNXsUR%rmCR4-QhZsR|N z((r!?rKurbHq*Wr_lIsBOu%-$QrMqcV1QKpzoGDE_f`KN)Ib02+?iNEt2o{Owc%Z# z;_UcMUG~F^mB-6QTub}*8*_o%o&4%~TgTW?G!}HK3*o<9KWgi+jATOB<^kffm099T!kW)Afq{m^(p4ndAlIiLD*BL`lt(Khxq_5;!3pz6P%J0@I={#3WbzdBjYJ%d@v1N>61bfmPnD7 z2q>VIR;N^3nf%&l67VCc@r|$#pK^qNK*P-E1ZqL|BlBdRu%=`sEmx3zqIGD@>bn!IU}uzH^;w9q#f9^o(^zGudh*SJk4 zqm;HeY(Amee|*zHSqv6s zEt%*0dFDZFOW54c5I5l`*agoP97SCnS9+lA1;5=Ki2ZljOZxw)?8&EdJB%*;n9ky| zUBH+fOB$E&2>t|&reU7>27%b%fbrssi5KPIU~)i0`1pQs6h$|0JrQDD`i(rdRSg8a>Z?QD@OB$xC0wT6bVwTBTK%k}~H z98TR+vg`OMfdeeT0gUMxrv(fga%s^vC|NRL%2(qHwd#2d? za&t9#yUjCkmB_s3DpSDAjC!QWAho@&O%L5J$CybLdd zHwm#RC+yFInkQagqA+K@MwY_ph>i(&$DznmQ=x!p|DuIi={;)?t2unVypTWx(T|zG zPs$Idg30x@j+qJTKQSE({|(oxFpj2RhI%K#E!2i zRzT7$Oa%t9ZeDVwO|`~kb;Q!8pvL2=2uYGb+MJQtKym#0WRO0Ff(r>noTl1IR-rl~ zAYA}*(#Mxy$kY^clXc|heRKRG{|ZF~{R~f9h4xn>^h^pV0e)CTH)~)21>DJcsMOXEo>mZbz0|`*o%(<3mwRfS*EG+XWqjGOou>wX^4(&z?NNgW-KuNGjEx3L3(Kkrm{35=6xB(HbUm0q z(vsa8z(ppAs2PoxWla~nH6!A>DMj?sdz{D|HHL4pg^prWRPVa);>fSLS^H}VohN248J2H@J;l~rH!qE!=4)K;x zP+c2s^IM|TQ^k(%7&Us*tL0$VxU+VtWhHlQZi*-H%{jdsiH>v}%!qV4-a(vedNOqh zegMM+1}L^Uyc9qu+ZtFiP8%NqdxUbHmK-y4zh{x&itJZ6W`J+gaE`<{Ek+u$r zT*4&E;7}|TFkYM^qlM0xDrll!Deio~gl$L~7MA~CfM`OWHsES+o+0!m$gSG{NXq!1 zK=dh>i!X5A&mGtqu5ntIe9gth@U|q9wY`aqDE}%scIU5}@sb+`XyUHDT55b3z1$Bg z`hOh*J^MR8QauqStcpUW-M87J{2E3T@!v{gn#v_Z!%jvti+?P@uu!ACV@|0!jeHW@05LK z*9rE-Z0+-Gl6eHb`eY;#yz8GMT0D2OQB)t7X7#H~r0!&x`R+92s>=GqvD0gA1|%t8 z$09PGLezD+NY3cSq;f1+zmPt@z}r?k6>`P(-eQ|QawqAh*VAWC{jg*Zm>SR8|25v6 zip1s{(3XMSn1gv$tGe@KIBWc#%I4m`BfH<27&Ncy&(qdiEoYy?L_6qD8#563xeyi% z)=Ja+-Dlyo0{BTi6o{v^6ECHVyPyMHctn1`P*>$_x8sX_R#z*lUFl> zl6>|&P0uM%WK!fTx=yVoW&q|BJ|dAu{15Z_O?o+ITBjR03^oc~|NnYF1?S6g8Q9k+ z3B4s+jPpOlH4{Ht{){G}dOC;A|B;^V-?)aAOne(a$GM*h`1tv2yl>t?uW49XTJoUb z{ih(xn#J_?f8~cL^z$CU_i0)6c$V*1nJfMXf0UzStQ_^k(sCrD7pw;JSI{uNzWgl? zD-O9E2x!B2Nc`zRufa{D>NdX7LA-E^(b#WLegJkQxM!Hoq1jyp!Jb$f;r291ubN&O zxT&+NuhkY}64>S8?C|#n_boVAyNte@=BcpUohx#;3K;aiO&d(n5+)@9C?ek@L`!6s zxa~sc8uvblVz&jydNagk`t!_zMLl6yG*ZM%l1I#JkTN3zzK2mWigTZW6tfE!;^JjI z;%7MTjZ7=Zx|~`|Ygkas>WpGG*o!{u^KEd`b-C;+$HiLb;L^qfLlEPNyIg5D9Bb^o z7`W3jP=NQM=R>$@ve97FS!`7Ox@EtwOK2b%&bBkC&&n~4a<1M2?J_=@zQyN-MurzZ zr5)M2>SRenqv}b58O4|?b{}(p1TSih#}C=LAD+*UIkqdMu7v&smXkA zh7C)*jL#oAupWTZn7$!0GYBQWdcIV~yI)Odp4>KX52hy&FI(e(mq5v<5s;=f0Vi1R zO9VQ=0Fq4N(M)GhFgK$CeJ9Exk)Erh2(G_7rNZja`?VV3dxs#brDM!*lL~NgYuJS< zNG_rD_XY&qEKeLstnGL+SvoU(j(M@bL909*_z|kwg!oo=mM-4-F$rgNlEQ)k=Un&_ zwHLoL?*_UjLjiI!0O`UleJ1!M+L_&92tJ{aV(Q249b*>Kug-~!IVW#^KCz_@tMFD1 z&wkU8VYJ^ZC)Ga6am9byt*ijwxubq zwizzYB9HM=HBHUzTXprL-BLG2lG-Z}fs!_?U$~-Uz?^+GcG+x4%-9=m>z`D{wo*F= z9cy7Khc(Peae$II0~zC;hcJf?;?gixy2oQ@`DU#mm9K!`C&8*^Ug((TOk~MBD8pwo zOS4V;)Q$END?qew$ja7a28)RFx=$^wxGoCq;nzM!YG>Eh#ssUF_vlHhva{*2#YY;! zWC3cNRpj^WRCa%QA&4h{RxSCv!m(a9U8%jmB`V?;^1#yNO9SC|6)RPvTu?elRLlL8 z+K8^t)3kfzPew zF}7PbuyskpLebYJ_oA1rp4$n)X~;2H*E7(&OI|O73-iI1RFQVzmycy->3xp}^Bi|w z-p%ia-CeQaz%8q}&(M$%9wZgyOd;Rf>G)0VRRFR(uz?UtH+S<{0s_QT-MiOstM?5^ zXJiudH3sg#)AA*==!QVeT2eSIK57M_n9P=I^E{kSv9Rwf8@{UnAb4qdO1*6gCBF=6 z?0#c;Uq+*a@#^GPO9DNXJgp!BYzU1EJ?=+hv68kBh0Yt*s=k45Ug5yK?;u@?S(2t^ zIdUc#28zBp<}cTt?w3!-4_^|Ci!t)bDXIIcG?K4$X|W)#U!=xc3rYGFGeSzFGNb!K za>X#$Ym6T>*2P$~qLr-a1Sj~x{TIlf*ceKha4;4zjf&W{I4J!+FMU-Qn5IXK|1JR_ zzk5^?I@}ES4A)QzCX$Phut4xIt~}md^Kz)##|T#AKP#%j+$0E8{y9^C?m`9`!+E3JRxB5RCs9}cZo^u8 zs;VsFaGTBK3CBjDs%!Q$v$4v?j zwRwp+=3H50_6mlSncn)g(ecT=1SY*csKd#u{m&6!en{oCF!`Rs{sqb9i1q6n0Gnsz zW64wIqxz-xK|$#0U{%pm0fH+p}qMTxcLr z6IN@Z@Rp+OguA5{fk9ZPJuCS^+BL~Vmft!Xddi3e@nDk9+m>>Zf|E~Qbn1DeGg_#b zVy}1FNX|GFZ_He2`-&{&SP?~O`wh4`BTv%yy7a&D^IYbhO*GLteaJ8qVh{c{h!cYPbd7UU_rh=OFeIA zqt)ODW#}8;J#YwI_2plC|E9BiX%IQdff1}iN%;b~Q0L40)n7GIYd_(2zlr+Lt7=B(DwiiF= z$#}WwTt)yb@P1!^lkFIgx)MO&uTFa=`F~T0&cWL;e)pz7J@+dl)uD%;RsoZY!hGxf ze`gn$X(jl~)Gug|)kv=F)!iRY!RqNk{W+_yNFgv;GkmEzVJhtP4lx8EV+G+yv1xT~ zcXN!D6l6C4CFD;ZawTku-&{T5vmLGT9;2>GnaT5d`#MlTV)a)m>AyJG_C9)%L~trL z!R3b>j);{kr7N8!bm2e}6X29UrIO>C0hQ9}m@>W5E8o(%LoBvxl!iFZJ-keTt$vHH zK)vOd@11)Fr1Rg;5Y1VVeTYst(}k~h75yS~?Z4981%MU)zTwON@`j1|qrIuAA0&Zd zZaGb03;Bi()aRd<7(jaGi!OgMnO!a48~)$Kpl2SZem~&7fRt?DX?1HeEA0C3Z`XWD z8a|61{v;a{kNNJ(MBrwqqx$_m-;0+zp(5j~)`!u*C{m@#iNYxtW~0|>6}}}H9arX~rOL;6E4d9G{PXk6iA|aLa~)Ap06#p+EkLeR{ zW0pxT0Cm3B{64eBh0PyJKSQR?;RV*D(zr3xHI?AhqPUr)#1TFpZYpQKrAzznzGGn6 z_*?c&R?+3-OJ1xx^O3HeaO{!exzAtivK2Ogbz)cS_%G3_)P$2_C1V<|`b=2`qt%JU zk%Fw1*Ul6UK0$gxBE+Q-P2(Zuc&9GoqMXv!&(b~g@JPJ{`7Da-66>fIOpEhXDXg!_?r-sJ&1pj zHG`RlhNp{LIq<*QDSJ0v>#$q(u36u(aai_@Dt8(HlwLWB=P!h-3s5Rb!Y;+(~FJ9ENm zQx#8#AZlR-CsQkv7QMQ=-NFjjU(f`7%&4Np%LzgE$Id91ODn5f=p&9Xa~1)&IK4=V z(_=o>jm@7i#^(=>o!@y~9?bXM&U6K%Mq3~)J>>+VcU_;XJ1x2XF#g%(#3;Y!wMpnN zMoQ)}i(_jK!u<2iDgx=V`Oi1I`Cq@;T?l9+#|IeXeld$R+PhqI=;!*OXPVyF!RdXL zzS6Vgwh}$e+VPV5d`+AM`qb8gr%NfFhkv8j)GhKr703qGIYY{IMGamby;t3UGxWbc z@*mE%!Jw7_<>Q`F{+i9LC&LcQX3+20oI0>Q5tC=yQp4|8t;R0Z3u z4NEB9-Lh$r?(PO@ltvl_0SW06kPZO>>5>wVZcti4N(2GvlslOV3SdpCd=aJsbJsXsS`ZaFANf*dsJ|^-2_a7R1K6p# z`}d3*2%{S9F(3u#BdGZSqGFRS?jG$Bhq6OMP8qJqPB)E`_FCa%%gneMWr)LbdMt z5RZyANcq9*S7|=E71g|R_l*Dq!l=aT39}7RpqY1QZ)l@u4w*~AVQtSmCx4-8?dHIR zIQ9x@5zEg$hop$dGZG=Kj)aUI9GchH{^Dby63Kh7#r8+FQ7QNc36z>h7i23`dlqP8 zYA;J93f_N$jdX27z_ftjL{J8uiNGX2Q@2b#dko;LLM3@6OdG+Wo zaPlpabWWf;U5Y)dq0i;vR8P1R1Zt9D55%OzEA-`>V6MYcWE(%~A$^q|CxbHm+ucmu zdo$rkiO|Oa3Q(L_n^g=__U}Icbh__TTdJ zzOWTg^Dr?pOG`^%*=DONEoA3qp2Fem1cO?`V#%soQ}1{$|9&AU_p+sm*q=Bz$7>x| z7JW}8Rzh>PLvV?bSL4wi5L5^j!y^9?U*5jS{nt zmfS0{L1-qx=?=Pulsjsp8`xkOYAaZSE)J0g!HCs_GE)-?LdfY!!Xmh7w$U&KkFQ|< zCF9K!l5G9X&jmA=_iBR;#B=j|4BHKf7CFe1DP|nETUdSJI+fa!==8(=4qxyC zgEd~a`&B>F>`giS?3Z$@^P=3+{I$mGUnv(0p>rnN{uF+`r8tf7dDhxlSWp*b59oac z5{i}7=V=aCz{v5o7q6g6St4bAmNO5s;q2n-#s=5uY`IQFs7&^^MmNqoXv6{KlSnH|9vT_hJ?j(-S(A6pIqbB-C_3!5>0RCS4CE0xOD0GdfPY5$M^#ew z5{d|4Yaan{6`qJ<&3GpW#iCwas#>Dp8n}M6q49n<{tlKi%(Mg09)%%ta%<+KlNVKX zakFNobHMwE^Wy5bg?GwVhe z-Xn%SVduS15*>Rws%J-Jp6b3_r0;7!&8uQ?{FelouzJjBGk-ydkhHg zFG+%k?rDuvU19%wA*SXr=p6beF=C@gopbHk;YiG%u_ zZZg!2`ZpS<&*B%h&(wRof}@wA_m>Kli8Jjo-UWnJ;wKL^uU^q8z6Z;ybf-hB7d_?z z3BtKqu^G%|a=k#A7EzT4CO3}a%Ek;m=H4bhZ9N<_YEcw}(ak#tlG%u$pgUA=z82R^ zKYr&o^o@kaVak^?G8}i$WL;P*8ftDrx1e~5}@jI�Lc zEcA;i)0&Z|OEt@hi9G=RSYJ<+u0d{s;$%6ZnK5yH25eO|w;$QX#b^YEu9z`xR$CeaaTFl<9-0jW@dOqk2$y?tDz^eMtZH1y~UVLpz87<1%uKNj4wD zruq|zk9v|y9JNrOOck4yF5i|!4TWr1B_W@rlPZf;Lm>)P?&NBZ2{K3GiUL&E*y8T+ zWofFpsFqZY!n+{nYR_NgT}N8%c5%f!Yne>mPhTO)?l48-va{<_-5vWzaK8cdJW?b& zP+j+!+Hqm^SVIEQZDBVb>lE`{EC+%XY5_z&b!J7D4HMUNJUKX_!?zA|jsB(rn>{+u z1=Uf@?D5QZ8HD})pkJe2UChAMniN%4GXaOU=8UH{@L(Ihil65c#Zbu-XXSteKG!@_`c14# z>3MgNsok?)1Q_5_`;J~)S;il@yhs`wJ|n*^54?u8EXsG2qzo(Hz9zy8Ta!JMLoi+S z!Yam^c3VkByz`>7V9dye%w;{HsdsSBgE9hx%(Wa%=|kur{8zhP{PhC}WYSeYOTwQ> zok&oa#Z7LD^wstZjop1 zC3VZh(XFW|q&+I>Kj>1aI{5hcoP_xuDpM)Qa!$NAtH#{h<*eyK`(unlv(#eTQ&RdW zv!z#_fr}|2%xncYr&quWc|(6e$Dg|>HZ0yshpiOzMNuNZ^365iPf6sMRSrGCy%G;~ z6L1f!8kS&K$T@l0Be2_! zkA;I5GX6Dd$FWScaAz{{NjLl%;EHlbZSf@vEVm>)Dv`M1irPKOG1jz}JMM~4^@3)BFg}`7U1+HcsW~((s--HE+tbbh;%u3WpuSi zFc`_|bg`GvvVmk3%@CL6Sb21mCwht0_7bD(d!3_4yK1FO&9fzi9@vTDr~pN$L9eTw zm!ifkmq(nV8>@~J-V<2ntpP|f&6Su6GW6!Scdt$lOMPr@_-jqO&`k|kHAtPIP&1~~ zijJV%Z%w-Ee7hwo1c#0|ondbjY|M!m&`byOYrT9mE}OcNq|P&5$3!bPp6#H_;1QWD zHnIqPACAIxnit*ql2&dUm3EL0W_-7E-OdZbeLsKi`XRmf6#B5GIYLlm2Bt=fgiG|!~SJN$H&mxj8oQnzFBe^gIC3ihh>b>iZT0TM>+85g9Rf zGT4qK5P5}WsjE2^_PrQcn>ib;3C~fI<WnfAJ{F-RVkKix@u9Uow}e&lMWTm%*hjd8 zdo$zXxX^RaIG1ywq0E`u8LIdqZA{&dLQDCG#CS7X1mJF`Sfvk`6|Uk(z58)0_doDxKR<$Pz=|fbYrLu^=`q08JoRZjoM!npbq*0^b8;Hl$OnnV<#bgjy)Rtb7^w4NM>ZBKn*o^mVIwSB zV>k1ZFF$Svs(3otjqge+llEvbJkJeOtzx%axvT^X|a(ct>UKVS(_qhNz_6i(&or597$?TRz)p+FZ|h~3ByOntWqPEZ9PTO zV*=atsOQ(bpn{Ih`z&!HK3ATmV+nm?7#24vWeJ*63=+%9lrmVMWWnTB!PXMi6DXmb z`Zr<+&`1nrU(OFL@bxt`OzCT_uT8OW&B>D@WwwLWV&vehW!EUKPY_C)&(``BJw)9dKO`JZxqIo+u zsO?H^^9&MK548B==-9RivXs*D;-VIPgxQ|zpN#;W^7dGPPMx~QiMrV7&DViGdR^;G zO?MR0qV>HweQ4yj!l*8L|9jnI@NeoKD3R6TO2G-L9^05r33@F+^d}=`F!-fH6r5{h z)v!^ZR8|`$PNYnJ9JKbKs(XD3d{s6(TX^qv!3|7@Yz2I)H^9l%jH{lW9Q1#LmTi9( z_ulSZ(C=OVMeI)xU_cR+7uiwG0lg5^NI;y6aUpuArjvfJLjUsmTUr5PS)z7}Dm2@7{5}dfIfTZK~JQ7pxApa|*9~89Stj zdu(mZZUbEq@=jt zC4>#w?S7UeOXOU9R>Sa1z{)1<|2WSLRIl`wTMr&5e?H^I3u>VWg8=oWG1dy^-Z0NV zE|JFBu46OU`+rs^7trdYy5cW_i!bqzGevr7WS3`FtBKrC@P|EUEWw@&; zEl>?}`Mt+f$c2cf@c%F&t-ju+XJS5XCvRrJF^2WRS&ejh)KQ=7dUDzVNqH>0`9Wcu zN+IN>92a(*Hf0>~cr9-~&gApRDMugBR+tL+3_=2-K$Yz#4tyS40$=Lle<9b%p_}lb}Jhr+P z4h>#!gLQDJKQa!sJteS?Q?*)?h^_w;13u|2>jEUWs<@ zc>6-}qB%eg%66z#b`GVI)upd8TatC1U-B{W zIKRV%jsy4)d2i(nEf4 zVB$^Gq#EH8TrGVR5OYlcD46jVei_~YxI5!M^7C@lo(9@EK~87@h`S*;c#IN2Pnihu zKL>z#7eHJc*qk3h7mNbJdTU=hNw6#hFb1w$bt};fq*<%-KSVza+*>SZ*lLa7owlOR z-S0-swIXR?)~n8_udlc4ia+qmZyD-mqaG#ZvWkw0>IV<`2cpW4i-UcT1-M2(uR8jG z=DMEr!F0Vq{q39iIsnhPMJ%syB4j*V@`*&fg?f5>b?D=ffF$SMj);7pP&~W<6m14D#cyICkKG=W>!Ilsh1?#(fkD zQ8p(BE3DXBoz->Uf=WV3^!jul;vKanGJJC=teDfdKXYeG5)2+`x><@^}ob*XBv}SCFg%8PH zVBieK`G(HOLbVC_|0Xr4V_t)P+}XAY+kX6cS%DiG@yjvF)fm;#{4P^SmZmIf*?r?4 ztd-f?5DhgQLjzK@)+#etH>r$@2C_=q6^&BS;M!6YZZtf>88=}(g;T9lU$ql_v&D(HwQ3L^i8~Xk|97=9&zIjseQTX_~sb~Mi zN>&(4E7%K)>W3rkowd1T7N8C|O|bqF7b&y*Y5G8d8;R;5oGwQ83xBl1(}OeZtoGk{Ss&!wH1{=z6pyXCpwz|LOHlqvvk`JNYt;71$I=q@&jG*PmvxRS z#6K#@D8>)QEz^na&!Dh2-a;Ob?|Q@i1XWrvq-3=lS7At`-!jD^DCTP2=z8K?b`k_W zS~W~_XE9(n=0{&lo5IYGyEJED0*hDDSo+0Q^2`M-eEU9sDsh`yXP+)()SwmtZPcOZ0{bM2h`XaB)TYAQ4H_V7og6`}dRa}og zPWEB_dTC2qC`V*92cps zHJ@*_eSKhL`#mhc5SNRRc`u7w3i}n7C3afOtqAmK_kj-|uEF~-7XLHae1}FGjlV}5 zz@f07a14fjR{!HWS?)mF`I>U}+ft*bX zk%Gf{5zHsUF-5E7X1x4?fOLMULSD!kQZbBxk{cW_&C2nkdOlrAj2D*Cn*w@Qgg`3~ zDkJ+>cLftGH(=nbVl0Lj(5UyVbf)ryfO<|JaNVIUE8e6iz2vLSX9z_c|2q1L4wR8CYUfX=5v_V z;&tvqg^XA1I3y0s6slfjH3U(GXAn4=dT!L0F50ppK$M9q9g>x*J-n_1S|NAQMNW*V zbE7KnhBkjWBMZqMMv%@{m`u8#oqkEA#+alu4=sL2(SI14wE^6R*1)g|OIB%E zbE6of+!1~W(D%_X^@w?m zuaN3AEFaXi&>qA)FOdzm+7Kf4Hr!q#D1V~8DO8Yy!1~FT6{kl~ z<(3%Gx&EQt`j^awW{k{G#A?*`gJRvP$xW-`HGoDYX!bNMXm!W3xLFyqkHvj2R1D0&tLW!w9;Vn&NJz+2AriHY*v!w4d7P z0F2}pxgNC!955_cx+e?k!ZjX_>Us1_L$FNAqN))#yawH;ptF>wy-f#oHlwUx+?>yJ zx)&tr`fX5f?n2p(t*a6s?0cYd9aysI^$_SC3#Kde&q{z|)a>yzu4L#5V8~yPa!lm8 zfrbyS_v?oj1tR<+T#59pUjTrLkVNN)*V59ecBYk$id%4Yb`FY-{~>hb%nr=;7^s{@ z_)VbwnWd)g-JG*Q?nU2M8gl)qT3%=M&zp1iPr7UHO+EILe!C*?zx?wRc>#~{gtr)X z(o#4CwDDfRNIq88Rtv?Bg2#}@RtQ|M8Xpb0JQl!RQK=(_Tfak~^1@y@NsJsbW2G~4 zwAgj)bJxy~&|}O(25~oMM3Jy#J-n|9wCbyCh}qVP2$3Q?)F}|iFd7kCBmK07*3Od0 z4YlD!`-e&`hDh)~0@=c0`ZKIzPG+O#@4>EIk4NaKh)3Q(YO_&hTc#T=;T}!nf%3-j z6Lu|^gM)*{9>rf!E4O6)ORDC*tE#HLf&NOws(O5U{9Eac1qV|vJ7B@sIWY}nM_8zL zl=qJ;=Of7k!!0?-MeJwARM~=uvSp7~nB!j?tS(40C^Zw|y#KRNObJ!Wyy^XK=({^F zHjsW5aoeTNe-g)uA*n9;*t;zhaXV~A>~fYd?g7%r7Y`&Du&{!lO8*IVm-ee0jjw>c z*&ol+{i%)ZL_pWd1soG^sAyUNN=73GHN5;Zv5G`*e>K~i8kgAsNtLd+?0yN;OH8|n z`G%XaA<-8bIVjYMmq=zP?ubjnG0WF zZM`~I!qszkqx#NRlJ=nr2n4Z$>E<6cxY2C+oS|1ryJQry1Pq7X6&jVN1hbJ%b&3}h zojS2BA02S=vfh@nG$=T*KuN?~INeWv`Sw2o+2X%15&lQNeTOA0Yr9sk3p>(s+LaKm zN_>P%vz1~;CQ%Bj1e+aDq|<(9dz2>Y?;PzHVd#R3Z(VkLrPtS^&0zPly612H1b;4O z+((l7t6xt&N8Ib%LN$C?DjCkQ1g)D$DRfywWDc`P`HZDU-&uG7RDqVvXX(&X#5pGp zt&iXVmij}6eQG|?b1(1Sue$#}*Js2y-3UpX_fDq&(wBPqN2l>|s1F zyp#9y)r-OWqK81O`b9eUFGX!C_kqYu448(ARANs)q=$0fKgEH~lz^Z;WMVo8eY-#8 zQFVy4v(R-#X~rwb_zXbM_8IUbA4~p`Ct;GceBOX5-1su7^m|l*s0|_zi3krK>DYcw z-RbHDvk9^GZ@)<}5zWw!rknX1%j}c!*InW-T6b@FW zPQ0ndlGC1jrBT}9ZHg|gW9?>vf7J)Q?Z&XTf1UP#4rBd3?eVYWNBj{;i*VqRoDM43 zm}WvXbU%^dSrD}JEPc0Oni)sLPB08w+XQ`!kVIcPpeT3sPqov{`K>>2r{N#_10|%m zWXl!t6bm5<+ma6&iK;;TZ>=-Pi3kR0Qj4eLoX&w?dKDb=9RHB2C>@1m2=4s((n8p4&4&LB@nbrbFkPw^c#meUBe z!G{)dzDTw5e@?qp)c>lR|Ci`%k&8AO13K|0$y62*DDej`zeWNrms zH(*qV`j>iz>z}W~)=^T`?L2m{K6|nAWyQ=&;acnQBiQ;IS@un-Frg2P{;b+Z@lMXp zAq8yPkxRV^P>pvaTJ0ib5|WfNhC5gx(;wr6n!;)f%Yq$ELb29a8m6_BmFT+yOd2Ms z5@>{L3@qV58cki#8Rh|7(`@;D5-vY&tk$P9r3IUAzSTRM*qBs1+;1kptR30cYZ0>6tLH zq~J0&A9(NzB%2Zr)$+24?u)Y3K1#&2cO&^x*fRN|_om^@=9dcZdX?eh4%>3Q+!=rB z)~0fNideUTH;}M6ntZb6NeLEHxn43GO(blNg!yi`E*$c%yJ#+51C3lgSTM?AkBHgw0C@Ozqe%5DLt{T1ivGeDM`ur%Mn`Q zY0u@u>LU%!(S)qRYFHW$pP4yW9&D5LY^2JqRSB)#&EJKQGb~2Ed4rFW-vjA z3-4#f7(&A&@?n49!g%UPi;%6B_@kBvZXcr=kWp{H|!gxqMSwpV6KhMu~?aY`l8)prV@?5O`(w z3#T)lUb)SE8B0YIfHC=occJ5yv&{a7ew-G9>K8F+VrU;VQ)~eMaz@b+9tZi@}jPKvlMpUA-W#C zI~jhwTgwKo&pauBDW^Gqu)xyZeiiF*(@yRV>mN`r?Dg!M>Vw$`%d8R!XV z-ojuw#xHx&dg#eFMChCFCB8w1%j?kj=6TV5c@ppr-U&Il&kC@p_c z%9urJ$6>d5G7&CoctSF2a|z20*Q-X1oYB5wJ3<>8N)Xl_s9Fg3;<3zZp<{CdC0tN8 z7a`*b&m8hil1Th;|Cgh9Lif)l!t?XK3I8vNzKkFRwvBH)YB|rHv%yS3TgE?Cm9{KU zxJ2*=T>6<3GDG#@aiVE&B1i0EAI^?^TXh>5mrD7%N_$?Qg*d=}jof z$z#Zwns^Gm{w75YR}FbupQz(Pr59G8hJ_YaGO~FA$$@dYq?)$wQ)i-dQ^4m^AW4}# zdZ%B5%V~Iq@TQ?A1LRG=jDdY-_E-q4!9zr*$!cfQzW0|qhhMHmxkgZ>?qY7p66{zqA$J{WENy+T?tK0)S&7Sh|H_*)pK@HE_EP-ihu{VRao8MH&5pq89H23CIoy zGJ>i}>5md3-6^0Ml)s;}?be*#$OK;Act$2m}N=cuc6zAA8lG87$4 ztq@y;0x9>bQR!2N86+@rltpSHme2J&yTn5`e#YLSxns8)ul!e``^qht!t|TK=lD%QV$#UmXw?2Y_hD|#QHUgEH3(mZ3lis%`B2{BMO_y~)aL+Strxc3 z+gDcWR-GE9!T}vjGcQYp{MOEp2eRd3K8CnrGfz&cVKa3gy-Wi$;3@=@Tk)+A=GQAk zJpF%GQ^;X|9z~zkKNEZPla{0Spo-SS0vM>@JH4n7W#%Q)Ds*!0r=7xMM`%6O0%gT+ zrhq@8p*lzAplCtyH5YXCUgR4t`{wugzbi33|3$m)h;GwE`FiOtmK5J`M2(2|ZPq4X zMX9hK?Y@-UJLgR}#DgxnL-weE<79L1c1>~l9!Mzo*oQfQ=yD7j9L8X*n0_z=udUHb zS)&s{9-d+uH|1MPUmNb;g)eI(lt-DfQnqnlG1m`F;qC4lpkns3Hk3#j{ zFD>~|^#2#qh!4M&UUB|6@9o$Dg4t6y&iBP$aG>!>g}LKVUHqC2Y1fLhcv4VJZ!}mV z;MROF9PO@)?&TO-tr?JK`pI#H4?OS*g;GlfyZD-46EZ9BeY5n|qFWuu3u`a4HDwCI zAdL7D)q;o+A*h6yE7zL)B3uyO{&iSFN6zG#~jS4t}>G;bs^qk_IKaj z!^6ECg=mr=u=oR+-aL|1b&#JGL*(TIt`h$mY23?$Z(Bv8dM|n{VC%80wrx5*z%1_m zKE=yYC7Xr^572!Ex9gH{-B>_#B=WQEx91}t`S#nx*xkQ9+)mR=!+8XE6Dm4xDzD#~ zvBf|%@DGBm|9U>XX-f#NYN4W5fZad;ccBKgk>CRb9@IwS*Rl)h6#;0*z8JaAGPm

%nkTKOOp0}YX;l@!_ z0b$ej$#ej}iC+J0#c3&Xg92BcfZ2WfC;eQ3Ij%BJ)h~zYi&13B?X!aEohgaeU*@Rn ztNi4f=zp?}%ae{!*Ofc}oJyfi0bmZ}f0S*rj08o)=#fQZ{VY-A(e!?PK1xP?GqyKI z2PQ5vG;=sVLAWy((Rv$LW~|OFVD2DahfDt&;1{bUAvJ$-UJ&GR(f_wxjsR;Va+Lu~Ki{?R&7 zn~8?b<;DM16$D_^P5|&URC&Lz90ueVH^XwqWB=ISNO~b957wZiml_7h8a0v>PfZT7 zH<8q7lyfiC^4{=4RpJ?#2LOLSm!B0pzQXbJB4P%D!>KIqme?prPeM^`? zcjEc1TN%yDdHQEbod}F2Ey@$P=_t{1=3z3)ux~D&4@P=(D86R}(cKTe`k68-5uFEEr64gJpWK&>ukXf5>3!4L;P12vcdo z_duUS_$X$*%Y!?X0+i~}m;(V{O(vV5WL}kC3+CjO7V*4%`oOupdL_5z0}nznWi?X1 zWwsQz)(%1r<62BGVmPlpBIHT(Gen%c`h}H6%l+Rz7c5;U!PFRK*!g5z_+{wC-N$Q8 zv-`*CJ7V95F=c3QyxSLYB?z?ww@P%KTt+F0i!iMzDMKGiXpNhUyna4gxzOJ=cTN!% zT3$GBOJBl|qblZGJr&kv>UthKwzJoul1VijP~XXT@)p76iR+zQG*k*xr*LPJRCd|Qky$el7A5&p77 zJhNM^^BVF@6M`jn4A)Jv!)BI~<=@i_;+Vl# zO0mbikCheNc}A%l%&iZmY}tXuiE2PyQGZ|m7h2o_DGEj<|2^Y zA4QKleTdbW&?u$Q12=NA^no+ zO#@H1i!n_P<6=!64otRGcYY;b4{N)cw`*X+2h-x4P~`Q(83NBLN*i4=$wX) zz$mj+!rTkfJf4ynJ`^{kq=7U1X!8*&2^(<&V&=$uagPix!ZlgQA7$OI)cDhXMy)qy zXcj{Mks4?Ev9`vs{QWbi^s9l9V>p@M=x#BX2wPTR{dyy&Ftzf8!5M_cYh_~p&tUm! z`NC#Q?PDd~wK|QI2DpNu)rU-2S$TXHeuxT+4XRpYaClmu z=~>ss?qcHUUDLE5vOCoTs$nyCjARdVup1@c>wkc6C+YAxNlX-H6?Z5Hzx3i3G%Chu z3O+^{toHEUy|=nTVyPx%i=!~PB@L=WW1xnaj$=~Qo4T7uwI$us+lmUv>6>f#<|nU+ z_$#kiK2VfbvLK4sT`UWz#b25#z^S*iMd$dI))g?2dN(P4_HN??v;^iiN|c3i@M@+& zFR1bxg?jKWDAeSz>dD9JHXTeTX&?0{S|HUkVOid)hy9z?Ht%x+)hcZbA>_Cw`OhRV zr6<<{1KAjfi`c5aDiY1p2XibS-f56t%)gJtU5I8bL5PPfQ6R02K>)2rJe+RUuidYLo*U45%HEJN=;%?|zG$k4%R_bLf9EhZ^c$TP6fCHHQ-+CeF6HRm(U ze7Ja{&}@mN%3?#MlQZf8y7ejoG!yWV(4!A@lPM&X& z_-F@5$Nf(!n!^)jypa9&a2eNo0v@VI*>1{l5p9KLnLnZHM4cfZP;S7Nyn_x1jN7DR zv7i}qbMab#f#yW!{}Xk|vjxeiu-9~|CvTN9(g7vLYl(vZ)WS+t|3bufdYtQ3s+K^N z7Mxwxgakk~;q7~A#ID`Z2Bxk zdHG>X+4!J(RW;fy8LzrxLpDO5cD2gfw1FB&#imrfb@K-`X8!QlSOI!Cg#1>hU=a;o z5cig!Z~ATQajZpsr4PVDz(%*iICG^4lXvQLO6r* zfa6o;^>$wBbYX`f0-WjR@$GE=QG&iTC>pMKcRgR(EKM7yew>l+tpYE9=4kt}Yhi}| z1+9kN8iUXY~f;Nxc{D+`1G-OP!?xJ~bE>{QNJ(IFfL&vg&vGv)mGepVPK} z3dU4G6NAXl=M(SxRPh;6ffF}TY|3z;3CXrnZ1TmFj1RLt?CG)AYjPh91XSvNSN;Q`X$T=0n0U#* z-Pam<^f{kGKhT)Lw#k~{Sb6U)I!ksz=tHi^x6HQrNN68`;(y-P3tifJwkY^Hdg#Kt zpU+gDsAqy?n! z^Xim=9Hi)n5aO_fwp>NF^{RyTI*73furDxGG^vQ03Fh-tu~Ott76~R-0u=<+3`bcY zvQ-f3>Mo2HdF%@U1oql1eogu|s0FmPnN&@5A38SadrV`CM1TJN9I zfoS>)9{Ur#fPmia_wQXhbLBN^*rD0=B@*F|!kr0an}(KBh`9o6P|vh}`2>@ZL3W`l zn#p8Vz>E7L;UfQbd7t=rUsq9?d8M7|?3W2L{pJn<1hE^sX#)gIFor%%qpw%oJ*0tx zSg~fz61?FFb!ZD1LHp4BrzYUaQ3jVp_xV+BP3-(r0rud9u9H;#R<|TEL#)IGh@WUP zxuebgjxNPk{riz!X8?uhU0|O@600~(9Fk~7zDXssSScnIt+;Tp24{I}>g_V}JT2ko zECCd9Sb?xhJfIFrnAQm#w=0&?NZ#>NpwB|JeV4!nk&Q zlRZ3?H7q(sYaLu$Z@T(8B9r4`*nXZj+`_RCX0AB?+lJ3?dYMFEhaSQM!C!ke>(qu{H}&+)>7W)gKIP2FCA) zC;tEW@VrkYP$^3CS}t5%lqM#lw$mydC($e)JbOwTO#%PZhdGG%fdB=`%kZ6r7L}#4 z7>eRP?KP>Nm$89Ib18p({E^Fa&n{=LUBa@9o0I6WrL5PA+s$MSM}sczbC!uvL1IOd zk6FsDdxuNkYKw`yeO`XMU#f?g-L66zjvOd}g(@TEt{+Tqr6(Lpx+KI=jA}jMT?%>$ zaKOLg!DQ=mt&SFIF%%h{25aCZbNdrT3`kM*r{A zTeF9}4X;=BI~07!WZT2$V-UleheUw;u8xCJA%0q3KgDkRMq-K121&21!RTFt z>1bB1j>osRQ;zb8a7C^Tt@yZ>-GZ=_g6g$LSSyoTYSIJ3cYT?nzSjvWPiv*1S-_HG zVXp7<9jCo-pK{Ckg8qh4_H#da#lCiAx{h}PS50sBIlHLfn&3c@>rhWWc0Y}xZ(19! z*5l1(p2m86evz{cgMfu1)yZfvEoB9pHar4712x=<96H!g&`y!#3##5UAmnzXP!c@y z+z>pw`v^z2Hs02;ITBj9urEKD`;@SaGih5Qoj(y3S{ThU1U5pA&{R)t*o0((!T4tH z3Er}?Q0}#>$+uEZ;h?PetW!B(8t@U zy&~Wt4^Go%cwpr)yAe;pU?&RE#KdIg$L_bU=9pO9=a;Tev|w9&zMdYLn0;edfUBM3 zyDf7hc)J+}=Ue;-qm_=`;gx5wtrRJw?d%@ASyLra0L#6ny*t|MulfN z)$TBik1u5clCbeOyDjc`CCRWJ>^_9G6%SuF!1e=elVPtlNFi}gr!>UvMes}VK6{eUo4BTmd_ zY$FkfWpWZ!7Nzt-8rtFnHjz(YLF)b`@NN5hv|)aXA)|;g)C%Iboh-3)mb~zG4kgt0 zR%r{KIF%z_739OZonSIm`dHMSHobEP99E??NFrO!d;y(nSLvBHo4_UgqL-cAb^PWML;KG-|ihR$9UyOVZR{ zBjYRRR26JU#p)zI6?_GdJZfUPj>$#v~;g`(ayih84z4{&g!Ug+1 zokOfbk{(q>-oOU+cP&+WXUSxSA_MT1l;E5uxNeiEOTn9s%{*>Dn}ln;vKZHZ z3TGvXM|+{Cw-8Q0S|gB9IpZ$FK#QuipYZDX&<0nLN3h1hlIr#BvbDIUZ4R;CX90Ae zMyf&dY-9;ZHlmJ+b9A5@e?Nms;q66hcY99T!=ArGa<)QQ{$jB%e*rlv|8TA!rGml{ z;LU~PFqFR`&KlK^5)9{t3(ApOatXnVRd=HLu1KmY-^os8AP4j)3P*j9!yYo9-=&^N zu2-Ea?NCEV2>?SzK|SS+1Gf8bb5BZ?#3FBnMAuWLy4rmb>T~nJ7cgbcCwmxTTws)3 zCYAMhtDk=2c>ue6w@9tk+7cmsfIbYhDt5kK={vP=YfZiyCAUu)fAW9b02bbyW-Tyc zCZk;6?;Cg%HKn8D&D$Rwe8{`vcnVH!4C0xFRI36W@OR4|8V&&W?%}Q5i7p@Vj*aaH zH$nT#Pqo)M-I(Ie#`o3kO|0q%9He*MW;btq6VkGhKy80KxrgPH&#pD`WYmohsQeUW zmj-ZZpBO$}n61x#!RIB)y}&bews)u`MhAF_ZQn;C-=7Hp*fXIXpKW0-X|(c=w57SE zOyPNhP(8tZ1?1{0e_bHBs9w^lr7F9wsKu$DT_0`5KCE?Ofn8z%&W_kI$wHBd49m^2 z)H_xfR1-+3B_w|PdHNACl1CD38O8(fO>aVq9|1`Ob_n07ok0W;!NN0*vdXyGs!GKo zhp;#AN2PPf*6-8MSywdItA6K7lgn`SK)^(sO@xc=Tr4FEAO7YZM{O!WUB&AUhmJo~ zDy6A5?Y__S^djV@EhzZ& zcF&T9O`$l^tmv>9gvqH>3)IqkDv4jmI5sPhU_AF9a!<+yi;lG8VaP;d`sAkeIxNzQ z_VW)$?LrFCDWFc-o-A1f!iUzd+a(Sn_c_;Bk3%p65?-n%HCa&YKLf(d&^I4umb-Ra zhigsm#CDxIEaYqubQP1Lb@37qfF8;J)xd&2`J<-JWYSNX*h5zEpb0@TFp%mux@Akq z_fFfUg}8#rJ#k<6MG5`44d1Eoi5 z>nIunctu@dQg^jSxV1xeH`haG@i4c!y(z`A8rj)pqjmg4*M_~a=lH7a9(P2)xuYa( z1$#un*y##kbosuIk~u;Mk3jAxW*vCa?}))T5wQIcn$ZU%e!IjIF(9e`n$L}V_QMWq zHdb&qH);e1*6p;NPV_TzW)Ev5>>iaEiC~_bUl3ZbIyc|xU$;9dW&(PVc$+Gb*cyp~ zB#N%x4-zY3E!iZV=u*epzk31HUJ5yfhI!0HE{xOkZ*o29W{30Qv#Q1LK&Kb?azknt zVxmwIBZ199u-Ur*=r9sjWIVb(bhDqTIZ11|?-I6l*@}B!rsCWY$eS%;XS_Hf;L7u|wt4WL@6ziJKi@_|2%SLrS-LI^H^+uC*~u&jMPu!>igFlz`}A;FyH zJJz$`=&mzubni>)ta+@`b=kY$Pg3$AmtI#IMRQ z%5R@aAmZgq2;qHLe#`8-1}4`Im(A&m4>HT&>LW(Bdo-Mg(g({9{%-lUt%cUuAqynwsR=K%mA0^SX9iarr>u`~y3Qvn+{e#k6R`Wj@qf~C*TO<~6| z!S|FJ_e|${H4K!eCfaH7UeFL@tc+dLhvl<-&*8h517gJ2$C@%X8EhyxAl?%R0f7)zJ1$MWbv~|| zWg`I%q1w|dB&Vu?ImH0`5plY?_O^a81LqgdKb-U{i4njS1YpWKLx$?^glwV7b+$>+ z)3mRe=zd%&l^{HPSm}cHe~5eQs3_a6Z&(qOFz6N#hAwGDN*V?PhM`lC22r|0L^_5J z5s5(t1q39d29)kby1RSm?*#O_uJ`_)=bvxAYu#(jAFNrxn%T#(_x{B`j?;`YFnI<3dZOisb^;D$szux`axXIpgN4M12v>CaP zUsaRLV&|Jy?Xm@*z_17@FJV|Y!#poh%^{Q_}51Fa(0bXq&@G!EwTRC0f? z27I}oXB4kE$f4={Z8%;(Nw?Xw{abcJDvsOQ2pBF3P?*Z_zg0e`5@D>8v);D9Vu;qo z{(S1Ai!nstXc7LXU-Km=Qz_tu-vy4g{KrW+2@IfTdxX0@u_(?1h8%$L(VxH!ePHR< zeG`v-6*t7;=q45$p*jn3aHf+kWpC2ej$00IKiGTU5-EK!}nh#kEUuHDJq?Lm6E zdiE!|`SJYfacWFeP>7T;7_^OJzfmj5jU{HgEa=a*&X)J`(r z&jru})>#jJeD(2EZ_Q78d6u9L9zN=5tpO^$>cFcHcn!ycHrYfj4cy*Q50FoE#}7Ol z4>yC>BQ?M_KHtC0pN3-*KJXK1;7{^_lq;ESN6jY_gFK5Qt;>-3S0xWXH$XA$3l zz`Oo`XTSf{@KMTODKv_*yt+`kj{2syNoi?)=3HymcaI;j`)_rMnt0T@;sbNLTTPO` zUH0k(O?v5b3dm}WnKzRIHrJl`5zeHt;#r@|NtvvO*O!str`IuU2SzE%;H{ogo;O=L zmhw#xs8f8a1_fjp+O#mfcML`q=t#vR3%;be9kdB{am=Wxpv(+NwCWjTV6$G8aw*7x zoa3kc&W-$dXQt+7+?TtzRn*9(+-ey!wBX(NcyB*zjj8go;6DTWIc#0OUxz86jE2;Z z_sP$Ut(C-PVE5#Bs{^B@aL@gkLgZX-X)(>arI}n0e@t9n*8~(yee3OiVVYq*)VLrW z?cn4Gg+0GF#%Jqgm&k?}KsK(IL{GS(OMvqRTGjc1I^9Ci`L;T9rFUzC&u|H-7FZ}^ zg|o>jL4#904oF{~d93lz?zJ+am3srrhx8ONsDj1$Qfxrw6TdFH zCQ@WuoaPU`CP&I9ay4zjQ_7(vJy9VccV8&kg%%>Gni zjj1S=5`6GP6@R|Ih)7m8Su0+cyd%u)J!L)(7C!J|EyUFoAQ#^pFDW=0A7V0I);z57 z-mg#@Kg&xpQy<^VJJ~oJJ0ag3I6K%RX{D0oj;-@cUlmMi0oNmb+=GA08rK~de1G=E z{k0c&fC)iyP$QkIjXM6a-hJv&Z7ZoT<&t$qYxSokfM?R*#F!}~o)D5gLnyHO*e;MXJw3)*@3=;0Cr+c1?R&=N5Tk9EX z$T+3IZv#Ir%t8YKK;7Kv!21+3i~WQC#AK&*8E-SRycz@Y<+RFh zY;t(}mxYa?W}>2Ue$gtt$+;#WX`SW*`8_z&2h(i#5fBl|Ql^3c~>mfx` zeEyAzhcyun%WT*J8Bz@rkJi@Yjb&xgsP9Dv$HU~HH`l0^>WqZn;l%KuT#d=4pKQl! zE0Dn}Ot6H1WK+Cx%R6UNOpTmP%8-i6&CrA#?@Redm-0lq%xRh6Yb$tZ9IB}1n0@AM z?u`9ry9J=$2E%7|nqm@l-%Qb17xL-E4){HLHNNSR!EaBIgg?NixrtBX7bjages(7u zk{$PR^MT^KJ)xuJqr`q06cG0KGXa+q?J*yoj=N6yaLxmx+N#HKey~*71VI7=mPE9Y=_^(3=kxDn-(SiJ9RMQx{S! zak<2AeZ zj3ji>JXSSVEu+lK?c0O19^BqxG*MEjruna=B&JjPs`9gC6%=RqejCLwE>G?hnT7a% zFK47EOEl8%bcOflxwG(Yx8`;`9p@vKrY&%YSlmQGB8B9uJ$6(>?Xt2r*7QLM<~!Ps z#`am*RgQhXakESY;j`0>AYKxw6h{M1@!c%hJWw{h$g4jSDCj8?!7sPIT&tn(-P*Qy z;&eS(Z;Lh$9{ji%gs{GlmyLyhF6OPJtK2&E`@`-GqHl?L2GAMFr&o>RNHZpkhtNZJ1Q5g8XP}BNZ{vyvEVk#s+_w zQpc3<_b0j>y09D}+82(k=Cr-e;@*)ccyTMXI?TDtkW%(>>3u`DW8L4v%S2ex+;(l(Ka^3NL5>Tbx+T0rQ`}?(mqDY z+r~~JqtD6&^fr-4z%m}e6o4wJL)5Rya&rTj*PpalUJdGQJ|E>Nx;U78ebLe5$m@P) zzxvGagaly0TzXdhiAEBd8_m%_(Ypb84z^2ruMTs-%rEzx+`nYDnMn4ZipmW3Gr6e)&%WSUI~Hv0UU~5eSV@BaVQWUD|KU6B*dIYoUPmi~Gz@l04>J=Wu_2e7fmF_qM}puqWX{S{0(>l)b z6IM>3!BJpdE}a^%bj@9gp9Y4&bY~7_s?RcG-3GqPgy)h{V>h5he0&mybe~n`;^%og zG4K!*T#={2#gv071ed_#nI!{B=)11@nQ!maBy>}4Mv0cbPyfPa7E`Uoj2^4Xk^-LX z-WGVXRxD8Hp-`2tMYHi0yqPZ$}Bs zd7e@!v~A(T$f2k>8zdE%3|ms!R>6$STQ0i`wM#{YSnqgb`fEGY& z@Q}O#im;H*&>bBFhL+zn+u;(@)9O8^m4|anD=M2#Dkg6p;Dbphq|^Q}C2Kn_<_`}I%9gL0zI8j)RX z=eq0e{f_lV$D3YsVeDQVNT0d$(bFGBEv+D6J>#BiQkImVlTZ5Www&wMtA8dRHvIO~ z=u6q>-0ZQAIgV{765jLM~ZOOr^#a$EiQ`p|w?;wv% z&0Y64I+$-6`i9|pQ-KKa&+Lg2lKqL&u0c5?IBs)`nBT=zCC2behYP!wg4Q1>p2{&Y zVkZ*vR|}s8M%7HL1_Q{ClCZuXyIW@YZ1A&K#BvuT59X*6%}SO)>f(CZTxYu{r)puE zQA^mFbV(NJIS>|FC5&Omn*mS^{cX{z?-wb*3r#s^dZesNw!Rx@1zt%CLg>cLXRt>#=RDfk8}Nx_RM zCVm!60jA&siJjR5fnN1skowQH?^1=W8e6E3B;PL5NqSN#%juJ2d6|g7a6| zR>J}9hp{jt(3@XOPT)YhJzr(uxpBa&E#c3ivc8?9PQ+Yx+UeWnLKtkFZ1hC0A%Ap5 zWE@PM4|lzr7aHo<-E`VOx^=~}b38;F^U)P@X!uylf3xJ`K)2|^xhDOr>=W=8JQ=)2 zNF&)&AlhCBGskd#=W*AB09A!^EqvrLqg(sjc;yJSHg&DoX{&xYq>>9PCTP%@=A-xX z<=`2q6+x#tzJ(K7OX34DBx6v)hNF3_#r}4o52)J)4SAADdtaZWQq;!0K2Yo!jyY!L z(W`G!z6=HSQDgsC9KIbM0*n3qUb%}gTWB|vsR3V}nxr<1Szn)^(>cLu1M9{8q+VVj z4}zfjpT-ain77XRQ0}5)H$oouF-B&Tf$J%S7SbI@ZL6f2u#lqTkqR5pV=k|UfnsNm zj%iJH)EaI0BPE9z8oq+eB!cPHarz0P2+Bn$pGky)iv!-qQ7IdErw_TWB|LcNb(2-v zU|7?yB3X_3gbF{iztNR6&48+5*@oD-F4qhFRxmmh`H~D|L`??4^`sOM_oUU{vZa{Y zX`b6Hi(z*uY=0(&^VHU6)K$z_wPhP2q|M&K)nKH~oDO+TC+4LW21&a$ZHwu2e_{&V61qfmBa@KA7}YI+)C+ z81dO{J5Q;UJILk|>iKc8@D6{KjpTAwIe1#$e8=|lInG2AEqC1YND4RzJ+Z@~AFMhn zGkp;YalP1)BfHp$I>~|LhM%RNp**@KMT()?om_E~JF?Nyo2_9d1kb|*9^hL@qge}` zqC&EMdmNqnW@TD5P!%bIzU^V2hH8cizMiO-R|rRmc?brjjjwlh334etzLMl*h`^Jg)X1&+?sjqqaOy*1kFH)g<8j8>K;VF=mH|~GW3g0w+=<$ zCZrun?>Qc{i6kCN#<8#jsK)lu<-`{o$F{bda6v<3di_XG>v9q^>%Wq%N#$5SIGuJL zDE!fQWOz+AOqKpn4e7dn72X2W**xqrMs&PkFI^;{o0C zVcck2{LXn#5tjmj`H1tJH*j@%l+iqW6Z#S{>B-@!*}FA9G8D1?^MQ1CAv=9i!jgY}+= z1es7Dfk7EG*WI6dcjtGR*m4DrV1@b})I7=MtMn)p9fZq3_XC|Iqi` zgwv{FpWWF_@1u>Z>B=}Ffc(MhhQLVx3yj}cE8NUe6kkW4om+=XM?+>QEG|1^R(iU7 zwOHTEIlj6ZH$#?-b0kZ%Y}I19{XEG0>AL{wYA>buQE(&YmbKiBjPSwT6pbt+L2=ak=@~ zeJKfizxR0N8ymDUJtP6XI)|q6$}X>#K+ivT~8dunxb z^7TS~kB^Imh(04`bkqr%7{4L!{6f|oO|$9qc8iTdgaglvqcCB-_jeN-%G$zJHPVR}-zzH8IB;FZYSw|G| zu1eTG8nbe0DUkU&?!dZzjZ!Or$hTHPCAeYZ|nvGl2e{A{3n^?$kS{!dSkkW zZy-h1kGZiQ-mHb@&)87tO6I!_Zr9V#Qk#JJO8}BMrSB#H{=_R_iHt? zSOnjK>Za54=N%EVsgZ^aJv^H)x{lGE^H#*A6~S)u(%MDsU?)}0E_eji4#C?8ii z9+zMo`BE-7pSGq&@2hH7G5F+Bo-9HuelL;ii5QOSN_xT6k0x*Ovz7s20g6N%@7PNq zpn?BqBM)N!8wf0zuOq#>81bM=vd-BK7ae4bgS6Ly*Sk^`yM)Jla9YntFB;DThGG_g z;I1@>JXS#XQI(T6UYwT-^zcJHRtkO8ZWxY|eR^Sa|IP*Y5>A4;L1?Ws`~I`0l!Auf z+?ftPAG1Eb)0JHcVau;^Vf4UNINQ2iYX!y?W^1F<5WjUW?R`Cm;D`)f}SwQ zN|eW(n%dA)Og&RDiQ+f2H(6Sfr%A>v`_PiEm#bxl=zmGvA|*!UGu4Lb-B^qYDwBlY zoApPJ7rN;r+L`tF^<44c_KL^kv-A(Z(&$y?62CT+;&jQ{P8d7zOjS$aDtO$f}wR-+OD?K6?EUfbYq zvvOc^;!B<4?xn?u2z4Lzw1_;$#gaH*@hFUFb-w;z@waZ4q{oqnZQA6A6het4`_w%} zH5w4sL?6<$s(dP)Nku(M)y{b%i7;08#BZ632Pouyz-ifuBv*$2*PUVae(eJwA~} z(?PS+Cjq(zn9I;mK*DuhspsI+&x8NM2qm?{;S3=pLwyS ziu3UJH-$teZGY(ZcZNTiY!-8f{Ob{{81I~{Z?>I}#x$AzVzB7#ZvRQSP{)RRJDdsV8@ACd(=r|y-m2(uBj zCYFo(2O%FSub*Lm^g>w&HQ|Rbl++JWS6-DLmy|zjcDM1wkl<1Eb8mN11}Cl~#F+rm zSMjrLM(RI}_FFGj&L<({u05&z29mFeka+iaez23>Iy07h2_DRH-ZDw}n1jUj5-nZ6 zY8B|a=}62kQ1HjMD;N2XfYh6QUpCZ82?W4|R!o4$W0z;nB_?1a?#dH!R7`lAt@z+b z;=Va&S}h>E(u-0*VFm`w<>W>q{=fuiw)i;Kf+C{wFKTp~}=sVO~hq+jB#VgeFymvppM58OuDIi=;1NL*93|KY5) z@);U`9#L_I*zY2|Z}C6fji@g-z7akCYnKZ^!m`ZUIv@Hgn(EtlCjh^{k_#a_+laJ?&s7piyG|qx;+syegUA~EMl*RLT@L_? zWI9&`2z%Jt>{N*G5%1_WhjkW!y?2v-$CY1do~;q{5LVWNwnTa5>8Z~Jvjd07P*3{s zjdSOBpe7gA>~j{@VrYKQ-pl*8uvcpn2c#c~i=joCB+q2xb*$R#Fb>MF<0@Vmn?W^eb3I`3VB?>>CL%3v7TKIU1XUS$2|Ch|_{txxvh14!-;?je!@}ugW;>)kp z$`%+IA5?s>!{d?XU`Qx<<{S_s=dvZ|HWGyCY%^Xo2Eg_+~%;d616lB8#q zX+4NUmdvux4X>VYUuJ_PF_Oq67NVlwIXCH(7nDiI3udPJlSgcJXqAJg?Ub(f-Sl|y z4NqYI%y!NSxoz$P9StbAQE4$rUkw9KSBihAU&X0E?{315wP}2^yzWu?+RAcc4bLp#c>TN9wHG|Tn z4~4o{kY?o%7ERWixJdphk#)=g5UQF3X!+$AX2~Qg*M$X~<_i#m%A$+-x$_jhq(ZIs zVtL!a{ZP&ryu)sN&kv$rbVk=x@d3~>|Rg2j(y1H*g2=lG(2 zGTg@}1=|SrK*l@#CWDyyY0W<4u2RHv(W6wuNFhX{5ZG6|5L)}YpRa0X!sQKWZgeY< zWob=`h7`h#(y_iVkoy1*!7c84m`F(>ju?C2VQmj6A@_qFcQK7&04|q_x60Gj9j-=b zu*#XL32q-Xx$ajZ8{>K`Ut(LU{=5-AKD)G15uqZ;xu~;hyT=p*MQFidJ&G|kOk5p@ zcrFbm`nS3Qefs%prX4+(U0z%geT*D_zLub~IobMN-1YRu#0CHD*wc6_q|a{6gMts1 zUnc=o;<}!G=LX6R`^FmY>am?PIlRBW+PN{%#{)A;%zeTC6183`Soi#*)}CFo($ywx zBJ83e{(Y)bYS*HjLLV7 z*@}<=V=BhZhjpug-oC|>pH*`28}oZ0V_m?!2wr!04%(!ukk_D5dpfnAQAi)z%k0o{ z^Y!Pa?Fq<$IzJ`MKw9=8P3NGiuVA!{`$^U6%|gf_GW-oa65g`OhXvhgqDIk7c#B?t zM>(Tm+{96zFRkY(-f~smJ%IAsF;kWS*&N66dRY-7=8b%J$Qi#Gi~9ICD3pG(0_a<; zk+%zVc4$+mI^ycdVE)?xqjz>gW1z$F@C^%3yY6 zX%@x$FLPXsV$J84*}2=4S!w%(3lAMyCqAT5IM@ zXrowdu5-WYg72?dxyc{~AQ`j3Kb09mTYDyOGOW{jV#hkdq@L0_q@1w5{-CWqKBKz2 z@m<%S1PrVdPochOsk!m7T41WY&t*3hUyZT4<{QwU27GpSIx-9x#YEDQSFZR5IdXpI zYleb4NT{!SAIJ5~pL(i0xC~bwz5Ao0oX+D#dTE zJ~a@dJFyD{7hMdJ`?wkz%cTC>0oD20 zw0+L4&KZZe26WZjEL71_@u+Lg#czmJJ}sZ3D*Z`9zb~|YU9-Jo46BD7G@g5R3SPOK ztz_eHP1d;72E=aKjE9{!MP{F7cYIhUd}>!pSKuR$hgjgQXc!_u|NT&+t2m+>f+I^m&PF*%n z-YRIr$J%J0s!90_37xL(BEvdXH~fXt{xoqKKbb$)WPR#C2-}KvTkXWohyKK_518ms zqWDAkhyj~Yz&z$#?BS$EY=ub#c;`;WrIQh<+-hj6JUNdfuFkacyynE>Xd$%~&&3SJ zsA@G`ZsR(VMP07iJ6o-F zNQssNNln8PQeh}|%|gUyJ?uv*mQDLT;fbm1Wwe{<4Ip@LEPZ7G4LSJNX3X9~FJ~OX zS14Oo5`4#d&nd6YA@YLPS%JL9^}EvX#9XHU2~p%AEjkp0kl+d=Sd5tB3kLqdI!1=L zt|pXL8l&Nqx9mQ>FGX%ETU_J|8%O`FY282wGm9n4%+$-UsCO&ir<-@ z2bg%uWFHGC5Ix%cUA>O?z09fQXkq8=H|d4I@oRu&aqrT<2i&#}s!boIztb`rvjSxo z8Wn{XgQwP31e||*DCk>$pNq)_d-IvVowgUuo5&s}c+!+hr`h&|8}$ToO|q~(l>Z)! z!ga!jD$)>cO$Ud23;&?^4dmyoAse(Zj2#Q3{9+cx-g81^%#|t7>crLBAuQ3*LQ)x_f$-q%XIZ@>8pk50 zwDQjPGK+VUvsl^`u)pn)oE3&K?xY!~zw^uuvY!Z4Z+RwsT-*2CX;aj(#05Owc%iTB z&eKXI5bDcK)|vhJ&C1Wuu7|UlT*A((z3W8uw7P`P~nk4$}-XSX(Ja(pr`J*_ob5qqMIIF zHctT`{Krf9l zUJjyOKSDn~!4At>d#@6|7wsm``9zdIsAajt!4f5*t-YBLrD*UhC)f;8bh_;2&L7w> zMNFY#TcQXV6%b)s<`{5usO)Kb#m06uy#NG*@lULc+x}q_PRv`CP8aKmYe=rXP)m(J z3Z?6_sll@{5upBdo(X7bCG4Ezg+r2=3j_*dl2}Bk^JZpVVrXW6Dd*R1czHXqMIFW- ze4<~Zxb^HBXG5?A!^M{*nQqkk+utPKf0T6N_K2qz-9pYidFs-=qi&+K(Y^e`O;x&g z2|!8aJuXjqQ7{w=0BO}XXHQae|893jYs8XZUonC4vFkm6ms4TKg~h$)#8+cR5jU~H zY31jE)~5WxNDugFcisLgHh%D%y#xK&enSHMfm|BEI%9FtLV$qOFJfvIQRnUnlywma z2z1_%RW=81={=H#J;2`z5=ZDo%R$soQ%kz87r#@=h2D_8;gahBtc}}ryMD3HzG(%` z9PcY7e$WerJYm550EySmtu${A`LvMra2K{Rq{8q^k0Elo@GgG(bI2aw zzbtcn9CWOtcD>JO{6;aarbkxzgZS-SFRX8w-jtpzyP?IeoN19`QMEr0CXIL0O`2>^>_h=O#-w-+Ne}p)EyvoeC$hW>mhrwr1glOSxvSu92~B@J zR#+Nwq$)^^j@A}qr*DQUWX@bFD^J{8M_2}S8W!}Os98C*iHZMb*SHKuIHXZtdxi)a z`@tCDJH48)H5e%N^N#Titd?*uSr-XKiV5;?D0Y$6fE}DBC5rH+TAy`2FTSLTqN=U! zP!F99%+>g|pSY?bV`oi{3b6eW;r2~eM4{d)&dhdp0~Qq$bMyKTh)Z9AE%Oq#mcLkdBWb?*{igBPg7W; z-uLu*V5PsVc?IGN)X&#dN>vZA!Q&w);(^|s*}hW~_W8gI<81s(o&$X24)Y(m!x!gI zwN76+Jp3y)u^c7++0ka1T< zQ-jE%)r00PUVpRH)vk6qP5)Cz$97;6gKXIcs((KbNRUn?-RtV z&W&N}E=suU6ghFcqtou{fM~4L+Dj`!$mBJ(8yW|6w|DxGL3QdWxN#2eda_=nKB~F) zHn!gWA=+#IBHI5E$g2V1|BQG_L!=CIhp?+{`!)b(_s!S|?`2m+56$jZi0piM>J)*I zFV>V}!y28&PTUkQw@4aiE4@?PFjF3NN|{-dC+A(NwLnyHDa%+uf^2nUWdp6# zWqag~Px$g$xl~*Txc);BDL=V`f;IyJf{Ad%pEdE9gS)FD(DRst##G8mLWYFNTUTEY`a#K9?u2fuKL*x_{toSkw-IS(w#jKht+< z1>ke`PGV|vMu7?6N1xNx-wu-|LDB?UC-t~3z+--{^f_0>KL)ww)Ia$5YD4~yalBGC zt^#gHECw{b-Ki3;rqu)JP)dmkZ{mu=^=2*D=zl_7e*QOflqAt_S!J)=RBEvU|vKN50(>D6Y_ucwEz z(f!cdxN=R3`M;ZY27S>u8k9N zs>xk8p&5OAqP9`OlnQT^>xQSL{TEx%{|j5Rbp6#6>BasEt#PM+=b)iGJOGR9n32tl zHKVHHUvM4&xZ)0!g4=5PE!UpUK&ns^+R*-Vv(98Xz%Uwq5M;%Sj%-2XovU zIZ#&PekS9kl|_K3A?sj$ckn`PuFG@TNlD;vzh|&I(bt=HaW?I5LF@_ZJ?`RdW9 z!RP6*P2D9pD8NRgeCj~TGc1pu`3#6DRwZ$T!8&gJK*MH<|{8nN)aFMUZE z9FgZlp z@?CQ(?dsohJswhxoYM<~mQfW?J@USr01pt8%rm&7Rr3-v2kJr4XM%!4)~i_&2aVYU zbtG)Mh=ZSnQ6(EB9@QG4dxnL$4ZnYXO2oO-pqGft^MY8U1q=upAQ-#|hliNum+_Cc z{fflP2;tk&{;1{8K6TvK492#-@shK%(Mi(ISKhMRr-5rU*2)|CUn~vSOe#H2Aq{U$ zqc@LO(Cx4)m4 zu=|+74DkVYPtN8A7Z|bI)FPRbWxp((-Ne20in>~T{f$%NUXzbe(&y$TF{*{G; z{-A~K{~}r-f_#EF6M-?p!tPD4KQOFIC$LLj;rx`z_y}ezu?vY}_LnUr#fyRBn`BbJ zTAa1BBVU6|;15>UsuqO25=ok^5vZLXv^&uuoGSNTICaq+*!WOB<=>27f_P~-GK*!d zx%ySkk0$C)x+!)%pJ|_;6?ELQy&GvDRhUe~V}uKH)q*J9UP(qHaqx|^uT@Zg@}zKt zn=)R3OFIR_(dAxxo?%Nz%Q3$$9{0K83WAn-7#&Hc!y0bSn}8 zkua6^z=#b&2|O0E&ilL8AhpZ3|U35=KHbm4a%aE*3=Wc@UUm zSOewzeEnLO+MPFh?SY93k#wXtjcxIyKm!KF8FB*@mhAT?%d-kTfPXQrLfRoB;|bm4 zJTKt|UtL7C7xPHb%o};cnS<_>iH2~B9di#J6v#4>=DWznU)%jrz$0(2hj)@O|7Do` zucYB0jJ9vKovEt8^T2?@q3V^gQ?)jZv_B$$Zr`=~RpjZ)^HdYR)&D7`f*hc~z&GA4 z;A-kH`Hp=65j2-5i>J8Uab4JZYcLHQQq+YTa?&&0XC^&{&^Q&ynEYB--!DYCgT*DK z-aR22Ld7LtQlcbR-K#GkKai~iG#FTx9)7W6X^>v-yBiWq>v{p>qf`z4hazul=2*DE z81OEo<}!^{(lnCPT0Mo93@I4-ufk&L>0bn!Gb8nnL*gin*B`af5Mjy_GV^V+Ki+R; zQ*Um1%!i-_=gHP|Gst{qtEkBTRIvHh_8osoDLC59-ij?iVIt7u&SsNl+Ub+8%ahs; zq(*=50`N^tyzwKO#4Bn0#*w! z7(9>nEN#Xs2DnHys#0$pjdz-)Z#LVd*S{)I_AcAiT6(&p%TzeJ8Z)^rOne-1O02MU za6ISoG%}zD7yzgT{HYD+bL8%-s`(wGJX7!|C@OpgF+teVK&G^aN&s$!#EJif<;WK9>hL%y&T1LB_KV=ZybMWSFfCo-V^AB!ZsFEEQvPHoRf zL>QgrdTh&PIFAR#5|@wun%@57ev@9FAQL-*w_?+ke4E}mcX?5@@m0y&1z-EO z$Br&(!Bgv)$yBLJC6~BywR2?7g%K7|&4>)xt|;a|8kc|hVzb5W@=wOuym=MI<-LQe z?FJ>ZjkPFU68>L`_AmVKZ=P|vnw=__h8KbLFae+z0O$BbpLk&wfXQoCW9TqtH0fd{ zLx%gUxE==JVatD~FrC7Ufr9wWr?%vCi#5EV^=$AA@O%*lmtx#h!F!m^W<&L=i_Uul zP$Y08753PC#P6!H&{I=`1AI*>S-{B+o+BZY`^(F}P-{O`!<6);5y=U>9xpD9W%ev7 z*iI!+G{3(zdr^OuRD5zGWh~|^EbT}PN5;SgHm4SD@X@K&!vxsJ&Q@ir%Ym39U@+}z z>6*0eSN+DOlF_gqGhEje*egcy%>H%ZD)*+7$ws+o>LWgxPld`3XD~w(bn+A4q5pWp zv3QVLZPRk0Wff|7jmo-B{(qI=x_*yc@%(PIxYDl>gpsj3A$6*75H)i5R#@jAo!VH0 zLdiqe>sE!5p*YHCplI6`CdhvFb22N?60QOU*T75cW_W0H7$+?=N@Moi<6y=w@1%RO zSf-IQ=DQ8=hyK~$3iDQZ#9)jz8mF1HCc&NIJA`H!`H^lcnAH!^0A@Mrie~CM8d5WP2W|_LQ6HTACJd zUS|KdqD$}vxO{yq)BioztNGfG8w?DL1vfj~3yeSp3Ow+6+k#30B}`oAujN9WwMhh^w&n|1r&~N8h`XM|GaN@au)RL}+wcF?oBl1z za2Ir635dWR+(I*zX(1{iO(So3Dmzi>es*ygs#nHxyxH0rJsxYg5s(Gf^g1$78k zylQ&_tcy^U<3_MtrD@_wjoX8b@oyC4sny2MVL4Ja$+I_J{$4v&=aP+kS+i5b!D!y} z=AajgQtSjTKTIFL4=|66h+tBf(>bJBXvR@Gw5nNaf*LG0xh{$GaO8IIy&WNr?CQHS z(*#SK$v+dEeXx|*yzi&ljvGf~_q3beftg-69;Lrik?7n1Jp{$}jg~X6Rr;=eWsr-g z`@KzS4}yVDQl-lgbmR}fQiypxO9X`Y7gYYP;16KSv3`QwrJ-DsP~K6s8U3b4t9TD6 z^{?t_Rmphnsdb3*eZ_=C8*>`A8$0f>bn1%+0JfOX(y?3CZgKoGmUv7$avVBbY_=39 z?Kob~24k%Zg*+J4^NT9Xq44uM?Z=5SNkxd^+YtAK!_Y^H=S_d5sWGL$#uwRtkAAkG zh5Clf#I%`n3A|_#_CYhDHT2;u%tY4irABdb1QL_~x80HcGNmDn3MxFl`>V zGT>f0atJw5SKf`l)iX`C7g zvz%5vnb`0u(t&I)ZlQNwfy%+pRzM79GVK0PJ{8!WJN`Zx{u@UD9pM{-0I9~ss~ORA z$;YV^vdd#V;epBip5&0JmZeYbcOYHT4zLg`Wm60ZFCZZO+#{P@ApZ@m{cQi&B zUQUoDoo4gY&$wd2t!Y-l^CQ_D^bmY)mPclY?jkO5T-&qu&b_JDxb^16QQUAv1?-wx zOw@aNdPA~lBf^#ggRh2O4@&P&1S(l?NPA@TaVuh+K4ZtSMI3u1X@?Y3#W|ZV$iv77tl6__GK1E&ekF=}4XXcpYmu)LZRqOtBM_+G*aA zI)}3B8v$I;VD{u?~o|7P%ZF`=L_NKImsG#9wq4HYP8e|(Fp z;Ksui!HhPVdugq;n(Ymbg(~neO;Q@(sXbFZ#Ms+Um86N1M1W>Vb`>6}MNdiJ1ZAu* zf%6+n!z*8RO*!v5?(1eF#Kj?XN2j}}LLh z74_vPNEN7nZp1AB1fVCgo(2wCWBSCnIOvAG*Y}Nn@fZ(10qdVqQZdhMV218XN z84yojZ?j)%@ZvasR5r^Mz?J)7+@XJn-t{TdD@OQ#(bSM5{*8YrVGfh&)womH`JhZb zRBZ((YKH)0#JJ&{>%5+EA=JfvBIWM3Ur1ftaU+$C2XgzCmPkt!)mr=2m^D?Sn<2pY znEAJ@kqbab?>^k-kPj}sZC&49WM(9C9An*0Y$ppMB#Ke5I$jp)}9=S=I8DVOZM7YuyCp3JP)xnbhJ85s~XZ_|Ar6FT&BN6^4) z3zYS2N8EelMTt zqnQS?#o@)YMUm=F2=569I&G`PMiuvNNvo5K1|JvqQw`)Q-&J{`hb$$VC7`JPhA*xh z8N%Rn6J{ZEmxX1b|3i}W6CJ44cYFDaN5Pi4QLGss0<#uzXz?R5>3RY_c8cD;Z_{|p zr~7rNP)?86G#S#9y%46>#%HT%yJ5||&k_wO0I6m|b*gPd{Sp1;rr#?;uojsHCcc=D z5~Wcx2uKyoKshOM;Vy*Fq2E4&hOg{fy@MvWOaS%_gxi$waWK8?B^EGH?H?qS5eCuF z)%tVX$l}SLF0BOmZ+2;RzYv8a4YsM>>8{sPzclIZO`dE8v;qRW?*BvFdq*{uZtvq` zFNjD-Kv1NEAX25p1}If(=n;@2EfkR+L-@X6Lde<4&tjs?9-TT?ke)c{$J|(7@fN?m!FYu2wO-cKrC>=bEg^mgb*wts36zjn-~0vGi}QT3PaEujLpTcHL3@8VvON6D^7vGtj6)&k-7zSW|FJN=hQ~V<|POqsV*HV<(x;J70L9FP$irZ^`&M z86_GnY{PUIP{BABGNYbmPh@E{e@qId_uscj2M;(w*-oBo%uikmUdNYpk_6T8~5e{}7uPf)DRBsr94*Ki`QmlJl8Bp(h=qda)5ois0Y%BV&LXP^nQX3n zT71se^_YLj`U9`J(Wgc9$pb)n*VR|T8Qg4aGFapHsy#{}NBhk4XD`mg=NZB>A$)XK z7DeoZ^VKrnP>rG){cpG*DwXOu`o3XNK)*qB{ERbv(TnMnXu3`-wbNEk<*w9~?3cao zDT_WXI$sYSWog}W-@fgP2E6ph+cy0(8MirJnB<+EGAR-;V+P#cA@}yO6H{PW$bmQz z@qzt|Xw7`}%r>Q8r%gWoP(l7*CPW|ai9L`f6BKrV=cL_53h}Fw!p#}CHU}X_!hO?6 zId@JZD7JZe>mW=&Xh^Bn9PTT^xR^bBY*lYsx+-%4TfJyusprlsuWHVSk3XX;VwlW& zLlelF$FpHZw*@j^Ow#SGzkiS481nYmtecfdH+=b$(5p}Ev9c`0pV1Mi zPk_{l0w)7_-+Zn^_M0q&5lZLmNs8@dZ^`Zu7l*w3GZ(LbdeY20=iix1@QFDnu(CP> z+|{7+lCrc?*yJL=gMne9iSYPU@kpr_Fx%a-+6#1@Beo^ zz?*C5cgGj!nu-E=R#A0*=E$f5b(glIAfP_^_~?Na|MOlMEn|T?Lua{2F>pCw!kmWq znY+dXy4hl9`EoN_?r?k)6|;K~CGRsv^KN`GZ@F6c=2E(7q<_9Ji1kXJkZ?}MCCfKi zm-n1vPS9+9X+T!vZ+Un|$LD@z{;96eOZj z9dmZP)dn{3U3t7)^Ik#aW&JP%o=Qt~zVr4CEIC=1MC`i79X{FKdF-Swnq#d$#S-Q3 zlA)E56*M~a&8;iz0+W>J5u+wBV|IMxv^rI+l~>mB@h4C711Xwj@QFZ&Gcgx(A7sSt zP+lW~NwvMf-&_gjPe1EmwL$+{LW?v0nw%y7hspWQ9_Fii(}GS!G!J;DQK|b6!&U{= z1ts3|zIs+D7Jk|2(}z9gLJ6kEZI^RwIe<3UX~nVw<6iqmnp?l|DnvMm3pga~Iq5+S zP4U!Jv1rWUg0IF$Hm-`m@9Q!}YPK1fj6^+6iyzQj6=!?FB`C_8GpT+ZE}5m-!zT|{ zl()7BhyS9Pz>}QW^Y{WVwhIquBw8iN3G9q#Rng4pE>L^)SSHliAEIwR&J`|vwhZ#h zBq~z-Id?~jmRMLu+l_V>S#~MS8NZK^W&H2jkC)jh_?~HlMAywlm7=ASV#h|}Lhg89 zeEEgJ_~DTfBN;&IWm{;Im=DZQ-pr>vd2;BXF8MfjhwakTlW#NC68ew$g{(_>O=?A^ zLEaN0O9p-&IkSf0@UzSb0U<8Q2WMV@2b4c)qKmN+2U5#V$$*UQ*advjq#9~{ifak#<$G9 z-)*GJZ3uHcR?Vw76U;+CA9rCyA-67b85^ZCG{kEf0>~7c3o!66_f``=H0A z<2Y0E{T<)JhdwMZR@0u@2ff4I$wy`cCfb?h3QJvu*Nb|``%o&`^J4=*XDbi%6}f3t zGQB@XxU#sIk@Oa}r!`etp1p;1vp(NWrEfH^@oVWSVfe{X3e8jfTsob73_?mM_4=9TR3<en=z8R3~yhmdGor;BN>D9+DtUzk{_4fB7~eo|T(A<^4ihW%@pBKbK}GP!5o3Ol-)4;_nLJ$L zS4$v|Uq21%7N}jrymR6{@D;z`a76Pfm_6oXD3#kL-y)JamF3C*F!Q9gkJuxKl}zmJ zdGC90cY*nW=*CAMH;M1FDY3nU-LJwvz1z(>eRSn-IyEwVnbMnoJl0?iY*H$U*B73U zQuJqS>I~u!hwrM8JNw{VL&PNwmM+uF2GuL6obby>A4H*r1%*ts?&TCdE|R#tygUt5 zI4>?eY0(Tx#N?(&J(HCy<2ea-m(pAk%Z|SkkE{3IDH;{88TYi6h5xY;Pfr#vz*1tn ztn}=fCIH5o^P!XVc}j5Mqwr7Ozqmf!4rpqOn^Rf2sasQ7pm~=E=j<{kR!^cD_scBb zDEiZbNrysE?&#;XCw=hI?T+;GrvIBl5KaaP=upSqMi(mHTuMqw6(Y?wf!^o1 z;IERBKK3%6mtjm0)@_&8G8Tnh|CC%5;v}=9m_RUG_OjP_MKCx|Oq`1Il?tB{1Df2&t{>lQ+ z+D?kHLgbPfKlIygE)C^YSv&D*(&E9hKCqtYhB;zWv#{YkB?rtDS+T(5fxwcBTVdYiZ|z8745MGaGR#V?0AX5 zmD0YuDrA-L2m&Tr|F!%r8?(e>Nc4K$^(W{Jk4H?7&2eUj+bv8I70@~y)Cda!IdXW z`{*$e|EAh2m`Tm~b;wCvv78x%iN1Z-b}6^ObsNFo-ktJ&{Yvfu-2ae!`*?G6z5*y6 zf#rRkGmAFds-(k>xwMewxjij!ri%#FB46WjbAtlj0E@Eg<>D)+Y&ewPG;ugI%lYk$ zRe8w4ZEc6Ur*`I^%?Ce@YnKl-A3@wx+v#`VxH5OGa+(^|){mn};edLX>q)xO*)n@% zPNMmgBhsa#rsrk8`v_)$?AtLAylO%&uh#S-gwQ%_@If7Ow?^_qG8ddUQq8BWteoA+ z+Fji;)-tj^ghX@e3}D(e*uD$cmT798aWHUMnXw2dXst74x81Q+Gw02KjbMa?|OAE2rO=ewjcC|ml-5;n!NnuBc zR)5JIs(UY1bkOk0=a^o{xnO(M-AB&-vSar<7N=!k5j@-mf1ix1zxQd3k`o*rgOk@5 zR*8iSI`#|0{`R5ON|$hg0z)O2w^t{-FR~!)@wI&Jmf-42d_7{mdtlcqP^#u$(i8Dp zjDJ4tLOY-~r$T;tT<3q}emPF8sJG-T2e6w>P}yrSln4Q>KO*))R<#vMJ(Szpbi%YG zGU+~-7~0roxn>@%NiDu24A`YhBa&T(~k zWpndu=qt{n7A0Gi_uK-nXHn4XTuOTxtHotP< zwDC=_iob(unyJ|VF$<+v0dQrS$Sr|aMX$ocaH0l6nX1Rw@JaPK-huUn@<{QZS*;og z%SyuFYnNN{ItKm~$$NGHcD=UmpiCgJ2c`S9Vfz}!=E9^W^^+_RMZiFlW?IB>%-KqObk%6jg)66x509&k4liKA{!=qMdNXwd?ozI>Ro(y7Au-gp z1hP51@XHl!^?}8r;gUBj6YL%Lgkn`V=ho&SVSzOsV83sXYxSQ7FA#18k){ktw}R}{ zymh7y8Wx{-^0DXRR=VZmH&cuc3qxzwWoYrxfgwvPuLC=aqqDAvW_-|hj~_(2fc;1t zt@WXB5^=IA^B_doKUbmC#l|*>PuZ6+j_qyr4GYWipX=mR#pF^w-wwt;}>il;t|P_l^!FZ_ATYkCI4@nmi>N=s8*{MmI8lSU#}M=5HwMv zcdzr9A5?V6H=bLD3NJE%q3LOc*heh{kCeMF-2zy1?Y@P{#hL z$Q466wI@bW3oU*XLF5`@9e$L@+H0Y;zGq(%L(Yj`;zz05 zMP>sZRXmpKAXGQkI#H)xeGK!}4-x4wt%_g)KM8mT*f|^B^MZ~2R>VkE=Ak}oZ|KvZ zVC4sKz*0Wzk8v86`p$h{jT985*(HwhGBBhovSyfB9TY??2QGj-%48mT?UuG>IP%7cjJqBV7^Z}oj86#A5I_!qE-%jsy$vn%NbVLos7Za=&^u90nod-ZjrQIs6y z*;i#Qd?2^w`EXNepej{VFI}Z}wFG|s2&mbXo%E-}nX5r#lBZZ=1eUmRVB|oDXbUTP zZH*nFkB*pHSfvBeiZCQEQN$~HW(AbehHNf4_@Or^u#b)tzKsxBy=*g>za^ynSQ=PA_=#A zpVa8XjP65cspa<(SyD5iJ{!J0xzrd)uWeWG{C>mX{L?1NPc6kb$D%%Z70X2#juf!d z+!h(}5HoA~g_rF0$%fH3H(+OHQYvMX*QW5xMSM$XE}87h)@3NCl)({~$C@K^#a)MA zGj2wSUv?@PUCa5VL`y}cwl`F=G48g zgjIUr%aBTr_1wOXG}G+E)z2T7(>`^I6})!V`;78(_eE@tWQbhQGSLd#JQf!=-h&=x z&Am07EaF}uZ#=B(rkJK&DydBUNEhWcQ8!#1I#(?S&-=Ne`&cay&_n4Xf^rJzK~qn3hLTB{XHw@qg?8SXNpGMmRFYxm+9W@*tQ{iMr|?5&b$aJ(V!;AqJ*0)Sd;^Mc zv!4%D`DMNl7$9hj(?hd5xWabFO5Mv0jlcgTsa7hSDE*L$k;dgcKb?Sm&ZiSW6NE`&#s@-2zaK7aH2s>ZgxxJyItl=h!btFT*hzMX|ZqNh~ z5?)WH87r(VX4@22cy+fGO=caPiXyg=@DX-;ATe!i@rd=c8t%i^{x2%jF!O3YvhWWo zpJEKg1HSoOOwe0JHLB;w2*Nj>t9Rm7Bk2w3&`{qo))%;h7mf8sT`WFs72}QU5$r1D zWsjxM>H4`!tpIw180&j6TUhEy#s1x~>DF|ZB`#7gW=T~d^(Y-}qfiDQ%-^l5={fV} zH>yvrLwi(le>Dxu4=m`jdC7Z@3Qtw*U;atI+n9q&glp)rkZbc+NZ!zt1AVhO`uT!f zqy^#EqvEC9iHYoW&7Xzp#`ILU=Tzn=vu;c=W8utAbyS=^wj1XL1kB|KtE$WcV2F>x zOywdRH+6A|>pF#zw4U;g~0L6#X>T&{p((d7Xd4%vD!zj#MGfc5VWQ;R43ERg z4+|&MO6Euh9YSH%;g>f3OQ7Sz9CXI}A4AfR#kFW`w8pf&CUPM)x4sY@$Iu4EMSb>f zvZz-Q5z-5eb&-By@NCF|2-IRFF>D|2AqG$}w{AwIOj}#V{y?${8!$-BDVWqVH;~4D87-)f=~Dp>Q)k%23YH;U%QH3NZMyAj z;FDsUFQRCq(9J9i1h5;Sfjrjq1V5UsBELFx5X7^AF8?Sj;3TJ6O&fXUwD=23-Ia?A zTYkRl+YVSvznokh^lnzyE-GiSo9jKyUbXCf6j|G${{1NWD0PN+^V#~Mo&jF+6F`cY z3joszyf-=J>I~Pc%;=_CLwk;h%uau zTAqJ+p2{C>)zF#y_}?7fZ{qF8)M>R*r(eVOZq&btO=mP=ptVOiM4?OsMtCV5BF$Q} zf(D#USNfvTrEBX44X5c1Ury+SYs9gYe_i`Df<;6R&7+4FP8CXo!;?v`T{>M6HqKto z^LJ2n-RN#%W9#$IH$>zvp9=ph#1;U4n3DXI4$KCQd6(n#@ZET0Qh33#6_D0RCj4M>4=*?W(MtCLyF? znGq#=6cL+=H%h2OXf_s%;m;c95DOlYrYQcp#v;`G>(z>|3`665F;sCCp(e1@5QNlsm-#7te<*rUs_@JAlHtj?AOytN z)u`|ETG-Y_sm-ISUbAb(?jlR{jkbfW){Q8u)0uh5V_ch;XzM0z{R3}!TTD}6afO}r z-z4wf-~NNgmJ%eA^noA{!^VMhehiCq&92LqO-(Tn_%5U=JE(jQN_%;e%Zh) zPt%0m0cU9VElkYrf#G(c-B7&wA+kf|7%A4dYBVZkRv3vfaIPdv)~JJNz!zHn%i&ZH zRKW?Gxf{J}$tG;Qs<93g+}ie~HFRXtjd&RcI$A@lMPi9bw0z$jxSdd@ zIA^ZOh3YrmGeG9{bk6USk>yFvZM=1>yn9(4E{3R<e?bn3v4P{F zbo8SaqHQ*A0**w_Y;$G?c{9YXqcma@*Vt4{uHqwFB`W#av^&Ei0Z1P7-6zgjhZ5Ff z!aNplu5;;88>VyXwTc$FlDug_zTNm}$4D}2wpSEJsoYsfQ@xL9UG-k_ISiV`nUK75 zZvN|`YAdj+dxh8~1lCkcTb+vEZ8&~)J9Q0-gCU_nd8@)E8KX=D7}o^*@uU1jf>Z0ozJxMW;eP!kYHR5)v>M0|a)+UIT{ zgEQ$$rimBv8_SGp$3!GoPFjz3f9<|>U&nAviP#&Ygwz+cR&jRoS#P=4s)hDfLb3og zcKCQsDPV;$AUkJx^lVJmwaZtZR$d##Ee9mAm-h{o3bieBz7UL7FHgtJ-14Ns$G1%=Tps)%Di^+J%qoGcu1<43WMF*^k$zYbIZ8_!nMaH8y}rAQe7 zjFsMyE_wmmoy?4#WY!Cd;GtftDH3)R+W;wyn?zta=tauN^LVvpS#Atdk%I@ zmP(13f~0#n*osqoefz2&yKN^c((Z+g@{u!53?MMD9&aDX$1J3%rUU!ZYFHFj6#uUW z=WT#<1;TQGqBY7$|9y$Cgb7Hifw%uzHN z60G2XHdgcQ&N!`NETh3%-j{uL>dyMfme3QMQTxHuT4;#Z*=l^z_+X&Vw^~069;0mK z@@w^Wm=NM}>QU%v*e?FkgOnZ|=l(JIa|Q*sA*YId{Lo|5bo3vgY<}6LH@dzu!gRXj z%HKmjXEmRsrsTP~-^JvrJ<*`Z>JJU?Md?Hf#t~hz$|~FQR@SK+Np%qfi-Fy1gM-3HtDGx<2k!}PGP<_7?}%q)K%>% zK)jL&BQ+O~8s`GHd2z;wplQfVVhcw1Fh#AAV;XvL)rU1NbjCf16UtvW~uTgiFt z2W)l>=+^8KFJ%*KdKsW|m87X(e0HjF8agMdF*^`!_B3_CN=8EmA+Hg`Ufw%x=Di0_ zN6-_{xVM?}PUr6gtvt{1=-ah~@gDYHxbC-BFttARKtB*wx}NrR6r~UXMtvKAiFYh) zvj%A;Pdi?3`~*$;Rt#8S!{o?EmS6sd3?-Wxm53C>;z4EnO$O+JEbN#(s$kUZw84T@ zc&#p*DKtei#m^)<7e1bI8s#ULX032cH!VO=*?*>(NGa9`1g8h!;1L&H)66Zc_qV#J zjTw0=)kMN^6CQ?W)|WW9)hIE&CC;rw=R%jVPYqX2mc6Gu@--({S5$#|p*UjMJ%=3% zXGd_bX0rR97f;E^t!w-0S~2rjY?B;6u#aqY?C=aSpX}dUq*Y%uU(i99gJX8g zD%0PZ_k9NwHzkQ~sOlpV(GK``yik;#=&*n5rNVZv?cAXNELJJKw6~5z|FB#13pm>4 z_yf!@R|Qzdk$sxC8qaBPDl?+Wz*ro_FZ!G=Az1-N>Q#uL=H~KWl#8i z{bo*ikAd_Lzvqk_UD5fB?JWo-GZYlF>hG2a3e7^lbeb@OrkRTu$;hIPK)ZaioE|H( zWdu!-h+j@lmw$`yWKZn4my>|bif}CSRjTP%U`@@4*CB-W9_>KR z?%9W6Sd&1VnMtYQZ_^=|qilhjfLtnox?QuZEr`5$aXale9-uY~CDsQLtc#GJy$$WK zpdiO}vHpqeq;2C*vk3HGa=Q)T)tD8c^axH5{en`B9n>Qh==7HFfCY1xW@etQga5h< z;0rkQ6?7MMRlynb9wBFj*}LhrJx!yud-CS%JU*FNF!&WH4uaAcaisLHPIr#=5%kGN zVLs^91J-iRb>B!#KsG#(Be{k&uz^lse_`TT}Bx60}(;)nW9 z$&{zL^$Fln_QTv1FV6%qPl5ZRR`UV0$aOvBSTsXR<7)+o>wR3YO0K2q)&2vwlUuj-HM`@MQ?WppPRa3P-=u=BYb5<(OdY)p)5%0jdwr=e%4p}R1 zZ{YRVrjm+DCE!8P*hq%&H-O_xC@&R>_W0V2tErR)xualwz!y4RI=i#h3D;l-1C9Xb zy<-HZ=E`^?))dI3i3|o1S=$6}8qy=zu^$#|;IkLy%;mTnqybmF3n`6}zLS}4l>Qzu z*m!iZB~BgQFz{hLCU|ZISL9oIl}S6{Vr85>If!DMp%{33$iuUEBY?dB)T&pVf=OV8zUJVR%9n zuJ(ZpSh7~-$hi;sF2}hep68c8DL&1X0b3owl7!r6Wi^lW`$8*%2#>PP-mpP3icZ*u zd>%lk6_?YdWAxA&X>PXx?$ZT15-DG6ky&2BW72g*)>uZlJ=`y~+K&rqRZ>--Q$-FJ zlNbSrV%YIbtY%%(J=`{TTB!skhOB6Bpz!PHccp!QDYqWh-filwBhz$ChJr%yV7)LX ztohPxcKiCmh<>KzPSQsC@|nrMUX{OG56Y5fH|!3??M)5+CiRjZhol{VzFk$BQmsc6 znTGJ7U~%s{i3MW4nN0{x_M9|zDryN&nkMJ19YVdcU0=~(RPwrciu#2hzqY4w8?eO; z5tcITGfsTeeOAq92!K*A&mK4$mv5-0wyTF~TRq<$iEA-z0!s1eOj4AgP&B?6HnrHC zRXKI3%gN`RFKd1UJ4x`?G)-kq)_UT`vl>_reDqwGnk*>fq9LgT&u0Jm0H_)>zrfv&7tRqN~>^WepAbwBYL%~Rado4)rfZnjU2kzum_v*045JE6H0MI~8w2Zvt7 z;hq9kFyP~Q(ntil=b7MG;e2jDd24ElsCyw;ITjo)3Qa8^Jz{L>C@N>3zCKfG%^J6| zS+aE`ougKrhPhh1LA~{Gre>H`KWTlM{)#W5*MCerHAK?ABYCjAy}L44=HQx#d)&hT z?}OC47_&oU-hz-JbTcD7aXw(Ii&uHR)a@SRR1x-^s(Gs3u7VtBsv-FLG) zNtF}kRH;QU4mGM%+^)(5+=iSii;3BEedoJD#2wJLdjTy2u^?OaDH{6mh3+1dqWN7j z>JJ_c=r66I<4#2|j1J8}0#Vy!EIWA>w`<34MaFMd0{P@X>eAkPzv-H#33vj|rJB{6 zYA$D-9$fv7$)rritdOO%)uCX?Rf>n}byR0UPk)|~%xPBetrTI-7mZ_uYf)<}OPpy@ z=C7d2gr3&HhE-E&RU|HnYVW@85@ap6vjO;U5jbaRcMSV^z>M0VIXPMWZz)wC&d&K{ zU-L>c*CRkinCrn|?8U^y(8k1G+NhYcPEBK9w6~6)9=i@=y8J>cf;Te>+#kKr*axw+ zoLN`lB)g3fJ_||?+NL@k0ky8XNU^3`f7wY^uI3yCKER}|vGUGBu%0~9HIrYOy$UP1 z_W>E=Ia&nkGUU0Kj@S;|J!exDTUS3zxU+)5nt0+552f4iod2K0^Tg|Za&LJbF3sM~C}?)R6gQoOFz&_qQJ*F!Ps-}Mr56OXEAUK{yj|R;$IckS zV(i8%QoLPG@%U2H($i;M_@lkiab6iqj?-&dQ{xH{Nz2g8wUQj7Mg*Ai*cX}){$|Cj zS^LHb_3!NPQ$UKFIPfoSN*wwRbb5CJu!xK6>szfco_QzgIVbbBeq=_;FGE#z*`WoHSLOKl+#D|MK3f3v^C7)`U=;Mx=6938dqB53X=ZexzFy;0sxO(d7fOj0RsnRK#uEQlms1LGbW!yQq8Ih?9q>g(fLo8lqaOWR>F zsL}y9GR`Gteyz~EIK;C9a871O!CX^bW}KfgrAMV;-Jkk1`R&!Vi-yY|{5c->4J37l zfE-_LCQb~=h3rT=er^A6#xo1;f5if76;pI`;}%U#v8~s0H5Y}3f+y@suDTwMin&0N zmOGQse!h4>SdJatm_<===PDUjX5{a+yQu#PJ=-#odj=w>o{(IjlO_afSbq>VR^~bC zg=q;EMm%I~vMmf6@dDYY*60TDK$JSba-f7Y=4mPBxDy@h)1>U-2nDvdV&#da(aDqm zZ*Om|rq4t7X_g;%qSl+oi5XQXBfVAaZe+eGVH8G@9VlpKfP#ipIt6>Mx^lniE@Y%+ z@WIe)9^^}D}B#K1Qzt*qEUq9J1 zfyW27pCk2p*)!`8aIkP?**-CFJub=>)6)2I{+6Hp$*yIGY$vAjXzxac910`XecOu$ zu8L7{t%5leMf-aFwnhT4orB9~lc06eXdO){=JH4ce2U~V7BueCw1OLZlboL>4~y|m zpf~sA9U~>InezCqD4V0twglhr9Gh;ykCzp*;U8q!?=pry+%y>mXlzfJdK$w8d1;GW zFYD=wZrmC|QUbE6uxZN~NLmTdac=0#Uz3``pX6bH`E)w}M+~)X#pxF4%;Mzd_EFXz zm8>W#wmzc%`NarPbrA;UTIkcO0oNLp)$xX^!n_(DB|!q8Z^tMdg@ zj~!+KqZ1}yecvMAh)(Lx_Xb0@#1XDQTkop_rYu}>E$VQJsjMct@%dBC;ANK7FyO{Y z%L$-6y@n_Xz0sM;%g)6-sRvs6!~xz+Meb<2vj3Yt7UK?9h>)B~Lc0cF%(x{lNQ>K; zC26Upa*UhAVfWc!xT!+t6*{i}D=XlTy{#)aYOzRxYeN*`SZ34a-;@kTP!vg}t#%I( z#x=0CxHvk<7Y5ZTCfFoOLRRtPv?Lui6NRnb5PjYS%Ou43(T3&@eTjf5h2F9#*&wNAc?`vO*TW*&$M6(w461 zli&A^g}2GgPDSpLo7>H(p{BkEyq?Ocu`3GTVN8E53kX+(UXS4&S8WWAPFC=x0{?d7 z>1Jk_FSV8HE#S3!(Q0e{p)SDMNw?cIBGFdCPS^upR zR)+nxY94CKdSsup*1(xlWxk&3&ei+wEbzmq5Zp!y$ei79xfiWuUD=rMr7XdETB{x{OPb=Y@7s&jh25{Z1kfUf~hq`I`Z%}eEB{| zNb7aKp<}Zl&Beji8UNCMbvpRHxhaQCG1^a9O-%L}+gPUc^N~A!HG*p@I5#hH=kEWX zTUEu;U4)Bq7|xB>q&{eZEwn;_(g965TyCvWlT&x`H}e99t~l~5uNNJ|Fx!h1s#n6i zMUj>*CjeV;d}qDHtaYN zMQcKPV8#k!7{;qIY3C@J)+^PGL(mbAZVYsM6y-Mq=ujLBO8#rq3tA!(23m^H!lFhn z)O-VIW$>Bm6*>3!j`hr29se!7c2*q z?G^V;YC>9fC`;6@@Sc6(rln zMj-o)n=!EiuNPj9dTc=y;d1_6-LQY=QfMaan*oIw+)g{MraTd{K9TG(J8nm%R*vuN zU)?AiH#$M?Yyx7&w=hVR@m*e6(YZMk18(}whGlSiy+U$y?zlP`x&$Rk%#i zkxV~?*_dAn(Jo$J*GP^0-ikM&l$uFVN?<&AHf!_8zzbqP*=Gka3*Nq?*FWo3Z(VV3Hx2w#qgD5$%y{KCO6fB&BW) z%^BbemK&mxnFb^m%8JSFhk?z~Qj_L-iY&i(GxZJpyyl;+Dvzpq>$(87jkkOaJeJh{ zAS`4xe9~iNv;@UiOum2o4?lZ<8Z>lYTwxe(jDfA56o!Ksyroc{BQg*A4p;b>nk{W) zPcCj}Ks6z1N>Sp{ED;D8AJ)m|gRFWp3lQFv8|>v#nPNHrdKldN^^ww#*SHFX7tmG& z=jZ%SMI*enAWwm^&+;$Re=op#HNTb=@`goq9?cra!&+@a@S`+C(+(zebcyE#<^3UA zpN`tu)cGomH<)HnK{Kr?XscPxjU+Naq-AU75#I2cXJtjq@;>%@Jdax+KP%$&8e&xW zm;`1yP7n};sxjB6Q#WQWD31=~PjVJ-q`Zo6cOwqHb(-Ig5_C=-`cnS)ed_ICqV#_t zIs4J^7!A~GrqQLBv+nQNeZSiG>E5F)lmgQ=!=tC}{O;!akAEz#)$HP0(o=oQqPldo zO02bUWaB*|^&lc(?h7jMGLn`TSn=Isc&SrGk2ih^n)C20$wHPkCYO+j^Z+ zqa--)erqZE;0o?y6nouT`H6^rdF8G2hR zkutVNbnxj-Mo~*ipN^`A|E*rtgSyd0kPMgPar2#ysbC<<1%c-24Y8lx$}yK`dF}rA z20YyN#Il!8G*J4YspiXM^)iEHUQm={%1!8---fdo3ObtxbvfQ_A=r+s0 z1RLq?V1tiKs#|Xwq}AUi6&DkwG0MjPntvS$1EcbscHVkk9pW7q+zoW4Hq_XAGpiK; zB(3_9*osRjc=hHQCZj?_z$=1*_i)7FV@xOGxpz>xbEXc#{$pdzbA4msQ85v!I@hM4 z>^;zCN`auZi(jls)kOgyVaf{;Y1Yy{0 zhYE~R%Is7wJN<*2`0dcUf+ax0UA1s?w0zF_c^Y|_iiXl{h>|JwhvoM4D-?NDg$+hP z3?fH+3o*-4aN;Vvw(BVp$L|FUz=0p3)T)SKM|dGE%bL(2oBcqWniyW7NC3T=9o{3u z2%FUv{7=8CsgTTs7a`}^ZtJAA;sky^Xit+%c+C!m%&)KpZrHq6$^;!9-DqjHk!t+I z1w42BNo{AyO_3la!eZjjhuToOe7s^N1>P3 z6d-C%9^%~uB=CmHFEJ*+&IkNPc>QP|gFPRk7vv!lH1$3RWm@Tp@uSp$wZj37Y7!JV z-f-j(-debOW3{UXnD1-Fet1$)I5F^!)LN9fAGEB9xFYJ_*CQ!fHkZ9dxe!b<@u6nJ zs)4qw;_Hokle>RA>F~Rpzh^T)8aqDA!~b(Jf-QS0C0%6hBgu{c0i;_W$aIMn_1~+4 zA_@CUriEJ1D zaKlWunjf$w=2mP?(*g_ws~;teCcG%>S86 z|B9aeEuPv^DE%Wm)!#@c9&^D8fvN%dn<7Ed3fFo>=@H5YNbsM&&Nqu$ghe>c6c za;q2r`M9t>5X%0O5XDQ1f_8xWZrE&yX{q%NG#3*=C|AB}QqFL!WN{nEh2N==XL?&C z4q?4p^C`r7x&ZITgy3I_RZ5k;g6;!^H>^6bOSXQrvL5@b_*}eEma*;-x3VzQRyceR zWso%hrlwRTSYW}LGHX!~>2fbIs%E4Egyb?|;=bkdITe(YHLkTyR0ssW{Ta`5wv}e# z8FkNh^kS}NUH=8e-}&)8HJnLc7e&B7{4h>FUifIOWV7_m(-DwrdW;?|L<>t~UO|Yn zGF+n-6%*yHm5+`Rvm&22ZA)Cf7j4nTTSVu^m}6pn*XI?k7g>|fca32+Hk#m9$+rS6 zIyGX+yyHRN`JJ&wgz{Q@4X*De{zt+fB3&~s+p0?j=-a=}ng?Lw02yD_m)b+opZ0D|TiS*8r zBR7jppBjCwv~zzuulBp-{p<2t7>JiHWI3mS&Vh3^{30_}7L5l+VH{H)-S}fwe|GEc zZY(Z{VdD-X(iYNT?F)i+4At-KBf1BWZR;B+b(Q4hlhDV>okKv1VPHbtddUeeYCATl z@BYl8{l=;(4Q$4DvyR}ctOSIdNM@%$jn@7CQ-7Nqd*qEJ*97L)L4%#olJZx=-TRez z<|;gugJ!)2>wE&6=5^Y%{}6QmUFoL2qY3qvX@mN~xq{YTSvkA3d=WNAchPUUufttZ z9tM*b4f!bCGjsFn#(Eu~Z%!}`Y46pnWQFBiDnU?Wy~WDw5b;jStSATfaT7`c(;uCw zZ~MOTjtcXC9|!!wZ~lGO%}Z)6_ywhJ7PRKrYU49IJhEY@v>vn8SU=kr7XdgzM?!Ad z7?Jz8{C`d3V1t;($ofa;is{fEKsn@|lFipm={KcRL8>^d{q1VR0?obISi2B5rVqjrk?x=Dngao;9NlxLEhc9JzKi~3oF00E7W`;RUD6LBb@n3kmts4=lWqsF8ZKB3~866^O7 zW+zO7b_v;N*#NP}BiOT`raqUMQ)5&Po?iRae6jh+4dm z!=2)3G{}aZf@N@vH{Y*)oc+fivh^Ru(5&U8*=q2tPq4%sy9HDw5W>R$=dKl7v^@jn-y7O>>vlVaFDnyr9xKvu=R1H6BTM#nv@cW+Wi zHR}aJb;so0t<{{TV~Gp+Kat;mNjMfMB`G_aI6RqGq{r8IFJcnbB}t_Rb`Xv**6Y(fA?4^Ot>f zfuQDD$_%smk>B5TDBDhwT;S$pgJqz9*2SqADwXh==aGMG+h8o7;{1o7QR7Y3f{{ z@;%yOWic^{p-hRVO#OGz^-JjaP{ccld-B8Y*bX1#=AM27a1pl#DSzmsz8y+g`qO_J z(?JB{--y2=!D3-w;fP4>4~xx|C10DBku|6NyzijmRP?Ud*yzIF11_Miy@#k#z1KHd z&E{U|EFzz~n_x)e95T*J!3xYt0;hdJw#Mabk5|7qT?jvEVQpksC#1@5-LvEd* zlGt&l&UZ|!xwvWl5k5Z%uQjb$n27Yi7cv9t?|NX$v9uF-Xa8nU@ka&Atv6dmR%sqy zwRn?4Rn12UN3SJrzhf3B^83}z-bW|m&*WtQ@D(r9{n#KRPbg)@yUP0xz^eZe zLi`SOaG2|9M}xcZW&FX8r$um!wc#zU*7io+*pHB4`EYKmMN@CSZ&et zqq682?h;De`CH*#yGP8!gKS!{ZpMrq@8o1CXU!bfn9F3x1D=N04W{XH_N_$yjL_i2nfZZJ#|5% zpb`cYD#+A9r%q}|GC-|$AUFpiSQUrZ3JO>dj?rQ%^vAA;!k`EAu$r2D{6WaqyZgQz zchC2{&-=blC#9uNF?aGK6V67sS51rzAgzQ~MFu^5<6ohY)+6^w#h)mK_E0PvBKiZk zk+yA=|GwQ3o8Dy(zjX*RJ&8JC3{QD^T`ox5<3Yu7MSB+#pvsDNL|-cK7xR|Rnw~{7 z=MdbM=M@kdS*thyU=f&d_R{SWf&EWvB`+as z+@?vzT+>q0i(t1+x7#ltCYN=kdru}S7Y3BuGw1obkN-f2q1xC9tBGa3c3g7a<=8Kg z+Pt+WsaZKg*?jg99(f*zl0Ujb#adZR)pqFqTR(z5RP$`h(t}C8jXnovTah34xqu9d zjeV{w!#;EUGPZsf`vt8wm_Lgi3^&rtdOb$r!J+HQVX%?ysJ!L5uB|jf ztJUGk;Ev#~F^uuXNFTjM#s07FK1=%vXIH&$pS|Hmdh@PxN8x#>>w;*o5grOwoH@n@ zR!&j!XLRmAG4B71(E7Rc5Pv;6V(N$Z)I12M{kZ^rbDMwFKjIkdi8xyH(7g`86KroF z3={%*g6;N!g+c&NFndt3C=(z*dU+LUNfZDp09IINGuVF!*D(iu0$NN=9G|XR3W)aEnFP_@C~FQe06hR!*iI`L;xmwuvHkk7#AhHQ zkE9+0u+kf&`Y4vMf|XNLw@l&?!>w>_e^-PP&=?=&p{&Ua#%`q0msVqTVd$LPIv%N| zt;ys_Z#qDgHJPR#Y0eNnYq?ipXlE*#{_tmz2HUl3pcs0>?&>RRTxBu#Y>oHy-7`H@ zVI8x-GugU#Z0`OD+RqEP}Z8zM>mcW^!C4v`^B(aUSZ?w(~`v0#&6rJA9 z;d6zS5jpanQ#TzJ5OZEUkjXt^miXUhnVZybPs7NI%vUlNL>4AV;=U2~99k=$mTWEz z+tnSzk(a^Ik&JVSkMPL$EpjWb!f86N(*JGuiaWK3~Y{^t!v`~BEzkwn$uye)IvPPO6R+#wfPmrAqDwx!yu z?1C8r_|?`Q!vj~?ZiHYNQ(?{ZT(S((7aNA00l*7Rl-0SD0A8^7D*!LpJ3oLIOaf$F z>_uM9r3utkoNkMfEDEHC?;yQ{flZpyR0ZNJ%vA8l-B4Xql))RHw^rYyF+)v51tIE= z7^~=>NGVHv=GIg08G%wXM#OxkY>>}uf78;qMBsEJpXLPVbtaN!#6-=F{4;OWU*-`fENHS zc+?>PF92R}XobcFFFh@=K%-upb!oflxH*Zd^!wHRg_(?)ZOd|05ff0&tn+*u5M;EH zv2jIAK<~wp_W?}?G#N*-2he0dlkuef0K5Qr`DEb*MkH3I>VjA2S0@mORVhAgLqWc5 zUUPD>H0FAf(r{|(K&|tg5DOhcQfs0%2~88JsTAeTd>_3Qdxn8huQ;o&8sgInITB6t zUNo5pF +# RFC-53: Use Lock-Free Message Queue Improving Hoodie Writing Efficiency + + +## Proposers +@zhangyue19921010 + +## Approvers +@leesf + +## Status + +JIRA: https://issues.apache.org/jira/browse/HUDI-3963 + + +## Abstract + +New option which use Lock-Free Message Queue called Disruptor as inner message queue to improve hoodie writing performance and optimize writing efficiency. + +Disruptor linked: https://lmax-exchange.github.io/disruptor/user-guide/index.html#_introduction + + +## Background + +Based on master branch, hoodie consumes upstream data (Kafka or S3 files) into the lake is a standard production-consumption model. +Currently, hoodie uses `LinkedBlockingQueue` as a inner message queue between Producer and Consumer. + +However, this lock model may become the bottleneck of application throughput when data volume is much larger. +What's worse is that even if we increase the number of the executors, it is still difficult to improve the throughput. + +In other words, users may encounter throughput bottlenecks when writing data into hudi in some scenarios, +for example the schema is relatively simple, but the volume of data is pretty large or users observed insufficient data throughput and low cpu usage, etc. + +This RFC is to solve the performance bottleneck problem caused by locking in some large data volume scenarios + +This RFC provides a new option which use Lock-Free Message Queue called Disruptor as inner message queue to +The advantages are that: + - Fully use all the cpu resources without lock blocking. + - Improving writing performance and efficiency + - Solve the potential performance bottlenecks causing by locking. + + +## Implementation + +![](DisruptorExecutor.png) + +This RFC mainly does two things: One is to do the code abstraction about hoodie consuming upstream data and writing into hudi format. +The other thing is to implement disruptor based producer, inner message queue executor and message handler based on this new abstraction. + +Firstly, briefly introduce code abstraction(take `[based-master]` as current logic/option, and `[rfc-new]` for new option provided by this rfc) +- [abstract] `HoodieMessageQueue`: Hold the inner message queue, control the initialization of the inner message queue, +control its life cycle, and provide a unified insert api, speed limit, memory control and other enrich functions. The current implementations are as follows: + - [based-master] `BoundedInMemoryQueue` which hold a `LinkedBlockingQueue` as inner message queue. + - [rfc-new] `DisruptorMessageQueue` which hold a lock free ringbuffer called disruptor as inner message queue. +- [interface] `HoodieProducer`: Controls the producer behaviors and life cycle of hoodie reading upstream data and writing it into the inner message queue. +The current implementations are as follows: + - [based-master][abstract] `BoundedInMemoryQueueProducer` Producer for `BoundedInMemoryQueue` + - [based-master] `IteratorBasedQueueProducer` Iterator based producer which pulls entry from iterator and produces items into the `LinkedBlockingQueue` + - [based-master] `FunctionBasedQueueProducer` Buffer producer which allows custom functions to insert entries to the `LinkedBlockingQueue` + - [rfc-new][abstract] `DisruptorBasedProducer`Producer for `DisruptorMessageQueue` + - [rfc-new] `IteratorBasedDisruptorProducer` Iterator based producer which pulls entry from iterator and produces items into the `DisruptorMessageQueue` + - [rfc-new] `FunctionBasedDisruptorQueueProducer` Buffer producer which allows custom functions to insert entries to the `DisruptorMessageQueue` + - [interface] `HoodieConsumer` Control hoodie to read the data from inner message queue and write them as hudi data files, and execute callback function. + The current implementations are as follows: + - [based-master] `BoundedInMemoryQueueConsumer` Consume entries directly from `LinkedBlockingQueue` and execute callback function. + - [rfc-new] `DisruptorMessageHandler` which hold the same `BoundedInMemoryQueueConsumer` instant mentioned before. Use `DisruptorMessageHandler` extracts each record in disruptor then + using `BoundedInMemoryQueueConsumer` writing hudi data file. +- [abstract] `HoodieExecutor`: Executor which orchestrates concurrent producers and consumers communicating through a inner message queue. +The current implementations are as follows: + - [based-master] `BoundedInMemoryExecutor` takes as input the size limit, queue producer(s), consumer and transformer and exposes API to orchestrate concurrent execution of these actors communicating through a central LinkedBlockingQueue. + - [rfc-new] `DisruptorExecutor` Control the initialization, life cycle of the disruptor, and coordinate the work of the producer, consumer, and ringbuffer related to the disruptor, etc. + +Secondly, This rfc implements disruptor related producers, message handlers and executor which use this lock-free message queue based on the above abstraction. Some compents are introduced in the first part. In this phase, we discuss how to use disruptor in hoodie writing stages. + +The Disruptor is a library that provides a concurrent ring buffer data structure. It is designed to provide a low-latency, high-throughput work queue in asynchronous event processing architectures. + +We use the Disruptor multi-producer single-consumer working model: +- Define `DisruptorPublisher` to register producers into Disruptor and control the produce behaviors including life cycle. +- Define `DisruptorMessageHandler` to register consumers into Disruptor and write consumption data from disruptor to hudi data file. +For example we will clear clear out the event after processing it to avoid to avoid unnecessary memory and GC pressure +- Define `HoodieDisruptorEvent` as the carrier of the hoodie message +- Define `HoodieDisruptorEventFactory`: Pre-populate all the hoodie events to fill the RingBuffer. +We can use `HoodieDisruptorEventFactory` to create `HoodieDisruptorEvent` storing the data for sharing during exchange or parallel coordination of an event. +- Expose some necessary parameters for the users with a proper default to tune in different scenarios. + +Finally, let me introduce the new parameters: + - `hoodie.write.executor.type`: Choose the type of executor to use, which orchestrates concurrent producers and consumers communicating through a inner message queue. + Default value is `BOUNDED_IN_MEMORY_EXECUTOR` which used a bounded in-memory queue `LinkedBlockingQueue`. + Also users could use `DISRUPTOR_EXECUTOR`, which use disruptor as a lock free message queue to gain better writing performance. + Although `DISRUPTOR_EXECUTOR` is still an experimental feature. + - `hoodie.write.buffer.size`: The size of the Disruptor Executor ring buffer, must be power of 2. Also the default/recommended value is 1024. + - `hoodie.write.wait.strategy`: Used for disruptor wait strategy. The Wait Strategy determines how a consumer will wait for events to be placed into the Disruptor by a producer. + More details are available in followed table about being optionally lock-free. + + Alternative Wait Strategies + + The default WaitStrategy used by the Disruptor is the `BlockingWaitStrategy`. Internally the `BlockingWaitStrategy` uses a typical lock and condition variable to handle thread wake-up. + The BlockingWaitStrategy is the slowest of the available wait strategies, but is the most conservative with the respect to CPU usage and will give the most consistent behaviour across the widest variety of deployment options. + + Knowledge of the deployed system can allow for additional performance by choosing a more appropriate wait strategy: + + `SleepingWaitStrategy`: + + Like the `BlockingWaitStrategy` the `SleepingWaitStrategy` it attempts to be conservative with CPU usage by using a simple busy wait loop. + The difference is that the `SleepingWaitStrategy` uses a call to `LockSupport.parkNanos(1)` in the middle of the loop. On a typical Linux system this will pause the thread for around 60µs. + + This has the benefits that the producing thread does not need to take any action other increment the appropriate counter and that it does not require the cost of signalling a condition variable. + However, the mean latency of moving the event between the producer and consumer threads will be higher. + + It works best in situations where low latency is not required, but a low impact on the producing thread is desired. A common use case is for asynchronous logging. + + `YieldingWaitStrategy` + + The `YieldingWaitStrategy` is one of two WaitStrategies that can be use in low-latency systems. It is designed for cases where there is the option to burn CPU cycles with the goal of improving latency. + + The `YieldingWaitStrategy` will busy spin, waiting for the sequence to increment to the appropriate value. Inside the body of the loop `Thread#yield()` will be called allowing other queued threads to run. + + This is the recommended wait strategy when you need very high performance, and the number of `EventHandler` threads is lower than the total number of logical cores, e.g. you have hyper-threading enabled. + + `BusySpinWaitStrategy` + + The `BusySpinWaitStrategy` is the highest performing WaitStrategy. Like the `YieldingWaitStrategy`, it can be used in low-latency systems, but puts the highest constraints on the deployment environment. + + This wait strategy should only be used if the number of `EventHandler` threads is lower than the number of physical cores on the box, e.g. hyper-threading should be disabled. + + +## Rollout/Adoption Plan + +Default executor is `BOUNDED_IN_MEMORY_EXECUTOR` which use a bounded in-memory queue using `LinkedBlockingQueue` same as master. + +So there is no impact on existing users. + + +## Test Plan +1. Add UT `TestDisruptorMessageQueue` and `TestDisruptorExecutionInSpark` to guard above logic, also validate data correctness. +2. Add Benchmark `BoundInMemoryExecutorBenchmark` benchmark with BoundInMemoryExecutor(based-master) and DisruptorExecutior(new option) + +## Future Plan + For now, this DisruptorExecutor is supported for spark insert and spark bulk insert operations as an experimental feature. So that there're also several further steps need to be done: + 1. Support DisruptorExecutor for spark upsert operation as multi-producers and single consumer. + 2. Support DisruptorExecutor for Flink writing operation. + 3. For some cases like bulk_insert and flink ingestion, we may support `DirectExecutor` which use no inner message queue and read messages from iterator directly + ,writing into hudi(remove the producer/consumer at all). + + From 57dbe57bed963f529b2921b6bb6253a90a1a45af Mon Sep 17 00:00:00 2001 From: watermelon12138 <49849410+watermelon12138@users.noreply.github.com> Date: Fri, 27 May 2022 14:08:54 +0800 Subject: [PATCH 23/44] [HUDI-4162] Fixed some constant mapping issues. (#5700) Co-authored-by: y00617041 --- .../src/main/scala/org/apache/hudi/DataSourceOptions.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DataSourceOptions.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DataSourceOptions.scala index ac4d0e5794392..a62a402b6ac22 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DataSourceOptions.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DataSourceOptions.scala @@ -540,13 +540,13 @@ object DataSourceWriteOptions { val HIVE_SYNC_ENABLED_OPT_KEY = HiveSyncConfig.HIVE_SYNC_ENABLED.key() /** @deprecated Use {@link META_SYNC_ENABLED} and its methods instead */ @Deprecated - val META_SYNC_ENABLED_OPT_KEY = HoodieSyncConfig.META_SYNC_DATABASE_NAME.key() + val META_SYNC_ENABLED_OPT_KEY = HoodieSyncConfig.META_SYNC_ENABLED.key() /** @deprecated Use {@link HIVE_DATABASE} and its methods instead */ @Deprecated val HIVE_DATABASE_OPT_KEY = HoodieSyncConfig.META_SYNC_DATABASE_NAME.key() /** @deprecated Use {@link HIVE_TABLE} and its methods instead */ @Deprecated - val HIVE_TABLE_OPT_KEY = HoodieSyncConfig.META_SYNC_DATABASE_NAME.key() + val HIVE_TABLE_OPT_KEY = HoodieSyncConfig.META_SYNC_TABLE_NAME.key() /** @deprecated Use {@link HIVE_BASE_FILE_FORMAT} and its methods instead */ @Deprecated val HIVE_BASE_FILE_FORMAT_OPT_KEY = HoodieSyncConfig.META_SYNC_BASE_FILE_FORMAT.key() From 1767ff5e7c7d7a06d0267380eb3b830392a3bf28 Mon Sep 17 00:00:00 2001 From: Alexey Kudinkin Date: Fri, 27 May 2022 02:36:30 -0700 Subject: [PATCH 24/44] [HUDI-4161] Make sure partition values are taken from partition path (#5699) --- .../apache/hudi/BaseFileOnlyRelation.scala | 10 +++++++++ .../org/apache/hudi/HoodieBaseRelation.scala | 4 ++-- .../org/apache/hudi/HoodieFileIndex.scala | 5 +---- .../hudi/functional/TestCOWDataSource.scala | 21 ++++++++++++------- .../hudi/functional/TestMORDataSource.scala | 14 ++++++------- 5 files changed, 34 insertions(+), 20 deletions(-) diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/BaseFileOnlyRelation.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/BaseFileOnlyRelation.scala index c57f46a7b6639..4160c34b0ce64 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/BaseFileOnlyRelation.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/BaseFileOnlyRelation.scala @@ -54,6 +54,16 @@ class BaseFileOnlyRelation(sqlContext: SQLContext, override type FileSplit = HoodieBaseFileSplit + // TODO(HUDI-3204) this is to override behavior (exclusively) for COW tables to always extract + // partition values from partition path + // For more details please check HUDI-4161 + // NOTE: This override has to mirror semantic of whenever this Relation is converted into [[HadoopFsRelation]], + // which is currently done for all cases, except when Schema Evolution is enabled + override protected val shouldExtractPartitionValuesFromPartitionPath: Boolean = { + val enableSchemaOnRead = !internalSchema.isEmptySchema + !enableSchemaOnRead + } + override lazy val mandatoryFields: Seq[String] = // TODO reconcile, record's key shouldn't be mandatory for base-file only relation Seq(recordKeyField) diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieBaseRelation.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieBaseRelation.scala index 4b7177f4d6326..08f87816d7c35 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieBaseRelation.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieBaseRelation.scala @@ -171,7 +171,7 @@ abstract class HoodieBaseRelation(val sqlContext: SQLContext, protected val shouldExtractPartitionValuesFromPartitionPath: Boolean = { // Controls whether partition columns (which are the source for the partition path values) should // be omitted from persistence in the data files. On the read path it affects whether partition values (values - // of partition columns) will be read from the data file ot extracted from partition path + // of partition columns) will be read from the data file or extracted from partition path val shouldOmitPartitionColumns = metaClient.getTableConfig.shouldDropPartitionColumns && partitionColumns.nonEmpty val shouldExtractPartitionValueFromPath = optParams.getOrElse(DataSourceReadOptions.EXTRACT_PARTITION_VALUES_FROM_PARTITION_PATH.key, @@ -419,7 +419,7 @@ abstract class HoodieBaseRelation(val sqlContext: SQLContext, } } catch { case NonFatal(e) => - logWarning(s"Failed to get the right partition InternalRow for file : ${file.toString}") + logWarning(s"Failed to get the right partition InternalRow for file: ${file.toString}", e) InternalRow.empty } } diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieFileIndex.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieFileIndex.scala index 08d0d722b2f68..d73e3a5d3b934 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieFileIndex.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieFileIndex.scala @@ -108,9 +108,6 @@ case class HoodieFileIndex(spark: SparkSession, * @return list of PartitionDirectory containing partition to base files mapping */ override def listFiles(partitionFilters: Seq[Expression], dataFilters: Seq[Expression]): Seq[PartitionDirectory] = { - val convertedPartitionFilters = - HoodieFileIndex.convertFilterForTimestampKeyGenerator(metaClient, partitionFilters) - // Look up candidate files names in the col-stats index, if all of the following conditions are true // - Data-skipping is enabled // - Col-Stats Index is present @@ -144,7 +141,7 @@ case class HoodieFileIndex(spark: SparkSession, Seq(PartitionDirectory(InternalRow.empty, candidateFiles)) } else { // Prune the partition path by the partition filters - val prunedPartitions = prunePartition(cachedAllInputFileSlices.keySet.asScala.toSeq, convertedPartitionFilters) + val prunedPartitions = prunePartition(cachedAllInputFileSlices.keySet.asScala.toSeq, partitionFilters) var totalFileSize = 0 var candidateFileSize = 0 diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestCOWDataSource.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestCOWDataSource.scala index 088ec1faabf73..7c86da0c9e362 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestCOWDataSource.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestCOWDataSource.scala @@ -144,7 +144,7 @@ class TestCOWDataSource extends HoodieClientTestBase { def testPrunePartitionForTimestampBasedKeyGenerator(): Unit = { val options = commonOpts ++ Map( "hoodie.compact.inline" -> "false", - DataSourceWriteOptions.TABLE_TYPE.key -> DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL, + DataSourceWriteOptions.TABLE_TYPE.key -> DataSourceWriteOptions.COW_TABLE_TYPE_OPT_VAL, DataSourceWriteOptions.KEYGENERATOR_CLASS_NAME.key -> "org.apache.hudi.keygen.TimestampBasedKeyGenerator", Config.TIMESTAMP_TYPE_FIELD_PROP -> "DATE_STRING", Config.TIMESTAMP_OUTPUT_DATE_FORMAT_PROP -> "yyyy/MM/dd", @@ -176,8 +176,11 @@ class TestCOWDataSource extends HoodieClientTestBase { // snapshot query val snapshotQueryRes = spark.read.format("hudi").load(basePath) - assertEquals(snapshotQueryRes.where("partition = '2022-01-01'").count, 20) - assertEquals(snapshotQueryRes.where("partition = '2022-01-02'").count, 30) + // TODO(HUDI-3204) we have to revert this to pre-existing behavior from 0.10 + //assertEquals(snapshotQueryRes.where("partition = '2022-01-01'").count, 20) + //assertEquals(snapshotQueryRes.where("partition = '2022-01-02'").count, 30) + assertEquals(snapshotQueryRes.where("partition = '2022/01/01'").count, 20) + assertEquals(snapshotQueryRes.where("partition = '2022/01/02'").count, 30) // incremental query val incrementalQueryRes = spark.read.format("hudi") @@ -961,10 +964,14 @@ class TestCOWDataSource extends HoodieClientTestBase { assert(firstDF.count() == 2) // data_date is the partition field. Persist to the parquet file using the origin values, and read it. - assertEquals( - Seq("2018-09-23", "2018-09-24"), - firstDF.select("data_date").map(_.get(0).toString).collect().sorted.toSeq - ) + // TODO(HUDI-3204) we have to revert this to pre-existing behavior from 0.10 + val expectedValues = if (useGlobbing) { + Seq("2018-09-23", "2018-09-24") + } else { + Seq("2018/09/23", "2018/09/24") + } + + assertEquals(expectedValues, firstDF.select("data_date").map(_.get(0).toString).collect().sorted.toSeq) assertEquals( Seq("2018/09/23", "2018/09/24"), firstDF.select("_hoodie_partition_path").map(_.get(0).toString).collect().sorted.toSeq diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestMORDataSource.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestMORDataSource.scala index 96514603efdcd..f9f14438933f3 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestMORDataSource.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestMORDataSource.scala @@ -17,11 +17,10 @@ package org.apache.hudi.functional -import org.apache.avro.generic.GenericRecord import org.apache.hadoop.fs.Path import org.apache.hudi.DataSourceWriteOptions._ import org.apache.hudi.common.config.HoodieMetadataConfig -import org.apache.hudi.common.model.{DefaultHoodieRecordPayload, HoodieRecord, HoodieRecordPayload, HoodieTableType} +import org.apache.hudi.common.model.{DefaultHoodieRecordPayload, HoodieTableType} import org.apache.hudi.common.table.HoodieTableMetaClient import org.apache.hudi.common.testutils.HoodieTestDataGenerator import org.apache.hudi.common.testutils.RawTripTestPayload.recordsToStrings @@ -30,9 +29,8 @@ import org.apache.hudi.index.HoodieIndex.IndexType import org.apache.hudi.keygen.NonpartitionedKeyGenerator import org.apache.hudi.keygen.constant.KeyGeneratorOptions.Config import org.apache.hudi.testutils.{DataSourceTestUtils, HoodieClientTestBase} -import org.apache.hudi.{AvroConversionUtils, DataSourceReadOptions, DataSourceWriteOptions, HoodieDataSourceHelpers, HoodieSparkUtils, SparkDatasetMixin} +import org.apache.hudi.{DataSourceReadOptions, DataSourceWriteOptions, HoodieDataSourceHelpers, SparkDatasetMixin} import org.apache.log4j.LogManager -import org.apache.spark.rdd.RDD import org.apache.spark.sql._ import org.apache.spark.sql.functions._ import org.apache.spark.sql.types.BooleanType @@ -41,7 +39,6 @@ import org.junit.jupiter.api.{AfterEach, BeforeEach, Test} import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.params.provider.CsvSource -import java.util import scala.collection.JavaConversions._ import scala.collection.JavaConverters._ @@ -864,8 +861,11 @@ class TestMORDataSource extends HoodieClientTestBase with SparkDatasetMixin { val readOptimizedQueryRes = spark.read.format("hudi") .option(DataSourceReadOptions.QUERY_TYPE.key, DataSourceReadOptions.QUERY_TYPE_READ_OPTIMIZED_OPT_VAL) .load(basePath) - assertEquals(readOptimizedQueryRes.where("partition = '2022-01-01'").count, 50) - assertEquals(readOptimizedQueryRes.where("partition = '2022-01-02'").count, 60) + // TODO(HUDI-3204) we have to revert this to pre-existing behavior from 0.10 + //assertEquals(readOptimizedQueryRes.where("partition = '2022-01-01'").count, 50) + //assertEquals(readOptimizedQueryRes.where("partition = '2022-01-02'").count, 60) + assertEquals(readOptimizedQueryRes.where("partition = '2022/01/01'").count, 50) + assertEquals(readOptimizedQueryRes.where("partition = '2022/01/02'").count, 60) // incremental query val incrementalQueryRes = spark.read.format("hudi") From 554caa34218e477bbd5eb9b724836f2c365c896d Mon Sep 17 00:00:00 2001 From: RexAn Date: Fri, 27 May 2022 19:45:09 +0800 Subject: [PATCH 25/44] [MINOR] Fix the issue when handling conf hoodie.datasource.write.operation=bulk_insert in sql mode (#5679) Co-authored-by: Rex An --- .../org/apache/spark/sql/hudi/ProvidesHoodieConfig.scala | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/ProvidesHoodieConfig.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/ProvidesHoodieConfig.scala index 131ebebe85a5a..b35e240426995 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/ProvidesHoodieConfig.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/ProvidesHoodieConfig.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.hudi import org.apache.hudi.DataSourceWriteOptions._ import org.apache.hudi.common.config.TypedProperties -import org.apache.hudi.common.model.OverwriteWithLatestAvroPayload +import org.apache.hudi.common.model.{OverwriteWithLatestAvroPayload, WriteOperationType} import org.apache.hudi.common.table.HoodieTableConfig import org.apache.hudi.config.HoodieWriteConfig.TBL_NAME import org.apache.hudi.config.{HoodieIndexConfig, HoodieWriteConfig} @@ -129,7 +129,8 @@ trait ProvidesHoodieConfig extends Logging { .getOrElse(classOf[ComplexKeyGenerator].getCanonicalName) val enableBulkInsert = parameters.getOrElse(DataSourceWriteOptions.SQL_ENABLE_BULK_INSERT.key, - DataSourceWriteOptions.SQL_ENABLE_BULK_INSERT.defaultValue()).toBoolean + DataSourceWriteOptions.SQL_ENABLE_BULK_INSERT.defaultValue()).toBoolean || + parameters.get(DataSourceWriteOptions.OPERATION.key).exists(_.equalsIgnoreCase(WriteOperationType.BULK_INSERT.value)) val dropDuplicate = sparkSession.conf .getOption(INSERT_DROP_DUPS.key).getOrElse(INSERT_DROP_DUPS.defaultValue).toBoolean From 93fe5a497e8f2f97d90f3dd66d966f9f4cc24bc9 Mon Sep 17 00:00:00 2001 From: Bo Cui Date: Sat, 28 May 2022 08:37:34 +0800 Subject: [PATCH 26/44] [HUDI-4151] flink split_reader supports rocksdb (#5675) * [HUDI-4151] flink split_reader supports rocksdb --- .../apache/hudi/table/format/FormatUtils.java | 35 +++++++++---------- .../format/mor/MergeOnReadInputFormat.java | 8 ++--- 2 files changed, 20 insertions(+), 23 deletions(-) diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/FormatUtils.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/FormatUtils.java index 478f94cb71f73..eb058597f8059 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/FormatUtils.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/FormatUtils.java @@ -30,6 +30,7 @@ import org.apache.hudi.common.util.queue.BoundedInMemoryQueueProducer; import org.apache.hudi.common.util.queue.FunctionBasedQueueProducer; import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.configuration.FlinkOptions; import org.apache.hudi.hadoop.config.HoodieRealtimeConfig; import org.apache.hudi.table.format.mor.MergeOnReadInputSplit; import org.apache.hudi.util.StreamerUtil; @@ -120,38 +121,34 @@ private static Object getVal(IndexedRecord record, int pos) { public static HoodieMergedLogRecordScanner logScanner( MergeOnReadInputSplit split, Schema logSchema, - Configuration config, - boolean withOperationField) { - FileSystem fs = FSUtils.getFs(split.getTablePath(), config); + org.apache.flink.configuration.Configuration flinkConf, + Configuration hadoopConf) { + HoodieWriteConfig writeConfig = StreamerUtil.getHoodieClientConfig(flinkConf); + FileSystem fs = FSUtils.getFs(split.getTablePath(), hadoopConf); return HoodieMergedLogRecordScanner.newBuilder() .withFileSystem(fs) .withBasePath(split.getTablePath()) .withLogFilePaths(split.getLogPaths().get()) .withReaderSchema(logSchema) .withLatestInstantTime(split.getLatestCommit()) - .withReadBlocksLazily( - string2Boolean( - config.get(HoodieRealtimeConfig.COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP, - HoodieRealtimeConfig.DEFAULT_COMPACTION_LAZY_BLOCK_READ_ENABLED))) + .withReadBlocksLazily(writeConfig.getCompactionLazyBlockReadEnabled()) .withReverseReader(false) - .withBufferSize( - config.getInt(HoodieRealtimeConfig.MAX_DFS_STREAM_BUFFER_SIZE_PROP, - HoodieRealtimeConfig.DEFAULT_MAX_DFS_STREAM_BUFFER_SIZE)) + .withBufferSize(writeConfig.getMaxDFSStreamBufferSize()) .withMaxMemorySizeInBytes(split.getMaxCompactionMemoryInBytes()) - .withSpillableMapBasePath( - config.get(HoodieRealtimeConfig.SPILLABLE_MAP_BASE_PATH_PROP, - HoodieRealtimeConfig.DEFAULT_SPILLABLE_MAP_BASE_PATH)) + .withDiskMapType(writeConfig.getCommonConfig().getSpillableDiskMapType()) + .withSpillableMapBasePath(writeConfig.getSpillableMapBasePath()) .withInstantRange(split.getInstantRange()) - .withOperationField(withOperationField) + .withOperationField(flinkConf.getBoolean(FlinkOptions.CHANGELOG_ENABLED)) .build(); } private static HoodieUnMergedLogRecordScanner unMergedLogScanner( MergeOnReadInputSplit split, Schema logSchema, - Configuration config, + org.apache.flink.configuration.Configuration flinkConf, + Configuration hadoopConf, HoodieUnMergedLogRecordScanner.LogRecordScannerCallback callback) { - FileSystem fs = FSUtils.getFs(split.getTablePath(), config); + FileSystem fs = FSUtils.getFs(split.getTablePath(), hadoopConf); return HoodieUnMergedLogRecordScanner.newBuilder() .withFileSystem(fs) .withBasePath(split.getTablePath()) @@ -160,11 +157,11 @@ private static HoodieUnMergedLogRecordScanner unMergedLogScanner( .withLatestInstantTime(split.getLatestCommit()) .withReadBlocksLazily( string2Boolean( - config.get(HoodieRealtimeConfig.COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP, + flinkConf.getString(HoodieRealtimeConfig.COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP, HoodieRealtimeConfig.DEFAULT_COMPACTION_LAZY_BLOCK_READ_ENABLED))) .withReverseReader(false) .withBufferSize( - config.getInt(HoodieRealtimeConfig.MAX_DFS_STREAM_BUFFER_SIZE_PROP, + flinkConf.getInteger(HoodieRealtimeConfig.MAX_DFS_STREAM_BUFFER_SIZE_PROP, HoodieRealtimeConfig.DEFAULT_MAX_DFS_STREAM_BUFFER_SIZE)) .withInstantRange(split.getInstantRange()) .withLogRecordScannerCallback(callback) @@ -198,7 +195,7 @@ public BoundedMemoryRecords( Functions.noop()); // Consumer of this record reader this.iterator = this.executor.getQueue().iterator(); - this.scanner = FormatUtils.unMergedLogScanner(split, logSchema, hadoopConf, + this.scanner = FormatUtils.unMergedLogScanner(split, logSchema, flinkConf, hadoopConf, record -> executor.getQueue().insertRecord(record)); // Start reading and buffering this.executor.startProducers(); 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 4f2de3648ed56..8eaa9d0b886f4 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 @@ -192,6 +192,7 @@ public void open(MergeOnReadInputSplit split) throws IOException { getLogFileIterator(split)); } else if (split.getMergeType().equals(FlinkOptions.REALTIME_PAYLOAD_COMBINE)) { this.iterator = new MergeIterator( + conf, hadoopConf, split, this.tableState.getRowType(), @@ -200,7 +201,6 @@ public void open(MergeOnReadInputSplit split) throws IOException { new Schema.Parser().parse(this.tableState.getRequiredAvroSchema()), this.requiredPos, this.emitDelete, - this.conf.getBoolean(FlinkOptions.CHANGELOG_ENABLED), this.tableState.getOperationPos(), getFullSchemaReader(split.getBasePath().get())); } else { @@ -323,7 +323,7 @@ private ClosableIterator getLogFileIterator(MergeOnReadInputSplit split final GenericRecordBuilder recordBuilder = new GenericRecordBuilder(requiredSchema); final AvroToRowDataConverters.AvroToRowDataConverter avroToRowDataConverter = AvroToRowDataConverters.createRowConverter(tableState.getRequiredRowType()); - final HoodieMergedLogRecordScanner scanner = FormatUtils.logScanner(split, tableSchema, hadoopConf, conf.getBoolean(FlinkOptions.CHANGELOG_ENABLED)); + final HoodieMergedLogRecordScanner scanner = FormatUtils.logScanner(split, tableSchema, conf, hadoopConf); final Iterator logRecordsKeyIterator = scanner.getRecords().keySet().iterator(); final int[] pkOffset = tableState.getPkOffsetsInRequired(); // flag saying whether the pk semantics has been dropped by user specified @@ -639,6 +639,7 @@ static class MergeIterator implements RecordIterator { private RowData currentRecord; MergeIterator( + Configuration finkConf, org.apache.hadoop.conf.Configuration hadoopConf, MergeOnReadInputSplit split, RowType tableRowType, @@ -647,12 +648,11 @@ static class MergeIterator implements RecordIterator { Schema requiredSchema, int[] requiredPos, boolean emitDelete, - boolean withOperationField, int operationPos, ParquetColumnarRowSplitReader reader) { // the reader should be with full schema this.tableSchema = tableSchema; this.reader = reader; - this.scanner = FormatUtils.logScanner(split, tableSchema, hadoopConf, withOperationField); + this.scanner = FormatUtils.logScanner(split, tableSchema, finkConf, hadoopConf); this.logKeysIterator = scanner.getRecords().keySet().iterator(); this.requiredSchema = requiredSchema; this.requiredPos = requiredPos; From 58014c147af677a426e0dd22fb59b8d92d0dd18a Mon Sep 17 00:00:00 2001 From: wangxianghu Date: Sat, 28 May 2022 11:13:24 +0400 Subject: [PATCH 27/44] [HUDI-4160] Make database regex of MaxwellJsonKafkaSourcePostProcessor optional (#5697) --- .../MaxwellJsonKafkaSourcePostProcessor.java | 21 ++++++++++--------- .../TestJsonKafkaSourcePostProcessor.java | 14 +++++++++++++ 2 files changed, 25 insertions(+), 10 deletions(-) diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/processor/maxwell/MaxwellJsonKafkaSourcePostProcessor.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/processor/maxwell/MaxwellJsonKafkaSourcePostProcessor.java index 42e2556ea59fa..e008c04e33d1c 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/processor/maxwell/MaxwellJsonKafkaSourcePostProcessor.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/processor/maxwell/MaxwellJsonKafkaSourcePostProcessor.java @@ -22,6 +22,7 @@ import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.util.DateTimeUtils; +import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.utilities.exception.HoodieSourcePostProcessException; import org.apache.hudi.utilities.sources.processor.JsonKafkaSourcePostProcessor; @@ -29,8 +30,6 @@ import com.fasterxml.jackson.databind.JsonNode; import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.node.ObjectNode; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; import org.apache.spark.api.java.JavaRDD; import java.util.Locale; @@ -49,12 +48,15 @@ */ public class MaxwellJsonKafkaSourcePostProcessor extends JsonKafkaSourcePostProcessor { - private static final Logger LOG = LogManager.getLogger(MaxwellJsonKafkaSourcePostProcessor.class); - private static final ObjectMapper MAPPER = new ObjectMapper(); + private final Option databaseRegex; + private final String tableRegex; + public MaxwellJsonKafkaSourcePostProcessor(TypedProperties props) { super(props); + databaseRegex = Option.ofNullable(props.getString(Config.DATABASE_NAME_REGEX_PROP.key(), null)); + tableRegex = props.getString(Config.TABLE_NAME_REGEX_PROP.key()); } // ------------------------------------------------------------------------ @@ -111,9 +113,6 @@ public JavaRDD process(JavaRDD maxwellJsonRecords) { // filter out target databases and tables if (isTargetTable(database, table)) { - - LOG.info(String.format("Maxwell source processor starts process table : %s.%s", database, table)); - ObjectNode result = (ObjectNode) inputJson.get(DATA); String type = inputJson.get(OPERATION_TYPE).textValue(); @@ -182,9 +181,11 @@ private String processDelete(JsonNode inputJson, ObjectNode result) { * @param table table the data belong to */ private boolean isTargetTable(String database, String table) { - String databaseRegex = this.props.getString(Config.DATABASE_NAME_REGEX_PROP.key()); - String tableRegex = this.props.getString(Config.TABLE_NAME_REGEX_PROP.key()); - return Pattern.matches(databaseRegex, database) && Pattern.matches(tableRegex, table); + if (!databaseRegex.isPresent()) { + return Pattern.matches(tableRegex, table); + } else { + return Pattern.matches(databaseRegex.get(), database) && Pattern.matches(tableRegex, table); + } } } diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/TestJsonKafkaSourcePostProcessor.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/TestJsonKafkaSourcePostProcessor.java index 80ac2f921ecd5..1d80b68449c79 100644 --- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/TestJsonKafkaSourcePostProcessor.java +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/TestJsonKafkaSourcePostProcessor.java @@ -178,6 +178,12 @@ public void testMaxwellJsonKafkaSourcePostProcessor() throws IOException { + "\"name\":\"andy\",\"age\":17,\"insert_time\":\"2022-03-12 08:31:56\"," + "\"update_time\":\"2022-03-12 08:31:56\"}}"; + // database hudi_02, table hudi_maxwell_01, insert + String hudi02Maxwell01Insert = "{\"database\":\"hudi_02\",\"table\":\"hudi_maxwell_01\",\"type\":\"insert\"," + + "\"ts\":1647073916,\"xid\":4990,\"commit\":true,\"data\":{\"id\":\"9bb17f316ee8488cb107621ddf0f3cb0\"," + + "\"name\":\"andy\",\"age\":17,\"insert_time\":\"2022-03-12 08:31:56\"," + + "\"update_time\":\"2022-03-12 08:31:56\"}}"; + // ------------------------------------------------------------------------ // Tests // ------------------------------------------------------------------------ @@ -248,6 +254,14 @@ public void testMaxwellJsonKafkaSourcePostProcessor() throws IOException { // ddl data will be ignored, ths count should be 0 long ddlDataNum = processor.process(ddlData).count(); assertEquals(0, ddlDataNum); + + // test table regex without database regex + props.remove(MaxwellJsonKafkaSourcePostProcessor.Config.DATABASE_NAME_REGEX_PROP.key()); + props.setProperty(MaxwellJsonKafkaSourcePostProcessor.Config.TABLE_NAME_REGEX_PROP.key(), "hudi_maxwell(_)?[0-9]{0,2}"); + + JavaRDD dataWithoutDatabaseRegex = jsc().parallelize(Arrays.asList(hudiMaxwell01Insert, hudi02Maxwell01Insert)); + long countWithoutDatabaseRegex = processor.process(dataWithoutDatabaseRegex).count(); + assertEquals(2, countWithoutDatabaseRegex); } /** From 8fa8f2603199638e2467fba0db8948a84455213e Mon Sep 17 00:00:00 2001 From: ForwardXu Date: Sat, 28 May 2022 22:56:39 +0800 Subject: [PATCH 28/44] [MINOR] Fix Hive and meta sync config for sql statement (#5316) --- .../org/apache/spark/sql/hudi/ProvidesHoodieConfig.scala | 5 +++++ .../spark/sql/hudi/command/MergeIntoHoodieTableCommand.scala | 1 + 2 files changed, 6 insertions(+) diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/ProvidesHoodieConfig.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/ProvidesHoodieConfig.scala index b35e240426995..eca73be0bb39d 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/ProvidesHoodieConfig.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/ProvidesHoodieConfig.scala @@ -76,6 +76,7 @@ trait ProvidesHoodieConfig extends Logging { OPERATION.key -> UPSERT_OPERATION_OPT_VAL, PARTITIONPATH_FIELD.key -> tableConfig.getPartitionFieldProp, HoodieSyncConfig.META_SYNC_ENABLED.key -> enableHive.toString, + HiveSyncConfig.HIVE_SYNC_ENABLED.key -> enableHive.toString, HiveSyncConfig.HIVE_SYNC_MODE.key -> hiveSyncConfig.syncMode, HoodieSyncConfig.META_SYNC_DATABASE_NAME.key -> hiveSyncConfig.databaseName, HoodieSyncConfig.META_SYNC_TABLE_NAME.key -> hiveSyncConfig.tableName, @@ -193,6 +194,7 @@ trait ProvidesHoodieConfig extends Logging { HoodieWriteConfig.COMBINE_BEFORE_INSERT.key -> String.valueOf(hasPrecombineColumn), HoodieSyncConfig.META_SYNC_PARTITION_FIELDS.key -> partitionFieldsStr, HoodieSyncConfig.META_SYNC_ENABLED.key -> enableHive.toString, + HiveSyncConfig.HIVE_SYNC_ENABLED.key -> enableHive.toString, HiveSyncConfig.HIVE_SYNC_MODE.key -> hiveSyncConfig.syncMode, HoodieSyncConfig.META_SYNC_DATABASE_NAME.key -> hiveSyncConfig.databaseName, HoodieSyncConfig.META_SYNC_TABLE_NAME.key -> hiveSyncConfig.tableName, @@ -257,6 +259,7 @@ trait ProvidesHoodieConfig extends Logging { val hiveSyncConfig = buildHiveSyncConfig(hoodieProps, hoodieCatalogTable) val options = hoodieCatalogTable.catalogProperties + val enableHive = isEnableHive(sparkSession) withSparkConf(sparkSession, options) { Map( @@ -269,6 +272,8 @@ trait ProvidesHoodieConfig extends Logging { SqlKeyGenerator.ORIGIN_KEYGEN_CLASS_NAME -> tableConfig.getKeyGeneratorClassName, OPERATION.key -> DataSourceWriteOptions.DELETE_OPERATION_OPT_VAL, PARTITIONPATH_FIELD.key -> tableConfig.getPartitionFieldProp, + HoodieSyncConfig.META_SYNC_ENABLED.key -> enableHive.toString, + HiveSyncConfig.HIVE_SYNC_ENABLED.key -> enableHive.toString, HiveSyncConfig.HIVE_SYNC_MODE.key -> hiveSyncConfig.syncMode, HiveSyncConfig.HIVE_SUPPORT_TIMESTAMP_TYPE.key -> hiveSyncConfig.supportTimestamp.toString, HoodieWriteConfig.DELETE_PARALLELISM_VALUE.key -> hoodieProps.getString(HoodieWriteConfig.DELETE_PARALLELISM_VALUE.key, "200"), diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/MergeIntoHoodieTableCommand.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/MergeIntoHoodieTableCommand.scala index 1376445bda966..f7c62adc6578e 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/MergeIntoHoodieTableCommand.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/MergeIntoHoodieTableCommand.scala @@ -467,6 +467,7 @@ case class MergeIntoHoodieTableCommand(mergeInto: MergeIntoTable) extends Hoodie KEYGENERATOR_CLASS_NAME.key -> classOf[SqlKeyGenerator].getCanonicalName, SqlKeyGenerator.ORIGIN_KEYGEN_CLASS_NAME -> tableConfig.getKeyGeneratorClassName, HoodieSyncConfig.META_SYNC_ENABLED.key -> enableHive.toString, + HiveSyncConfig.HIVE_SYNC_ENABLED.key -> enableHive.toString, HiveSyncConfig.HIVE_SYNC_MODE.key -> hiveSyncConfig.syncMode, HoodieSyncConfig.META_SYNC_DATABASE_NAME.key -> targetTableDb, HoodieSyncConfig.META_SYNC_TABLE_NAME.key -> targetTableName, From 48062a570817ba776f23ff95963d540fddbe1eac Mon Sep 17 00:00:00 2001 From: uday08bce Date: Sat, 28 May 2022 17:20:52 +0200 Subject: [PATCH 29/44] [HUDI-4166] Added SimpleClient plugin for integ test (#5710) --- packaging/hudi-integ-test-bundle/pom.xml | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/packaging/hudi-integ-test-bundle/pom.xml b/packaging/hudi-integ-test-bundle/pom.xml index ce18681fc2d81..ad435967a4057 100644 --- a/packaging/hudi-integ-test-bundle/pom.xml +++ b/packaging/hudi-integ-test-bundle/pom.xml @@ -174,6 +174,12 @@ org.apache.curator:curator-framework org.apache.curator:curator-client org.apache.curator:curator-recipes + + io.prometheus:simpleclient + io.prometheus:simpleclient_httpserver + io.prometheus:simpleclient_dropwizard + io.prometheus:simpleclient_pushgateway + io.prometheus:simpleclient_common From 62d792368bd1ebf2dead648b436e623e9a1a003e Mon Sep 17 00:00:00 2001 From: Carter Shanklin Date: Sat, 28 May 2022 08:26:14 -0700 Subject: [PATCH 30/44] [HUDI-3551] Add the Oracle Cloud Infrastructure (oci) Object Storage URI scheme (#4952) --- .../main/java/org/apache/hudi/common/fs/StorageSchemes.java | 4 +++- .../java/org/apache/hudi/common/fs/TestStorageSchemes.java | 1 + 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/hudi-common/src/main/java/org/apache/hudi/common/fs/StorageSchemes.java b/hudi-common/src/main/java/org/apache/hudi/common/fs/StorageSchemes.java index 5dcd0b156aa32..10619f8b3afaf 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/fs/StorageSchemes.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/fs/StorageSchemes.java @@ -67,7 +67,9 @@ public enum StorageSchemes { // JuiceFileSystem JFS("jfs", true), // Baidu Object Storage - BOS("bos", false); + BOS("bos", false), + // Oracle Cloud Infrastructure Object Storage + OCI("oci", false); private String scheme; private boolean supportsAppend; diff --git a/hudi-common/src/test/java/org/apache/hudi/common/fs/TestStorageSchemes.java b/hudi-common/src/test/java/org/apache/hudi/common/fs/TestStorageSchemes.java index 186ac62d372b7..80e64f8d77f47 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/fs/TestStorageSchemes.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/fs/TestStorageSchemes.java @@ -49,6 +49,7 @@ public void testStorageSchemes() { assertFalse(StorageSchemes.isAppendSupported("bos")); assertFalse(StorageSchemes.isAppendSupported("ks3")); assertTrue(StorageSchemes.isAppendSupported("ofs")); + assertTrue(StorageSchemes.isAppendSupported("oci")); assertThrows(IllegalArgumentException.class, () -> { StorageSchemes.isAppendSupported("s2"); }, "Should throw exception for unsupported schemes"); From 0a72458291ace85faec5155a2d7865b95f420b9b Mon Sep 17 00:00:00 2001 From: Raymond Xu <2701446+xushiyan@users.noreply.github.com> Date: Sat, 28 May 2022 12:13:37 -0700 Subject: [PATCH 31/44] [HUDI-3551] Fix testStorageSchemes for oci storage (#5711) --- .../test/java/org/apache/hudi/common/fs/TestStorageSchemes.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/hudi-common/src/test/java/org/apache/hudi/common/fs/TestStorageSchemes.java b/hudi-common/src/test/java/org/apache/hudi/common/fs/TestStorageSchemes.java index 80e64f8d77f47..85f3ce65ec277 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/fs/TestStorageSchemes.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/fs/TestStorageSchemes.java @@ -49,7 +49,7 @@ public void testStorageSchemes() { assertFalse(StorageSchemes.isAppendSupported("bos")); assertFalse(StorageSchemes.isAppendSupported("ks3")); assertTrue(StorageSchemes.isAppendSupported("ofs")); - assertTrue(StorageSchemes.isAppendSupported("oci")); + assertFalse(StorageSchemes.isAppendSupported("oci")); assertThrows(IllegalArgumentException.class, () -> { StorageSchemes.isAppendSupported("s2"); }, "Should throw exception for unsupported schemes"); From 7e86884604057c0a7a373d4e84d8f3e112170ca7 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E8=8B=8F=E6=89=BF=E7=A5=A5?= Date: Sun, 29 May 2022 13:35:47 +0800 Subject: [PATCH 32/44] [HUDI-4086] Use CustomizedThreadFactory in async compaction and clustering (#5563) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Co-authored-by: 苏承祥 --- .../hudi/async/AsyncClusteringService.java | 13 ++++--------- .../apache/hudi/async/AsyncCompactService.java | 17 ++++++----------- 2 files changed, 10 insertions(+), 20 deletions(-) diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/async/AsyncClusteringService.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/async/AsyncClusteringService.java index 7fece5c885f8a..1e4d4d1f593af 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/async/AsyncClusteringService.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/async/AsyncClusteringService.java @@ -24,6 +24,7 @@ import org.apache.hudi.common.engine.EngineProperty; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.util.CustomizedThreadFactory; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.exception.HoodieIOException; @@ -42,13 +43,12 @@ */ public abstract class AsyncClusteringService extends HoodieAsyncTableService { + public static final String CLUSTERING_POOL_NAME = "hoodiecluster"; private static final long serialVersionUID = 1L; private static final Logger LOG = LogManager.getLogger(AsyncClusteringService.class); - public static final String CLUSTERING_POOL_NAME = "hoodiecluster"; - private final int maxConcurrentClustering; - private transient BaseClusterer clusteringClient; protected transient HoodieEngineContext context; + private transient BaseClusterer clusteringClient; public AsyncClusteringService(HoodieEngineContext context, BaseHoodieWriteClient writeClient) { this(context, writeClient, false); @@ -69,12 +69,7 @@ public AsyncClusteringService(HoodieEngineContext context, BaseHoodieWriteClient @Override protected Pair startService() { ExecutorService executor = Executors.newFixedThreadPool(maxConcurrentClustering, - r -> { - Thread t = new Thread(r, "async_clustering_thread"); - t.setDaemon(isRunInDaemonMode()); - return t; - }); - + new CustomizedThreadFactory("async_clustering_thread", isRunInDaemonMode())); return Pair.of(CompletableFuture.allOf(IntStream.range(0, maxConcurrentClustering).mapToObj(i -> CompletableFuture.supplyAsync(() -> { try { // Set Compactor Pool Name for allowing users to prioritize compaction diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/async/AsyncCompactService.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/async/AsyncCompactService.java index f1f7f416e466c..a62beae02bbdb 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/async/AsyncCompactService.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/async/AsyncCompactService.java @@ -22,6 +22,7 @@ import org.apache.hudi.common.engine.EngineProperty; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.util.CustomizedThreadFactory; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.exception.HoodieIOException; @@ -39,17 +40,15 @@ */ public abstract class AsyncCompactService extends HoodieAsyncTableService { - private static final long serialVersionUID = 1L; - private static final Logger LOG = LogManager.getLogger(AsyncCompactService.class); - /** * This is the job pool used by async compaction. */ public static final String COMPACT_POOL_NAME = "hoodiecompact"; - + private static final long serialVersionUID = 1L; + private static final Logger LOG = LogManager.getLogger(AsyncCompactService.class); private final int maxConcurrentCompaction; - private transient BaseCompactor compactor; protected transient HoodieEngineContext context; + private transient BaseCompactor compactor; public AsyncCompactService(HoodieEngineContext context, BaseHoodieWriteClient client) { this(context, client, false); @@ -70,11 +69,7 @@ public AsyncCompactService(HoodieEngineContext context, BaseHoodieWriteClient cl @Override protected Pair startService() { ExecutorService executor = Executors.newFixedThreadPool(maxConcurrentCompaction, - r -> { - Thread t = new Thread(r, "async_compact_thread"); - t.setDaemon(isRunInDaemonMode()); - return t; - }); + new CustomizedThreadFactory("async_compact_thread", isRunInDaemonMode())); return Pair.of(CompletableFuture.allOf(IntStream.range(0, maxConcurrentCompaction).mapToObj(i -> CompletableFuture.supplyAsync(() -> { try { // Set Compactor Pool Name for allowing users to prioritize compaction @@ -107,9 +102,9 @@ protected Pair startService() { }, executor)).toArray(CompletableFuture[]::new)), executor); } - /** * Check whether compactor thread needs to be stopped. + * * @return */ protected boolean shouldStopCompactor() { From 329da34ee0f8d935fa7ec1853104addfb5b45400 Mon Sep 17 00:00:00 2001 From: Danny Chan Date: Mon, 30 May 2022 13:08:02 +0800 Subject: [PATCH 33/44] [HUDI-4163] Catch general exception instead of IOException while fetching rollback plan during rollback (#5703) If the avro file is corrupted, an InvalidAvroMagicException throws. --- .../main/java/org/apache/hudi/client/BaseHoodieWriteClient.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieWriteClient.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieWriteClient.java index 251ff97799ffa..5c485bed0581d 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieWriteClient.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieWriteClient.java @@ -1118,7 +1118,7 @@ protected Map> getPendingRollbackInfos HoodieRollbackPlan rollbackPlan; try { rollbackPlan = RollbackUtils.getRollbackPlan(metaClient, rollbackInstant); - } catch (IOException e) { + } catch (Exception e) { if (rollbackInstant.isRequested()) { LOG.warn("Fetching rollback plan failed for " + rollbackInstant + ", deleting the plan since it's in REQUESTED state", e); try { From 918c4f4e0ba5dd30fc733cc3e208650e2154dd5c Mon Sep 17 00:00:00 2001 From: Jin Xing Date: Mon, 30 May 2022 19:09:26 +0800 Subject: [PATCH 34/44] [HUDI-4149] Drop-Table fails when underlying table directory is broken (#5672) --- .../catalyst/catalog/HoodieCatalogTable.scala | 73 +++++++----- .../hudi/command/DropHoodieTableCommand.scala | 46 ++++---- .../src/test/resources/sql-statements.sql | 2 + .../apache/spark/sql/hudi/TestDropTable.scala | 111 ++++++++++++++++++ 4 files changed, 182 insertions(+), 50 deletions(-) diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/catalyst/catalog/HoodieCatalogTable.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/catalyst/catalog/HoodieCatalogTable.scala index 76cea362a3b53..3dbb358fbb05e 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/catalyst/catalog/HoodieCatalogTable.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/catalyst/catalog/HoodieCatalogTable.scala @@ -39,9 +39,12 @@ import scala.collection.JavaConverters._ import scala.collection.mutable /** - * A wrapper of hoodie CatalogTable instance and hoodie Table. + * Table definition for SQL funcitonalities. Depending on the way of data generation, + * meta of Hudi table can be from Spark catalog or meta directory on filesystem. + * [[HoodieCatalogTable]] takes both meta sources into consideration when handling + * EXTERNAL and MANAGED tables. */ -class HoodieCatalogTable(val spark: SparkSession, val table: CatalogTable) extends Logging { +class HoodieCatalogTable(val spark: SparkSession, var table: CatalogTable) extends Logging { assert(table.provider.map(_.toLowerCase(Locale.ROOT)).orNull == "hudi", "It's not a Hudi table") @@ -117,23 +120,9 @@ class HoodieCatalogTable(val spark: SparkSession, val table: CatalogTable) exten lazy val baseFileFormat: String = metaClient.getTableConfig.getBaseFileFormat.name() /** - * The schema of table. - * Make StructField nullable and fill the comments in. + * Table schema */ - lazy val tableSchema: StructType = { - val resolver = spark.sessionState.conf.resolver - val originSchema = getTableSqlSchema(metaClient, includeMetadataFields = true).getOrElse(table.schema) - val fields = originSchema.fields.map { f => - val nullableField: StructField = f.copy(nullable = true) - val catalogField = findColumnByName(table.schema, nullableField.name, resolver) - if (catalogField.isDefined) { - catalogField.get.getComment().map(nullableField.withComment).getOrElse(nullableField) - } else { - nullableField - } - } - StructType(fields) - } + lazy val tableSchema: StructType = table.schema /** * The schema without hoodie meta fields @@ -168,12 +157,14 @@ class HoodieCatalogTable(val spark: SparkSession, val table: CatalogTable) exten def isPartitionedTable: Boolean = table.partitionColumnNames.nonEmpty /** - * init hoodie table for create table (as select) + * Initializes table meta on filesystem when applying CREATE TABLE clause. */ def initHoodieTable(): Unit = { logInfo(s"Init hoodie.properties for ${table.identifier.unquotedString}") val (finalSchema, tableConfigs) = parseSchemaAndConfigs() + table = table.copy(schema = finalSchema) + // Save all the table config to the hoodie.properties. val properties = new Properties() properties.putAll(tableConfigs.asJava) @@ -199,7 +190,10 @@ class HoodieCatalogTable(val spark: SparkSession, val table: CatalogTable) exten } /** - * @return schema, table parameters in which all parameters aren't sql-styled. + * Derives the SQL schema and configurations for a Hudi table: + * 1. Columns in the schema fall under two categories -- the data columns described in + * CREATE TABLE clause and meta columns enumerated in [[HoodieRecord#HOODIE_META_COLUMNS]]; + * 2. Configurations derived come from config file, PROPERTIES and OPTIONS in CREATE TABLE clause. */ private def parseSchemaAndConfigs(): (StructType, Map[String, String]) = { val globalProps = DFSPropertiesConfiguration.getGlobalProps.asScala.toMap @@ -216,24 +210,25 @@ class HoodieCatalogTable(val spark: SparkSession, val table: CatalogTable) exten val catalogTableProps = HoodieOptionConfig.mappingSqlOptionToTableConfig(catalogProperties) validateTableConfig(spark, catalogTableProps, convertMapToHoodieConfig(existingTableConfig)) - val options = extraTableConfig(spark, hoodieTableExists, currentTableConfig) ++ + val options = extraTableConfig(hoodieTableExists, currentTableConfig) ++ HoodieOptionConfig.mappingSqlOptionToTableConfig(sqlOptions) ++ currentTableConfig - ValidationUtils.checkArgument(tableSchema.nonEmpty || table.schema.nonEmpty, - s"Missing schema for Create Table: $catalogTableName") - val schema = if (tableSchema.nonEmpty) { - tableSchema - } else { + val schemaFromMetaOpt = loadTableSchemaByMetaClient() + val schema = if (schemaFromMetaOpt.nonEmpty) { + schemaFromMetaOpt.get + } else if (table.schema.nonEmpty) { addMetaFields(table.schema) + } else { + throw new AnalysisException( + s"Missing schema fields when applying CREATE TABLE clause for ${catalogTableName}") } - (schema, options) case (_, false) => ValidationUtils.checkArgument(table.schema.nonEmpty, s"Missing schema for Create Table: $catalogTableName") val schema = table.schema - val options = extraTableConfig(spark, isTableExists = false, globalTableConfigs) ++ + val options = extraTableConfig(tableExists = false, globalTableConfigs) ++ HoodieOptionConfig.mappingSqlOptionToTableConfig(sqlOptions) (addMetaFields(schema), options) @@ -253,10 +248,10 @@ class HoodieCatalogTable(val spark: SparkSession, val table: CatalogTable) exten (finalSchema, tableConfigs) } - private def extraTableConfig(sparkSession: SparkSession, isTableExists: Boolean, + private def extraTableConfig(tableExists: Boolean, originTableConfig: Map[String, String] = Map.empty): Map[String, String] = { val extraConfig = mutable.Map.empty[String, String] - if (isTableExists) { + if (tableExists) { val allPartitionPaths = getPartitionPaths if (originTableConfig.contains(HoodieTableConfig.HIVE_STYLE_PARTITIONING_ENABLE.key)) { extraConfig(HoodieTableConfig.HIVE_STYLE_PARTITIONING_ENABLE.key) = @@ -287,6 +282,24 @@ class HoodieCatalogTable(val spark: SparkSession, val table: CatalogTable) exten extraConfig.toMap } + private def loadTableSchemaByMetaClient(): Option[StructType] = { + val resolver = spark.sessionState.conf.resolver + getTableSqlSchema(metaClient, includeMetadataFields = true).map(originSchema => { + // Load table schema from meta on filesystem, and fill in 'comment' + // information from Spark catalog. + val fields = originSchema.fields.map { f => + val nullableField: StructField = f.copy(nullable = true) + val catalogField = findColumnByName(table.schema, nullableField.name, resolver) + if (catalogField.isDefined) { + catalogField.get.getComment().map(nullableField.withComment).getOrElse(nullableField) + } else { + nullableField + } + } + StructType(fields) + }) + } + // This code is forked from org.apache.spark.sql.hive.HiveExternalCatalog#verifyDataSchema private def verifyDataSchema(tableIdentifier: TableIdentifier, tableType: CatalogTableType, dataSchema: Seq[StructField]): Unit = { diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/DropHoodieTableCommand.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/DropHoodieTableCommand.scala index c24d0fd992d97..a0252861dbf63 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/DropHoodieTableCommand.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/DropHoodieTableCommand.scala @@ -23,39 +23,44 @@ import org.apache.hudi.common.fs.FSUtils import org.apache.hudi.common.model.HoodieTableType import org.apache.hudi.sync.common.util.ConfigUtils import org.apache.spark.sql._ -import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.catalyst.{QualifiedTableName, TableIdentifier} import org.apache.spark.sql.catalyst.catalog._ -import scala.util.control.NonFatal - +/** + * Physical plan node for dropping a table. + */ case class DropHoodieTableCommand( tableIdentifier: TableIdentifier, ifExists: Boolean, isView: Boolean, - purge: Boolean) -extends HoodieLeafRunnableCommand { + purge: Boolean) extends HoodieLeafRunnableCommand { - val MOR_SNAPSHOT_TABLE_SUFFIX = "_rt" - val MOR_READ_OPTIMIZED_TABLE_SUFFIX = "_ro" + private val MOR_SNAPSHOT_TABLE_SUFFIX = "_rt" + private val MOR_READ_OPTIMIZED_TABLE_SUFFIX = "_ro" override def run(sparkSession: SparkSession): Seq[Row] = { - val fullTableName = s"${tableIdentifier.database}.${tableIdentifier.table}" - logInfo(s"start execute drop table command for $fullTableName") - sparkSession.catalog.refreshTable(tableIdentifier.unquotedString) - - try { - // drop catalog table for this hoodie table - dropTableInCatalog(sparkSession, tableIdentifier, ifExists, purge) - } catch { - case NonFatal(e) => - logWarning(s"Failed to drop catalog table in metastore: ${e.getMessage}") + logInfo(s"Start executing 'DROP TABLE' on ${tableIdentifier.unquotedString}" + + s" (ifExists=${ifExists}, purge=${purge}).") + if (!sparkSession.catalog.tableExists(tableIdentifier.unquotedString)) { + sparkSession.catalog.refreshTable(tableIdentifier.unquotedString) } + val qualifiedTableName = QualifiedTableName( + tableIdentifier.database.getOrElse(sparkSession.sessionState.catalog.getCurrentDatabase), + tableIdentifier.table) + sparkSession.sessionState.catalog.invalidateCachedTable(qualifiedTableName) + + dropTableInCatalog(sparkSession, tableIdentifier, ifExists, purge) - logInfo(s"Finish execute drop table command for $fullTableName") + logInfo(s"Finished executing 'DROP TABLE' on ${tableIdentifier.unquotedString}.") Seq.empty[Row] } - def dropTableInCatalog(sparkSession: SparkSession, + /** + * Drops table in Spark catalog. Note that RO & RT table could coexist with a MOR table. + * If `purge` enabled, RO & RT table and corresponding data directory on filesystem will + * all be removed. + */ + private def dropTableInCatalog(sparkSession: SparkSession, tableIdentifier: TableIdentifier, ifExists: Boolean, purge: Boolean): Unit = { @@ -67,7 +72,8 @@ extends HoodieLeafRunnableCommand { val catalog = sparkSession.sessionState.catalog // Drop table in the catalog - if (HoodieTableType.MERGE_ON_READ == hoodieCatalogTable.tableType && purge) { + if (hoodieCatalogTable.hoodieTableExists && + HoodieTableType.MERGE_ON_READ == hoodieCatalogTable.tableType && purge) { val (rtTableOpt, roTableOpt) = getTableRTAndRO(catalog, hoodieCatalogTable) rtTableOpt.foreach(table => catalog.dropTable(table.identifier, true, false)) roTableOpt.foreach(table => catalog.dropTable(table.identifier, true, false)) diff --git a/hudi-spark-datasource/hudi-spark/src/test/resources/sql-statements.sql b/hudi-spark-datasource/hudi-spark/src/test/resources/sql-statements.sql index 3e92d31e3a3b4..449ba2e2e67b0 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/resources/sql-statements.sql +++ b/hudi-spark-datasource/hudi-spark/src/test/resources/sql-statements.sql @@ -34,6 +34,7 @@ set hoodie.delete.shuffle.parallelism = 1; # CTAS create table h0 using hudi options(type = '${tableType}', primaryKey = 'id') +location '${tmpDir}/h0' as select 1 as id, 'a1' as name, 10 as price; +----------+ | ok | @@ -46,6 +47,7 @@ select id, name, price from h0; create table h0_p using hudi partitioned by(dt) options(type = '${tableType}', primaryKey = 'id') +location '${tmpDir}/h0_p' as select cast('2021-05-07 00:00:00' as timestamp) as dt, 1 as id, 'a1' as name, 10 as price; +----------+ diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestDropTable.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestDropTable.scala index 174835cbac0bf..1beb78e27e7f7 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestDropTable.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestDropTable.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.hudi +import org.apache.hadoop.fs.{LocalFileSystem, Path} +import org.apache.hudi.common.fs.FSUtils import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.catalog.SessionCatalog @@ -230,6 +232,115 @@ class TestDropTable extends HoodieSparkSqlTestBase { } } + test("Drop an EXTERNAL table which path is lost.") { + withTempDir { tmp => + val tableName = generateTableName + val tablePath = s"${tmp.getCanonicalPath}/$tableName" + val filesystem = FSUtils.getFs(tablePath, spark.sparkContext.hadoopConfiguration); + spark.sql( + s""" + |create table $tableName ( + |id int, + |ts int, + |value string + |)using hudi + | location '$tablePath' + | tblproperties ( + | primaryKey = 'id', + | preCombineField = 'ts' + | ) + |""".stripMargin) + + assert(filesystem.exists(new Path(tablePath)), s"Table path doesn't exists (${tablePath}).") + + filesystem.delete(new Path(tablePath), true) + spark.sql(s"drop table ${tableName}") + checkAnswer("show tables")() + } + } + + test("Drop an MOR table and related RT & RO when path is lost.") { + withTempDir { tmp => + val tableName = generateTableName + val tablePath = s"${tmp.getCanonicalPath}/$tableName" + val filesystem = FSUtils.getFs(tablePath, spark.sparkContext.hadoopConfiguration); + spark.sql( + s""" + |create table $tableName ( + |id int, + |ts int, + |value string + |)using hudi + | location '$tablePath' + | tblproperties ( + | primaryKey = 'id', + | preCombineField = 'ts', + | type = 'mor' + | ) + |""".stripMargin) + assert(filesystem.exists(new Path(tablePath)), s"Table path doesn't exist (${tablePath}).") + + spark.sql( + s""" + |create table ${tableName}_ro using hudi + | location '${tmp.getCanonicalPath}/$tableName' + | tblproperties ( + | type = 'mor', + | primaryKey = 'id', + | preCombineField = 'ts' + | ) + """.stripMargin) + alterSerdeProperties(spark.sessionState.catalog, TableIdentifier(s"${tableName}_ro"), + Map("hoodie.query.as.ro.table" -> "true")) + + spark.sql( + s""" + |create table ${tableName}_rt using hudi + | location '${tmp.getCanonicalPath}/$tableName' + | tblproperties ( + | type = 'mor', + | primaryKey = 'id', + | preCombineField = 'ts' + | ) + """.stripMargin) + alterSerdeProperties(spark.sessionState.catalog, TableIdentifier(s"${tableName}_rt"), + Map("hoodie.query.as.ro.table" -> "false")) + + filesystem.delete(new Path(tablePath), true) + spark.sql(s"drop table ${tableName}") + spark.sql(s"drop table ${tableName}_ro") + spark.sql(s"drop table ${tableName}_rt") + checkAnswer("show tables")() + } + } + + + test("Drop an MANAGED table which path is lost.") { + val tableName = generateTableName + spark.sql( + s""" + |create table $tableName ( + |id int, + |ts int, + |value string + |)using hudi + | tblproperties ( + | primaryKey = 'id', + | preCombineField = 'ts' + | ) + |""".stripMargin) + + val tablePath = new Path( + spark.sessionState.catalog.getTableMetadata(TableIdentifier(tableName)).location) + + val filesystem = FSUtils.getFs(tablePath, spark.sparkContext.hadoopConfiguration); + assert(filesystem.exists(tablePath), s"Table path doesn't exists ($tablePath).") + + filesystem.delete(tablePath, true) + spark.sql(s"drop table ${tableName}") + checkAnswer("show tables")() + } + private def alterSerdeProperties(sessionCatalog: SessionCatalog, tableIdt: TableIdentifier, newProperties: Map[String, String]): Unit = { val catalogTable = spark.sessionState.catalog.getTableMetadata(tableIdt) From 795a99ba73183b5f7697bb3964843a43a64e74e9 Mon Sep 17 00:00:00 2001 From: Kumud Kumar Srivatsava Tirupati Date: Tue, 31 May 2022 20:27:50 +0530 Subject: [PATCH 35/44] [HUDI-4107] Added --sync-tool-classes config option in HoodieMultiTableDeltaStreamer (#5597) * added --sync-tool-classes config option in multitable delta streamer * added a testcase to assert if syncClientToolClassNames is getting picked to the deltastreamer execution context --- .../apache/hudi/utilities/deltastreamer/DeltaSync.java | 2 +- .../utilities/deltastreamer/HoodieDeltaStreamer.java | 10 +++++++--- .../deltastreamer/HoodieMultiTableDeltaStreamer.java | 5 +++++ .../functional/TestHoodieMultiTableDeltaStreamer.java | 9 +++++++++ 4 files changed, 22 insertions(+), 4 deletions(-) diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/DeltaSync.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/DeltaSync.java index 0ae72f94b82e0..736e416162d21 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/DeltaSync.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/DeltaSync.java @@ -691,7 +691,7 @@ private String getSyncClassShortName(String syncClassName) { } private void syncMeta(HoodieDeltaStreamerMetrics metrics) { - Set syncClientToolClasses = new HashSet<>(Arrays.asList(cfg.syncClientToolClass.split(","))); + Set syncClientToolClasses = new HashSet<>(Arrays.asList(cfg.syncClientToolClassNames.split(","))); // for backward compatibility if (cfg.enableHiveSync) { cfg.enableMetaSync = true; diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/HoodieDeltaStreamer.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/HoodieDeltaStreamer.java index 7a688b50c7097..a22a3581ae94a 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/HoodieDeltaStreamer.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/HoodieDeltaStreamer.java @@ -306,7 +306,7 @@ public static class Config implements Serializable { public Boolean enableMetaSync = false; @Parameter(names = {"--sync-tool-classes"}, description = "Meta sync client tool, using comma to separate multi tools") - public String syncClientToolClass = HiveSyncTool.class.getName(); + public String syncClientToolClassNames = HiveSyncTool.class.getName(); @Parameter(names = {"--max-pending-compactions"}, description = "Maximum number of outstanding inflight/requested compactions. Delta Sync will not happen unless" @@ -442,6 +442,8 @@ public boolean equals(Object o) { && operation == config.operation && Objects.equals(filterDupes, config.filterDupes) && Objects.equals(enableHiveSync, config.enableHiveSync) + && Objects.equals(enableMetaSync, config.enableMetaSync) + && Objects.equals(syncClientToolClassNames, config.syncClientToolClassNames) && Objects.equals(maxPendingCompactions, config.maxPendingCompactions) && Objects.equals(maxPendingClustering, config.maxPendingClustering) && Objects.equals(continuousMode, config.continuousMode) @@ -466,8 +468,8 @@ public int hashCode() { baseFileFormat, propsFilePath, configs, sourceClassName, sourceOrderingField, payloadClassName, schemaProviderClassName, transformerClassNames, sourceLimit, operation, filterDupes, - enableHiveSync, maxPendingCompactions, maxPendingClustering, continuousMode, - minSyncIntervalSeconds, sparkMaster, commitOnErrors, + enableHiveSync, enableMetaSync, syncClientToolClassNames, maxPendingCompactions, maxPendingClustering, + continuousMode, minSyncIntervalSeconds, sparkMaster, commitOnErrors, deltaSyncSchedulingWeight, compactSchedulingWeight, clusterSchedulingWeight, deltaSyncSchedulingMinShare, compactSchedulingMinShare, clusterSchedulingMinShare, forceDisableCompaction, checkpoint, initialCheckpointProvider, help); @@ -491,6 +493,8 @@ public String toString() { + ", operation=" + operation + ", filterDupes=" + filterDupes + ", enableHiveSync=" + enableHiveSync + + ", enableMetaSync=" + enableMetaSync + + ", syncClientToolClassNames=" + syncClientToolClassNames + ", maxPendingCompactions=" + maxPendingCompactions + ", maxPendingClustering=" + maxPendingClustering + ", continuousMode=" + continuousMode diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/HoodieMultiTableDeltaStreamer.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/HoodieMultiTableDeltaStreamer.java index 376c9cfae3730..84aee29dec81c 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/HoodieMultiTableDeltaStreamer.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/HoodieMultiTableDeltaStreamer.java @@ -27,6 +27,7 @@ import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.util.ValidationUtils; +import org.apache.hudi.hive.HiveSyncTool; import org.apache.hudi.sync.common.HoodieSyncConfig; import org.apache.hudi.utilities.IdentitySplitter; import org.apache.hudi.exception.HoodieException; @@ -203,6 +204,7 @@ static String getTableWithDatabase(TableExecutionContext context) { static void deepCopyConfigs(Config globalConfig, HoodieDeltaStreamer.Config tableConfig) { tableConfig.enableHiveSync = globalConfig.enableHiveSync; tableConfig.enableMetaSync = globalConfig.enableMetaSync; + tableConfig.syncClientToolClassNames = globalConfig.syncClientToolClassNames; tableConfig.schemaProviderClassName = globalConfig.schemaProviderClassName; tableConfig.sourceOrderingField = globalConfig.sourceOrderingField; tableConfig.sourceClassName = globalConfig.sourceClassName; @@ -325,6 +327,9 @@ public static class Config implements Serializable { @Parameter(names = {"--enable-sync"}, description = "Enable syncing meta") public Boolean enableMetaSync = false; + @Parameter(names = {"--sync-tool-classes"}, description = "Meta sync client tool, using comma to separate multi tools") + public String syncClientToolClassNames = HiveSyncTool.class.getName(); + @Parameter(names = {"--max-pending-compactions"}, description = "Maximum number of outstanding inflight/requested compactions. Delta Sync will not happen unless" + "outstanding compactions is less than this number") diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieMultiTableDeltaStreamer.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieMultiTableDeltaStreamer.java index cc2c96f2c8516..8f54b0d34dccc 100644 --- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieMultiTableDeltaStreamer.java +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieMultiTableDeltaStreamer.java @@ -72,10 +72,19 @@ static HoodieMultiTableDeltaStreamer.Config getConfig(String fileName, String co } config.enableHiveSync = enableHiveSync; config.enableMetaSync = enableMetaSync; + config.syncClientToolClassNames = "com.example.DummySyncTool1,com.example.DummySyncTool2"; return config; } } + @Test + public void testMetaSyncConfig() throws IOException { + HoodieMultiTableDeltaStreamer.Config cfg = TestHelpers.getConfig(PROPS_FILENAME_TEST_SOURCE1, dfsBasePath + "/config", TestDataSource.class.getName(), true, true, null); + HoodieMultiTableDeltaStreamer streamer = new HoodieMultiTableDeltaStreamer(cfg, jsc); + TableExecutionContext executionContext = streamer.getTableExecutionContexts().get(1); + assertEquals("com.example.DummySyncTool1,com.example.DummySyncTool2", executionContext.getConfig().syncClientToolClassNames); + } + @Test public void testInvalidHiveSyncProps() throws IOException { HoodieMultiTableDeltaStreamer.Config cfg = TestHelpers.getConfig(PROPS_INVALID_HIVE_SYNC_TEST_SOURCE1, dfsBasePath + "/config", TestDataSource.class.getName(), true, true, null); From 0d069b5e57e96984d508434a9b76c48a282bd45b Mon Sep 17 00:00:00 2001 From: Danny Chan Date: Wed, 1 Jun 2022 16:17:36 +0800 Subject: [PATCH 36/44] [HUDI-4174] Add hive conf dir option for flink sink (#5725) --- .../apache/hudi/configuration/FlinkOptions.java | 6 ++++++ .../hudi/configuration/HadoopConfigurations.java | 14 ++++++++++++++ .../hudi/sink/StreamWriteOperatorCoordinator.java | 10 +++++++++- .../apache/hudi/sink/utils/HiveSyncContext.java | 4 +++- 4 files changed, 32 insertions(+), 2 deletions(-) diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/configuration/FlinkOptions.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/configuration/FlinkOptions.java index 3de4bd4f757b8..57cb8daa44579 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/configuration/FlinkOptions.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/configuration/FlinkOptions.java @@ -769,6 +769,12 @@ private FlinkOptions() { .noDefaultValue() .withDescription("Serde properties to hive table, the data format is k1=v1\nk2=v2"); + public static final ConfigOption HIVE_SYNC_CONF_DIR = ConfigOptions + .key("hive_sync.conf.dir") + .stringType() + .noDefaultValue() + .withDescription("The hive configuration directory, where the hive-site.xml lies in, the file should be put on the client machine"); + // ------------------------------------------------------------------------- // Utilities // ------------------------------------------------------------------------- diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/configuration/HadoopConfigurations.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/configuration/HadoopConfigurations.java index 72f20311504d0..d15ef280f532a 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/configuration/HadoopConfigurations.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/configuration/HadoopConfigurations.java @@ -19,6 +19,8 @@ package org.apache.hudi.configuration; import org.apache.flink.configuration.Configuration; +import org.apache.hadoop.fs.Path; + import org.apache.hudi.util.FlinkClientUtil; import java.util.Map; @@ -51,4 +53,16 @@ public static org.apache.hadoop.conf.Configuration getHadoopConf(Configuration c options.forEach(hadoopConf::set); return hadoopConf; } + + /** + * Creates a Hive configuration with configured dir path or empty if no Hive conf dir is set. + */ + public static org.apache.hadoop.conf.Configuration getHiveConf(Configuration conf) { + String explicitDir = conf.getString(FlinkOptions.HIVE_SYNC_CONF_DIR, System.getenv("HIVE_CONF_DIR")); + org.apache.hadoop.conf.Configuration hadoopConf = new org.apache.hadoop.conf.Configuration(); + if (explicitDir != null) { + hadoopConf.addResource(new Path(explicitDir, "hive-site.xml")); + } + return hadoopConf; + } } diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/StreamWriteOperatorCoordinator.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/StreamWriteOperatorCoordinator.java index 39976e5ee2dc4..75e8beaef17cf 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/StreamWriteOperatorCoordinator.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/StreamWriteOperatorCoordinator.java @@ -20,6 +20,7 @@ import org.apache.hudi.client.HoodieFlinkWriteClient; import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.common.config.SerializableConfiguration; import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.common.table.HoodieTableMetaClient; @@ -28,6 +29,7 @@ import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.ValidationUtils; import org.apache.hudi.configuration.FlinkOptions; +import org.apache.hudi.configuration.HadoopConfigurations; import org.apache.hudi.configuration.OptionsResolver; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.sink.event.CommitAckEvent; @@ -82,6 +84,11 @@ public class StreamWriteOperatorCoordinator */ private final Configuration conf; + /** + * Hive config options. + */ + private final SerializableConfiguration hiveConf; + /** * Coordinator context. */ @@ -160,6 +167,7 @@ public StreamWriteOperatorCoordinator( this.conf = conf; this.context = context; this.parallelism = context.currentParallelism(); + this.hiveConf = new SerializableConfiguration(HadoopConfigurations.getHiveConf(conf)); } @Override @@ -314,7 +322,7 @@ public void subtaskReady(int i, SubtaskGateway subtaskGateway) { private void initHiveSync() { this.hiveSyncExecutor = NonThrownExecutor.builder(LOG).waitForTasksFinish(true).build(); - this.hiveSyncContext = HiveSyncContext.create(conf); + this.hiveSyncContext = HiveSyncContext.create(conf, this.hiveConf); } private void syncHiveAsync() { diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/utils/HiveSyncContext.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/utils/HiveSyncContext.java index bd837efc8737d..9fc5323d46a2d 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/utils/HiveSyncContext.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/utils/HiveSyncContext.java @@ -20,6 +20,7 @@ import org.apache.flink.annotation.VisibleForTesting; import org.apache.hudi.aws.sync.AwsGlueCatalogSyncTool; +import org.apache.hudi.common.config.SerializableConfiguration; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.configuration.FlinkOptions; import org.apache.hudi.configuration.HadoopConfigurations; @@ -58,7 +59,7 @@ public HiveSyncTool hiveSyncTool() { return new HiveSyncTool(this.syncConfig, this.hiveConf, this.fs); } - public static HiveSyncContext create(Configuration conf) { + public static HiveSyncContext create(Configuration conf, SerializableConfiguration serConf) { HiveSyncConfig syncConfig = buildSyncConfig(conf); org.apache.hadoop.conf.Configuration hadoopConf = HadoopConfigurations.getHadoopConf(conf); String path = conf.getString(FlinkOptions.PATH); @@ -67,6 +68,7 @@ public static HiveSyncContext create(Configuration conf) { if (!FlinkOptions.isDefaultValueDefined(conf, FlinkOptions.HIVE_SYNC_METASTORE_URIS)) { hadoopConf.set(HiveConf.ConfVars.METASTOREURIS.varname, conf.getString(FlinkOptions.HIVE_SYNC_METASTORE_URIS)); } + hiveConf.addResource(serConf.get()); hiveConf.addResource(hadoopConf); return new HiveSyncContext(syncConfig, hiveConf, fs); } From dfcd6d9a8605e93fcb21b8cc27ac474d4ffa2a76 Mon Sep 17 00:00:00 2001 From: Sagar Sumit Date: Wed, 1 Jun 2022 18:00:29 +0530 Subject: [PATCH 37/44] [HUDI-4011] Add hudi-aws-bundle (#5674) Co-authored-by: Raymond Xu <2701446+xushiyan@users.noreply.github.com> --- packaging/hudi-aws-bundle/pom.xml | 295 ++++++++++++++++++ .../java/org/apache/hudi/aws/bundle/Main.java | 29 ++ packaging/hudi-integ-test-bundle/pom.xml | 1 - packaging/hudi-spark-bundle/pom.xml | 1 - packaging/hudi-utilities-bundle/pom.xml | 1 - packaging/hudi-utilities-slim-bundle/pom.xml | 1 - pom.xml | 1 + 7 files changed, 325 insertions(+), 4 deletions(-) create mode 100644 packaging/hudi-aws-bundle/pom.xml create mode 100644 packaging/hudi-aws-bundle/src/main/java/org/apache/hudi/aws/bundle/Main.java diff --git a/packaging/hudi-aws-bundle/pom.xml b/packaging/hudi-aws-bundle/pom.xml new file mode 100644 index 0000000000000..3cbf4b7252353 --- /dev/null +++ b/packaging/hudi-aws-bundle/pom.xml @@ -0,0 +1,295 @@ + + + + + + hudi + org.apache.hudi + 0.12.0-SNAPSHOT + ../../pom.xml + + 4.0.0 + + hudi-aws-bundle + jar + + + true + ${project.parent.basedir} + + + + + + org.apache.rat + apache-rat-plugin + + + org.apache.maven.plugins + maven-shade-plugin + ${maven-shade-plugin.version} + + + package + + shade + + + ${shadeSources} + ${project.build.directory}/dependency-reduced-pom.xml + + + + + + true + + + META-INF/LICENSE + target/classes/META-INF/LICENSE + + + + + + org.apache.hudi:hudi-common + org.apache.hudi:hudi-hadoop-mr + org.apache.hudi:hudi-sync-common + org.apache.hudi:hudi-hive-sync + org.apache.hudi:hudi-aws + org.apache.parquet:parquet-avro + org.apache.avro:avro + com.amazonaws:dynamodb-lock-client + com.amazonaws:aws-java-sdk-cloudwatch + com.amazonaws:aws-java-sdk-dynamodb + com.amazonaws:aws-java-sdk-core + com.amazonaws:aws-java-sdk-glue + com.beust:jcommander + commons-io:commons-io + org.apache.hbase:hbase-common + org.apache.hbase:hbase-client + org.apache.hbase:hbase-hadoop-compat + org.apache.hbase:hbase-hadoop2-compat + org.apache.hbase:hbase-metrics + org.apache.hbase:hbase-metrics-api + org.apache.hbase:hbase-protocol-shaded + org.apache.hbase:hbase-server + org.apache.hbase.thirdparty:hbase-shaded-miscellaneous + org.apache.hbase.thirdparty:hbase-shaded-netty + org.apache.hbase.thirdparty:hbase-shaded-protobuf + org.apache.htrace:htrace-core4 + + + + + com.esotericsoftware.kryo. + org.apache.hudi.com.esotericsoftware.kryo. + + + com.esotericsoftware.minlog. + org.apache.hudi.com.esotericsoftware.minlog. + + + com.beust.jcommander. + org.apache.hudi.com.beust.jcommander. + + + org.apache.commons.io. + org.apache.hudi.org.apache.commons.io. + + + org.apache.hadoop.hbase. + org.apache.hudi.org.apache.hadoop.hbase. + + org.apache.hadoop.hbase.KeyValue$KeyComparator + + + + org.apache.hbase. + org.apache.hudi.org.apache.hbase. + + + org.apache.htrace. + org.apache.hudi.org.apache.htrace. + + + org.objenesis. + org.apache.hudi.org.objenesis. + + + com.amazonaws. + org.apache.hudi.com.amazonaws. + + + org.apache.parquet.avro. + org.apache.hudi.org.apache.parquet.avro. + + + org.apache.avro. + org.apache.hudi.org.apache.avro. + + + + org.apache.hadoop.metrics2.MetricHistogram + org.apache.hudi.org.apache.hadoop.metrics2.MetricHistogram + + + + org.apache.hadoop.metrics2.MetricsExecutor + org.apache.hudi.org.apache.hadoop.metrics2.MetricsExecutor + + + + org.apache.hadoop.metrics2.impl.JmxCacheBuster + org.apache.hudi.org.apache.hadoop.metrics2.impl.JmxCacheBuster + + + org.apache.hadoop.metrics2.lib.DefaultMetricsSystemHelper + org.apache.hudi.org.apache.hadoop.metrics2.lib.DefaultMetricsSystemHelper + + + + org.apache.hadoop.metrics2.lib.DynamicMetricsRegistry + org.apache.hudi.org.apache.hadoop.metrics2.lib.DynamicMetricsRegistry + + + + org.apache.hadoop.metrics2.lib.MetricsExecutorImpl + org.apache.hudi.org.apache.hadoop.metrics2.lib.MetricsExecutorImpl + + + + org.apache.hadoop.metrics2.lib.MutableFastCounter + org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableFastCounter + + + + org.apache.hadoop.metrics2.lib.MutableHistogram + org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableHistogram + + + + org.apache.hadoop.metrics2.lib.MutableRangeHistogram + org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableRangeHistogram + + + + org.apache.hadoop.metrics2.lib.MutableSizeHistogram + org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableSizeHistogram + + + + org.apache.hadoop.metrics2.lib.MutableTimeHistogram + org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableTimeHistogram + + + + org.apache.hadoop.metrics2.util.MetricQuantile + org.apache.hudi.org.apache.hadoop.metrics2.util.MetricQuantile + + + + org.apache.hadoop.metrics2.util.MetricSampleQuantiles + org.apache.hudi.org.apache.hadoop.metrics2.util.MetricSampleQuantiles + + + + false + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + META-INF/services/javax.* + **/*.proto + hbase-webapps/** + + + + ${project.artifactId}-${project.version} + + + + + + + + src/main/resources + + + src/test/resources + + + + + + + + org.apache.hudi + hudi-common + ${project.version} + + + + org.apache.hadoop + * + + + + + org.apache.hudi + hudi-hive-sync + ${project.version} + + + javax.servlet + servlet-api + + + + + org.apache.hudi + hudi-aws + ${project.version} + + + + org.apache.parquet + parquet-avro + ${parquet.version} + compile + + + org.apache.avro + avro + ${avro.version} + compile + + + diff --git a/packaging/hudi-aws-bundle/src/main/java/org/apache/hudi/aws/bundle/Main.java b/packaging/hudi-aws-bundle/src/main/java/org/apache/hudi/aws/bundle/Main.java new file mode 100644 index 0000000000000..b5b167e30ab99 --- /dev/null +++ b/packaging/hudi-aws-bundle/src/main/java/org/apache/hudi/aws/bundle/Main.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.aws.bundle; + +import org.apache.hudi.common.util.ReflectionUtils; + +public class Main { + + public static void main(String[] args) { + ReflectionUtils.getTopLevelClassesInClasspath(Main.class).forEach(System.out::println); + } +} diff --git a/packaging/hudi-integ-test-bundle/pom.xml b/packaging/hudi-integ-test-bundle/pom.xml index ad435967a4057..3181ceefe4e09 100644 --- a/packaging/hudi-integ-test-bundle/pom.xml +++ b/packaging/hudi-integ-test-bundle/pom.xml @@ -83,7 +83,6 @@ org.apache.hudi:hudi-sync-common org.apache.hudi:hudi-hadoop-mr org.apache.hudi:hudi-timeline-service - org.apache.hudi:hudi-aws org.apache.hudi:hudi-integ-test org.apache.hbase:hbase-common diff --git a/packaging/hudi-spark-bundle/pom.xml b/packaging/hudi-spark-bundle/pom.xml index d6a5eb6924618..8b551478278ff 100644 --- a/packaging/hudi-spark-bundle/pom.xml +++ b/packaging/hudi-spark-bundle/pom.xml @@ -79,7 +79,6 @@ org.apache.hudi:hudi-sync-common org.apache.hudi:hudi-hadoop-mr org.apache.hudi:hudi-timeline-service - org.apache.hudi:hudi-aws javax.servlet:javax.servlet-api com.beust:jcommander diff --git a/packaging/hudi-utilities-bundle/pom.xml b/packaging/hudi-utilities-bundle/pom.xml index a18808678b636..209a06514b624 100644 --- a/packaging/hudi-utilities-bundle/pom.xml +++ b/packaging/hudi-utilities-bundle/pom.xml @@ -102,7 +102,6 @@ org.apache.hudi:hudi-sync-common org.apache.hudi:hudi-hadoop-mr org.apache.hudi:hudi-timeline-service - org.apache.hudi:hudi-aws com.yammer.metrics:metrics-core com.beust:jcommander diff --git a/packaging/hudi-utilities-slim-bundle/pom.xml b/packaging/hudi-utilities-slim-bundle/pom.xml index 993e2ad7fd912..1a6f6f106bcfd 100644 --- a/packaging/hudi-utilities-slim-bundle/pom.xml +++ b/packaging/hudi-utilities-slim-bundle/pom.xml @@ -95,7 +95,6 @@ org.apache.hudi:hudi-utilities_${scala.binary.version} org.apache.hudi:hudi-hadoop-mr org.apache.hudi:hudi-timeline-service - org.apache.hudi:hudi-aws com.yammer.metrics:metrics-core com.beust:jcommander diff --git a/pom.xml b/pom.xml index 1188ec620aa39..c079c70bfb386 100644 --- a/pom.xml +++ b/pom.xml @@ -48,6 +48,7 @@ packaging/hudi-hadoop-mr-bundle packaging/hudi-datahub-sync-bundle packaging/hudi-hive-sync-bundle + packaging/hudi-aws-bundle packaging/hudi-gcp-bundle packaging/hudi-spark-bundle packaging/hudi-presto-bundle From 7276d0eaa662298b64a6aca7e7180556f03603b1 Mon Sep 17 00:00:00 2001 From: Qi Ji Date: Wed, 1 Jun 2022 22:35:40 +0800 Subject: [PATCH 38/44] [HUDI-3670] free temp views in sql transformers (#5080) --- .../hudi/utilities/transform/FlatteningTransformer.java | 6 ++++-- .../hudi/utilities/transform/SqlFileBasedTransformer.java | 4 +++- .../hudi/utilities/transform/SqlQueryBasedTransformer.java | 6 ++++-- 3 files changed, 11 insertions(+), 5 deletions(-) diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/transform/FlatteningTransformer.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/transform/FlatteningTransformer.java index cf7b67b449fe0..8aa032666e9cd 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/transform/FlatteningTransformer.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/transform/FlatteningTransformer.java @@ -49,8 +49,10 @@ public Dataset apply(JavaSparkContext jsc, SparkSession sparkSession, Datas // tmp table name doesn't like dashes String tmpTable = TMP_TABLE.concat(UUID.randomUUID().toString().replace("-", "_")); LOG.info("Registering tmp table : " + tmpTable); - rowDataset.registerTempTable(tmpTable); - return sparkSession.sql("select " + flattenSchema(rowDataset.schema(), null) + " from " + tmpTable); + rowDataset.createOrReplaceTempView(tmpTable); + Dataset transformed = sparkSession.sql("select " + flattenSchema(rowDataset.schema(), null) + " from " + tmpTable); + sparkSession.catalog().dropTempView(tmpTable); + return transformed; } public String flattenSchema(StructType schema, String prefix) { diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/transform/SqlFileBasedTransformer.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/transform/SqlFileBasedTransformer.java index 04264bf4cb3d9..a53b50431c8d2 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/transform/SqlFileBasedTransformer.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/transform/SqlFileBasedTransformer.java @@ -76,7 +76,7 @@ public Dataset apply( // tmp table name doesn't like dashes final String tmpTable = TMP_TABLE.concat(UUID.randomUUID().toString().replace("-", "_")); LOG.info("Registering tmp table : " + tmpTable); - rowDataset.registerTempTable(tmpTable); + rowDataset.createOrReplaceTempView(tmpTable); try (final Scanner scanner = new Scanner(fs.open(new Path(sqlFile)), "UTF-8")) { Dataset rows = null; @@ -95,6 +95,8 @@ public Dataset apply( return rows; } catch (final IOException ioe) { throw new HoodieIOException("Error reading transformer SQL file.", ioe); + } finally { + sparkSession.catalog().dropTempView(tmpTable); } } diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/transform/SqlQueryBasedTransformer.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/transform/SqlQueryBasedTransformer.java index 7e5ed05f26b93..e39ca74631483 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/transform/SqlQueryBasedTransformer.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/transform/SqlQueryBasedTransformer.java @@ -60,9 +60,11 @@ public Dataset apply(JavaSparkContext jsc, SparkSession sparkSession, Datas // tmp table name doesn't like dashes String tmpTable = TMP_TABLE.concat(UUID.randomUUID().toString().replace("-", "_")); LOG.info("Registering tmp table : " + tmpTable); - rowDataset.registerTempTable(tmpTable); + rowDataset.createOrReplaceTempView(tmpTable); String sqlStr = transformerSQL.replaceAll(SRC_PATTERN, tmpTable); LOG.debug("SQL Query for transformation : (" + sqlStr + ")"); - return sparkSession.sql(sqlStr); + Dataset transformed = sparkSession.sql(sqlStr); + sparkSession.catalog().dropTempView(tmpTable); + return transformed; } } From 7f8630cc57fbb9d29e8dc7ca87b582264da073fd Mon Sep 17 00:00:00 2001 From: Danny Chan Date: Thu, 2 Jun 2022 09:48:48 +0800 Subject: [PATCH 39/44] [HUDI-4167] Remove the timeline refresh with initializing hoodie table (#5716) The timeline refresh on table initialization invokes the fs view #sync, which has two actions now: 1. reload the timeline of the fs view, so that the next fs view request is based on this timeline metadata 2. if this is a local fs view, clear all the local states; if this is a remote fs view, send request to sync the remote fs view But, let's see the construction, the meta client is instantiated freshly so the timeline is already the latest, the table is also constructed freshly, so the fs view has no local states, that means, the #sync is unnecessary totally. In this patch, the metadata lifecycle and data set fs view are kept in sync, when the fs view is refreshed, the underneath metadata is also refreshed synchronouly. The freshness of the metadata follows the same rules as data fs view: 1. if the fs view is local, the visibility is based on the client table metadata client's latest commit 2. if the fs view is remote, the timeline server would #sync the fs view and metadata together based on the lagging server local timeline From the perspective of client, no need to care about the refresh action anymore no matter whether the metadata table is enabled or not. That make the client logic more clear and less error-prone. Removes the timeline refresh has another benefit: if avoids unncecessary #refresh of the remote fs view, if all the clients send request to #sync the remote fs view, the server would encounter conflicts and the client encounters a response error. --- .../apache/hudi/cli/commands/SparkMain.java | 2 +- .../hudi/client/BaseHoodieWriteClient.java | 28 ++++++++----------- .../apache/hudi/config/HoodieWriteConfig.java | 9 ++++-- .../hudi/client/HoodieFlinkWriteClient.java | 3 +- .../apache/hudi/table/HoodieFlinkTable.java | 10 ------- .../bloom/TestFlinkHoodieBloomIndex.java | 2 +- .../hudi/client/HoodieJavaWriteClient.java | 4 +-- .../hudi/client/SparkRDDWriteClient.java | 12 ++++---- .../apache/hudi/table/HoodieSparkTable.java | 17 +---------- .../hudi/client/TestTableSchemaEvolution.java | 8 +++++- .../TestHoodieBackedTableMetadata.java | 2 +- .../table/TestHoodieMergeOnReadTable.java | 2 +- .../testutils/HoodieClientTestHarness.java | 2 +- .../view/RocksDbBasedFileSystemView.java | 1 + .../metadata/HoodieBackedTableMetadata.java | 21 +++++++++++--- .../timeline/service/TimelineService.java | 3 +- 16 files changed, 59 insertions(+), 67 deletions(-) diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/SparkMain.java b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/SparkMain.java index 9fe83f1995c2b..43fe168587ac1 100644 --- a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/SparkMain.java +++ b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/SparkMain.java @@ -296,7 +296,7 @@ protected static int deleteMarker(JavaSparkContext jsc, String instantTime, Stri SparkRDDWriteClient client = createHoodieClient(jsc, basePath, false); HoodieWriteConfig config = client.getConfig(); HoodieEngineContext context = client.getEngineContext(); - HoodieSparkTable table = HoodieSparkTable.create(config, context, true); + HoodieSparkTable table = HoodieSparkTable.create(config, context); WriteMarkersFactory.get(config.getMarkersType(), table, instantTime) .quietDeleteMarkerDir(context, config.getMarkersDeleteParallelism()); return 0; diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieWriteClient.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieWriteClient.java index 5c485bed0581d..455cb644c7d47 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieWriteClient.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieWriteClient.java @@ -296,11 +296,7 @@ private void saveInternalSchema(HoodieTable table, String instantTime, HoodieCom } } - protected HoodieTable createTable(HoodieWriteConfig config, Configuration hadoopConf) { - return createTable(config, hadoopConf, false); - } - - protected abstract HoodieTable createTable(HoodieWriteConfig config, Configuration hadoopConf, boolean refreshTimeline); + protected abstract HoodieTable createTable(HoodieWriteConfig config, Configuration hadoopConf); void emitCommitMetrics(String instantTime, HoodieCommitMetadata metadata, String actionType) { try { @@ -365,7 +361,7 @@ public void bootstrap(Option> extraMetadata) { */ protected void rollbackFailedBootstrap() { LOG.info("Rolling back pending bootstrap if present"); - HoodieTable table = createTable(config, hadoopConf, config.isMetadataTableEnabled()); + HoodieTable table = createTable(config, hadoopConf); HoodieTimeline inflightTimeline = table.getMetaClient().getCommitsTimeline().filterPendingExcludingCompaction(); Option instant = Option.fromJavaOptional( inflightTimeline.getReverseOrderedInstants().map(HoodieInstant::getTimestamp).findFirst()); @@ -634,7 +630,7 @@ protected void autoArchiveOnCommit(HoodieTable table, boolean acquireLockForArch * Run any pending compactions. */ public void runAnyPendingCompactions() { - runAnyPendingCompactions(createTable(config, hadoopConf, config.isMetadataTableEnabled())); + runAnyPendingCompactions(createTable(config, hadoopConf)); } /** @@ -644,7 +640,7 @@ public void runAnyPendingCompactions() { * @param comment - Comment for the savepoint */ public void savepoint(String user, String comment) { - HoodieTable table = createTable(config, hadoopConf, config.isMetadataTableEnabled()); + HoodieTable table = createTable(config, hadoopConf); if (table.getCompletedCommitsTimeline().empty()) { throw new HoodieSavepointException("Could not savepoint. Commit timeline is empty"); } @@ -668,7 +664,7 @@ public void savepoint(String user, String comment) { * @param comment - Comment for the savepoint */ public void savepoint(String instantTime, String user, String comment) { - HoodieTable table = createTable(config, hadoopConf, config.isMetadataTableEnabled()); + HoodieTable table = createTable(config, hadoopConf); table.savepoint(context, instantTime, user, comment); } @@ -680,7 +676,7 @@ public void savepoint(String instantTime, String user, String comment) { * @return true if the savepoint was deleted successfully */ public void deleteSavepoint(String savepointTime) { - HoodieTable table = createTable(config, hadoopConf, config.isMetadataTableEnabled()); + HoodieTable table = createTable(config, hadoopConf); SavepointHelpers.deleteSavepoint(table, savepointTime); } @@ -1012,7 +1008,7 @@ public boolean scheduleCompactionAtInstant(String instantTime, Option scheduleIndexing(List partitionTypes) { String instantTime = HoodieActiveTimeline.createNewInstantTime(); - Option indexPlan = createTable(config, hadoopConf, config.isMetadataTableEnabled()) + Option indexPlan = createTable(config, hadoopConf) .scheduleIndexing(context, instantTime, partitionTypes); return indexPlan.isPresent() ? Option.of(instantTime) : Option.empty(); } @@ -1024,7 +1020,7 @@ public Option scheduleIndexing(List partitionType * @return {@link Option} after successful indexing. */ public Option index(String indexInstantTime) { - return createTable(config, hadoopConf, config.isMetadataTableEnabled()).index(context, indexInstantTime); + return createTable(config, hadoopConf).index(context, indexInstantTime); } /** @@ -1339,17 +1335,17 @@ private Option scheduleTableServiceInternal(String instantTime, Option clusteringPlan = createTable(config, hadoopConf, config.isMetadataTableEnabled()) + Option clusteringPlan = createTable(config, hadoopConf) .scheduleClustering(context, instantTime, extraMetadata); return clusteringPlan.isPresent() ? Option.of(instantTime) : Option.empty(); case COMPACT: LOG.info("Scheduling compaction at instant time :" + instantTime); - Option compactionPlan = createTable(config, hadoopConf, config.isMetadataTableEnabled()) + Option compactionPlan = createTable(config, hadoopConf) .scheduleCompaction(context, instantTime, extraMetadata); return compactionPlan.isPresent() ? Option.of(instantTime) : Option.empty(); case CLEAN: LOG.info("Scheduling cleaning at instant time :" + instantTime); - Option cleanerPlan = createTable(config, hadoopConf, config.isMetadataTableEnabled()) + Option cleanerPlan = createTable(config, hadoopConf) .scheduleCleaning(context, instantTime, extraMetadata); return cleanerPlan.isPresent() ? Option.of(instantTime) : Option.empty(); default: @@ -1702,6 +1698,6 @@ private void commitTableChange(InternalSchema newSchema, HoodieTableMetaClient m // try to save history schemas FileBasedInternalSchemaStorageManager schemasManager = new FileBasedInternalSchemaStorageManager(metaClient); schemasManager.persistHistorySchemaStr(instantTime, SerDeHelper.inheritSchemas(newSchema, historySchemaStr)); - commitStats(instantTime, Collections.EMPTY_LIST, Option.of(extraMeta), commitActionType); + commitStats(instantTime, Collections.emptyList(), Option.of(extraMeta), commitActionType); } } 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 42208a0734aa7..1603965ea987f 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 @@ -364,8 +364,8 @@ public class HoodieWriteConfig extends HoodieConfig { public static final ConfigProperty REFRESH_TIMELINE_SERVER_BASED_ON_LATEST_COMMIT = ConfigProperty .key("hoodie.refresh.timeline.server.based.on.latest.commit") - .defaultValue(false) - .withDocumentation("Refresh timeline in timeline server based on latest commit apart from timeline hash difference. By default (false), "); + .defaultValue(true) + .withDocumentation("Refresh timeline in timeline server based on latest commit apart from timeline hash difference. By default (true)."); public static final ConfigProperty INITIAL_CONSISTENCY_CHECK_INTERVAL_MS = ConfigProperty .key("hoodie.consistency.check.initial_interval_ms") @@ -2499,6 +2499,11 @@ public Builder withAutoAdjustLockConfigs(boolean autoAdjustLockConfigs) { return this; } + public Builder withRefreshTimelineServerBasedOnLatestCommit(boolean refreshTimelineServerBasedOnLatestCommit) { + writeConfig.setValue(REFRESH_TIMELINE_SERVER_BASED_ON_LATEST_COMMIT, Boolean.toString(refreshTimelineServerBasedOnLatestCommit)); + return this; + } + protected void setDefaults() { writeConfig.setDefaultValue(MARKERS_TYPE, getDefaultMarkersType(engineType)); // Check for mandatory properties diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/HoodieFlinkWriteClient.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/HoodieFlinkWriteClient.java index ddfbabaf36ae9..b68cf97e9aa35 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/HoodieFlinkWriteClient.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/HoodieFlinkWriteClient.java @@ -117,8 +117,7 @@ public boolean commit(String instantTime, List writeStatuses, Optio } @Override - protected HoodieTable createTable(HoodieWriteConfig config, Configuration hadoopConf, - boolean refreshTimeline) { + protected HoodieTable createTable(HoodieWriteConfig config, Configuration hadoopConf) { return HoodieFlinkTable.create(config, (HoodieFlinkEngineContext) context); } diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkTable.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkTable.java index 6eae15e7e1aff..26149918c6549 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkTable.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkTable.java @@ -62,13 +62,6 @@ public static HoodieFlinkTable create(HoodieW public static HoodieFlinkTable create(HoodieWriteConfig config, HoodieFlinkEngineContext context, HoodieTableMetaClient metaClient) { - return HoodieFlinkTable.create(config, context, metaClient, config.isMetadataTableEnabled()); - } - - public static HoodieFlinkTable create(HoodieWriteConfig config, - HoodieFlinkEngineContext context, - HoodieTableMetaClient metaClient, - boolean refreshTimeline) { final HoodieFlinkTable hoodieFlinkTable; switch (metaClient.getTableType()) { case COPY_ON_WRITE: @@ -80,9 +73,6 @@ public static HoodieFlinkTable create(HoodieW default: throw new HoodieException("Unsupported table type :" + metaClient.getTableType()); } - if (refreshTimeline) { - hoodieFlinkTable.getHoodieView().sync(); - } return hoodieFlinkTable; } diff --git a/hudi-client/hudi-flink-client/src/test/java/org/apache/hudi/index/bloom/TestFlinkHoodieBloomIndex.java b/hudi-client/hudi-flink-client/src/test/java/org/apache/hudi/index/bloom/TestFlinkHoodieBloomIndex.java index 50adabbd585ea..e23ee4ad58e6e 100644 --- a/hudi-client/hudi-flink-client/src/test/java/org/apache/hudi/index/bloom/TestFlinkHoodieBloomIndex.java +++ b/hudi-client/hudi-flink-client/src/test/java/org/apache/hudi/index/bloom/TestFlinkHoodieBloomIndex.java @@ -104,7 +104,7 @@ private HoodieWriteConfig makeConfig(boolean rangePruning, boolean treeFiltering public void testLoadInvolvedFiles(boolean rangePruning, boolean treeFiltering, boolean bucketizedChecking) throws Exception { HoodieWriteConfig config = makeConfig(rangePruning, treeFiltering, bucketizedChecking); HoodieBloomIndex index = new HoodieBloomIndex(config, ListBasedHoodieBloomIndexHelper.getInstance()); - HoodieTable hoodieTable = HoodieFlinkTable.create(config, context, metaClient, false); + HoodieTable hoodieTable = HoodieFlinkTable.create(config, context, metaClient); HoodieFlinkWriteableTestTable testTable = HoodieFlinkWriteableTestTable.of(hoodieTable, SCHEMA); // Create some partitions, and put some files diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/client/HoodieJavaWriteClient.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/client/HoodieJavaWriteClient.java index 7f5dc19baf274..fbfb85bab3b8f 100644 --- a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/client/HoodieJavaWriteClient.java +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/client/HoodieJavaWriteClient.java @@ -89,9 +89,7 @@ public boolean commit(String instantTime, } @Override - protected HoodieTable createTable(HoodieWriteConfig config, - Configuration hadoopConf, - boolean refreshTimeline) { + protected HoodieTable createTable(HoodieWriteConfig config, Configuration hadoopConf) { return HoodieJavaTable.create(config, context); } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/SparkRDDWriteClient.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/SparkRDDWriteClient.java index 7f9ec05e3c5eb..fe6ea975e3111 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/SparkRDDWriteClient.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/SparkRDDWriteClient.java @@ -123,10 +123,8 @@ public boolean commit(String instantTime, JavaRDD writeStatuses, Op } @Override - protected HoodieTable createTable(HoodieWriteConfig config, - Configuration hadoopConf, - boolean refreshTimeline) { - return HoodieSparkTable.create(config, context, refreshTimeline); + protected HoodieTable createTable(HoodieWriteConfig config, Configuration hadoopConf) { + return HoodieSparkTable.create(config, context); } @Override @@ -333,7 +331,7 @@ protected void completeCompaction(HoodieCommitMetadata metadata, @Override protected HoodieWriteMetadata> compact(String compactionInstantTime, boolean shouldComplete) { - HoodieSparkTable table = HoodieSparkTable.create(config, context, true); + HoodieSparkTable table = HoodieSparkTable.create(config, context); preWrite(compactionInstantTime, WriteOperationType.COMPACT, table.getMetaClient()); HoodieTimeline pendingCompactionTimeline = table.getActiveTimeline().filterPendingCompactionTimeline(); HoodieInstant inflightInstant = HoodieTimeline.getCompactionInflightInstant(compactionInstantTime); @@ -352,7 +350,7 @@ protected HoodieWriteMetadata> compact(String compactionIns @Override public HoodieWriteMetadata> cluster(String clusteringInstant, boolean shouldComplete) { - HoodieSparkTable table = HoodieSparkTable.create(config, context, config.isMetadataTableEnabled()); + HoodieSparkTable table = HoodieSparkTable.create(config, context); preWrite(clusteringInstant, WriteOperationType.CLUSTER, table.getMetaClient()); HoodieTimeline pendingClusteringTimeline = table.getActiveTimeline().filterPendingReplaceTimeline(); HoodieInstant inflightInstant = HoodieTimeline.getReplaceCommitInflightInstant(clusteringInstant); @@ -434,7 +432,7 @@ protected HoodieTable doInitTable(HoodieTableMetaClient metaClient, Option HoodieSparkTable create(HoodieWriteConfig config, HoodieEngineContext context) { - return create(config, context, false); - } - - public static HoodieSparkTable create(HoodieWriteConfig config, HoodieEngineContext context, - boolean refreshTimeline) { HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder().setConf(context.getHadoopConf().get()).setBasePath(config.getBasePath()) .setLoadActiveTimelineOnLoad(true).setConsistencyGuardConfig(config.getConsistencyGuardConfig()) .setLayoutVersion(Option.of(new TimelineLayoutVersion(config.getTimelineLayoutVersion()))) .setFileSystemRetryConfig(config.getFileSystemRetryConfig()) .setProperties(config.getProps()).build(); - return HoodieSparkTable.create(config, (HoodieSparkEngineContext) context, metaClient, refreshTimeline); + return HoodieSparkTable.create(config, (HoodieSparkEngineContext) context, metaClient); } public static HoodieSparkTable create(HoodieWriteConfig config, HoodieSparkEngineContext context, HoodieTableMetaClient metaClient) { - return create(config, context, metaClient, false); - } - - public static HoodieSparkTable create(HoodieWriteConfig config, - HoodieSparkEngineContext context, - HoodieTableMetaClient metaClient, - boolean refreshTimeline) { HoodieSparkTable hoodieSparkTable; switch (metaClient.getTableType()) { case COPY_ON_WRITE: @@ -89,9 +77,6 @@ public static HoodieSparkTable create(HoodieW default: throw new HoodieException("Unsupported table type :" + metaClient.getTableType()); } - if (refreshTimeline) { - hoodieSparkTable.getHoodieView().sync(); - } return hoodieSparkTable; } diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestTableSchemaEvolution.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestTableSchemaEvolution.java index 1cb7bcbfc4fcb..98bcb11033c5b 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestTableSchemaEvolution.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestTableSchemaEvolution.java @@ -515,7 +515,13 @@ private HoodieWriteConfig.Builder getWriteConfigBuilder(String schema) { return getConfigBuilder(schema) .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(IndexType.INMEMORY).build()) .withCompactionConfig(HoodieCompactionConfig.newBuilder().withMaxNumDeltaCommitsBeforeCompaction(1).build()) - .withAvroSchemaValidate(true); + .withAvroSchemaValidate(true) + // The test has rollback instants on the timeline, + // these rollback instants use real time as instant time, whose instant time is always greater than + // the normal commits instant time, this breaks the refresh rule introduced in HUDI-2761: + // The last client instant is always the rollback instant but not the normal commit. + // Always refresh the timeline when client and server have different timeline. + .withRefreshTimelineServerBasedOnLatestCommit(false); } @Override diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieBackedTableMetadata.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieBackedTableMetadata.java index 9a8fc55a20028..e19c8fc1a2ee5 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieBackedTableMetadata.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieBackedTableMetadata.java @@ -111,7 +111,7 @@ private void verifyBaseMetadataTable() throws IOException { assertEquals(fsPartitions, metadataPartitions, "Partitions should match"); // Files within each partition should match - HoodieTable table = HoodieSparkTable.create(writeConfig, context, true); + HoodieTable table = HoodieSparkTable.create(writeConfig, context); TableFileSystemView tableView = table.getHoodieView(); List fullPartitionPaths = fsPartitions.stream().map(partition -> basePath + "/" + partition).collect(Collectors.toList()); Map partitionToFilesMap = tableMetadata.getAllFilesInPartitions(fullPartitionPaths); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/TestHoodieMergeOnReadTable.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/TestHoodieMergeOnReadTable.java index b9f025223b7df..0ce6ca0ee923b 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/TestHoodieMergeOnReadTable.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/TestHoodieMergeOnReadTable.java @@ -247,7 +247,7 @@ public void testLogFileCountsAfterCompaction(boolean preserveCommitMeta) throws assertEquals(allPartitions.size(), testTable.listAllBaseFiles().length); // Verify that all data file has one log file - HoodieTable table = HoodieSparkTable.create(config, context(), metaClient, true); + HoodieTable table = HoodieSparkTable.create(config, context(), metaClient); for (String partitionPath : dataGen.getPartitionPaths()) { List groupedLogFiles = table.getSliceView().getLatestFileSlices(partitionPath).collect(Collectors.toList()); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestHarness.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestHarness.java index 4504c552c95d6..d0365dced199e 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestHarness.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestHarness.java @@ -559,7 +559,7 @@ public void validateMetadata(HoodieTestTable testTable, List inflightCom // Files within each partition should match metaClient = HoodieTableMetaClient.reload(metaClient); - HoodieTable table = HoodieSparkTable.create(writeConfig, engineContext, true); + HoodieTable table = HoodieSparkTable.create(writeConfig, engineContext); TableFileSystemView tableView = table.getHoodieView(); List fullPartitionPaths = fsPartitions.stream().map(partition -> basePath + "/" + partition).collect(Collectors.toList()); Map partitionToFilesMap = tableMetadata.getAllFilesInPartitions(fullPartitionPaths); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/view/RocksDbBasedFileSystemView.java b/hudi-common/src/main/java/org/apache/hudi/common/table/view/RocksDbBasedFileSystemView.java index af0dc130162aa..02a406e7e0763 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/view/RocksDbBasedFileSystemView.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/view/RocksDbBasedFileSystemView.java @@ -199,6 +199,7 @@ protected void resetViewState() { LOG.info("Deleting all rocksdb data associated with table filesystem view"); rocksDB.close(); rocksDB = new RocksDBDAO(metaClient.getBasePath(), config.getRocksdbBasePath()); + schemaHelper.getAllColumnFamilies().forEach(rocksDB::addColumnFamily); } @Override diff --git a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadata.java b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadata.java index cf941bb70cc3b..e8937b39dc7f1 100644 --- a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadata.java +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadata.java @@ -550,10 +550,7 @@ private List getRollbackedCommits(HoodieInstant instant, HoodieActiveTim @Override public void close() { - for (Pair partitionFileSlicePair : partitionReaders.keySet()) { - close(partitionFileSlicePair); - } - partitionReaders.clear(); + closePartitionReaders(); } /** @@ -567,6 +564,16 @@ private synchronized void close(Pair partitionFileSlicePair) { closeReader(readers); } + /** + * Close and clear all the partitions readers. + */ + private void closePartitionReaders() { + for (Pair partitionFileSlicePair : partitionReaders.keySet()) { + close(partitionFileSlicePair); + } + partitionReaders.clear(); + } + private void closeReader(Pair readers) { if (readers != null) { try { @@ -624,5 +631,11 @@ public Option getLatestCompactionTime() { public void reset() { initIfNeeded(); dataMetaClient.reloadActiveTimeline(); + if (metadataMetaClient != null) { + metadataMetaClient.reloadActiveTimeline(); + } + // the cached reader has max instant time restriction, they should be cleared + // because the metadata timeline may have changed. + closePartitionReaders(); } } diff --git a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/TimelineService.java b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/TimelineService.java index 40669f50e42d6..2ff21682213c2 100644 --- a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/TimelineService.java +++ b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/TimelineService.java @@ -150,7 +150,7 @@ public static class Builder { private int markerBatchNumThreads = 20; private long markerBatchIntervalMs = 50L; private int markerParallelism = 100; - private boolean refreshTimelineBasedOnLatestCommit = false; + private boolean refreshTimelineBasedOnLatestCommit = true; public Builder() { } @@ -240,6 +240,7 @@ public Config build() { config.markerBatchNumThreads = this.markerBatchNumThreads; config.markerBatchIntervalMs = this.markerBatchIntervalMs; config.markerParallelism = this.markerParallelism; + config.refreshTimelineBasedOnLatestCommit = this.refreshTimelineBasedOnLatestCommit; return config; } } From 51602a34f7b2c296b1f1438e85ef2f12c4dff741 Mon Sep 17 00:00:00 2001 From: KnightChess <981159963@qq.com> Date: Thu, 2 Jun 2022 20:28:21 +0800 Subject: [PATCH 40/44] [HUDI-4179] Cluster with sort cloumns invalid (#5739) --- .../execution/bulkinsert/RDDCustomColumnsSortPartitioner.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/RDDCustomColumnsSortPartitioner.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/RDDCustomColumnsSortPartitioner.java index b1cbe47a6b72e..dc80498c7a964 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/RDDCustomColumnsSortPartitioner.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/RDDCustomColumnsSortPartitioner.java @@ -68,7 +68,7 @@ record -> { if (recordValue == null) { return StringUtils.EMPTY_STRING; } else { - return StringUtils.objToString(record); + return StringUtils.objToString(recordValue); } }, true, outputSparkPartitions); From 69f28b66e3ee9f511a3e5b262fb3c71a41cf4c03 Mon Sep 17 00:00:00 2001 From: yanenze Date: Wed, 20 Apr 2022 00:04:59 +0800 Subject: [PATCH 41/44] #1 --- .../ScheduleCompactionsActionExecutor.java | 184 ++++++++++++++++++ .../action/compact/ScheduleConstant.java | 12 ++ 2 files changed, 196 insertions(+) create mode 100644 hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/compact/ScheduleCompactionsActionExecutor.java create mode 100644 hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/compact/ScheduleConstant.java diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/compact/ScheduleCompactionsActionExecutor.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/compact/ScheduleCompactionsActionExecutor.java new file mode 100644 index 0000000000000..e3e732eae9ef8 --- /dev/null +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/compact/ScheduleCompactionsActionExecutor.java @@ -0,0 +1,184 @@ +package org.apache.hudi.table.action.compact; + +import org.apache.hudi.avro.model.HoodieCompactionPlan; +import org.apache.hudi.common.engine.EngineType; +import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.model.HoodieFileGroupId; +import org.apache.hudi.common.model.HoodieRecordPayload; +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.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.HoodieCompactionException; +import org.apache.hudi.exception.HoodieIOException; +import org.apache.hudi.table.HoodieTable; +import org.apache.hudi.table.action.BaseActionExecutor; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; + +import java.io.IOException; +import java.text.ParseException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.atomic.AtomicLong; +import java.util.stream.Collectors; + +public class ScheduleCompactionsActionExecutor extends BaseActionExecutor>> { + + private static final Logger LOG = LogManager.getLogger(ScheduleCompactionActionExecutor.class); + + private final Option> extraMetadata; + private final HoodieFlinkMergeOnReadTableCompactor compactor; + + public ScheduleCompactionsActionExecutor(HoodieEngineContext context, + HoodieWriteConfig config, + HoodieTable table, + String instantTime, + Option> extraMetadata, + HoodieFlinkMergeOnReadTableCompactor compactor) { + super(context, config, table, instantTime); + this.extraMetadata = extraMetadata; + this.compactor = compactor; + } + + @Override + public Option> execute() { + if (!config.getWriteConcurrencyMode().supportsOptimisticConcurrencyControl() + && !config.getFailedWritesCleanPolicy().isLazy()) { + // TODO(yihua): this validation is removed for Java client used by kafka-connect. Need to revisit this. + if (config.getEngineType() != EngineType.JAVA) { + // if there are inflight writes, their instantTime must not be less than that of compaction instant time + table.getActiveTimeline().getCommitsTimeline().filterPendingExcludingCompaction().firstInstant() + .ifPresent(earliestInflight -> ValidationUtils.checkArgument( + HoodieTimeline.compareTimestamps(earliestInflight.getTimestamp(), HoodieTimeline.GREATER_THAN, instantTime), + "Earliest write inflight instant time must be later than compaction time. Earliest :" + earliestInflight + + ", Compaction scheduled at " + instantTime)); + } + // Committed and pending compaction instants should have strictly lower timestamps + List conflictingInstants = table.getActiveTimeline() + .getWriteTimeline().filterCompletedAndCompactionInstants().getInstants() + .filter(instant -> HoodieTimeline.compareTimestamps( + instant.getTimestamp(), HoodieTimeline.GREATER_THAN_OR_EQUALS, instantTime)) + .collect(Collectors.toList()); + ValidationUtils.checkArgument(conflictingInstants.isEmpty(), + "Following instants have timestamps >= compactionInstant (" + instantTime + ") Instants :" + + conflictingInstants); + } + + List plans = scheduleCompaction(); + AtomicLong num = new AtomicLong(0); + plans.stream().forEach(plan -> { + if (plan != null && (plan.getOperations() != null) && (!plan.getOperations().isEmpty())) { + extraMetadata.ifPresent(plan::setExtraMetadata); + HoodieInstant compactionInstant = + new HoodieInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, String.valueOf(Long.valueOf(instantTime) + num.getAndAdd(1))); + try { + table.getActiveTimeline().saveToCompactionRequested(compactionInstant, + TimelineMetadataUtils.serializeCompactionPlan(plan)); + } catch (IOException ioe) { + throw new HoodieIOException("Exception scheduling compaction", ioe); + } + } + }); + return Option.of(plans); + } + + private List scheduleCompaction() { + LOG.info("Checking if compaction needs to be run on " + config.getBasePath()); + // judge if we need to compact according to num delta commits and time elapsed + boolean compactable = needCompact(config.getInlineCompactTriggerStrategy()); + if (compactable) { + LOG.info("Generating compaction plan for merge on read table " + config.getBasePath()); + try { + SyncableFileSystemView fileSystemView = (SyncableFileSystemView) table.getSliceView(); + Set fgInPendingCompactionAndClustering = fileSystemView.getPendingCompactionOperations() + .map(instantTimeOpPair -> instantTimeOpPair.getValue().getFileGroupId()) + .collect(Collectors.toSet()); + // exclude files in pending clustering from compaction. + fgInPendingCompactionAndClustering.addAll(fileSystemView.getFileGroupsInPendingClustering().map(Pair::getLeft).collect(Collectors.toSet())); + context.setJobStatus(this.getClass().getSimpleName(), "Compaction: generating compaction plan"); + return compactor.generateCompactionPlans(context, table, config, instantTime, fgInPendingCompactionAndClustering); + } catch (IOException e) { + throw new HoodieCompactionException("Could not schedule compaction " + config.getBasePath(), e); + } + } + return new ArrayList(); + } + + + private Pair getLatestDeltaCommitInfo() { + Option lastCompaction = table.getActiveTimeline().getCommitTimeline() + .filterCompletedInstants().lastInstant(); + HoodieTimeline deltaCommits = table.getActiveTimeline().getDeltaCommitTimeline(); + + String latestInstantTs; + final int deltaCommitsSinceLastCompaction; + if (lastCompaction.isPresent()) { + latestInstantTs = lastCompaction.get().getTimestamp(); + deltaCommitsSinceLastCompaction = deltaCommits.findInstantsAfter(latestInstantTs, Integer.MAX_VALUE).countInstants(); + } else { + + latestInstantTs = deltaCommits.firstInstant().get().getTimestamp(); + deltaCommitsSinceLastCompaction = deltaCommits.findInstantsAfterOrEquals(latestInstantTs, Integer.MAX_VALUE).countInstants(); + } + return Pair.of(deltaCommitsSinceLastCompaction, latestInstantTs); + } + + private boolean needCompact(CompactionTriggerStrategy compactionTriggerStrategy) { + boolean compactable; + // get deltaCommitsSinceLastCompaction and lastCompactionTs + Pair latestDeltaCommitInfo = getLatestDeltaCommitInfo(); + int inlineCompactDeltaCommitMax = config.getInlineCompactDeltaCommitMax(); + int inlineCompactDeltaSecondsMax = config.getInlineCompactDeltaSecondsMax(); + switch (compactionTriggerStrategy) { + case NUM_COMMITS: + compactable = inlineCompactDeltaCommitMax <= latestDeltaCommitInfo.getLeft(); + if (compactable) { + LOG.info(String.format("The delta commits >= %s, trigger compaction scheduler.", inlineCompactDeltaCommitMax)); + } + break; + case TIME_ELAPSED: + compactable = inlineCompactDeltaSecondsMax <= parsedToSeconds(instantTime) - parsedToSeconds(latestDeltaCommitInfo.getRight()); + if (compactable) { + LOG.info(String.format("The elapsed time >=%ss, trigger compaction scheduler.", inlineCompactDeltaSecondsMax)); + } + break; + case NUM_OR_TIME: + compactable = inlineCompactDeltaCommitMax <= latestDeltaCommitInfo.getLeft() + || inlineCompactDeltaSecondsMax <= parsedToSeconds(instantTime) - parsedToSeconds(latestDeltaCommitInfo.getRight()); + if (compactable) { + LOG.info(String.format("The delta commits >= %s or elapsed_time >=%ss, trigger compaction scheduler.", inlineCompactDeltaCommitMax, + inlineCompactDeltaSecondsMax)); + } + break; + case NUM_AND_TIME: + compactable = inlineCompactDeltaCommitMax <= latestDeltaCommitInfo.getLeft() + && inlineCompactDeltaSecondsMax <= parsedToSeconds(instantTime) - parsedToSeconds(latestDeltaCommitInfo.getRight()); + if (compactable) { + LOG.info(String.format("The delta commits >= %s and elapsed_time >=%ss, trigger compaction scheduler.", inlineCompactDeltaCommitMax, + inlineCompactDeltaSecondsMax)); + } + break; + default: + throw new HoodieCompactionException("Unsupported compaction trigger strategy: " + config.getInlineCompactTriggerStrategy()); + } + return compactable; + } + + private Long parsedToSeconds(String time) { + long timestamp; + try { + timestamp = HoodieActiveTimeline.parseDateFromInstantTime(time).getTime() / 1000; + } catch (ParseException e) { + throw new HoodieCompactionException(e.getMessage(), e); + } + return timestamp; + } +} diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/compact/ScheduleConstant.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/compact/ScheduleConstant.java new file mode 100644 index 0000000000000..aae99723aec0e --- /dev/null +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/compact/ScheduleConstant.java @@ -0,0 +1,12 @@ +package org.apache.hudi.table.action.compact; + +import java.util.concurrent.atomic.AtomicInteger; + +/** + * @Author ez + * @Description + * @createTime 2022/3/17 + */ +public class ScheduleConstant { + static AtomicInteger countDeltaCommit = new AtomicInteger(0); +} From bdde6aea670d3e39b6bde9a09b33c04d22002296 Mon Sep 17 00:00:00 2001 From: yanenze Date: Wed, 20 Apr 2022 00:15:14 +0800 Subject: [PATCH 42/44] #1 --- .../ScheduleCompactionsActionExecutor.java | 184 ------------------ .../action/compact/ScheduleConstant.java | 12 -- 2 files changed, 196 deletions(-) delete mode 100644 hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/compact/ScheduleCompactionsActionExecutor.java delete mode 100644 hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/compact/ScheduleConstant.java diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/compact/ScheduleCompactionsActionExecutor.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/compact/ScheduleCompactionsActionExecutor.java deleted file mode 100644 index e3e732eae9ef8..0000000000000 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/compact/ScheduleCompactionsActionExecutor.java +++ /dev/null @@ -1,184 +0,0 @@ -package org.apache.hudi.table.action.compact; - -import org.apache.hudi.avro.model.HoodieCompactionPlan; -import org.apache.hudi.common.engine.EngineType; -import org.apache.hudi.common.engine.HoodieEngineContext; -import org.apache.hudi.common.model.HoodieFileGroupId; -import org.apache.hudi.common.model.HoodieRecordPayload; -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.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.HoodieCompactionException; -import org.apache.hudi.exception.HoodieIOException; -import org.apache.hudi.table.HoodieTable; -import org.apache.hudi.table.action.BaseActionExecutor; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; - -import java.io.IOException; -import java.text.ParseException; -import java.util.ArrayList; -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.concurrent.atomic.AtomicLong; -import java.util.stream.Collectors; - -public class ScheduleCompactionsActionExecutor extends BaseActionExecutor>> { - - private static final Logger LOG = LogManager.getLogger(ScheduleCompactionActionExecutor.class); - - private final Option> extraMetadata; - private final HoodieFlinkMergeOnReadTableCompactor compactor; - - public ScheduleCompactionsActionExecutor(HoodieEngineContext context, - HoodieWriteConfig config, - HoodieTable table, - String instantTime, - Option> extraMetadata, - HoodieFlinkMergeOnReadTableCompactor compactor) { - super(context, config, table, instantTime); - this.extraMetadata = extraMetadata; - this.compactor = compactor; - } - - @Override - public Option> execute() { - if (!config.getWriteConcurrencyMode().supportsOptimisticConcurrencyControl() - && !config.getFailedWritesCleanPolicy().isLazy()) { - // TODO(yihua): this validation is removed for Java client used by kafka-connect. Need to revisit this. - if (config.getEngineType() != EngineType.JAVA) { - // if there are inflight writes, their instantTime must not be less than that of compaction instant time - table.getActiveTimeline().getCommitsTimeline().filterPendingExcludingCompaction().firstInstant() - .ifPresent(earliestInflight -> ValidationUtils.checkArgument( - HoodieTimeline.compareTimestamps(earliestInflight.getTimestamp(), HoodieTimeline.GREATER_THAN, instantTime), - "Earliest write inflight instant time must be later than compaction time. Earliest :" + earliestInflight - + ", Compaction scheduled at " + instantTime)); - } - // Committed and pending compaction instants should have strictly lower timestamps - List conflictingInstants = table.getActiveTimeline() - .getWriteTimeline().filterCompletedAndCompactionInstants().getInstants() - .filter(instant -> HoodieTimeline.compareTimestamps( - instant.getTimestamp(), HoodieTimeline.GREATER_THAN_OR_EQUALS, instantTime)) - .collect(Collectors.toList()); - ValidationUtils.checkArgument(conflictingInstants.isEmpty(), - "Following instants have timestamps >= compactionInstant (" + instantTime + ") Instants :" - + conflictingInstants); - } - - List plans = scheduleCompaction(); - AtomicLong num = new AtomicLong(0); - plans.stream().forEach(plan -> { - if (plan != null && (plan.getOperations() != null) && (!plan.getOperations().isEmpty())) { - extraMetadata.ifPresent(plan::setExtraMetadata); - HoodieInstant compactionInstant = - new HoodieInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, String.valueOf(Long.valueOf(instantTime) + num.getAndAdd(1))); - try { - table.getActiveTimeline().saveToCompactionRequested(compactionInstant, - TimelineMetadataUtils.serializeCompactionPlan(plan)); - } catch (IOException ioe) { - throw new HoodieIOException("Exception scheduling compaction", ioe); - } - } - }); - return Option.of(plans); - } - - private List scheduleCompaction() { - LOG.info("Checking if compaction needs to be run on " + config.getBasePath()); - // judge if we need to compact according to num delta commits and time elapsed - boolean compactable = needCompact(config.getInlineCompactTriggerStrategy()); - if (compactable) { - LOG.info("Generating compaction plan for merge on read table " + config.getBasePath()); - try { - SyncableFileSystemView fileSystemView = (SyncableFileSystemView) table.getSliceView(); - Set fgInPendingCompactionAndClustering = fileSystemView.getPendingCompactionOperations() - .map(instantTimeOpPair -> instantTimeOpPair.getValue().getFileGroupId()) - .collect(Collectors.toSet()); - // exclude files in pending clustering from compaction. - fgInPendingCompactionAndClustering.addAll(fileSystemView.getFileGroupsInPendingClustering().map(Pair::getLeft).collect(Collectors.toSet())); - context.setJobStatus(this.getClass().getSimpleName(), "Compaction: generating compaction plan"); - return compactor.generateCompactionPlans(context, table, config, instantTime, fgInPendingCompactionAndClustering); - } catch (IOException e) { - throw new HoodieCompactionException("Could not schedule compaction " + config.getBasePath(), e); - } - } - return new ArrayList(); - } - - - private Pair getLatestDeltaCommitInfo() { - Option lastCompaction = table.getActiveTimeline().getCommitTimeline() - .filterCompletedInstants().lastInstant(); - HoodieTimeline deltaCommits = table.getActiveTimeline().getDeltaCommitTimeline(); - - String latestInstantTs; - final int deltaCommitsSinceLastCompaction; - if (lastCompaction.isPresent()) { - latestInstantTs = lastCompaction.get().getTimestamp(); - deltaCommitsSinceLastCompaction = deltaCommits.findInstantsAfter(latestInstantTs, Integer.MAX_VALUE).countInstants(); - } else { - - latestInstantTs = deltaCommits.firstInstant().get().getTimestamp(); - deltaCommitsSinceLastCompaction = deltaCommits.findInstantsAfterOrEquals(latestInstantTs, Integer.MAX_VALUE).countInstants(); - } - return Pair.of(deltaCommitsSinceLastCompaction, latestInstantTs); - } - - private boolean needCompact(CompactionTriggerStrategy compactionTriggerStrategy) { - boolean compactable; - // get deltaCommitsSinceLastCompaction and lastCompactionTs - Pair latestDeltaCommitInfo = getLatestDeltaCommitInfo(); - int inlineCompactDeltaCommitMax = config.getInlineCompactDeltaCommitMax(); - int inlineCompactDeltaSecondsMax = config.getInlineCompactDeltaSecondsMax(); - switch (compactionTriggerStrategy) { - case NUM_COMMITS: - compactable = inlineCompactDeltaCommitMax <= latestDeltaCommitInfo.getLeft(); - if (compactable) { - LOG.info(String.format("The delta commits >= %s, trigger compaction scheduler.", inlineCompactDeltaCommitMax)); - } - break; - case TIME_ELAPSED: - compactable = inlineCompactDeltaSecondsMax <= parsedToSeconds(instantTime) - parsedToSeconds(latestDeltaCommitInfo.getRight()); - if (compactable) { - LOG.info(String.format("The elapsed time >=%ss, trigger compaction scheduler.", inlineCompactDeltaSecondsMax)); - } - break; - case NUM_OR_TIME: - compactable = inlineCompactDeltaCommitMax <= latestDeltaCommitInfo.getLeft() - || inlineCompactDeltaSecondsMax <= parsedToSeconds(instantTime) - parsedToSeconds(latestDeltaCommitInfo.getRight()); - if (compactable) { - LOG.info(String.format("The delta commits >= %s or elapsed_time >=%ss, trigger compaction scheduler.", inlineCompactDeltaCommitMax, - inlineCompactDeltaSecondsMax)); - } - break; - case NUM_AND_TIME: - compactable = inlineCompactDeltaCommitMax <= latestDeltaCommitInfo.getLeft() - && inlineCompactDeltaSecondsMax <= parsedToSeconds(instantTime) - parsedToSeconds(latestDeltaCommitInfo.getRight()); - if (compactable) { - LOG.info(String.format("The delta commits >= %s and elapsed_time >=%ss, trigger compaction scheduler.", inlineCompactDeltaCommitMax, - inlineCompactDeltaSecondsMax)); - } - break; - default: - throw new HoodieCompactionException("Unsupported compaction trigger strategy: " + config.getInlineCompactTriggerStrategy()); - } - return compactable; - } - - private Long parsedToSeconds(String time) { - long timestamp; - try { - timestamp = HoodieActiveTimeline.parseDateFromInstantTime(time).getTime() / 1000; - } catch (ParseException e) { - throw new HoodieCompactionException(e.getMessage(), e); - } - return timestamp; - } -} diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/compact/ScheduleConstant.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/compact/ScheduleConstant.java deleted file mode 100644 index aae99723aec0e..0000000000000 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/compact/ScheduleConstant.java +++ /dev/null @@ -1,12 +0,0 @@ -package org.apache.hudi.table.action.compact; - -import java.util.concurrent.atomic.AtomicInteger; - -/** - * @Author ez - * @Description - * @createTime 2022/3/17 - */ -public class ScheduleConstant { - static AtomicInteger countDeltaCommit = new AtomicInteger(0); -} From 4caedfc668e0eda698249deb6453d4aa68ea9a3f Mon Sep 17 00:00:00 2001 From: yanenze Date: Mon, 23 May 2022 15:47:30 +0800 Subject: [PATCH 43/44] #improve the flink sink operator name for better identify tables to write --- .../org/apache/hudi/sink/utils/Pipelines.java | 18 +++++++++--------- 1 file changed, 9 insertions(+), 9 deletions(-) diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/utils/Pipelines.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/utils/Pipelines.java index 91ac2beadc080..94b3132481f91 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/utils/Pipelines.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/utils/Pipelines.java @@ -114,7 +114,7 @@ public static DataStreamSink bulkInsert(Configuration conf, RowType rowT conf.getInteger(FlinkOptions.WRITE_SORT_MEMORY) * 1024L * 1024L); } return dataStream - .transform("bucket_bulk_insert", TypeInformation.of(Object.class), operatorFactory) + .transform("bucket_bulk_insert" + ":" + conf.getString(FlinkOptions.TABLE_NAME), TypeInformation.of(Object.class), operatorFactory) .uid("uid_bucket_bulk_insert" + conf.getString(FlinkOptions.TABLE_NAME)) .setParallelism(conf.getInteger(FlinkOptions.WRITE_TASKS)) .addSink(DummySink.INSTANCE) @@ -137,7 +137,7 @@ public static DataStreamSink bulkInsert(Configuration conf, RowType rowT SortOperatorGen sortOperatorGen = new SortOperatorGen(rowType, partitionFields); // sort by partition keys dataStream = dataStream - .transform("partition_key_sorter", + .transform("partition_key_sorter" + ":" + conf.getString(FlinkOptions.TABLE_NAME), TypeInformation.of(RowData.class), sortOperatorGen.createSortOperator()) .setParallelism(conf.getInteger(FlinkOptions.WRITE_TASKS)); @@ -146,7 +146,7 @@ public static DataStreamSink bulkInsert(Configuration conf, RowType rowT } } return dataStream - .transform("hoodie_bulk_insert_write", + .transform("hoodie_bulk_insert_write" + ":" + conf.getString(FlinkOptions.TABLE_NAME), TypeInformation.of(Object.class), operatorFactory) // follow the parallelism of upstream operators to avoid shuffle @@ -190,7 +190,7 @@ public static DataStreamSink append( WriteOperatorFactory operatorFactory = AppendWriteOperator.getFactory(conf, rowType); return dataStream - .transform("hoodie_append_write", TypeInformation.of(Object.class), operatorFactory) + .transform("hoodie_append_write" + ":" + conf.getString(FlinkOptions.TABLE_NAME), TypeInformation.of(Object.class), operatorFactory) .uid("uid_hoodie_stream_write" + conf.getString(FlinkOptions.TABLE_NAME)) .setParallelism(conf.getInteger(FlinkOptions.WRITE_TASKS)) .addSink(DummySink.INSTANCE) @@ -250,7 +250,7 @@ private static DataStream streamBootstrap( if (conf.getBoolean(FlinkOptions.INDEX_BOOTSTRAP_ENABLED) || bounded) { dataStream1 = dataStream1 .transform( - "index_bootstrap", + "index_bootstrap" + ":" + conf.getString(FlinkOptions.TABLE_NAME), TypeInformation.of(HoodieRecord.class), new BootstrapOperator<>(conf)) .setParallelism(conf.getOptional(FlinkOptions.INDEX_BOOTSTRAP_TASKS).orElse(defaultParallelism)) @@ -277,7 +277,7 @@ private static DataStream boundedBootstrap( return rowDataToHoodieRecord(conf, rowType, dataStream) .transform( - "batch_index_bootstrap", + "batch_index_bootstrap" + ":" + conf.getString(FlinkOptions.TABLE_NAME), TypeInformation.of(HoodieRecord.class), new BatchBootstrapOperator<>(conf)) .setParallelism(conf.getOptional(FlinkOptions.INDEX_BOOTSTRAP_TASKS).orElse(defaultParallelism)) @@ -322,7 +322,7 @@ public static DataStream hoodieStreamWrite(Configuration conf, int defau String indexKeyFields = conf.getString(FlinkOptions.INDEX_KEY_FIELD); BucketIndexPartitioner partitioner = new BucketIndexPartitioner<>(bucketNum, indexKeyFields); return dataStream.partitionCustom(partitioner, HoodieRecord::getKey) - .transform("bucket_write", TypeInformation.of(Object.class), operatorFactory) + .transform("bucket_write" + ":" + conf.getString(FlinkOptions.TABLE_NAME), TypeInformation.of(Object.class), operatorFactory) .uid("uid_bucket_write" + conf.getString(FlinkOptions.TABLE_NAME)) .setParallelism(conf.getInteger(FlinkOptions.WRITE_TASKS)); } else { @@ -331,7 +331,7 @@ public static DataStream hoodieStreamWrite(Configuration conf, int defau // Key-by record key, to avoid multiple subtasks write to a bucket at the same time .keyBy(HoodieRecord::getRecordKey) .transform( - "bucket_assigner", + "bucket_assigner" + ":" + conf.getString(FlinkOptions.TABLE_NAME), TypeInformation.of(HoodieRecord.class), new KeyedProcessOperator<>(new BucketAssignFunction<>(conf))) .uid("uid_bucket_assigner_" + conf.getString(FlinkOptions.TABLE_NAME)) @@ -365,7 +365,7 @@ public static DataStream hoodieStreamWrite(Configuration conf, int defau * @return the compaction pipeline */ public static DataStreamSink compact(Configuration conf, DataStream dataStream) { - return dataStream.transform("compact_plan_generate", + return dataStream.transform("compact_plan_generate" + ":" + conf.getString(FlinkOptions.TABLE_NAME), TypeInformation.of(CompactionPlanEvent.class), new CompactionPlanOperator(conf)) .setParallelism(1) // plan generate must be singleton From 297f9361285f532e528e3a20d5b1e9eb7c8d55b6 Mon Sep 17 00:00:00 2001 From: yanenze Date: Fri, 3 Jun 2022 11:58:36 +0800 Subject: [PATCH 44/44] #improve the flink sink operator name for better identify tables to write --- .../org/apache/hudi/sink/utils/Pipelines.java | 24 +++++++++++-------- .../apache/hudi/utils/TestStreamerUtil.java | 15 ++++++++---- 2 files changed, 24 insertions(+), 15 deletions(-) diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/utils/Pipelines.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/utils/Pipelines.java index 94b3132481f91..c9e7ac545c881 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/utils/Pipelines.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/utils/Pipelines.java @@ -114,7 +114,7 @@ public static DataStreamSink bulkInsert(Configuration conf, RowType rowT conf.getInteger(FlinkOptions.WRITE_SORT_MEMORY) * 1024L * 1024L); } return dataStream - .transform("bucket_bulk_insert" + ":" + conf.getString(FlinkOptions.TABLE_NAME), TypeInformation.of(Object.class), operatorFactory) + .transform(writeOpIdentifier("bucket_bulk_insert", conf) , TypeInformation.of(Object.class), operatorFactory) .uid("uid_bucket_bulk_insert" + conf.getString(FlinkOptions.TABLE_NAME)) .setParallelism(conf.getInteger(FlinkOptions.WRITE_TASKS)) .addSink(DummySink.INSTANCE) @@ -137,7 +137,7 @@ public static DataStreamSink bulkInsert(Configuration conf, RowType rowT SortOperatorGen sortOperatorGen = new SortOperatorGen(rowType, partitionFields); // sort by partition keys dataStream = dataStream - .transform("partition_key_sorter" + ":" + conf.getString(FlinkOptions.TABLE_NAME), + .transform("partition_key_sorter", TypeInformation.of(RowData.class), sortOperatorGen.createSortOperator()) .setParallelism(conf.getInteger(FlinkOptions.WRITE_TASKS)); @@ -146,7 +146,7 @@ public static DataStreamSink bulkInsert(Configuration conf, RowType rowT } } return dataStream - .transform("hoodie_bulk_insert_write" + ":" + conf.getString(FlinkOptions.TABLE_NAME), + .transform(writeOpIdentifier("hoodie_bulk_insert_write", conf), TypeInformation.of(Object.class), operatorFactory) // follow the parallelism of upstream operators to avoid shuffle @@ -190,7 +190,7 @@ public static DataStreamSink append( WriteOperatorFactory operatorFactory = AppendWriteOperator.getFactory(conf, rowType); return dataStream - .transform("hoodie_append_write" + ":" + conf.getString(FlinkOptions.TABLE_NAME), TypeInformation.of(Object.class), operatorFactory) + .transform(writeOpIdentifier( "hoodie_append_write", conf), TypeInformation.of(Object.class), operatorFactory) .uid("uid_hoodie_stream_write" + conf.getString(FlinkOptions.TABLE_NAME)) .setParallelism(conf.getInteger(FlinkOptions.WRITE_TASKS)) .addSink(DummySink.INSTANCE) @@ -250,7 +250,7 @@ private static DataStream streamBootstrap( if (conf.getBoolean(FlinkOptions.INDEX_BOOTSTRAP_ENABLED) || bounded) { dataStream1 = dataStream1 .transform( - "index_bootstrap" + ":" + conf.getString(FlinkOptions.TABLE_NAME), + "index_bootstrap", TypeInformation.of(HoodieRecord.class), new BootstrapOperator<>(conf)) .setParallelism(conf.getOptional(FlinkOptions.INDEX_BOOTSTRAP_TASKS).orElse(defaultParallelism)) @@ -277,7 +277,7 @@ private static DataStream boundedBootstrap( return rowDataToHoodieRecord(conf, rowType, dataStream) .transform( - "batch_index_bootstrap" + ":" + conf.getString(FlinkOptions.TABLE_NAME), + "batch_index_bootstrap", TypeInformation.of(HoodieRecord.class), new BatchBootstrapOperator<>(conf)) .setParallelism(conf.getOptional(FlinkOptions.INDEX_BOOTSTRAP_TASKS).orElse(defaultParallelism)) @@ -322,7 +322,7 @@ public static DataStream hoodieStreamWrite(Configuration conf, int defau String indexKeyFields = conf.getString(FlinkOptions.INDEX_KEY_FIELD); BucketIndexPartitioner partitioner = new BucketIndexPartitioner<>(bucketNum, indexKeyFields); return dataStream.partitionCustom(partitioner, HoodieRecord::getKey) - .transform("bucket_write" + ":" + conf.getString(FlinkOptions.TABLE_NAME), TypeInformation.of(Object.class), operatorFactory) + .transform( writeOpIdentifier("bucket_write", conf), TypeInformation.of(Object.class), operatorFactory) .uid("uid_bucket_write" + conf.getString(FlinkOptions.TABLE_NAME)) .setParallelism(conf.getInteger(FlinkOptions.WRITE_TASKS)); } else { @@ -331,14 +331,14 @@ public static DataStream hoodieStreamWrite(Configuration conf, int defau // Key-by record key, to avoid multiple subtasks write to a bucket at the same time .keyBy(HoodieRecord::getRecordKey) .transform( - "bucket_assigner" + ":" + conf.getString(FlinkOptions.TABLE_NAME), + "bucket_assigner" , TypeInformation.of(HoodieRecord.class), new KeyedProcessOperator<>(new BucketAssignFunction<>(conf))) .uid("uid_bucket_assigner_" + conf.getString(FlinkOptions.TABLE_NAME)) .setParallelism(conf.getOptional(FlinkOptions.BUCKET_ASSIGN_TASKS).orElse(defaultParallelism)) // shuffle by fileId(bucket id) .keyBy(record -> record.getCurrentLocation().getFileId()) - .transform("stream_write", TypeInformation.of(Object.class), operatorFactory) + .transform(writeOpIdentifier("stream_write", conf) , TypeInformation.of(Object.class), operatorFactory) .uid("uid_stream_write" + conf.getString(FlinkOptions.TABLE_NAME)) .setParallelism(conf.getInteger(FlinkOptions.WRITE_TASKS)); } @@ -365,7 +365,7 @@ public static DataStream hoodieStreamWrite(Configuration conf, int defau * @return the compaction pipeline */ public static DataStreamSink compact(Configuration conf, DataStream dataStream) { - return dataStream.transform("compact_plan_generate" + ":" + conf.getString(FlinkOptions.TABLE_NAME), + return dataStream.transform("compact_plan_generate" , TypeInformation.of(CompactionPlanEvent.class), new CompactionPlanOperator(conf)) .setParallelism(1) // plan generate must be singleton @@ -385,6 +385,10 @@ public static DataStreamSink clean(Configuration conf, DataStream