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 8db0e0f3694c9..9e9793045fba2 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 @@ -25,6 +25,7 @@ import org.apache.hudi.async.AsyncArchiveService; import org.apache.hudi.async.AsyncCleanerService; import org.apache.hudi.avro.HoodieAvroUtils; +import org.apache.hudi.avro.model.HoodieBuildPlan; import org.apache.hudi.avro.model.HoodieCleanMetadata; import org.apache.hudi.avro.model.HoodieCleanerPlan; import org.apache.hudi.avro.model.HoodieClusteringPlan; @@ -45,6 +46,7 @@ import org.apache.hudi.common.HoodiePendingRollbackInfo; import org.apache.hudi.common.config.HoodieCommonConfig; import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.model.HoodieBuildCommitMetadata; import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy; import org.apache.hudi.common.model.HoodieKey; @@ -142,6 +144,7 @@ public abstract class BaseHoodieWriteClient cluster(String clusteringInstant, boolean shouldComplete); + /** + * Schedule a new build instant with passed-in instant time. + * + * @param instantTime Build instant time + * @param extraMetadata Extra metadata to be stored + * @return True if build instant scheduled successfully + * @throws HoodieIOException + */ + public boolean scheduleBuildAtInstant(String instantTime, Option> extraMetadata) throws HoodieIOException { + return scheduleTableService(instantTime, extraMetadata, TableServiceType.BUILD).isPresent(); + } + + /** + * Perform build for the plan stored in metadata + * + * @param instantTime Build instant time + * @param shouldComplete + * @return + */ + public abstract HoodieBuildCommitMetadata build(String instantTime, boolean shouldComplete); + /** * Schedule table services such as clustering, compaction & cleaning. * - * @param extraMetadata Metadata to pass onto the scheduled service instant + * @param extraMetadata Metadata to pass onto the scheduled service instant * @param tableServiceType Type of table service to schedule * @return */ @@ -1458,6 +1483,11 @@ private Option scheduleTableServiceInternal(String instantTime, Option cleanerPlan = createTable(config, hadoopConf) .scheduleCleaning(context, instantTime, extraMetadata); return cleanerPlan.isPresent() ? Option.of(instantTime) : Option.empty(); + case BUILD: + LOG.info("Scheduling build at instant time: " + instantTime); + Option buildPlan = createTable(config, hadoopConf) + .scheduleBuild(context, instantTime, extraMetadata); + return buildPlan.isPresent() ? Option.of(instantTime) : Option.empty(); default: throw new IllegalArgumentException("Invalid TableService " + tableServiceType); } @@ -1488,9 +1518,9 @@ protected Option inlineScheduleClustering(Option> ex /** * Finalize Write operation. * - * @param table HoodieTable + * @param table HoodieTable * @param instantTime Instant Time - * @param stats Hoodie Write Stat + * @param stats Hoodie Write Stat */ protected void finalizeWrite(HoodieTable table, String instantTime, List stats) { try { 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 466bfeaeba15a..24ac714c6795d 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 @@ -25,6 +25,7 @@ import org.apache.hudi.common.config.ConfigClassProperty; import org.apache.hudi.common.config.ConfigGroups; import org.apache.hudi.common.config.ConfigProperty; +import org.apache.hudi.common.config.HoodieBuildConfig; import org.apache.hudi.common.config.HoodieCommonConfig; import org.apache.hudi.common.config.HoodieConfig; import org.apache.hudi.common.config.HoodieMetadataConfig; @@ -2216,6 +2217,10 @@ public boolean areReleaseResourceEnabled() { return getBooleanOrDefault(RELEASE_RESOURCE_ENABLE); } + public String getBuildPartitionSelected() { + return getString(HoodieBuildConfig.PARTITION_SELECTED); + } + /** * Layout configs. */ diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/HoodieMetrics.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/HoodieMetrics.java index 7b34bb481763f..878ffd3f8d947 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/HoodieMetrics.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/HoodieMetrics.java @@ -48,6 +48,7 @@ public class HoodieMetrics { private String conflictResolutionTimerName = null; private String conflictResolutionSuccessCounterName = null; private String conflictResolutionFailureCounterName = null; + public String buildTimerName = null; private HoodieWriteConfig config; private String tableName; private Timer rollbackTimer = null; @@ -62,6 +63,7 @@ public class HoodieMetrics { private Timer conflictResolutionTimer = null; private Counter conflictResolutionSuccessCounter = null; private Counter conflictResolutionFailureCounter = null; + private Timer buildTimer = null; public HoodieMetrics(HoodieWriteConfig config) { this.config = config; @@ -80,6 +82,7 @@ public HoodieMetrics(HoodieWriteConfig config) { this.conflictResolutionTimerName = getMetricsName("timer", "conflict_resolution"); this.conflictResolutionSuccessCounterName = getMetricsName("counter", "conflict_resolution.success"); this.conflictResolutionFailureCounterName = getMetricsName("counter", "conflict_resolution.failure"); + this.buildTimerName = getMetricsName("timer", HoodieTimeline.BUILD_ACTION); } } @@ -157,6 +160,13 @@ public Timer.Context getConflictResolutionCtx() { return conflictResolutionTimer == null ? null : conflictResolutionTimer.time(); } + public Timer.Context getBuildCtx() { + if (config.isMetricsOn() && buildTimer == null) { + buildTimer = createTimer(indexTimerName); + } + return buildTimer == null ? null : buildTimer.time(); + } + public void updateMetricsForEmptyData(String actionType) { if (!config.isMetricsOn() || !config.getMetricsReporterType().equals(MetricsReporterType.PROMETHEUS_PUSHGATEWAY)) { // No-op if metrics are not of type PROMETHEUS_PUSHGATEWAY. @@ -264,6 +274,13 @@ public void updateIndexMetrics(final String action, final long durationInMs) { } } + public void updateBuildMetrics(final String action, final long durationInMs) { + if (config.isMetricsOn()) { + LOG.info(String.format("Sending build metrics (%s.duration, %d)", action, durationInMs)); + Metrics.registerGauge(getMetricsName(HoodieTimeline.BUILD_ACTION, String.format("%s.duration", action)), durationInMs); + } + } + String getMetricsName(String action, String metric) { return config == null ? null : String.format("%s.%s.%s", config.getMetricReporterMetricsNamePrefix(), action, metric); } 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 e3045fbece837..52f83cc55ab7b 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 @@ -19,6 +19,7 @@ package org.apache.hudi.table; import org.apache.hudi.avro.HoodieAvroUtils; +import org.apache.hudi.avro.model.HoodieBuildPlan; import org.apache.hudi.avro.model.HoodieCleanMetadata; import org.apache.hudi.avro.model.HoodieCleanerPlan; import org.apache.hudi.avro.model.HoodieClusteringPlan; @@ -41,6 +42,7 @@ import org.apache.hudi.common.fs.ConsistencyGuardConfig; import org.apache.hudi.common.fs.FailSafeConsistencyGuard; import org.apache.hudi.common.fs.OptimisticConsistencyGuard; +import org.apache.hudi.common.model.HoodieBuildCommitMetadata; import org.apache.hudi.common.model.HoodieFileFormat; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecordPayload; @@ -559,8 +561,29 @@ public abstract HoodieRestoreMetadata restore(HoodieEngineContext context, * Schedules Restore for the table to the given instant. */ public abstract Option scheduleRestore(HoodieEngineContext context, - String restoreInstantTime, - String instantToRestore); + String restoreInstantTime, + String instantToRestore); + + /** + * Schedule build at passed-in instant time + * + * @param context HoodieEngineContext + * @param instantTime Instant time for scheduling build + * @param extraMetadata Additional metadata to write into plan + * @return Build plan + */ + public abstract Option scheduleBuild(HoodieEngineContext context, + String instantTime, + Option> extraMetadata); + + /** + * Execute build for the table + * + * @param context HoodieEngineContext + * @param instantTime Build instant time + * @return HoodieWriteMetadata + */ + public abstract HoodieBuildCommitMetadata build(HoodieEngineContext context, String instantTime); public void rollbackInflightCompaction(HoodieInstant inflightInstant) { rollbackInflightCompaction(inflightInstant, s -> Option.empty()); @@ -594,6 +617,18 @@ public void rollbackInflightClustering(HoodieInstant inflightInstant, rollbackInflightInstant(inflightInstant, getPendingRollbackInstantFunc); } + /** + * Rollback inflight build instant to requested build instant + * + * @param inflightInstant Inflight build instant + * @param getPendingRollbackInstantFunc Function to get rollback instant + */ + public void rollbackInflightBuild(HoodieInstant inflightInstant, + Function> getPendingRollbackInstantFunc) { + ValidationUtils.checkArgument(inflightInstant.getAction().equals(HoodieTimeline.BUILD_ACTION)); + rollbackInflightInstant(inflightInstant, getPendingRollbackInstantFunc); + } + /** * Rollback inflight instant to requested instant * diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/build/BuildPlanActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/build/BuildPlanActionExecutor.java new file mode 100644 index 0000000000000..52baaf9213026 --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/build/BuildPlanActionExecutor.java @@ -0,0 +1,219 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.hudi.table.action.build; + +import org.apache.hudi.avro.model.HoodieBuildPlan; +import org.apache.hudi.avro.model.HoodieBuildTask; +import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.model.FileSlice; +import org.apache.hudi.common.model.HoodieRecordPayload; +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.CollectionUtils; +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.exception.HoodieIOException; +import org.apache.hudi.secondary.index.HoodieSecondaryIndex; +import org.apache.hudi.secondary.index.SecondaryIndexUtils; +import org.apache.hudi.table.HoodieTable; +import org.apache.hudi.table.action.BaseActionExecutor; + +import org.apache.hadoop.fs.Path; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.function.Predicate; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +/** + * Create plan for build action, the main metadata for plan is organized + * as follows: + * @formatter:off + * { + * tasks: [ + * (baseFilePathA, [secondaryIndexM, secondaryIndexN, ...]), + * (baseFilePathB, [secondaryIndexN, secondaryIndexY, ...]), + * ... + * ], + * ... + * } + * @formatter:on + * + * For every base file, there is one build action task, and the task knows + * which secondary indexes should be generated, so we can read the file + * once to build all the relevant secondary indexes. + * + * todo: scenarios as follows can be optimized: + * t1: write commit + * t2: generate build plan => buildTaskSetA + * t3: write commit + * t4: generate build plan => buildTaskSetB + * + * Some tasks in buildTaskSetA may out of date for there are new file slices + * generated in the same file group when scheduling build at t4 + */ +public class BuildPlanActionExecutor + extends BaseActionExecutor> { + private static final Logger LOG = LoggerFactory.getLogger(BuildPlanActionExecutor.class); + + private static final int BUILD_PLAN_VERSION_1 = 1; + public static final int CURRENT_BUILD_PLAN_VERSION = BUILD_PLAN_VERSION_1; + + private final Option> extraMetadata; + + public BuildPlanActionExecutor( + HoodieEngineContext context, + HoodieWriteConfig config, + HoodieTable table, + String instantTime, + Option> extraMetadata) { + super(context, config, table, instantTime); + this.extraMetadata = extraMetadata; + } + + @Override + public Option execute() { + Option buildPlan = createBuildPlan(); + if (buildPlan.isPresent()) { + HoodieInstant buildInstant = + new HoodieInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.BUILD_ACTION, instantTime); + try { + table.getActiveTimeline().saveToPendingBuildAction(buildInstant, + TimelineMetadataUtils.serializeBuildPlan(buildPlan.get())); + } catch (IOException e) { + throw new HoodieIOException("Generate build plan failed", e); + } + } + + return buildPlan; + } + + private Option createBuildPlan() { + Option> secondaryIndexes = + SecondaryIndexUtils.getSecondaryIndexes(table.getMetaClient()); + if (!secondaryIndexes.isPresent() || CollectionUtils.isNullOrEmpty(secondaryIndexes.get())) { + LOG.info("No secondary index defined for this table: {}", + table.getMetaClient().getTableConfig().getTableName()); + return Option.empty(); + } + + List partitionPaths = FSUtils.getAllPartitionPaths( + context, table.getConfig().getMetadataConfig(), table.getMetaClient().getBasePathV2().toString()); + String partitionSelectedStr = table.getConfig().getBuildPartitionSelected(); + if (!StringUtils.isNullOrEmpty(partitionSelectedStr)) { + List partitionSelected = Arrays.asList(partitionSelectedStr.trim().split(",")); + partitionPaths = partitionPaths.stream().filter(partitionSelected::contains).collect(Collectors.toList()); + } + + if (CollectionUtils.isNullOrEmpty(partitionPaths)) { + LOG.info("No partition needs to build secondary index"); + return Option.empty(); + } + + // Notice: here we won't check whether there are new commits since last build action + // for new indexes may be added during this time + + List buildTasks = partitionPaths.stream() + .flatMap(partitionPath -> + secondaryIndexes.get().stream().flatMap(secondaryIndex -> + getEligibleFileSlices(partitionPath, secondaryIndex) + .map(fileSlice -> Pair.of(secondaryIndex, fileSlice)))) + .collect(Collectors.groupingBy(Pair::getRight, + Collectors.mapping(Pair::getLeft, Collectors.toList()))) + .entrySet().stream() + .map(entry -> { + // Convert List to json string + String indexMetas = SecondaryIndexUtils.toJsonString(entry.getValue()); + + String partitionPath = entry.getKey().getPartitionPath(); + String baseFileName = entry.getKey().getBaseFile().get().getFileName(); + String filePath; + if (StringUtils.isNullOrEmpty(partitionPath)) { + filePath = new Path(baseFileName).toString(); + } else { + filePath = new Path(partitionPath, baseFileName).toString(); + } + return HoodieBuildTask.newBuilder() + .setPartition(entry.getKey().getPartitionPath()) + .setBaseFilePath(filePath) + .setIndexMetas(indexMetas) + .setVersion(CURRENT_BUILD_PLAN_VERSION) + .build(); + }).collect(Collectors.toList()); + + // No new commits since last build and no indexes be added during this time + if (CollectionUtils.isNullOrEmpty(buildTasks)) { + return Option.empty(); + } + + HoodieBuildPlan buildPlan = HoodieBuildPlan.newBuilder() + .setTasks(buildTasks) + .setExtraMetadata(extraMetadata.orElse(Collections.emptyMap())) + .setVersion(CURRENT_BUILD_PLAN_VERSION) + .build(); + return Option.ofNullable(buildPlan); + } + + private Stream getEligibleFileSlices( + String partitionPath, HoodieSecondaryIndex secondaryIndex) { + SyncableFileSystemView fileSystemView = (SyncableFileSystemView) table.getSliceView(); + Stream pendingBaseFilePaths = fileSystemView.getPendingSecondaryIndexBaseFiles() + .filter(pair -> pair.getLeft().equals(secondaryIndex)) + .flatMap(pair -> pair.getValue().keySet().stream()); + + Stream completedBaseFilePaths = fileSystemView.getSecondaryIndexBaseFiles() + .filter(pair -> pair.getLeft().equals(secondaryIndex)) + .flatMap(pair -> pair.getRight().keySet().stream()); + + Set baseFilePaths = Stream.concat(pendingBaseFilePaths, completedBaseFilePaths) + .collect(Collectors.toSet()); + + // The file slices which have base file and base file not exists in + // pending/completed secondary index are eligible. + Predicate predicate = fileSlice -> { + if (fileSlice.getBaseFile().isPresent()) { + String baseFilePath; + if (StringUtils.isNullOrEmpty(fileSlice.getPartitionPath())) { + baseFilePath = fileSlice.getBaseFile().get().getFileName(); + } else { + baseFilePath = fileSlice.getPartitionPath() + "/" + fileSlice.getBaseFile().get().getFileName(); + } + + return !baseFilePaths.contains(baseFilePath); + } + + return false; + }; + + return table.getSliceView().getLatestFileSlices(partitionPath).filter(predicate); + } +} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/build/BuildTaskExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/build/BuildTaskExecutor.java new file mode 100644 index 0000000000000..0bf2a402f3186 --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/build/BuildTaskExecutor.java @@ -0,0 +1,242 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.hudi.table.action.build; + +import org.apache.hudi.avro.HoodieAvroUtils; +import org.apache.hudi.avro.model.HoodieBuildTask; +import org.apache.hudi.common.config.HoodieBuildTaskConfig; +import org.apache.hudi.common.config.SerializableConfiguration; +import org.apache.hudi.common.config.SerializableSchema; +import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.model.BuildStatus; +import org.apache.hudi.common.util.BuildUtils; +import org.apache.hudi.common.util.ClosableIterator; +import org.apache.hudi.common.util.CollectionUtils; +import org.apache.hudi.common.util.HoodieTimer; +import org.apache.hudi.common.util.ValidationUtils; +import org.apache.hudi.exception.HoodieBuildException; +import org.apache.hudi.exception.HoodieIOException; +import org.apache.hudi.io.storage.HoodieFileReaderFactory; +import org.apache.hudi.secondary.index.HoodieSecondaryIndex; +import org.apache.hudi.secondary.index.SecondaryIndexBuilder; +import org.apache.hudi.secondary.index.SecondaryIndexFactory; +import org.apache.hudi.secondary.index.SecondaryIndexType; +import org.apache.hudi.secondary.index.SecondaryIndexUtils; + +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericRecord; +import org.apache.avro.generic.IndexedRecord; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.LinkedHashMap; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +public class BuildTaskExecutor { + private static final Logger LOG = LoggerFactory.getLogger(BuildTaskExecutor.class); + + private final String partition; + private final Path baseFilePath; + private final String indexFolderPath; + private final List combinedSecondaryIndices; + private final List indexBuilders; + private final ClosableIterator recordIterator; + + private final int batchSize = 1000; + private final GenericRecord[] reusedRecords; + + private long totalRecords = 0; + private final HoodieTimer buildTimer; + + public BuildTaskExecutor(HoodieBuildTask buildTask, + String basePath, + String indexFolderPath, + SerializableSchema schema, + SerializableConfiguration conf) { + this.partition = buildTask.getPartition(); + this.baseFilePath = new Path(buildTask.getBaseFilePath()); + this.indexFolderPath = indexFolderPath; + + Schema avroSchema = schema.get(); + Configuration hadoopConf = conf.get(); + + List secondaryIndices = SecondaryIndexUtils.fromJsonString(buildTask.getIndexMetas()); + combinedSecondaryIndices = combineSecondaryIndices(secondaryIndices); + // Init index builder for every combined secondary indices + this.indexBuilders = combinedSecondaryIndices.stream().map(indexInfo -> + initBuilderForIndex(indexInfo, avroSchema, hadoopConf) + ).collect(Collectors.toList()); + + // Create record iterator for this base file + Schema readerSchema = HoodieAvroUtils.addMetadataFields(avroSchema); + try { + Path baseFileFullPath = new Path(basePath, buildTask.getBaseFilePath()); + recordIterator = HoodieFileReaderFactory.getFileReader(hadoopConf, baseFileFullPath) + .getRecordIterator(readerSchema); + } catch (IOException e) { + throw new HoodieBuildException("Fail to get record iterator for: " + baseFilePath, e); + } + + reusedRecords = new GenericRecord[batchSize]; + buildTimer = new HoodieTimer(); + + LOG.info("Init build task executor ok, basePath:{}, file: {}, indexSaveDir: {}, batchSize: {}", + basePath, baseFilePath, indexFolderPath, batchSize); + } + + public BuildStatus execute() { + buildTimer.startTimer(); + int recordNum = 0; + while (recordIterator.hasNext()) { + GenericRecord next = (GenericRecord) recordIterator.next(); + reusedRecords[recordNum++] = next; + + if (recordNum == batchSize) { + addBatch(reusedRecords, recordNum); + totalRecords += recordNum; + recordNum = 0; + } + } + + if (recordNum != 0) { + addBatch(reusedRecords, recordNum); + totalRecords += recordNum; + } + + // Close all index builders + indexBuilders.forEach(SecondaryIndexBuilder::close); + LOG.info("Finish building indexes for file: {}, timeCost: {}ms", + baseFilePath, buildTimer.endTimer()); + + return BuildStatus.builder() + .setPartition(partition) + .setBaseFilePath(baseFilePath.toString()) + .setTotalRecords(totalRecords) + .setSecondaryIndexes(combinedSecondaryIndices) + .build(); + } + + /** + * Different fields to build lucene secondary index can be combined as a single lucene + * secondary index to reduce index data files and simplify the query logic. So here we + * try to combine multi lucene secondary indices to one lucene secondary index, the + * index name of combined lucene secondary index is the first lucene secondary index, + * and combined lucene secondary index has all origin lucene secondary indices' options. + * + * @param secondaryIndices All secondary indices need build for this file + * @return Combined secondary indices + */ + private List combineSecondaryIndices(List secondaryIndices) { + List combinedSecondaryIndices = new ArrayList<>(); + Map> groupedIndices = + secondaryIndices.stream().collect(Collectors.groupingBy(HoodieSecondaryIndex::getIndexType)); + List luceneIndices = groupedIndices.get(SecondaryIndexType.LUCENE); + if (!CollectionUtils.isNullOrEmpty(luceneIndices)) { + LinkedHashMap> columns = new LinkedHashMap<>(); + Map options = new HashMap<>(); + luceneIndices.forEach(index -> { + columns.putAll(index.getColumns()); + options.putAll(index.getOptions()); + }); + + HoodieSecondaryIndex secondaryIndex = HoodieSecondaryIndex.builder() + .setIndexName(luceneIndices.get(0).getIndexName()) + .setColumns(columns) + .setIndexType(SecondaryIndexType.LUCENE.name()) + .setOptions(options) + .build(); + + combinedSecondaryIndices.add(secondaryIndex); + groupedIndices.remove(SecondaryIndexType.LUCENE); + } + groupedIndices.values().forEach(combinedSecondaryIndices::addAll); + + return combinedSecondaryIndices; + } + + /** + * Init index builder for given secondary index + * + * @param secondaryIndex HoodieSecondaryIndex + * @param schema Avro schema for this table + * @param conf Hadoop conf + * @return IndexBuilder + */ + private SecondaryIndexBuilder initBuilderForIndex( + HoodieSecondaryIndex secondaryIndex, + Schema schema, + Configuration conf) { + SecondaryIndexType indexType = secondaryIndex.getIndexType(); + + LinkedList indexFields = new LinkedList<>(); + secondaryIndex.getColumns().forEach((colName, options) -> { + Schema.Field field = schema.getField(colName); + ValidationUtils.checkArgument(field != null, + "Field not exists: " + colName + ", schema: " + schema); + indexFields.add(field); + }); + + Path indexSaveDir = BuildUtils.getIndexSaveDir(indexFolderPath, indexType.name(), baseFilePath.getName()); + try { + FileSystem fs = FSUtils.getFs(indexSaveDir, conf); + if (!fs.exists(indexSaveDir)) { + fs.mkdirs(indexSaveDir); + LOG.info("Create index save dir ok: {}", indexSaveDir); + } + } catch (IOException e) { + throw new HoodieBuildException("Fail to make index save dir"); + } + + HoodieBuildTaskConfig config = HoodieBuildTaskConfig.builder() + .setIndexSaveDir(indexSaveDir.toString()) + .setIndexType(indexType) + .setIndexFields(indexFields) + .setConf(conf) + .build(); + + return SecondaryIndexFactory.getIndexBuilder(config); + } + + /** + * Batch add records to index builders + * + * @param records Records to be added + * @param size The size of records + */ + private void addBatch(GenericRecord[] records, int size) { + indexBuilders.forEach(indexBuilder -> { + try { + indexBuilder.addBatch(records, size); + } catch (IOException e) { + throw new HoodieIOException("Add records to index builder failed, baseFile: " + + baseFilePath + ", builderName: " + indexBuilder.getName(), e); + } + }); + } +} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseCommitActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseCommitActionExecutor.java index 5f406d8920165..2b4c0cbd019e7 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseCommitActionExecutor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseCommitActionExecutor.java @@ -157,7 +157,6 @@ protected String getCommitActionType() { return table.getMetaClient().getCommitActionType(); } - /** * Check if any validators are configured and run those validations. If any of the validations fail, throws HoodieValidationException. */ diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/BaseRollbackActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/BaseRollbackActionExecutor.java index eb888bdec7730..d58fb1160f432 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/BaseRollbackActionExecutor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/BaseRollbackActionExecutor.java @@ -205,8 +205,11 @@ public List doRollbackAndGetStats(HoodieRollbackPlan hoodieR final boolean isPendingClustering = Objects.equals(HoodieTimeline.REPLACE_COMMIT_ACTION, instantToRollback.getAction()) && !instantToRollback.isCompleted() && ClusteringUtils.getClusteringPlan(table.getMetaClient(), instantToRollback).isPresent(); + + final boolean isPendingBuild = Objects.equals(HoodieTimeline.BUILD_ACTION, instantToRollback.getAction()) + && !instantToRollback.isCompleted(); validateSavepointRollbacks(); - if (!isPendingCompaction && !isPendingClustering) { + if (!isPendingCompaction && !isPendingClustering && !isPendingBuild) { validateRollbackCommitSequence(); } 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 d3578a8cffa36..c60451d687aa0 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 @@ -24,6 +24,7 @@ import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.FileSlice; +import org.apache.hudi.common.model.HoodieBuildCommitMetadata; import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; @@ -491,6 +492,11 @@ private void completeClustering( LOG.info("Clustering successfully on commit " + clusteringCommitTime); } + @Override + public HoodieBuildCommitMetadata build(String instantTime, boolean shouldComplete) { + throw new HoodieNotSupportedException("Build is not supported in HoodieFlinkWriteClient"); + } + @Override protected HoodieTable doInitTable(HoodieTableMetaClient metaClient, Option instantTime, boolean initialMetadataTableIfNecessary) { // Create a Hoodie table which encapsulated the commits and files visible 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 2608161ab0954..1e93e00badf56 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 @@ -18,6 +18,7 @@ package org.apache.hudi.table; +import org.apache.hudi.avro.model.HoodieBuildPlan; import org.apache.hudi.avro.model.HoodieCleanMetadata; import org.apache.hudi.avro.model.HoodieCleanerPlan; import org.apache.hudi.avro.model.HoodieClusteringPlan; @@ -33,6 +34,7 @@ import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieBaseFile; +import org.apache.hudi.common.model.HoodieBuildCommitMetadata; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; @@ -414,4 +416,14 @@ public Iterator> handleInsert( createHandle.write(); return Collections.singletonList(createHandle.close()).iterator(); } + + @Override + public Option scheduleBuild(HoodieEngineContext context, String instantTime, Option> extraMetadata) { + throw new HoodieNotSupportedException("Schedule build is not supported for flink table yet"); + } + + @Override + public HoodieBuildCommitMetadata build(HoodieEngineContext context, String instantTime) { + throw new HoodieNotSupportedException("Build is not supported for flink table yet"); + } } 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 b6951bc6b7874..421f9a18154f7 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 @@ -22,6 +22,7 @@ import org.apache.hudi.client.embedded.EmbeddedTimelineService; import org.apache.hudi.common.data.HoodieListData; import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.model.HoodieBuildCommitMetadata; import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; @@ -220,7 +221,7 @@ protected void completeCompaction(HoodieCommitMetadata metadata, @Override protected HoodieWriteMetadata> compact(String compactionInstantTime, - boolean shouldComplete) { + boolean shouldComplete) { throw new HoodieNotSupportedException("Compact is not supported in HoodieJavaClient"); } @@ -229,6 +230,11 @@ public HoodieWriteMetadata> cluster(final String clusteringIns throw new HoodieNotSupportedException("Cluster is not supported in HoodieJavaClient"); } + @Override + public HoodieBuildCommitMetadata build(String instantTime, boolean shouldComplete) { + throw new HoodieNotSupportedException("Build is not supported in HoodieJavaClient"); + } + @Override protected HoodieTable doInitTable(HoodieTableMetaClient metaClient, Option instantTime, boolean initialMetadataTableIfNecessary) { // new JavaUpgradeDowngrade(metaClient, config, context).run(metaClient, HoodieTableVersion.current(), config, context, instantTime); diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/HoodieJavaCopyOnWriteTable.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/HoodieJavaCopyOnWriteTable.java index c98d30adb7858..9c31b85da66e1 100644 --- a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/HoodieJavaCopyOnWriteTable.java +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/HoodieJavaCopyOnWriteTable.java @@ -18,6 +18,7 @@ package org.apache.hudi.table; +import org.apache.hudi.avro.model.HoodieBuildPlan; import org.apache.hudi.avro.model.HoodieCleanMetadata; import org.apache.hudi.avro.model.HoodieCleanerPlan; import org.apache.hudi.avro.model.HoodieClusteringPlan; @@ -33,6 +34,7 @@ import org.apache.hudi.client.common.HoodieJavaEngineContext; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieBaseFile; +import org.apache.hudi.common.model.HoodieBuildCommitMetadata; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; @@ -49,6 +51,8 @@ import org.apache.hudi.metadata.MetadataPartitionType; import org.apache.hudi.table.action.HoodieWriteMetadata; import org.apache.hudi.table.action.bootstrap.HoodieBootstrapWriteMetadata; +import org.apache.hudi.table.action.build.BuildPlanActionExecutor; +import org.apache.hudi.table.action.build.JavaExecuteBuildCommitActionExecutor; import org.apache.hudi.table.action.clean.CleanActionExecutor; import org.apache.hudi.table.action.clean.CleanPlanActionExecutor; import org.apache.hudi.table.action.cluster.ClusteringPlanActionExecutor; @@ -70,6 +74,7 @@ import org.apache.hudi.table.action.rollback.CopyOnWriteRollbackActionExecutor; import org.apache.hudi.table.action.rollback.RestorePlanActionExecutor; import org.apache.hudi.table.action.savepoint.SavepointActionExecutor; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -312,4 +317,14 @@ public Iterator> handleInsert( createHandle.write(); return Collections.singletonList(createHandle.close()).iterator(); } + + @Override + public Option scheduleBuild(HoodieEngineContext context, String instantTime, Option> extraMetadata) { + return new BuildPlanActionExecutor<>(context, config, this, instantTime, extraMetadata).execute(); + } + + @Override + public HoodieBuildCommitMetadata build(HoodieEngineContext context, String instantTime) { + return new JavaExecuteBuildCommitActionExecutor<>(context, config, this, instantTime).execute(); + } } diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/build/JavaExecuteBuildCommitActionExecutor.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/build/JavaExecuteBuildCommitActionExecutor.java new file mode 100644 index 0000000000000..26027909ea3ec --- /dev/null +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/build/JavaExecuteBuildCommitActionExecutor.java @@ -0,0 +1,84 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.hudi.table.action.build; + +import org.apache.hudi.avro.model.HoodieBuildPlan; +import org.apache.hudi.avro.model.HoodieBuildTask; +import org.apache.hudi.common.config.SerializableConfiguration; +import org.apache.hudi.common.config.SerializableSchema; +import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.model.BuildStatus; +import org.apache.hudi.common.model.HoodieBuildCommitMetadata; +import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.table.TableSchemaResolver; +import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.util.BuildUtils; +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.exception.HoodieBuildException; +import org.apache.hudi.table.HoodieTable; +import org.apache.hudi.table.action.BaseActionExecutor; + +import org.apache.avro.Schema; + +import java.util.ArrayList; +import java.util.List; + +public class JavaExecuteBuildCommitActionExecutor extends BaseActionExecutor { + + private HoodieBuildPlan buildPlan; + + public JavaExecuteBuildCommitActionExecutor(HoodieEngineContext context, HoodieWriteConfig config, HoodieTable table, String instantTime) { + super(context, config, table, instantTime); + this.buildPlan = BuildUtils.getBuildPlan(table.getMetaClient(), HoodieTimeline.getBuildRequestedInstant(instantTime)) + .map(Pair::getRight) + .orElseThrow(() -> new HoodieBuildException("No plan found for this build:" + instantTime)); + } + + @Override + public HoodieBuildCommitMetadata execute() { + HoodieInstant requestInstant = HoodieTimeline.getBuildRequestedInstant(instantTime); + table.getActiveTimeline().transitionBuildRequestedToInflight(requestInstant, Option.empty()); + table.getMetaClient().reloadActiveTimeline(); + + Schema schema; + try { + schema = new TableSchemaResolver(table.getMetaClient()).getTableAvroSchema(); + } catch (Exception e) { + throw new HoodieBuildException("Fail to get table schema for build action", e); + } + SerializableSchema serializableSchema = new SerializableSchema(schema); + + String indexFolderPath = table.getMetaClient().getIndexFolderPath(); + List buildTasks = buildPlan.getTasks(); + SerializableConfiguration conf = new SerializableConfiguration(context.getHadoopConf().get()); + + List buildStatuses = new ArrayList<>(); + buildTasks.forEach(buildTask -> { + BuildStatus buildStatus = new BuildTaskExecutor(buildTask, table.getConfig().getBasePath(), + indexFolderPath, serializableSchema, conf).execute(); + buildStatuses.add(buildStatus); + }); + + return BuildUtils.convertToCommitMetadata(buildStatuses); + } +} 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 c200abee5e746..8604b7d5fb528 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 @@ -27,6 +27,7 @@ import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.fs.HoodieWrapperFileSystem; import org.apache.hudi.common.metrics.Registry; +import org.apache.hudi.common.model.HoodieBuildCommitMetadata; import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; @@ -44,6 +45,7 @@ import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.data.HoodieJavaRDD; +import org.apache.hudi.exception.HoodieBuildException; import org.apache.hudi.exception.HoodieClusteringException; import org.apache.hudi.exception.HoodieWriteConflictException; import org.apache.hudi.exception.HoodieException; @@ -68,6 +70,7 @@ import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; +import java.io.IOException; import java.nio.charset.StandardCharsets; import java.util.List; import java.util.Map; @@ -481,6 +484,55 @@ private void validateClusteringCommit(HoodieWriteMetadata> } } + @Override + public HoodieBuildCommitMetadata build(String instantTime, boolean shouldComplete) { + HoodieSparkTable table = HoodieSparkTable.create(config, context); + preWrite(instantTime, WriteOperationType.BUILD, table.getMetaClient()); + HoodieTimeline pendingBuildTimeline = table.getActiveTimeline().filterPendingBuildTimeline(); + HoodieInstant inflightInstant = HoodieTimeline.getBuildInflightInstant(instantTime); + + // This secondary index instant should be REQUESTED state. + // If it is INFLIGHT state, we should roll back it + if (pendingBuildTimeline.containsInstant(inflightInstant)) { + table.rollbackInflightBuild(inflightInstant, commitToRollback -> + getPendingRollbackInfo(table.getMetaClient(), commitToRollback, false)); + table.getMetaClient().reloadActiveTimeline(); + } + buildTimer = metrics.getBuildCtx(); + LOG.info("Starting build secondary index at " + instantTime); + HoodieBuildCommitMetadata buildMetadata = table.build(context, instantTime); + if (shouldComplete) { + completeTableService(TableServiceType.BUILD, buildMetadata, table, instantTime); + } + return buildMetadata; + } + + /** + * Persistence build commit metadata to completed build instant + * + * @param metadata Build commit metadata + * @param table HoodieTable + * @param buildCommitTime Build instant time + */ + private void completeBuild(HoodieBuildCommitMetadata metadata, + HoodieTable table, + String buildCommitTime) { + if (metadata.getCommittedIndexesInfo().size() <= 0) { + throw new HoodieBuildException("No committed index info for this build: " + buildCommitTime); + } + + HoodieInstant inflightInstant = HoodieTimeline.getBuildInflightInstant(buildCommitTime); + try { + this.txnManager.beginTransaction(Option.of(inflightInstant), Option.empty()); + table.getActiveTimeline().transitionBuildInflightToCompleted(inflightInstant, + Option.of(metadata.toJsonString().getBytes(StandardCharsets.UTF_8))); + } catch (IOException e) { + throw new HoodieBuildException("Unable to transition build inflight to completed", e); + } finally { + this.txnManager.endTransaction(Option.of(inflightInstant)); + } + } + private void updateTableMetadata(HoodieTable table, HoodieCommitMetadata commitMetadata, HoodieInstant hoodieInstant) { boolean isTableServiceAction = table.isTableServiceAction(hoodieInstant.getAction()); @@ -531,6 +583,9 @@ private void completeTableService(TableServiceType tableServiceType, HoodieCommi case LOG_COMPACT: completeLogCompaction(metadata, table, commitInstant); break; + case BUILD: + completeBuild((HoodieBuildCommitMetadata) metadata, table, commitInstant); + break; default: throw new IllegalArgumentException("This table service is not valid " + tableServiceType); } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkCopyOnWriteTable.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkCopyOnWriteTable.java index 115aea06f2a2f..cfbbb467fa5f2 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkCopyOnWriteTable.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkCopyOnWriteTable.java @@ -18,6 +18,7 @@ package org.apache.hudi.table; +import org.apache.hudi.avro.model.HoodieBuildPlan; import org.apache.hudi.avro.model.HoodieCleanMetadata; import org.apache.hudi.avro.model.HoodieCleanerPlan; import org.apache.hudi.avro.model.HoodieClusteringPlan; @@ -35,6 +36,7 @@ import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieBaseFile; +import org.apache.hudi.common.model.HoodieBuildCommitMetadata; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; @@ -55,6 +57,8 @@ import org.apache.hudi.table.action.HoodieWriteMetadata; import org.apache.hudi.table.action.bootstrap.HoodieBootstrapWriteMetadata; import org.apache.hudi.table.action.bootstrap.SparkBootstrapCommitActionExecutor; +import org.apache.hudi.table.action.build.BuildPlanActionExecutor; +import org.apache.hudi.table.action.build.SparkExecuteBuildCommitActionExecutor; import org.apache.hudi.table.action.clean.CleanActionExecutor; import org.apache.hudi.table.action.clean.CleanPlanActionExecutor; import org.apache.hudi.table.action.cluster.ClusteringPlanActionExecutor; @@ -77,6 +81,7 @@ import org.apache.hudi.table.action.rollback.CopyOnWriteRollbackActionExecutor; import org.apache.hudi.table.action.rollback.RestorePlanActionExecutor; import org.apache.hudi.table.action.savepoint.SavepointActionExecutor; + import org.apache.log4j.LogManager; import org.apache.log4j.Logger; @@ -300,4 +305,14 @@ public HoodieRestoreMetadata restore(HoodieEngineContext context, String restore public Option scheduleRestore(HoodieEngineContext context, String restoreInstantTime, String instantToRestore) { return new RestorePlanActionExecutor<>(context, config, this, restoreInstantTime, instantToRestore).execute(); } + + @Override + public Option scheduleBuild(HoodieEngineContext context, String instantTime, Option> extraMetadata) { + return new BuildPlanActionExecutor<>(context, config, this, instantTime, extraMetadata).execute(); + } + + @Override + public HoodieBuildCommitMetadata build(HoodieEngineContext context, String instantTime) { + return new SparkExecuteBuildCommitActionExecutor<>(context, config, this, instantTime).execute(); + } } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/build/SparkExecuteBuildCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/build/SparkExecuteBuildCommitActionExecutor.java new file mode 100644 index 0000000000000..d103cf42a7393 --- /dev/null +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/build/SparkExecuteBuildCommitActionExecutor.java @@ -0,0 +1,113 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.hudi.table.action.build; + +import org.apache.hudi.avro.model.HoodieBuildPlan; +import org.apache.hudi.avro.model.HoodieBuildTask; +import org.apache.hudi.client.common.HoodieSparkEngineContext; +import org.apache.hudi.common.config.SerializableConfiguration; +import org.apache.hudi.common.config.SerializableSchema; +import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.model.BuildStatus; +import org.apache.hudi.common.model.HoodieBuildCommitMetadata; +import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.table.TableSchemaResolver; +import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.util.BuildUtils; +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.data.HoodieJavaRDD; +import org.apache.hudi.exception.HoodieBuildException; +import org.apache.hudi.table.HoodieTable; +import org.apache.hudi.table.action.BaseActionExecutor; + +import org.apache.avro.Schema; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.catalyst.QualifiedTableName; +import org.apache.spark.sql.catalyst.catalog.SessionCatalog; + +import java.util.ArrayList; +import java.util.List; + +public class SparkExecuteBuildCommitActionExecutor + extends BaseActionExecutor { + + private HoodieBuildPlan buildPlan; + + public SparkExecuteBuildCommitActionExecutor(HoodieEngineContext context, + HoodieWriteConfig config, + HoodieTable table, + String instantTime) { + super(context, config, table, instantTime); + this.buildPlan = BuildUtils.getBuildPlan(table.getMetaClient(), + HoodieTimeline.getBuildRequestedInstant(instantTime)) + .map(Pair::getRight) + .orElseThrow(() -> new HoodieBuildException("No plan found for this build:" + instantTime)); + } + + @Override + public HoodieBuildCommitMetadata execute() { + HoodieInstant requestInstant = HoodieTimeline.getBuildRequestedInstant(instantTime); + table.getActiveTimeline().transitionBuildRequestedToInflight(requestInstant, Option.empty()); + table.getMetaClient().reloadActiveTimeline(); + + Schema schema; + try { + schema = new TableSchemaResolver(table.getMetaClient()).getTableAvroSchema(); + } catch (Exception e) { + throw new HoodieBuildException("Fail to get table schema for build action", e); + } + SerializableSchema serializableSchema = new SerializableSchema(schema); + + String indexFolderPath = table.getMetaClient().getIndexFolderPath(); + List buildTasks = buildPlan.getTasks(); + SerializableConfiguration conf = new SerializableConfiguration(context.getHadoopConf().get()); + JavaSparkContext sparkContext = HoodieSparkEngineContext.getSparkContext(context); + HoodieJavaRDD buildStatusRDD = + HoodieJavaRDD.of(sparkContext.parallelize(buildTasks, buildTasks.size()) + .mapPartitions(buildTasksPartition -> { + List buildStatuses = new ArrayList<>(); + buildTasksPartition.forEachRemaining(buildTask -> { + BuildStatus buildStatus = new BuildTaskExecutor(buildTask, table.getConfig().getBasePath(), + indexFolderPath, serializableSchema, conf).execute(); + buildStatuses.add(buildStatus); + }); + + return buildStatuses.iterator(); + })); + + // Invalidate cached table for queries do not access related table through {@code DefaultSource} + SessionCatalog sessionCatalog = SparkSession.active().sessionState().catalog(); + String databaseName = table.getMetaClient().getTableConfig().getDatabaseName(); + String tableName = table.getMetaClient().getTableConfig().getTableName(); + if (StringUtils.isNullOrEmpty(databaseName)) { + databaseName = sessionCatalog.getCurrentDatabase(); + } + QualifiedTableName qualifiedTableName = new QualifiedTableName(databaseName, tableName); + sessionCatalog.invalidateCachedTable(qualifiedTableName); + + List buildStatuses = buildStatusRDD.collectAsList(); + return BuildUtils.convertToCommitMetadata(buildStatuses); + } +} diff --git a/hudi-common/pom.xml b/hudi-common/pom.xml index 200b759ed234d..eea66303b911c 100644 --- a/hudi-common/pom.xml +++ b/hudi-common/pom.xml @@ -27,6 +27,8 @@ ${project.parent.basedir} + 8.11.1 + 0.9.28 @@ -93,6 +95,8 @@ ${basedir}/src/main/avro/HoodieMetadata.avsc ${basedir}/src/main/avro/HoodieIndexPartitionInfo.avsc ${basedir}/src/main/avro/HoodieIndexPlan.avsc + ${basedir}/src/main/avro/HoodieBuildTask.avsc + ${basedir}/src/main/avro/HoodieBuildPlan.avsc ${basedir}/src/main/avro/HoodieIndexCommitMetadata.avsc ${basedir}/src/main/avro/HoodieArchivedMetaEntry.avsc @@ -113,6 +117,18 @@ log4j-1.2-api + + org.apache.lucene + lucene-core + ${lucene.version} + + + + org.roaringbitmap + RoaringBitmap + ${roaringbitmap.version} + + com.fasterxml.jackson.core diff --git a/hudi-common/src/main/avro/HoodieBuildPlan.avsc b/hudi-common/src/main/avro/HoodieBuildPlan.avsc new file mode 100644 index 0000000000000..69e3d50b71add --- /dev/null +++ b/hudi-common/src/main/avro/HoodieBuildPlan.avsc @@ -0,0 +1,54 @@ +/* + * 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. + */ +{ + "namespace": "org.apache.hudi.avro.model", + "type": "record", + "name": "HoodieBuildPlan", + "fields": [ + { + "name": "tasks", + "type": [ + "null", + { + "type": "array", + "items": "HoodieBuildTask" + } + ], + "default": null + }, + { + "name": "extraMetadata", + "type": [ + "null", + { + "type": "map", + "values": "string" + } + ], + "default": null + }, + { + "name": "version", + "type": [ + "int", + "null" + ], + "default": 1 + } + ] +} diff --git a/hudi-common/src/main/avro/HoodieBuildTask.avsc b/hudi-common/src/main/avro/HoodieBuildTask.avsc new file mode 100644 index 0000000000000..326ddaf60807f --- /dev/null +++ b/hudi-common/src/main/avro/HoodieBuildTask.avsc @@ -0,0 +1,56 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +{ + "namespace": "org.apache.hudi.avro.model", + "type": "record", + "name": "HoodieBuildTask", + "fields": [ + { + "name": "partition", + "type": [ + "null", + "string" + ], + "default": null + }, + { + "name": "baseFilePath", + "type": [ + "null", + "string" + ], + "default": null + }, + { + "name": "indexMetas", + "type": [ + "null", + "string" + ], + "default": null + }, + { + "name": "version", + "type": [ + "int", + "null" + ], + "default": 1 + } + ] +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieBuildConfig.java b/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieBuildConfig.java new file mode 100644 index 0000000000000..f7f1a1b6fcc8c --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieBuildConfig.java @@ -0,0 +1,50 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.hudi.common.config; + +import java.util.Properties; + +public class HoodieBuildConfig extends HoodieConfig { + public static final ConfigProperty PARTITION_SELECTED = ConfigProperty + .key("hoodie.build.partition.selected") + .noDefaultValue() + .sinceVersion("0.13.0") + .withDocumentation("Partitions to run build"); + + public static final ConfigProperty BUILD_BATCH_ADD_SIZE = ConfigProperty + .key("hoodie.build.batch.add.size") + .defaultValue(1000) + .withDocumentation("Batch size when add records to index builder"); + + public HoodieBuildConfig() { + } + + public HoodieBuildConfig(Properties props) { + super(props); + } + + public int getPartitionSelected() { + return getIntOrDefault(PARTITION_SELECTED); + } + + public int getBatchSize() { + return getIntOrDefault(BUILD_BATCH_ADD_SIZE); + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieBuildTaskConfig.java b/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieBuildTaskConfig.java new file mode 100644 index 0000000000000..3aca2be9f50dd --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieBuildTaskConfig.java @@ -0,0 +1,154 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.hudi.common.config; + +import org.apache.hudi.secondary.index.SecondaryIndexType; + +import org.apache.avro.Schema; +import org.apache.hadoop.conf.Configuration; + +import java.util.Arrays; +import java.util.LinkedList; +import java.util.Properties; + +public class HoodieBuildTaskConfig extends HoodieConfig { + + public static final ConfigProperty LUCENE_INDEX_RAM_BUFFER_SIZE_MB = ConfigProperty + .key("hoodie.build.task.lucene.ram.buffer.size.mb") + .defaultValue(32.0) + .withDocumentation("Ram buffer size from build lucene index"); + + public static final ConfigProperty LUCENE_INDEX_MERGE_FACTOR = ConfigProperty + .key("hoodie.build.task.lucene.merge.factor") + .defaultValue(10) + .withDocumentation("Determines how often segment indices are merged by addDocument()"); + + public static final ConfigProperty LUCENE_INDEX_MAX_MERGE_MB = ConfigProperty + .key("hoodie.build.task.lucene.max.merge.mb") + .defaultValue(102400.0) + .withDocumentation("Determines the largest segment (measured by total byte size of the segment's files, in MB) that may be merged with other segments"); + + public static final ConfigProperty LUCENE_INDEX_LOG_ENABLED = ConfigProperty + .key("hoodie.build.task.lucene.log.enabled") + .defaultValue(true) + .withDocumentation("Whether to log information about merges, deletes and a message when maxFieldLength is reached"); + + public static final ConfigProperty BUILD_BATCH_ADD_SIZE = ConfigProperty + .key("hoodie.build.batch.add.size") + .defaultValue(1000) + .withDocumentation("Batch size when add records to index builder"); + + + private String indexSaveDir; + private SecondaryIndexType indexType; + private LinkedList indexFields; + private Configuration conf; + + public HoodieBuildTaskConfig(String indexSaveDir, + SecondaryIndexType indexType, + LinkedList indexFields, + Configuration conf) { + this.indexSaveDir = indexSaveDir; + this.indexType = indexType; + this.indexFields = indexFields; + this.conf = conf; + } + + public HoodieBuildTaskConfig(Properties props) { + super(props); + } + + public double getLuceneIndexRamBufferSizeMB() { + return getDoubleOrDefault(LUCENE_INDEX_RAM_BUFFER_SIZE_MB); + } + + public int getLuceneIndexMergeFactor() { + return getIntOrDefault(LUCENE_INDEX_MERGE_FACTOR); + } + + public double getLuceneIndexMaxMergeMB() { + return getDoubleOrDefault(LUCENE_INDEX_MAX_MERGE_MB); + } + + public boolean isLuceneIndexLogEnabled() { + return getBooleanOrDefault(LUCENE_INDEX_LOG_ENABLED); + } + + public String getIndexSaveDir() { + return indexSaveDir; + } + + public SecondaryIndexType getIndexType() { + return indexType; + } + + public LinkedList getIndexFields() { + return indexFields; + } + + public Configuration getConf() { + return conf; + } + + public static Builder builder() { + return new Builder(); + } + + @Override + public String toString() { + return "HoodieBuildTaskConfig{" + + "indexSaveDir='" + indexSaveDir + '\'' + + ", indexType=" + indexType + + ", indexFields=" + Arrays.toString(indexFields.toArray()) + + ", conf=" + conf + + '}'; + } + + public static class Builder { + private String indexSaveDir; + private SecondaryIndexType indexType; + private LinkedList indexFields; + private Configuration conf; + + public Builder setIndexSaveDir(String indexSaveDir) { + this.indexSaveDir = indexSaveDir; + return this; + } + + public Builder setIndexType(SecondaryIndexType indexType) { + this.indexType = indexType; + return this; + } + + public Builder setIndexFields(LinkedList indexFields) { + this.indexFields = indexFields; + return this; + } + + public Builder setConf(Configuration conf) { + this.conf = conf; + return this; + } + + public HoodieBuildTaskConfig build() { + return new HoodieBuildTaskConfig(indexSaveDir, indexType, indexFields, conf); + } + } +} 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 91f0671cf9b89..7c3e649d91f0c 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 @@ -189,6 +189,11 @@ public Double getDouble(ConfigProperty configProperty) { return rawValue.map(v -> Double.parseDouble(v.toString())).orElse(null); } + public Double getDoubleOrDefault(ConfigProperty configProperty) { + Option rawValue = getRawValue(configProperty); + return rawValue.map(v -> Double.parseDouble(v.toString())).orElse((Double) configProperty.defaultValue()); + } + public String getStringOrDefault(ConfigProperty configProperty) { return getStringOrDefault(configProperty, configProperty.defaultValue().toString()); } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieSecondaryIndexConfig.java b/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieSecondaryIndexConfig.java new file mode 100644 index 0000000000000..314fc9f0ff83b --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieSecondaryIndexConfig.java @@ -0,0 +1,24 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.hudi.common.config; + +public class HoodieSecondaryIndexConfig { + public static final String HOODIE_SECONDARY_INDEX_DATA = "hoodie.secondary.index.data"; +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/BuildStatus.java b/hudi-common/src/main/java/org/apache/hudi/common/model/BuildStatus.java new file mode 100644 index 0000000000000..2fbe8ce71ccc7 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/BuildStatus.java @@ -0,0 +1,106 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.hudi.common.model; + +import org.apache.hudi.secondary.index.HoodieSecondaryIndex; + +import java.util.List; + +public class BuildStatus { + private final String partition; + private final String fileName; + private final List secondaryIndexes; + + private final long totalRecords; + private final long totalBytes; + + public BuildStatus(String partition, + String fileName, + List secondaryIndexes, + long totalRecords, + long totalBytes) { + this.partition = partition; + this.fileName = fileName; + this.secondaryIndexes = secondaryIndexes; + this.totalRecords = totalRecords; + this.totalBytes = totalBytes; + } + + public String getPartition() { + return partition; + } + + public String getFileName() { + return fileName; + } + + public List getSecondaryIndexes() { + return secondaryIndexes; + } + + public long getTotalRecords() { + return totalRecords; + } + + public long getTotalBytes() { + return totalBytes; + } + + public static Builder builder() { + return new Builder(); + } + + public static class Builder { + private String partition; + private String baseFilePath; + private List secondaryIndexes; + private long totalRecords; + private long totalBytes; + + public Builder setPartition(String partition) { + this.partition = partition; + return this; + } + + public Builder setBaseFilePath(String baseFilePath) { + this.baseFilePath = baseFilePath; + return this; + } + + public Builder setSecondaryIndexes(List secondaryIndexes) { + this.secondaryIndexes = secondaryIndexes; + return this; + } + + public Builder setTotalRecords(long totalRecords) { + this.totalRecords = totalRecords; + return this; + } + + public Builder setTotalBytes(long totalBytes) { + this.totalBytes = totalBytes; + return this; + } + + public BuildStatus build() { + return new BuildStatus(partition, baseFilePath, secondaryIndexes, totalRecords, totalBytes); + } + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieBuildCommitMetadata.java b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieBuildCommitMetadata.java new file mode 100644 index 0000000000000..c5ebe3504b3fb --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieBuildCommitMetadata.java @@ -0,0 +1,46 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.hudi.common.model; + +import org.apache.hudi.secondary.index.HoodieSecondaryIndex; + +import com.fasterxml.jackson.annotation.JsonIgnoreProperties; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +@JsonIgnoreProperties(ignoreUnknown = true) +public class HoodieBuildCommitMetadata extends HoodieCommitMetadata { + private static final Logger LOG = LoggerFactory.getLogger(HoodieBuildCommitMetadata.class); + + // partitionPath -> baseFileName -> List + private Map>> committedIndexesInfo = new HashMap<>(); + + public Map>> getCommittedIndexesInfo() { + return committedIndexesInfo; + } + + public void setCommittedIndexesInfo(Map>> committedIndexesInfo) { + this.committedIndexesInfo = committedIndexesInfo; + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/TableServiceType.java b/hudi-common/src/main/java/org/apache/hudi/common/model/TableServiceType.java index 2aa5d08f06a8e..b5266bec704a2 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/model/TableServiceType.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/TableServiceType.java @@ -24,7 +24,7 @@ * Supported runtime table services. */ public enum TableServiceType { - ARCHIVE, COMPACT, CLUSTER, CLEAN, LOG_COMPACT; + ARCHIVE, COMPACT, CLUSTER, CLEAN, LOG_COMPACT, BUILD; public String getAction() { switch (this) { @@ -39,6 +39,8 @@ public String getAction() { return HoodieTimeline.REPLACE_COMMIT_ACTION; case LOG_COMPACT: return HoodieTimeline.LOG_COMPACTION_ACTION; + case BUILD: + return HoodieTimeline.BUILD_ACTION; default: throw new IllegalArgumentException("Unknown table service " + this); } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/WriteOperationType.java b/hudi-common/src/main/java/org/apache/hudi/common/model/WriteOperationType.java index 3be3004e5364e..c1e79f7989192 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/model/WriteOperationType.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/WriteOperationType.java @@ -51,6 +51,9 @@ public enum WriteOperationType { INDEX("index"), + // build + BUILD("build"), + // alter schema ALTER_SCHEMA("alter_schema"), // log compact @@ -95,6 +98,8 @@ public static WriteOperationType fromValue(String value) { return COMPACT; case "index": return INDEX; + case "build": + return BUILD; case "alter_schema": return ALTER_SCHEMA; case "unknown": diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java b/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java index 0938a61522a23..41f3a837797ec 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java @@ -96,6 +96,7 @@ public class HoodieTableMetaClient implements Serializable { public static final String SCHEMA_FOLDER_NAME = ".schema"; public static final String MARKER_EXTN = ".marker"; + public static final String INDEX_FOLDER_NAME = ".index"; // NOTE: Since those two parameters lay on the hot-path of a lot of computations, we // use tailored extension of the {@code Path} class allowing to avoid repetitive @@ -282,6 +283,15 @@ public String getArchivePath() { return getMetaPath() + Path.SEPARATOR + archiveFolder; } + /** + * Get lucene index folder path + * + * @return Lucene index folder path + */ + public String getIndexFolderPath() { + return new Path(metaPath.get(), INDEX_FOLDER_NAME).toString(); + } + /** * @return Table Config */ diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieActiveTimeline.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieActiveTimeline.java index 0ef46031ec1b0..03f7326479b7e 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieActiveTimeline.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieActiveTimeline.java @@ -74,6 +74,7 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline { ROLLBACK_EXTENSION, REQUESTED_ROLLBACK_EXTENSION, INFLIGHT_ROLLBACK_EXTENSION, REQUESTED_REPLACE_COMMIT_EXTENSION, INFLIGHT_REPLACE_COMMIT_EXTENSION, REPLACE_COMMIT_EXTENSION, REQUESTED_INDEX_COMMIT_EXTENSION, INFLIGHT_INDEX_COMMIT_EXTENSION, INDEX_COMMIT_EXTENSION, + REQUESTED_BUILD_COMMIT_EXTENSION, INFLIGHT_BUILD_COMMIT_EXTENSION, BUILD_COMMIT_EXTENSION, REQUESTED_SAVE_SCHEMA_ACTION_EXTENSION, INFLIGHT_SAVE_SCHEMA_ACTION_EXTENSION, SAVE_SCHEMA_ACTION_EXTENSION)); private static final Set NOT_PARSABLE_TIMESTAMPS = new HashSet(3) {{ @@ -610,6 +611,38 @@ public HoodieInstant transitionReplaceInflightToComplete(HoodieInstant inflightI return commitInstant; } + /** + * Transition build requested to inflight + * + * @param requestedInstant Requested instant + * @param data Metadata + * @return inflight instant + */ + public HoodieInstant transitionBuildRequestedToInflight(HoodieInstant requestedInstant, Option data) { + ValidationUtils.checkArgument(requestedInstant.getAction().equals(HoodieTimeline.BUILD_ACTION)); + ValidationUtils.checkArgument(requestedInstant.isRequested()); + HoodieInstant inflightInstant = new HoodieInstant(State.INFLIGHT, BUILD_ACTION, requestedInstant.getTimestamp()); + // Then write to timeline + transitionState(requestedInstant, inflightInstant, data); + return inflightInstant; + } + + /** + * Transition build inflight to completed + * + * @param inflightInstant Inflight instant + * @param data Metadata + * @return completed instant + */ + public HoodieInstant transitionBuildInflightToCompleted(HoodieInstant inflightInstant, Option data) { + ValidationUtils.checkArgument(inflightInstant.getAction().equals(HoodieTimeline.BUILD_ACTION)); + ValidationUtils.checkArgument(inflightInstant.isInflight()); + HoodieInstant completedInstant = new HoodieInstant(State.COMPLETED, BUILD_ACTION, inflightInstant.getTimestamp()); + // Then write to timeline + transitionState(inflightInstant, completedInstant, data); + return completedInstant; + } + private void transitionState(HoodieInstant fromInstant, HoodieInstant toInstant, Option data) { transitionState(fromInstant, toInstant, data, false); } @@ -808,6 +841,15 @@ public void saveToPendingIndexAction(HoodieInstant instant, Option conte createFileInMetaPath(instant.getFileName(), content, false); } + /** + * Save content for inflight/requested build instant. + */ + public void saveToPendingBuildAction(HoodieInstant instant, Option content) { + ValidationUtils.checkArgument(instant.getAction().equals(HoodieTimeline.BUILD_ACTION), + String.format("%s is not equal to %s action", instant.getAction(), BUILD_ACTION)); + createFileInMetaPath(instant.getFileName(), content, false); + } + protected void createFileInMetaPath(String filename, Option content, boolean allowOverwrite) { Path fullPath = getInstantFileNamePath(filename); if (allowOverwrite || metaClient.getTimelineLayoutVersion().isNullVersion()) { diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieArchivedTimeline.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieArchivedTimeline.java index c98920e3ed8f0..3b4b24aa57b42 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieArchivedTimeline.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieArchivedTimeline.java @@ -364,7 +364,7 @@ private int getArchivedFileSuffix(FileStatus f) { @Override public HoodieDefaultTimeline getWriteTimeline() { // filter in-memory instants - Set validActions = CollectionUtils.createSet(COMMIT_ACTION, DELTA_COMMIT_ACTION, COMPACTION_ACTION, LOG_COMPACTION_ACTION, REPLACE_COMMIT_ACTION); + Set validActions = CollectionUtils.createSet(COMMIT_ACTION, DELTA_COMMIT_ACTION, COMPACTION_ACTION, LOG_COMPACTION_ACTION, REPLACE_COMMIT_ACTION, BUILD_ACTION); return new HoodieDefaultTimeline(getInstantsAsStream().filter(i -> readCommits.containsKey(i.getTimestamp())) .filter(s -> validActions.contains(s.getAction())), details); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieDefaultTimeline.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieDefaultTimeline.java index d9db86942abf9..768b60f932bd8 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieDefaultTimeline.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieDefaultTimeline.java @@ -136,7 +136,7 @@ public HoodieDefaultTimeline filterCompletedInstantsOrRewriteTimeline() { @Override public HoodieDefaultTimeline getWriteTimeline() { - Set validActions = CollectionUtils.createSet(COMMIT_ACTION, DELTA_COMMIT_ACTION, COMPACTION_ACTION, LOG_COMPACTION_ACTION, REPLACE_COMMIT_ACTION); + Set validActions = CollectionUtils.createSet(COMMIT_ACTION, DELTA_COMMIT_ACTION, COMPACTION_ACTION, LOG_COMPACTION_ACTION, REPLACE_COMMIT_ACTION, BUILD_ACTION); return new HoodieDefaultTimeline(getInstantsAsStream().filter(s -> validActions.contains(s.getAction())), details); } @@ -246,6 +246,18 @@ public HoodieTimeline filterCompletedIndexTimeline() { return new HoodieDefaultTimeline(getInstantsAsStream().filter(s -> s.getAction().equals(INDEXING_ACTION) && s.isCompleted()), details); } + @Override + public HoodieTimeline filterPendingBuildTimeline() { + return new HoodieDefaultTimeline(instants.stream().filter(s -> + s.getAction().equals(BUILD_ACTION) && !s.isCompleted()), details); + } + + @Override + public HoodieTimeline filterCompletedBuildTimeline() { + return new HoodieDefaultTimeline(instants.stream().filter(s -> + s.getAction().equals(BUILD_ACTION) && s.isCompleted()), details); + } + /** * Get all instants (commits, delta commits) that produce new data, in the active timeline. */ diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieInstant.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieInstant.java index 0115742e07a08..7f46b18d176a6 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieInstant.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieInstant.java @@ -187,6 +187,10 @@ public String getFileName() { return isInflight() ? HoodieTimeline.makeInflightSchemaFileName(timestamp) : isRequested() ? HoodieTimeline.makeRequestSchemaFileName(timestamp) : HoodieTimeline.makeSchemaFileName(timestamp); + } else if (HoodieTimeline.BUILD_ACTION.equals(action)) { + return isInflight() ? HoodieTimeline.makeInflightBuildFileName(timestamp) + : isRequested() ? HoodieTimeline.makeRequestedBuildFileName(timestamp) + : HoodieTimeline.makeBuildFileName(timestamp); } throw new IllegalArgumentException("Cannot get file name for unknown action " + action); } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieTimeline.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieTimeline.java index d245be793b7c2..0975f683f4dd5 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieTimeline.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieTimeline.java @@ -58,6 +58,7 @@ public interface HoodieTimeline extends Serializable { String REQUESTED_EXTENSION = ".requested"; String RESTORE_ACTION = "restore"; String INDEXING_ACTION = "indexing"; + String BUILD_ACTION = "build"; // only for schema save String SCHEMA_COMMIT_ACTION = "schemacommit"; @@ -92,6 +93,9 @@ public interface HoodieTimeline extends Serializable { String INFLIGHT_INDEX_COMMIT_EXTENSION = "." + INDEXING_ACTION + INFLIGHT_EXTENSION; String REQUESTED_INDEX_COMMIT_EXTENSION = "." + INDEXING_ACTION + REQUESTED_EXTENSION; String INDEX_COMMIT_EXTENSION = "." + INDEXING_ACTION; + String INFLIGHT_BUILD_COMMIT_EXTENSION = "." + BUILD_ACTION + INFLIGHT_EXTENSION; + String REQUESTED_BUILD_COMMIT_EXTENSION = "." + BUILD_ACTION + REQUESTED_EXTENSION; + String BUILD_COMMIT_EXTENSION = "." + BUILD_ACTION; String SAVE_SCHEMA_ACTION_EXTENSION = "." + SCHEMA_COMMIT_ACTION; String INFLIGHT_SAVE_SCHEMA_ACTION_EXTENSION = "." + SCHEMA_COMMIT_ACTION + INFLIGHT_EXTENSION; String REQUESTED_SAVE_SCHEMA_ACTION_EXTENSION = "." + SCHEMA_COMMIT_ACTION + REQUESTED_EXTENSION; @@ -172,7 +176,7 @@ public interface HoodieTimeline extends Serializable { HoodieTimeline filterCompletedInstantsOrRewriteTimeline(); /** - * Timeline to just include commits (commit/deltacommit), compaction and replace actions. + * Timeline to just include commits (commit/deltacommit), compaction, replace and build actions. * * @return */ @@ -270,6 +274,16 @@ public interface HoodieTimeline extends Serializable { */ HoodieTimeline filterCompletedIndexTimeline(); + /** + * Filter this timeline to just include requested and inflight build instants. + */ + HoodieTimeline filterPendingBuildTimeline(); + + /** + * Filter this timeline to just include completed build instants. + */ + HoodieTimeline filterCompletedBuildTimeline(); + /** * If the timeline has any instants. * @@ -445,6 +459,14 @@ static HoodieInstant getIndexInflightInstant(final String timestamp) { return new HoodieInstant(State.INFLIGHT, INDEXING_ACTION, timestamp); } + static HoodieInstant getBuildRequestedInstant(final String timestamp) { + return new HoodieInstant(State.REQUESTED, BUILD_ACTION, timestamp); + } + + static HoodieInstant getBuildInflightInstant(String timestamp) { + return new HoodieInstant(State.INFLIGHT, BUILD_ACTION, timestamp); + } + /** * Returns the inflight instant corresponding to the instant being passed. Takes care of changes in action names * between inflight and completed instants (compaction <=> commit) and (logcompaction <==> deltacommit). @@ -592,6 +614,18 @@ static String makeRequestedIndexFileName(String instant) { return StringUtils.join(instant, HoodieTimeline.REQUESTED_INDEX_COMMIT_EXTENSION); } + static String makeBuildFileName(String instant) { + return StringUtils.join(instant, HoodieTimeline.BUILD_COMMIT_EXTENSION); + } + + static String makeInflightBuildFileName(String instant) { + return StringUtils.join(instant, HoodieTimeline.INFLIGHT_BUILD_COMMIT_EXTENSION); + } + + static String makeRequestedBuildFileName(String instant) { + return StringUtils.join(instant, HoodieTimeline.REQUESTED_BUILD_COMMIT_EXTENSION); + } + static String makeSchemaFileName(String instantTime) { return StringUtils.join(instantTime, HoodieTimeline.SAVE_SCHEMA_ACTION_EXTENSION); } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/TimelineMetadataUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/TimelineMetadataUtils.java index c0550fef6fe08..19afd68631259 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/TimelineMetadataUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/TimelineMetadataUtils.java @@ -19,6 +19,7 @@ package org.apache.hudi.common.table.timeline; import org.apache.hudi.avro.HoodieAvroUtils; +import org.apache.hudi.avro.model.HoodieBuildPlan; import org.apache.hudi.avro.model.HoodieCleanMetadata; import org.apache.hudi.avro.model.HoodieCleanerPlan; import org.apache.hudi.avro.model.HoodieCompactionPlan; @@ -150,6 +151,10 @@ public static Option serializeIndexCommitMetadata(HoodieIndexCommitMetad return serializeAvroMetadata(indexCommitMetadata, HoodieIndexCommitMetadata.class); } + public static Option serializeBuildPlan(HoodieBuildPlan buildPlan) throws IOException { + return serializeAvroMetadata(buildPlan, HoodieBuildPlan.class); + } + public static Option serializeAvroMetadata(T metadata, Class clazz) throws IOException { DatumWriter datumWriter = new SpecificDatumWriter<>(clazz); @@ -201,6 +206,10 @@ public static HoodieIndexCommitMetadata deserializeIndexCommitMetadata(byte[] by return deserializeAvroMetadata(bytes, HoodieIndexCommitMetadata.class); } + public static HoodieBuildPlan deserializeBuildPlan(byte[] bytes) throws IOException { + return deserializeAvroMetadata(bytes, HoodieBuildPlan.class); + } + public static T deserializeAvroMetadata(byte[] bytes, Class clazz) throws IOException { DatumReader reader = new SpecificDatumReader<>(clazz); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/dto/BaseFileToInstantDTO.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/dto/BaseFileToInstantDTO.java new file mode 100644 index 0000000000000..f18fe57e1f41d --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/dto/BaseFileToInstantDTO.java @@ -0,0 +1,48 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.hudi.common.table.timeline.dto; + +import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.util.collection.Pair; + +import com.fasterxml.jackson.annotation.JsonIgnoreProperties; +import com.fasterxml.jackson.annotation.JsonProperty; + +import java.io.Serializable; + +@JsonIgnoreProperties(ignoreUnknown = true) +public class BaseFileToInstantDTO implements Serializable { + @JsonProperty("baseFile") + private String baseFile; + + @JsonProperty("instant") + private InstantDTO instant; + + public static BaseFileToInstantDTO fromBaseFileToInstant(String baseFile, HoodieInstant instant) { + BaseFileToInstantDTO dto = new BaseFileToInstantDTO(); + dto.baseFile = baseFile; + dto.instant = InstantDTO.fromInstant(instant); + return dto; + } + + public static Pair toBaseFileInstant(BaseFileToInstantDTO dto) { + return Pair.of(dto.baseFile, InstantDTO.toInstant(dto.instant)); + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/dto/SecondaryIndexBaseFilesDTO.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/dto/SecondaryIndexBaseFilesDTO.java new file mode 100644 index 0000000000000..d1b3aed79ed52 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/dto/SecondaryIndexBaseFilesDTO.java @@ -0,0 +1,63 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.hudi.common.table.timeline.dto; + +import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.secondary.index.HoodieSecondaryIndex; + +import com.fasterxml.jackson.annotation.JsonIgnoreProperties; +import com.fasterxml.jackson.annotation.JsonProperty; + +import java.io.Serializable; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +@JsonIgnoreProperties(ignoreUnknown = true) +public class SecondaryIndexBaseFilesDTO implements Serializable { + @JsonProperty("secondaryIndex") + private SecondaryIndexDto secondaryIndex; + + @JsonProperty("baseFileToInstant") + private List baseFileToInstant; + + public static SecondaryIndexBaseFilesDTO fromSecondaryIndexBaseFiles( + HoodieSecondaryIndex secondaryIndex, + Map baseFileToInstant) { + SecondaryIndexBaseFilesDTO dto = new SecondaryIndexBaseFilesDTO(); + dto.secondaryIndex = SecondaryIndexDto.fromSecondaryIndex(secondaryIndex); + dto.baseFileToInstant = baseFileToInstant.entrySet().stream() + .map(x -> BaseFileToInstantDTO.fromBaseFileToInstant(x.getKey(), x.getValue())) + .collect(Collectors.toList()); + return dto; + } + + public static Pair> toSecondaryIndexBaseFiles( + SecondaryIndexBaseFilesDTO dto) { + HoodieSecondaryIndex secondaryIndex = SecondaryIndexDto.toSecondaryIndex(dto.secondaryIndex); + Map baseFileToInstant = new HashMap<>(); + dto.baseFileToInstant.stream() + .map(BaseFileToInstantDTO::toBaseFileInstant) + .forEach(y -> baseFileToInstant.put(y.getKey(), y.getValue())); + return Pair.of(secondaryIndex, baseFileToInstant); + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/dto/SecondaryIndexDto.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/dto/SecondaryIndexDto.java new file mode 100644 index 0000000000000..595bb0094085d --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/dto/SecondaryIndexDto.java @@ -0,0 +1,59 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.hudi.common.table.timeline.dto; + +import org.apache.hudi.secondary.index.HoodieSecondaryIndex; +import org.apache.hudi.secondary.index.SecondaryIndexType; + +import com.fasterxml.jackson.annotation.JsonIgnoreProperties; +import com.fasterxml.jackson.annotation.JsonProperty; + +import java.io.Serializable; +import java.util.LinkedHashMap; +import java.util.Map; + +@JsonIgnoreProperties(ignoreUnknown = true) +public class SecondaryIndexDto implements Serializable { + @JsonProperty("indexName") + private String indexName; + + @JsonProperty("indexType") + private byte indexType; + + @JsonProperty("columns") + private LinkedHashMap> columns; + + @JsonProperty("options") + private Map options; + + public static SecondaryIndexDto fromSecondaryIndex(HoodieSecondaryIndex secondaryIndex) { + SecondaryIndexDto dto = new SecondaryIndexDto(); + dto.indexName = secondaryIndex.getIndexName(); + dto.indexType = secondaryIndex.getIndexType().getValue(); + dto.columns = secondaryIndex.getColumns(); + dto.options = secondaryIndex.getOptions(); + return dto; + } + + public static HoodieSecondaryIndex toSecondaryIndex(SecondaryIndexDto dto) { + SecondaryIndexType indexType = SecondaryIndexType.of(dto.indexType); + return new HoodieSecondaryIndex(dto.indexName, indexType, dto.columns, dto.options); + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/view/AbstractTableFileSystemView.java b/hudi-common/src/main/java/org/apache/hudi/common/table/view/AbstractTableFileSystemView.java index 6e456e8305e26..5f4119935b833 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/view/AbstractTableFileSystemView.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/view/AbstractTableFileSystemView.java @@ -25,6 +25,8 @@ import org.apache.hudi.common.model.CompactionOperation; import org.apache.hudi.common.model.FileSlice; import org.apache.hudi.common.model.HoodieBaseFile; +import org.apache.hudi.common.model.HoodieBuildCommitMetadata; +import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.model.HoodieFileGroup; import org.apache.hudi.common.model.HoodieFileGroupId; import org.apache.hudi.common.model.HoodieLogFile; @@ -32,6 +34,7 @@ import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.util.BuildUtils; import org.apache.hudi.common.util.ClusteringUtils; import org.apache.hudi.common.util.CompactionUtils; import org.apache.hudi.common.util.HoodieTimer; @@ -39,6 +42,7 @@ import org.apache.hudi.common.util.ValidationUtils; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.exception.HoodieIOException; +import org.apache.hudi.secondary.index.HoodieSecondaryIndex; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; @@ -52,6 +56,7 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Comparator; +import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; @@ -107,7 +112,7 @@ protected void init(HoodieTableMetaClient metaClient, HoodieTimeline visibleActi this.metaClient = metaClient; refreshTimeline(visibleActiveTimeline); resetFileGroupsReplaced(visibleCommitsAndCompactionTimeline); - this.bootstrapIndex = BootstrapIndex.getBootstrapIndex(metaClient); + this.bootstrapIndex = BootstrapIndex.getBootstrapIndex(metaClient); // Load Pending Compaction Operations resetPendingCompactionOperations(CompactionUtils.getAllPendingCompactionOperations(metaClient).values().stream() .map(e -> Pair.of(e.getKey(), CompactionOperation.convertFromAvroRecordInstance(e.getValue())))); @@ -117,6 +122,10 @@ protected void init(HoodieTableMetaClient metaClient, HoodieTimeline visibleActi resetBootstrapBaseFileMapping(Stream.empty()); resetFileGroupsInPendingClustering(ClusteringUtils.getAllFileGroupsInPendingClusteringPlans(metaClient)); + + // Load completed/pending secondary instants + resetSecondaryIndexBaseFiles(visibleCommitsAndCompactionTimeline); + resetPendingSecondaryIndexBaseFiles(BuildUtils.getAllPendingBuildSecondaryIndexes(metaClient)); } /** @@ -249,6 +258,62 @@ private void resetFileGroupsReplaced(HoodieTimeline timeline) { + replacedFileGroups.size() + " replaced file groups"); } + /** + * Collect committed secondary indexes' info by looking at all secondary + * index commit instants. The metadata in commit instants as follows: + * partition -> baseFile -> List + *

+ * After conversion, base files which have been built secondary index will + * be organized as follows format: + * HoodieSecondaryIndex -> baseFile -> HoodieInstant + * + * @param timeline Active hoodie timeline + */ + private void resetSecondaryIndexBaseFiles(HoodieTimeline timeline) { + HoodieTimer hoodieTimer = new HoodieTimer(); + hoodieTimer.startTimer(); + // for each REPLACE instant, get map of (partitionPath -> deleteFileGroup) + HoodieTimeline secondaryIndexTimeline = timeline.filterCompletedBuildTimeline(); + + // HoodieSecondaryIndex -> BaseFileName -> Instant + Map> secondaryIndexToFileName = new HashMap<>(); + secondaryIndexTimeline.getInstants().forEach(instant -> { + try { + byte[] metadataBytes = metaClient.getActiveTimeline().getInstantDetails(instant).get(); + HoodieBuildCommitMetadata secondaryIndexCommitMetadata = + HoodieCommitMetadata.fromBytes(metadataBytes, HoodieBuildCommitMetadata.class); + + // Map>> + Map>> committedSecondaryIndexes = + secondaryIndexCommitMetadata.getCommittedIndexesInfo(); + + committedSecondaryIndexes.forEach((partitionPath, fileSecondaryIndexInfo) -> + // BaseFileName -> List + fileSecondaryIndexInfo.forEach((baseFileName, secondaryIndexes) -> + secondaryIndexes.forEach(secondaryIndex -> { + Map fileNameToInstant = + secondaryIndexToFileName.computeIfAbsent(secondaryIndex, HoodieSecondaryIndex -> new HashMap<>()); + fileNameToInstant.put(baseFileName, instant); + }))); + } catch (HoodieIOException ex) { + if (ex.getIOException() instanceof FileNotFoundException) { + // Secondary index instant could be deleted by archive and FileNotFoundException could be + // thrown during getInstantDetails function, so that we need to catch the FileNotFoundException + // here and continue + LOG.warn(ex.getMessage()); + } else { + throw ex; + } + } catch (IOException e) { + throw new HoodieIOException("Error reading secondary index commit metadata for " + instant); + } + }); + + resetSecondaryIndexBaseFiles(secondaryIndexToFileName); + LOG.info("Took " + hoodieTimer.endTimer() + " ms to read " + secondaryIndexTimeline.countInstants() + + " instants, " + secondaryIndexToFileName.size() + " secondary indexes"); + } + @Override public void close() { try { @@ -829,6 +894,26 @@ public final Stream> getFileGroupsInPendi } } + @Override + public Stream>> getPendingSecondaryIndexBaseFiles() { + try { + readLock.lock(); + return fetchPendingSecondaryIndexBaseFiles(); + } finally { + readLock.unlock(); + } + } + + @Override + public Stream>> getSecondaryIndexBaseFiles() { + try { + readLock.lock(); + return fetchSecondaryIndexBaseFiles(); + } finally { + readLock.unlock(); + } + } + // Fetch APIs to be implemented by concrete sub-classes /** @@ -1040,6 +1125,26 @@ protected abstract Option> getPendingLogCompac */ protected abstract Option getReplaceInstant(final HoodieFileGroupId fileGroupId); + /** + * Track base files in secondary index. + */ + protected abstract void resetSecondaryIndexBaseFiles(Map> secondaryIndexBaseFiles); + + /** + * Track base files in pending secondary index + */ + protected abstract void resetPendingSecondaryIndexBaseFiles(Map> pendingSecondaryIndexBaseFiles); + + /** + * Get all committed secondary index base files + */ + protected abstract Stream>> fetchSecondaryIndexBaseFiles(); + + /** + * Get all pending secondary index base files + */ + protected abstract Stream>> fetchPendingSecondaryIndexBaseFiles(); + /** * Check if the view is already closed. */ diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/view/HoodieTableFileSystemView.java b/hudi-common/src/main/java/org/apache/hudi/common/table/view/HoodieTableFileSystemView.java index ea72f305b0738..39f86a166883f 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/view/HoodieTableFileSystemView.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/view/HoodieTableFileSystemView.java @@ -29,6 +29,7 @@ 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.secondary.index.HoodieSecondaryIndex; import org.apache.hadoop.fs.FileStatus; import org.apache.log4j.LogManager; @@ -82,6 +83,16 @@ public class HoodieTableFileSystemView extends IncrementalTimelineSyncFileSystem */ protected Map fgIdToPendingClustering; + /** + * Track base files in secondary index. + */ + protected Map> secondaryIndexToBaseFiles; + + /** + * Track base files in pending secondary index. + */ + protected Map> pendingSecondaryIndexToBaseFiles; + /** * Flag to determine if closed. */ @@ -168,11 +179,21 @@ protected Map createFileIdToPendingClusteringM return fgInpendingClustering; } + protected Map> createSecondaryIndexToBaseFilesMap( + final Map> secondaryIndexToBaseFiles) { + return new ConcurrentHashMap<>(secondaryIndexToBaseFiles); + } + + protected Map> createPendingSecondaryIndexToBaseFilesMap( + final Map> pendingSecondaryIndexToBaseFiles) { + return new ConcurrentHashMap<>(pendingSecondaryIndexToBaseFiles); + } + /** * Create a file system view, as of the given timeline, with the provided file statuses. */ public HoodieTableFileSystemView(HoodieTableMetaClient metaClient, HoodieTimeline visibleActiveTimeline, - FileStatus[] fileStatuses) { + FileStatus[] fileStatuses) { this(metaClient, visibleActiveTimeline); addFilesToView(fileStatuses); } @@ -413,6 +434,26 @@ protected Option getReplaceInstant(final HoodieFileGroupId fileGr return Option.ofNullable(fgIdToReplaceInstants.get(fileGroupId)); } + @Override + protected void resetSecondaryIndexBaseFiles(Map> secondaryIndexBaseFiles) { + this.secondaryIndexToBaseFiles = createSecondaryIndexToBaseFilesMap(secondaryIndexBaseFiles); + } + + @Override + protected void resetPendingSecondaryIndexBaseFiles(Map> pendingSecondaryIndexBaseFiles) { + this.pendingSecondaryIndexToBaseFiles = createPendingSecondaryIndexToBaseFilesMap(pendingSecondaryIndexBaseFiles); + } + + @Override + protected Stream>> fetchSecondaryIndexBaseFiles() { + return secondaryIndexToBaseFiles.entrySet().stream().map(x -> Pair.of(x.getKey(), x.getValue())); + } + + @Override + protected Stream>> fetchPendingSecondaryIndexBaseFiles() { + return pendingSecondaryIndexToBaseFiles.entrySet().stream().map(x -> Pair.of(x.getKey(), x.getValue())); + } + /** * Get the latest file slices for a given partition including the inflight ones. * @@ -434,6 +475,8 @@ public void close() { this.fgIdToBootstrapBaseFile = null; this.fgIdToReplaceInstants = null; this.fgIdToPendingClustering = null; + this.secondaryIndexToBaseFiles = null; + this.pendingSecondaryIndexToBaseFiles = null; closed = true; } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/view/PriorityBasedFileSystemView.java b/hudi-common/src/main/java/org/apache/hudi/common/table/view/PriorityBasedFileSystemView.java index 62edc4daa33e0..d16229727942d 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/view/PriorityBasedFileSystemView.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/view/PriorityBasedFileSystemView.java @@ -31,6 +31,7 @@ import org.apache.hudi.common.util.Functions.Function3; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.secondary.index.HoodieSecondaryIndex; import org.apache.http.HttpStatus; import org.apache.http.client.HttpResponseException; @@ -39,6 +40,7 @@ import java.io.Serializable; import java.util.List; +import java.util.Map; import java.util.stream.Stream; /** @@ -234,6 +236,16 @@ public Stream> getFileGroupsInPendingClus return execute(preferredView::getFileGroupsInPendingClustering, secondaryView::getFileGroupsInPendingClustering); } + @Override + public Stream>> getPendingSecondaryIndexBaseFiles() { + return execute(preferredView::getPendingSecondaryIndexBaseFiles, secondaryView::getPendingSecondaryIndexBaseFiles); + } + + @Override + public Stream>> getSecondaryIndexBaseFiles() { + return execute(preferredView::getSecondaryIndexBaseFiles, secondaryView::getSecondaryIndexBaseFiles); + } + @Override public void close() { preferredView.close(); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/view/RemoteHoodieTableFileSystemView.java b/hudi-common/src/main/java/org/apache/hudi/common/table/view/RemoteHoodieTableFileSystemView.java index 759ef70c6d80f..af4b636669a40 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/view/RemoteHoodieTableFileSystemView.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/view/RemoteHoodieTableFileSystemView.java @@ -32,6 +32,7 @@ import org.apache.hudi.common.table.timeline.dto.FileGroupDTO; import org.apache.hudi.common.table.timeline.dto.FileSliceDTO; import org.apache.hudi.common.table.timeline.dto.InstantDTO; +import org.apache.hudi.common.table.timeline.dto.SecondaryIndexBaseFilesDTO; import org.apache.hudi.common.table.timeline.dto.TimelineDTO; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.RetryHelper; @@ -39,6 +40,7 @@ import org.apache.hudi.common.util.ValidationUtils; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.exception.HoodieRemoteException; +import org.apache.hudi.secondary.index.HoodieSecondaryIndex; import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; @@ -100,9 +102,14 @@ public class RemoteHoodieTableFileSystemView implements SyncableFileSystemView, public static final String ALL_REPLACED_FILEGROUPS_PARTITION = String.format("%s/%s", BASE_URL, "filegroups/replaced/partition/"); - + public static final String PENDING_CLUSTERING_FILEGROUPS = String.format("%s/%s", BASE_URL, "clustering/pending/"); + public static final String PENDING_SECONDARY_INDEX_BASE_FILES + = String.format("%s/%s", BASE_URL, "secondaryindex/pending/"); + public static final String COMPLETED_SECONDARY_INDEX_BASE_FILES + = String.format("%s/%s", BASE_URL, "secondaryindex/completed/"); + public static final String LAST_INSTANT = String.format("%s/%s", BASE_URL, "timeline/instant/last"); public static final String LAST_INSTANTS = String.format("%s/%s", BASE_URL, "timeline/instants/last"); @@ -454,13 +461,42 @@ public Stream> getFileGroupsInPendingClus Map paramsMap = getParams(); try { List dtos = executeRequest(PENDING_CLUSTERING_FILEGROUPS, paramsMap, - new TypeReference>() {}, RequestMethod.GET); + new TypeReference>() { + }, RequestMethod.GET); return dtos.stream().map(ClusteringOpDTO::toClusteringOperation); } catch (IOException e) { throw new HoodieRemoteException(e); } } + @Override + public Stream>> getPendingSecondaryIndexBaseFiles() { + Map paramsMap = getParams(); + try { + List dtos = + executeRequest(PENDING_SECONDARY_INDEX_BASE_FILES, paramsMap, + new TypeReference>() { + }, RequestMethod.GET); + return dtos.stream().map(SecondaryIndexBaseFilesDTO::toSecondaryIndexBaseFiles); + } catch (IOException e) { + throw new HoodieRemoteException(e); + } + } + + @Override + public Stream>> getSecondaryIndexBaseFiles() { + Map paramsMap = getParams(); + try { + List dtos = + executeRequest(COMPLETED_SECONDARY_INDEX_BASE_FILES, paramsMap, + new TypeReference>() { + }, RequestMethod.GET); + return dtos.stream().map(SecondaryIndexBaseFilesDTO::toSecondaryIndexBaseFiles); + } catch (IOException e) { + throw new HoodieRemoteException(e); + } + } + @Override public void close() { closed = true; 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 005b6d1fe9118..c79a3de3b075b 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 @@ -33,6 +33,7 @@ import org.apache.hudi.common.util.ValidationUtils; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.common.util.collection.RocksDBDAO; +import org.apache.hudi.secondary.index.HoodieSecondaryIndex; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; @@ -543,6 +544,52 @@ protected Option getReplaceInstant(final HoodieFileGroupId fileGr return Option.ofNullable(replacedInstant); } + @Override + protected void resetSecondaryIndexBaseFiles(Map> secondaryIndexBaseFiles) { + LOG.info("Resetting secondary index base files to ROCKSDB based file-system view at " + + config.getRocksdbBasePath() + ", total num: " + secondaryIndexBaseFiles.size()); + + // Delete all completed secondary index base files + rocksDB.prefixDelete(schemaHelper.getColFamilyForCompletedSecondaryIndexBaseFiles(), + schemaHelper.getPrefixForSecondaryIndexBaseFiles()); + // Now add new entries + rocksDB.writeBatch(batch -> + secondaryIndexBaseFiles.forEach((key, value) -> + rocksDB.>putInBatch( + batch, schemaHelper.getColFamilyForCompletedSecondaryIndexBaseFiles(), key, (HashMap) value))); + LOG.info("Resetting secondary index base files to ROCKSDB based file-system view complete"); + } + + @Override + protected void resetPendingSecondaryIndexBaseFiles(Map> pendingSecondaryIndexBaseFiles) { + LOG.info("Resetting pending secondary index base files to ROCKSDB based file-system view at " + + config.getRocksdbBasePath() + ", total num: " + pendingSecondaryIndexBaseFiles.size()); + + // Delete all replaced file groups + rocksDB.prefixDelete(schemaHelper.getColFamilyForPendingSecondaryIndexBaseFiles(), + schemaHelper.getPrefixForSecondaryIndexBaseFiles()); + // Now add new entries + rocksDB.writeBatch(batch -> + pendingSecondaryIndexBaseFiles.forEach((key, value) -> + rocksDB.>putInBatch( + batch, schemaHelper.getColFamilyForPendingSecondaryIndexBaseFiles(), key, (HashMap) value))); + LOG.info("Resetting pending secondary index base files to ROCKSDB based file-system view complete"); + } + + @Override + protected Stream>> fetchSecondaryIndexBaseFiles() { + return rocksDB.>>prefixSearch( + schemaHelper.getColFamilyForCompletedSecondaryIndexBaseFiles(), "") + .map(Pair::getValue); + } + + @Override + protected Stream>> fetchPendingSecondaryIndexBaseFiles() { + return rocksDB.>>prefixSearch( + schemaHelper.getColFamilyForPendingSecondaryIndexBaseFiles(), "") + .map(Pair::getValue); + } + private Stream getFileGroups(Stream sliceStream) { return sliceStream.map(s -> Pair.of(Pair.of(s.getPartitionPath(), s.getFileId()), s)) .collect(Collectors.groupingBy(Pair::getKey)).entrySet().stream().map(slicePair -> { diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/view/TableFileSystemView.java b/hudi-common/src/main/java/org/apache/hudi/common/table/view/TableFileSystemView.java index 18c9a9af99817..fbb489592132d 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/view/TableFileSystemView.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/view/TableFileSystemView.java @@ -27,8 +27,10 @@ import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.secondary.index.HoodieSecondaryIndex; import java.util.List; +import java.util.Map; import java.util.stream.Stream; /** @@ -192,4 +194,14 @@ interface SliceView extends SliceViewWithLatestSlice { * Filegroups that are in pending clustering. */ Stream> getFileGroupsInPendingClustering(); + + /** + * Stream all the pending base files to build secondary index + */ + Stream>> getPendingSecondaryIndexBaseFiles(); + + /** + * Stream all the base files that have built secondary index + */ + Stream>> getSecondaryIndexBaseFiles(); } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/BuildUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/util/BuildUtils.java new file mode 100644 index 0000000000000..be5360b7f8720 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/BuildUtils.java @@ -0,0 +1,244 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.hudi.common.util; + +import org.apache.hudi.avro.model.HoodieBuildPlan; +import org.apache.hudi.avro.model.HoodieBuildTask; +import org.apache.hudi.common.model.BuildStatus; +import org.apache.hudi.common.model.HoodieBuildCommitMetadata; +import org.apache.hudi.common.model.WriteOperationType; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.table.timeline.TimelineMetadataUtils; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.exception.HoodieBuildException; +import org.apache.hudi.exception.HoodieIOException; +import org.apache.hudi.secondary.index.HoodieSecondaryIndex; +import org.apache.hudi.secondary.index.SecondaryIndexUtils; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.parquet.hadoop.util.SerializationUtil; + +import java.io.IOException; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static org.apache.hudi.common.config.HoodieSecondaryIndexConfig.HOODIE_SECONDARY_INDEX_DATA; +import static org.apache.hudi.common.table.HoodieTableMetaClient.INDEX_FOLDER_NAME; +import static org.apache.hudi.common.table.HoodieTableMetaClient.METAFOLDER_NAME; + +public class BuildUtils { + + public static String getIndexFolderPath(String basePath) { + return new Path(new Path(basePath, METAFOLDER_NAME), INDEX_FOLDER_NAME).toString(); + } + + /** + * Get index save path + * + * @param indexFolderPath Table base path + * @param indexType Index type name + * @param fileName File name + * @return Index save dir + */ + public static Path getIndexSaveDir(String indexFolderPath, String indexType, String fileName) { + int index = fileName.indexOf("."); + String fileNameWithoutExtension; + if (index >= 0) { + fileNameWithoutExtension = fileName.substring(0, index); + } else { + fileNameWithoutExtension = fileName; + } + + return new Path(new Path(indexFolderPath, indexType), fileNameWithoutExtension); + } + + public static Map> getBaseFileIndexInfo(Configuration config) { + try { + return SerializationUtil.readObjectFromConfAsBase64(HOODIE_SECONDARY_INDEX_DATA, config); + } catch (IOException e) { + throw new HoodieBuildException("Fail to get base file index info", e); + } + } + + public static Map> getBaseFileIndexInfo( + HoodieTableMetaClient metaClient) { + Map> indexedBaseFiles = new HashMap<>(); + getAllCompletedBuildSecondaryIndexes(metaClient) + .forEach((k, v) -> indexedBaseFiles.put(k, v.keySet())); + + return indexedBaseFiles.entrySet().stream() + .flatMap(entry -> + entry.getValue().stream().map(baseFile -> Pair.of(baseFile, entry.getKey()))) + .collect(Collectors.groupingBy(Pair::getLeft, Collectors.mapping(Pair::getRight, Collectors.toList()))); + } + + /** + * Get all pending build secondary indexes + * + * @param metaClient HoodieTableMetaClient + * @return All pending build secondary indexes + */ + public static Map> getAllPendingBuildSecondaryIndexes( + HoodieTableMetaClient metaClient) { + return getAllBuildSecondaryIndexes(metaClient, false); + } + + /** + * Get all completed build secondary indexes + * + * @param metaClient HoodieTableMetaClient + * @return All completed build secondary indexes + */ + public static Map> getAllCompletedBuildSecondaryIndexes( + HoodieTableMetaClient metaClient) { + return getAllBuildSecondaryIndexes(metaClient, true); + } + + /** + * Get all completed or pending build secondary indexes + * + * @param metaClient HoodieTableMetaClient + * @param isCompleted Mark whether completed or pending build should be filtered out + * @return All completed or pending build secondary indexes + */ + private static Map> getAllBuildSecondaryIndexes( + HoodieTableMetaClient metaClient, boolean isCompleted) { + Map> xBuildSecondaryIndexes = new HashMap<>(); + getAllBuildPlans(metaClient, isCompleted).forEach(pair -> { + HoodieInstant instant = pair.getLeft(); + pair.getRight().getTasks().forEach(task -> { + String baseFilePath = task.getBaseFilePath(); + SecondaryIndexUtils.fromJsonString(task.getIndexMetas()).forEach(secondaryIndex -> { + Map baseFileToInstant = xBuildSecondaryIndexes.computeIfAbsent(secondaryIndex, + HoodieSecondaryIndex -> new HashMap<>()); + baseFileToInstant.put(baseFilePath, instant); + }); + }); + }); + + return xBuildSecondaryIndexes; + } + + /** + * Get all pending build plans + * + * @param metaClient HoodieTableMetaClient + * @return All pending build plans + */ + public static Stream> getAllPendingBuildPlans( + HoodieTableMetaClient metaClient) { + return getAllBuildPlans(metaClient, false); + } + + /** + * Get all completed build plans + * + * @param metaClient HoodieTableMetaClient + * @return All completed build plans + */ + public static Stream> getAllCompletedBuildPlans( + HoodieTableMetaClient metaClient) { + return getAllBuildPlans(metaClient, true); + } + + /** + * Get all completed or pending build plans + * + * @param metaClient HoodieTableMetaClient + * @param isCompleted Mark whether completed or pending build should be filtered out + * @return All completed or pending build plans + */ + private static Stream> getAllBuildPlans( + HoodieTableMetaClient metaClient, boolean isCompleted) { + HoodieTimeline timeline; + if (isCompleted) { + timeline = metaClient.getActiveTimeline().filterCompletedBuildTimeline(); + } else { + timeline = metaClient.getActiveTimeline().filterPendingBuildTimeline(); + } + + return timeline.getInstantsAsStream() + .map(instant -> getBuildPlan(metaClient, instant)) + .filter(Option::isPresent) + .map(Option::get); + } + + /** + * Get pending build plan of passed-in instant + * + * @param metaClient HoodieTableMetaClient + * @param requestedInstant Requested hoodie instant + * @return HoodieBuildPlan + */ + public static Option> getBuildPlan( + HoodieTableMetaClient metaClient, + HoodieInstant requestedInstant) { + HoodieInstant requestInstant = requestedInstant.isRequested() ? requestedInstant + : HoodieTimeline.getRequestedInstant(requestedInstant); + Option instantDetails = metaClient.getActiveTimeline().getInstantDetails(requestInstant); + if (!instantDetails.isPresent() || instantDetails.get().length == 0) { + return Option.empty(); + } + HoodieBuildPlan buildPlan = null; + try { + buildPlan = TimelineMetadataUtils.deserializeBuildPlan(instantDetails.get()); + } catch (IOException e) { + throw new HoodieIOException("Error reading build plan", e); + } + + return Option.of(Pair.of(requestedInstant, buildPlan)); + } + + public static HoodieBuildCommitMetadata convertToCommitMetadata(List buildStatuses) { + Map>> committedIndexesInfo = new HashMap<>(); + buildStatuses.stream().collect(Collectors.groupingBy(BuildStatus::getPartition)) + .forEach((partition, value) -> { + Map> fileToIndexes = new HashMap<>(); + value.forEach(buildStatus -> + fileToIndexes.put(buildStatus.getFileName(), buildStatus.getSecondaryIndexes())); + committedIndexesInfo.put(partition, fileToIndexes); + }); + + HoodieBuildCommitMetadata buildCommitMetadata = new HoodieBuildCommitMetadata(); + buildCommitMetadata.setOperationType(WriteOperationType.BUILD); + buildCommitMetadata.setCommittedIndexesInfo(committedIndexesInfo); + + return buildCommitMetadata; + } + + public static List extractPartitions(List buildTasks) { + if (CollectionUtils.isNullOrEmpty(buildTasks)) { + return Collections.emptyList(); + } + + return buildTasks.stream() + .map(HoodieBuildTask::getPartition) + .sorted() + .collect(Collectors.toList()); + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/RocksDBSchemaHelper.java b/hudi-common/src/main/java/org/apache/hudi/common/util/RocksDBSchemaHelper.java index 45b2a13eb72ae..cd9f471ef5e4f 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/RocksDBSchemaHelper.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/RocksDBSchemaHelper.java @@ -40,6 +40,9 @@ * Partitions : Key = "part=" Value = Boolean * * Pending Compactions Key = "part=,id=" Value = Pair + * + * Pending Secondary Index Base Files Key = "index_name=" Value = "Map" + * Completed Secondary Index Base Files Key = "index_name=" Value = "Map" */ public class RocksDBSchemaHelper { @@ -50,6 +53,8 @@ public class RocksDBSchemaHelper { private final String colFamilyForStoredPartitions; private final String colFamilyForReplacedFileGroups; private final String colFamilyForPendingClusteringFileGroups; + private final String colFamilyForPendingSecondaryIndexBaseFiles; + private final String colFamilyForCompletedSecondaryIndexBaseFiles; public RocksDBSchemaHelper(HoodieTableMetaClient metaClient) { this.colFamilyForBootstrapBaseFile = "hudi_bootstrap_basefile_" + metaClient.getBasePath().replace("/", "_"); @@ -59,12 +64,15 @@ public RocksDBSchemaHelper(HoodieTableMetaClient metaClient) { this.colFamilyForView = "hudi_view_" + metaClient.getBasePath().replace("/", "_"); this.colFamilyForReplacedFileGroups = "hudi_replaced_fg" + metaClient.getBasePath().replace("/", "_"); this.colFamilyForPendingClusteringFileGroups = "hudi_pending_clustering_fg" + metaClient.getBasePath().replace("/", "_"); + this.colFamilyForPendingSecondaryIndexBaseFiles = "hudi_pending_secondary_index_base_files" + metaClient.getBasePath().replace("/", "_"); + this.colFamilyForCompletedSecondaryIndexBaseFiles = "hudi_completed_secondary_index_base_files" + metaClient.getBasePath().replace("/", "_"); } public List getAllColumnFamilies() { return Arrays.asList(getColFamilyForView(), getColFamilyForPendingCompaction(), getColFamilyForPendingLogCompaction(), getColFamilyForBootstrapBaseFile(), getColFamilyForStoredPartitions(), getColFamilyForReplacedFileGroups(), - getColFamilyForFileGroupsInPendingClustering()); + getColFamilyForFileGroupsInPendingClustering(), getColFamilyForCompletedSecondaryIndexBaseFiles(), + getColFamilyForPendingSecondaryIndexBaseFiles()); } public String getKeyForPartitionLookup(String partition) { @@ -129,6 +137,10 @@ private String getPartitionFileIdBasedLookup(HoodieFileGroupId fgId) { return String.format("part=%s,id=%s", fgId.getPartitionPath(), fgId.getFileId()); } + public String getPrefixForSecondaryIndexBaseFiles() { + return "secondary_index="; + } + public String getColFamilyForView() { return colFamilyForView; } @@ -156,4 +168,12 @@ public String getColFamilyForReplacedFileGroups() { public String getColFamilyForFileGroupsInPendingClustering() { return colFamilyForPendingClusteringFileGroups; } + + public String getColFamilyForPendingSecondaryIndexBaseFiles() { + return colFamilyForPendingSecondaryIndexBaseFiles; + } + + public String getColFamilyForCompletedSecondaryIndexBaseFiles() { + return colFamilyForCompletedSecondaryIndexBaseFiles; + } } diff --git a/hudi-common/src/main/java/org/apache/hudi/exception/HoodieBuildException.java b/hudi-common/src/main/java/org/apache/hudi/exception/HoodieBuildException.java new file mode 100644 index 0000000000000..f9945d0ab7086 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/exception/HoodieBuildException.java @@ -0,0 +1,30 @@ +/* + * 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.exception; + +public class HoodieBuildException extends HoodieException { + public HoodieBuildException(String message) { + super(message); + } + + public HoodieBuildException(String message, Throwable t) { + super(message, t); + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/secondary/index/HoodieSecondaryIndex.java b/hudi-common/src/main/java/org/apache/hudi/secondary/index/HoodieSecondaryIndex.java index 4a93242474ba1..5035c7787ebd0 100644 --- a/hudi-common/src/main/java/org/apache/hudi/secondary/index/HoodieSecondaryIndex.java +++ b/hudi-common/src/main/java/org/apache/hudi/secondary/index/HoodieSecondaryIndex.java @@ -19,16 +19,16 @@ package org.apache.hudi.secondary.index; -import org.apache.hudi.exception.HoodieSecondaryIndexException; - +import java.io.Serializable; import java.util.Comparator; import java.util.LinkedHashMap; import java.util.Map; +import java.util.Objects; /** * Represents a single secondary index. */ -public class HoodieSecondaryIndex { +public class HoodieSecondaryIndex implements Serializable { private String indexName; private SecondaryIndexType indexType; @@ -60,7 +60,7 @@ public SecondaryIndexType getIndexType() { return indexType; } - public Map> getColumns() { + public LinkedHashMap> getColumns() { return columns; } @@ -73,15 +73,23 @@ public static Builder builder() { } private void validate() { - switch (indexType) { - case LUCENE: - if (columns.size() != 1) { - throw new HoodieSecondaryIndexException("Lucene index only support single column"); - } - break; - default: - return; + } + + @Override + public boolean equals(Object obj) { + if (obj == this) { + return true; } + + if (Objects.isNull(obj) || !(obj instanceof HoodieSecondaryIndex)) { + return false; + } + + HoodieSecondaryIndex other = (HoodieSecondaryIndex) obj; + return this.indexName.equals(other.getIndexName()) + && this.indexType == other.getIndexType() + && this.columns.equals(other.getColumns()) + && this.options.equals(other.getOptions()); } @Override diff --git a/hudi-common/src/main/java/org/apache/hudi/secondary/index/SecondaryIndexBuilder.java b/hudi-common/src/main/java/org/apache/hudi/secondary/index/SecondaryIndexBuilder.java new file mode 100644 index 0000000000000..e3171759000d0 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/secondary/index/SecondaryIndexBuilder.java @@ -0,0 +1,34 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.hudi.secondary.index; + +import org.apache.avro.generic.GenericRecord; + +import java.io.IOException; + +public interface SecondaryIndexBuilder { + void addBatch(GenericRecord[] records, int size) throws IOException; + + void addRow(GenericRecord record) throws IOException; + + String getName(); + + void close(); +} diff --git a/hudi-common/src/main/java/org/apache/hudi/secondary/index/SecondaryIndexFactory.java b/hudi-common/src/main/java/org/apache/hudi/secondary/index/SecondaryIndexFactory.java new file mode 100644 index 0000000000000..5b31cd44a2fcc --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/secondary/index/SecondaryIndexFactory.java @@ -0,0 +1,36 @@ +/* + * 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.secondary.index; + +import org.apache.hudi.common.config.HoodieBuildTaskConfig; +import org.apache.hudi.exception.HoodieSecondaryIndexException; +import org.apache.hudi.secondary.index.lucene.LuceneIndexBuilder; + +public class SecondaryIndexFactory { + public static SecondaryIndexBuilder getIndexBuilder(HoodieBuildTaskConfig indexConfig) { + switch (indexConfig.getIndexType()) { + case LUCENE: + return new LuceneIndexBuilder(indexConfig); + default: + throw new HoodieSecondaryIndexException( + "Unknown hoodie secondary index type: " + indexConfig.getIndexType()); + } + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/secondary/index/SecondaryIndexReader.java b/hudi-common/src/main/java/org/apache/hudi/secondary/index/SecondaryIndexReader.java new file mode 100644 index 0000000000000..199a287a62e90 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/secondary/index/SecondaryIndexReader.java @@ -0,0 +1,30 @@ +/* + * 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.secondary.index; + +import org.roaringbitmap.RoaringBitmap; + +public interface SecondaryIndexReader { + SecondaryIndexReader open(HoodieSecondaryIndex secondaryIndex, String indexPath); + + RoaringBitmap queryTerm(); + + void close(); +} diff --git a/hudi-common/src/main/java/org/apache/hudi/secondary/index/SecondaryIndexUtils.java b/hudi-common/src/main/java/org/apache/hudi/secondary/index/SecondaryIndexUtils.java index d134a457caf38..e50b765a05eea 100644 --- a/hudi-common/src/main/java/org/apache/hudi/secondary/index/SecondaryIndexUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/secondary/index/SecondaryIndexUtils.java @@ -31,6 +31,7 @@ import com.fasterxml.jackson.databind.DeserializationFeature; import com.fasterxml.jackson.databind.ObjectMapper; +import java.io.IOException; import java.util.List; /** @@ -56,13 +57,9 @@ public static Option> getSecondaryIndexes(HoodieTable * @return List */ public static List fromJsonString(String jsonStr) { - try { - return SecondaryIndexUtils.fromJsonString(jsonStr, - new TypeReference>() { - }); - } catch (Exception e) { - throw new HoodieSecondaryIndexException("Fail to get secondary indexes", e); - } + return SecondaryIndexUtils.fromJsonString(jsonStr, + new TypeReference>() { + }); } public static String toJsonString(Object value) { @@ -73,12 +70,16 @@ public static String toJsonString(Object value) { } } - public static T fromJsonString(String jsonStr, TypeReference type) throws Exception { + public static T fromJsonString(String jsonStr, TypeReference type) { if (jsonStr == null || jsonStr.isEmpty()) { return null; } - return getObjectMapper().readValue(jsonStr, type); + try { + return getObjectMapper().readValue(jsonStr, type); + } catch (IOException e) { + throw new HoodieSecondaryIndexException("Fail to parse json string"); + } } public static ObjectMapper getObjectMapper() { diff --git a/hudi-common/src/main/java/org/apache/hudi/secondary/index/lucene/LuceneIndexBuilder.java b/hudi-common/src/main/java/org/apache/hudi/secondary/index/lucene/LuceneIndexBuilder.java new file mode 100644 index 0000000000000..425315ff64218 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/secondary/index/lucene/LuceneIndexBuilder.java @@ -0,0 +1,206 @@ +/* + * 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.secondary.index.lucene; + +import org.apache.hudi.common.config.HoodieBuildTaskConfig; +import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.util.StringUtils; +import org.apache.hudi.exception.HoodieBuildException; +import org.apache.hudi.exception.HoodieSecondaryIndexException; +import org.apache.hudi.internal.schema.Type; +import org.apache.hudi.internal.schema.convert.AvroInternalSchemaConverter; +import org.apache.hudi.secondary.index.SecondaryIndexBuilder; +import org.apache.hudi.secondary.index.lucene.hadoop.HdfsDirectory; + +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericRecord; +import org.apache.avro.util.Utf8; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.lucene.document.Document; +import org.apache.lucene.document.DoublePoint; +import org.apache.lucene.document.Field; +import org.apache.lucene.document.FloatPoint; +import org.apache.lucene.document.IntPoint; +import org.apache.lucene.document.LongPoint; +import org.apache.lucene.document.StringField; +import org.apache.lucene.index.ConcurrentMergeScheduler; +import org.apache.lucene.index.DirectoryReader; +import org.apache.lucene.index.IndexWriter; +import org.apache.lucene.index.IndexWriterConfig; +import org.apache.lucene.index.KeepOnlyLastCommitDeletionPolicy; +import org.apache.lucene.index.LogByteSizeMergePolicy; +import org.apache.lucene.store.Directory; +import org.apache.lucene.util.BytesRef; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.LinkedList; +import java.util.stream.IntStream; + +public class LuceneIndexBuilder implements SecondaryIndexBuilder { + private static final Logger LOG = LoggerFactory.getLogger(LuceneIndexBuilder.class); + + private final String name; + private final Configuration conf; + private final String[] fieldNames; + private final Type.TypeID[] fieldTypes; + private final String indexSaveDir; + private final Directory directory; + private IndexWriter indexWriter; + private final Document reusedDoc = new Document(); + + public LuceneIndexBuilder(HoodieBuildTaskConfig indexConfig) { + this.name = "lucene-index-builder-" + System.nanoTime(); + this.conf = indexConfig.getConf(); + this.indexSaveDir = indexConfig.getIndexSaveDir(); + try { + Path path = new Path(indexConfig.getIndexSaveDir()); + String scheme = path.toUri().getScheme(); + if (!StringUtils.isNullOrEmpty(scheme)) { + String disableCacheName = String.format("fs.%s.impl.disable.cache", scheme); + conf.set(disableCacheName, "true"); + } + this.directory = new HdfsDirectory(path, conf); + IndexWriterConfig indexWriteConfig = getIndexWriteConfig(indexConfig); + this.indexWriter = new IndexWriter(directory, indexWriteConfig); + } catch (Exception e) { + throw new HoodieBuildException("Init lucene index builder failed", e); + } + + LinkedList indexFields = indexConfig.getIndexFields(); + fieldNames = new String[indexFields.size()]; + fieldTypes = new Type.TypeID[indexFields.size()]; + IntStream.range(0, indexFields.size()).forEach(i -> { + Schema.Field field = indexFields.get(i); + fieldTypes[i] = AvroInternalSchemaConverter.buildTypeFromAvroSchema(field.schema()).typeId(); + fieldNames[i] = field.name(); + }); + LOG.info("Init lucene index builder ok, name: {}, indexConfig: {}", name, indexConfig); + } + + @Override + public void addBatch(GenericRecord[] records, int size) throws IOException { + for (int i = 0; i < size; i++) { + addRow(records[i]); + } + } + + @Override + public void addRow(GenericRecord record) throws IOException { + buildDocument(reusedDoc, record); + indexWriter.addDocument(reusedDoc); + } + + @Override + public String getName() { + return name; + } + + @Override + public void close() { + if (indexWriter != null) { + try { + indexWriter.forceMerge(1, true); + indexWriter.close(); + } catch (IOException e) { + LOG.error("Fail to close lucene index writer", e); + } + + indexWriter = null; + } + } + + private void buildDocument(Document reusedDoc, GenericRecord record) { + reusedDoc.clear(); + for (int i = 0; i < fieldNames.length; i++) { + reusedDoc.add( + createField(fieldNames[i], fieldTypes[i], record.get(fieldNames[i]))); + } + } + + private Field createField(String fieldName, Type.TypeID fieldType, Object value) { + switch (fieldType) { + case BOOLEAN: + return new IntPoint(fieldName, (Boolean) value ? 1 : 0); + case INT: + return new IntPoint(fieldName, (Integer) value); + case LONG: + return new LongPoint(fieldName, (Long) value); + case FLOAT: + return new FloatPoint(fieldName, (Float) value); + case DOUBLE: + return new DoublePoint(fieldName, (Double) value); + case STRING: + byte[] bytes = ((Utf8) value).getBytes(); + return new StringField(fieldName, + new BytesRef(bytes, 0, bytes.length), Field.Store.NO); + case DECIMAL: + case UUID: + return new StringField(fieldName, "", Field.Store.NO); + case DATE: + case TIME: + case TIMESTAMP: + return new LongPoint(fieldName, 1); + default: + throw new HoodieSecondaryIndexException( + "Unsupported field type, field: " + fieldType + ", type: " + fieldType); + } + } + + /** + * Convert hoodie build task config to lucene index writer config + * + * @param secondaryIndexConfig HoodieBuildTaskConfig + * @return IndexWriterConfig + */ + private IndexWriterConfig getIndexWriteConfig(HoodieBuildTaskConfig secondaryIndexConfig) { + IndexWriterConfig config = new IndexWriterConfig(); + + config.setUseCompoundFile(true); + config.setCommitOnClose(true); + config.setRAMBufferSizeMB(secondaryIndexConfig.getLuceneIndexRamBufferSizeMB()); + config.setIndexDeletionPolicy(new KeepOnlyLastCommitDeletionPolicy()); + + ConcurrentMergeScheduler scheduler = new ConcurrentMergeScheduler(); + scheduler.setMaxMergesAndThreads(6, 1); + config.setMergeScheduler(scheduler); + + LogByteSizeMergePolicy mergePolicy = new LogByteSizeMergePolicy(); + mergePolicy.setMergeFactor(secondaryIndexConfig.getLuceneIndexMergeFactor()); + mergePolicy.setMaxMergeMB(secondaryIndexConfig.getLuceneIndexMaxMergeMB()); + config.setMergePolicy(mergePolicy); + config.setInfoStream(new LuceneIndexInfoStream(secondaryIndexConfig, name)); + + try { + if (DirectoryReader.indexExists(directory)) { + FSUtils.getFs(indexSaveDir, conf).delete(new Path(indexSaveDir), true); + LOG.info("Delete index dir: {}", indexSaveDir); + } + } catch (IOException e) { + throw new HoodieSecondaryIndexException("Fail to delete lucene index dir: " + indexSaveDir, e); + } + + config.setOpenMode(IndexWriterConfig.OpenMode.CREATE); + + return config; + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/secondary/index/lucene/LuceneIndexInfoStream.java b/hudi-common/src/main/java/org/apache/hudi/secondary/index/lucene/LuceneIndexInfoStream.java new file mode 100644 index 0000000000000..be3943aae8222 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/secondary/index/lucene/LuceneIndexInfoStream.java @@ -0,0 +1,56 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.hudi.secondary.index.lucene; + +import org.apache.hudi.common.config.HoodieBuildTaskConfig; + +import org.apache.lucene.util.InfoStream; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; + +public class LuceneIndexInfoStream extends InfoStream { + private static final Logger LOG = LoggerFactory.getLogger(LuceneIndexInfoStream.class); + + private final String uniqueKey; + private final boolean logEnable; + + public LuceneIndexInfoStream(HoodieBuildTaskConfig secondaryIndexConfig, String uniqueKey) { + this.uniqueKey = uniqueKey; + this.logEnable = secondaryIndexConfig.isLuceneIndexLogEnabled(); + } + + @Override + public void message(final String component, final String message) { + LOG.info("Lucene index info, uniqueKey:{}, component:{}, message:{}", + uniqueKey, component, message); + } + + @Override + public boolean isEnabled(final String component) { + return logEnable; + } + + @Override + public void close() throws IOException { + + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/secondary/index/lucene/LuceneIndexReader.java b/hudi-common/src/main/java/org/apache/hudi/secondary/index/lucene/LuceneIndexReader.java new file mode 100644 index 0000000000000..fa925d18a5366 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/secondary/index/lucene/LuceneIndexReader.java @@ -0,0 +1,84 @@ +/* + * 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.secondary.index.lucene; + +import org.apache.hudi.common.util.StringUtils; +import org.apache.hudi.exception.HoodieSecondaryIndexException; +import org.apache.hudi.secondary.index.HoodieSecondaryIndex; +import org.apache.hudi.secondary.index.SecondaryIndexReader; +import org.apache.hudi.secondary.index.lucene.hadoop.HdfsDirectory; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.lucene.index.DirectoryReader; +import org.apache.lucene.search.IndexSearcher; +import org.apache.lucene.store.Directory; +import org.roaringbitmap.RoaringBitmap; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; + +public class LuceneIndexReader implements SecondaryIndexReader { + private static final Logger LOG = LoggerFactory.getLogger(LuceneIndexReader.class); + + private String indexPath; + private DirectoryReader reader; + private IndexSearcher searcher; + + public LuceneIndexReader(String indexPath, Configuration conf) { + this.indexPath = indexPath; + try { + Path path = new Path(indexPath); + String scheme = path.toUri().getScheme(); + if (!StringUtils.isNullOrEmpty(scheme)) { + String disableCacheName = String.format("fs.%s.impl.disable.cache", scheme); + conf.set(disableCacheName, "true"); + } + Directory directory = new HdfsDirectory(path, conf); + reader = DirectoryReader.open(directory); + } catch (Exception e) { + throw new HoodieSecondaryIndexException("Init lucene index reader failed", e); + } + searcher = new IndexSearcher(reader); + } + + @Override + public SecondaryIndexReader open(HoodieSecondaryIndex secondaryIndex, String indexPath) { + return null; + } + + @Override + public RoaringBitmap queryTerm() { + return null; + } + + @Override + public void close() { + searcher = null; + if (reader != null) { + try { + reader.close(); + } catch (IOException e) { + LOG.error("Fail to close lucene index reader"); + } + } + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/secondary/index/lucene/hadoop/BufferStore.java b/hudi-common/src/main/java/org/apache/hudi/secondary/index/lucene/hadoop/BufferStore.java new file mode 100644 index 0000000000000..500bc842738aa --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/secondary/index/lucene/hadoop/BufferStore.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.secondary.index.lucene.hadoop; + +import java.util.concurrent.ArrayBlockingQueue; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; + +public class BufferStore implements Store { + + private static final Store EMPTY = + new Store() { + + @Override + public byte[] takeBuffer(int bufferSize) { + return new byte[bufferSize]; + } + + @Override + public void putBuffer(byte[] buffer) { + } + }; + + private static final ConcurrentMap BUFFER_STORES = + new ConcurrentHashMap<>(8192, 0.75f, 512); + + private final BlockingQueue buffers; + + private final int bufferSize; + + /** + * For testing only + * + * @lucene.internal + */ + static void clearBufferStores() { + BUFFER_STORES.clear(); + } + + public static synchronized void initNewBuffer(int bufferSize, long totalAmount) { + if (totalAmount == 0) { + return; + } + BufferStore bufferStore = BUFFER_STORES.get(bufferSize); + if (bufferStore == null) { + long count = totalAmount / bufferSize; + if (count > Integer.MAX_VALUE) { + count = Integer.MAX_VALUE; + } + BufferStore store = new BufferStore(bufferSize, (int) count); + BUFFER_STORES.put(bufferSize, store); + } + } + + private BufferStore(int bufferSize, int count) { + this.bufferSize = bufferSize; + buffers = setupBuffers(bufferSize, count); + } + + private static BlockingQueue setupBuffers(int bufferSize, int count) { + BlockingQueue queue = new ArrayBlockingQueue<>(count); + for (int i = 0; i < count; i++) { + queue.add(new byte[bufferSize]); + } + return queue; + } + + public static Store instance(int bufferSize) { + BufferStore bufferStore = BUFFER_STORES.get(bufferSize); + if (bufferStore == null) { + return EMPTY; + } + return bufferStore; + } + + @Override + public byte[] takeBuffer(int bufferSize) { + if (this.bufferSize != bufferSize) { + throw new RuntimeException( + "Buffer with length [" + + bufferSize + + "] does not match buffer size of [" + + bufferSize + + "]"); + } + return newBuffer(buffers.poll()); + } + + @Override + public void putBuffer(byte[] buffer) { + if (buffer == null) { + return; + } + if (buffer.length != bufferSize) { + throw new RuntimeException( + "Buffer with length [" + + buffer.length + + "] does not match buffer size of [" + + bufferSize + + "]"); + } + } + + private byte[] newBuffer(byte[] buf) { + if (buf != null) { + return buf; + } + return new byte[bufferSize]; + } +} + diff --git a/hudi-common/src/main/java/org/apache/hudi/secondary/index/lucene/hadoop/CustomBufferedIndexInput.java b/hudi-common/src/main/java/org/apache/hudi/secondary/index/lucene/hadoop/CustomBufferedIndexInput.java new file mode 100644 index 0000000000000..72b74855f21c7 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/secondary/index/lucene/hadoop/CustomBufferedIndexInput.java @@ -0,0 +1,260 @@ +/* + * 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.secondary.index.lucene.hadoop; + +import org.apache.lucene.store.BufferedIndexInput; +import org.apache.lucene.store.IndexInput; +import org.apache.lucene.store.IndexOutput; + +import java.io.EOFException; +import java.io.IOException; + +public abstract class CustomBufferedIndexInput extends IndexInput { + + public static final int BUFFER_SIZE = + Integer.getInteger("solr.hdfs.readbuffer.size.default", 32768); + + private int bufferSize = BUFFER_SIZE; + + protected byte[] buffer; + + private long bufferStart = 0; // position in file of buffer + private int bufferLength = 0; // end of valid bytes + private int bufferPosition = 0; // next byte to read + + private final Store store; + + @Override + public byte readByte() throws IOException { + if (bufferPosition >= bufferLength) { + refill(); + } + return buffer[bufferPosition++]; + } + + public CustomBufferedIndexInput(String resourceDesc) { + this(resourceDesc, BUFFER_SIZE); + } + + public CustomBufferedIndexInput(String resourceDesc, int bufferSize) { + super(resourceDesc); + checkBufferSize(bufferSize); + this.bufferSize = bufferSize; + this.store = BufferStore.instance(bufferSize); + } + + private void checkBufferSize(int bufferSize) { + if (bufferSize <= 0) { + throw new IllegalArgumentException( + "bufferSize must be greater than 0 (got " + bufferSize + ")"); + } + } + + @Override + public void readBytes(byte[] b, int offset, int len) throws IOException { + readBytes(b, offset, len, true); + } + + @Override + public void readBytes(byte[] b, int offset, int len, boolean useBuffer) throws IOException { + + if (len <= (bufferLength - bufferPosition)) { + // the buffer contains enough data to satisfy this request + if (len > 0) { + // to allow b to be null if len is 0... + System.arraycopy(buffer, bufferPosition, b, offset, len); + } + bufferPosition += len; + } else { + // the buffer does not have enough data. First serve all we've got. + int available = bufferLength - bufferPosition; + if (available > 0) { + System.arraycopy(buffer, bufferPosition, b, offset, available); + offset += available; + len -= available; + bufferPosition += available; + } + // and now, read the remaining 'len' bytes: + if (useBuffer && len < bufferSize) { + // If the amount left to read is small enough, and + // we are allowed to use our buffer, do it in the usual + // buffered way: fill the buffer and copy from it: + refill(); + if (bufferLength < len) { + // Throw an exception when refill() could not read len bytes: + System.arraycopy(buffer, 0, b, offset, bufferLength); + throw new EOFException("read past EOF: " + this); + } else { + System.arraycopy(buffer, 0, b, offset, len); + bufferPosition = len; + } + } else { + // The amount left to read is larger than the buffer + // or we've been asked to not use our buffer - + // there's no performance reason not to read it all + // at once. Note that unlike the previous code of + // this function, there is no need to do a seek + // here, because there's no need to reread what we + // had in the buffer. + long after = bufferStart + bufferPosition + len; + if (after > length()) { + throw new EOFException("read past EOF: " + this); + } + readInternal(b, offset, len); + bufferStart = after; + bufferPosition = 0; + bufferLength = 0; // trigger refill() on read + } + } + } + + @Override + public final short readShort() throws IOException { + // this can make JVM less confused (see LUCENE-10366 / SOLR-15943) + return super.readShort(); + } + + @Override + public final int readInt() throws IOException { + // this can make JVM less confused (see LUCENE-10366 / SOLR-15943) + return super.readInt(); + } + + @Override + public final long readLong() throws IOException { + // this can make JVM less confused (see LUCENE-10366 / SOLR-15943) + return super.readLong(); + } + + @Override + public final int readVInt() throws IOException { + // this can make JVM less confused (see LUCENE-10366 / SOLR-15943) + return super.readVInt(); + } + + @Override + public final long readVLong() throws IOException { + // this can make JVM less confused (see LUCENE-10366 / SOLR-15943) + return super.readVLong(); + } + + private void refill() throws IOException { + long start = bufferStart + bufferPosition; + long end = start + bufferSize; + if (end > length()) { + // don't read past EOF + end = length(); + } + int newLength = (int) (end - start); + if (newLength <= 0) { + throw new EOFException("read past EOF: " + this); + } + + if (buffer == null) { + buffer = store.takeBuffer(bufferSize); + seekInternal(bufferStart); + } + readInternal(buffer, 0, newLength); + bufferLength = newLength; + bufferStart = start; + bufferPosition = 0; + } + + @Override + public final void close() throws IOException { + closeInternal(); + store.putBuffer(buffer); + buffer = null; + } + + protected abstract void closeInternal() throws IOException; + + /** + * Expert: implements buffer refill. Reads bytes from the current position in the input. + * + * @param b the array to read bytes into + * @param offset the offset in the array to start storing bytes + * @param length the number of bytes to read + */ + protected abstract void readInternal(byte[] b, int offset, int length) throws IOException; + + @Override + public long getFilePointer() { + return bufferStart + bufferPosition; + } + + @Override + public void seek(long pos) throws IOException { + if (pos >= bufferStart && pos < (bufferStart + bufferLength)) { + bufferPosition = (int) (pos - bufferStart); // seek + } else { + bufferStart = pos; + bufferPosition = 0; + bufferLength = 0; // trigger refill() on read() + seekInternal(pos); + } + } + + /** + * Expert: implements seek. Sets current position in this file, where the next {@link + * #readInternal(byte[], int, int)} will occur. + * + * @see #readInternal(byte[], int, int) + */ + protected abstract void seekInternal(long pos) throws IOException; + + @Override + public IndexInput clone() { + CustomBufferedIndexInput clone = (CustomBufferedIndexInput) super.clone(); + + clone.buffer = null; + clone.bufferLength = 0; + clone.bufferPosition = 0; + clone.bufferStart = getFilePointer(); + + return clone; + } + + @Override + public IndexInput slice(String sliceDescription, long offset, long length) throws IOException { + return BufferedIndexInput.wrap(sliceDescription, this, offset, length); + } + + /** + * Flushes the in-memory bufer to the given output, copying at most numBytes. + * + *

NOTE: this method does not refill the buffer, however it does advance the buffer + * position. + * + * @return the number of bytes actually flushed from the in-memory buffer. + */ + protected int flushBuffer(IndexOutput out, long numBytes) throws IOException { + int toCopy = bufferLength - bufferPosition; + if (toCopy > numBytes) { + toCopy = (int) numBytes; + } + if (toCopy > 0) { + out.writeBytes(buffer, bufferPosition, toCopy); + bufferPosition += toCopy; + } + return toCopy; + } +} + diff --git a/hudi-common/src/main/java/org/apache/hudi/secondary/index/lucene/hadoop/HdfsDirectory.java b/hudi-common/src/main/java/org/apache/hudi/secondary/index/lucene/hadoop/HdfsDirectory.java new file mode 100644 index 0000000000000..4ffe79d2077da --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/secondary/index/lucene/hadoop/HdfsDirectory.java @@ -0,0 +1,311 @@ +/* + * 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.secondary.index.lucene.hadoop; + +import org.apache.hudi.common.util.FileIOUtils; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.FileAlreadyExistsException; +import org.apache.hadoop.fs.FileContext; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hdfs.DistributedFileSystem; +import org.apache.hadoop.hdfs.protocol.HdfsConstants; +import org.apache.lucene.store.BaseDirectory; +import org.apache.lucene.store.IOContext; +import org.apache.lucene.store.IndexInput; +import org.apache.lucene.store.IndexOutput; +import org.apache.lucene.store.LockFactory; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.lang.invoke.MethodHandles; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.Set; +import java.util.concurrent.atomic.AtomicLong; + +public class HdfsDirectory extends BaseDirectory { + private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); + public static final int DEFAULT_BUFFER_SIZE = 4096; + + private static final String LF_EXT = ".lf"; + protected final Path hdfsDirPath; + protected final Configuration configuration; + + private final FileSystem fileSystem; + private final FileContext fileContext; + + private final int bufferSize; + + /** + * Used to generate temp file names in {@link #createTempOutput}. + */ + private final AtomicLong nextTempFileCounter = new AtomicLong(); + + public HdfsDirectory(Path hdfsDirPath, Configuration configuration) throws IOException { + this(hdfsDirPath, HdfsLockFactory.INSTANCE, configuration, DEFAULT_BUFFER_SIZE); + } + + public HdfsDirectory( + Path hdfsDirPath, LockFactory lockFactory, Configuration configuration, int bufferSize) + throws IOException { + super(lockFactory); + this.hdfsDirPath = hdfsDirPath; + this.configuration = configuration; + this.bufferSize = bufferSize; + fileSystem = FileSystem.get(hdfsDirPath.toUri(), configuration); + fileContext = FileContext.getFileContext(hdfsDirPath.toUri(), configuration); + + if (fileSystem instanceof DistributedFileSystem) { + // Make sure dfs is not in safe mode + while (((DistributedFileSystem) fileSystem).setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_GET, true)) { + LOG.warn("The NameNode is in SafeMode - Solr will wait 5 seconds and try again."); + try { + Thread.sleep(5000); + } catch (InterruptedException e) { + Thread.interrupted(); + // continue + } + } + } + + try { + if (!fileSystem.exists(hdfsDirPath)) { + boolean success = fileSystem.mkdirs(hdfsDirPath); + if (!success) { + throw new RuntimeException("Could not create directory: " + hdfsDirPath); + } + } + } catch (Exception e) { + FileIOUtils.closeQuietly(fileSystem); + throw new RuntimeException("Problem creating directory: " + hdfsDirPath, e); + } + } + + @Override + public void close() throws IOException { + LOG.info("Closing hdfs directory {}", hdfsDirPath); + fileSystem.close(); + isOpen = false; + } + + /** + * Check whether this directory is open or closed. This check may return stale results in the form + * of false negatives. + * + * @return true if the directory is definitely closed, false if the directory is open or is + * pending closure + */ + public boolean isClosed() { + return !isOpen; + } + + @Override + public IndexOutput createOutput(String name, IOContext context) throws IOException { + try { + return new HdfsFileWriter(getFileSystem(), new Path(hdfsDirPath, name), name); + } catch (FileAlreadyExistsException e) { + java.nio.file.FileAlreadyExistsException ex = + new java.nio.file.FileAlreadyExistsException(e.getMessage()); + ex.initCause(e); + throw ex; + } + } + + @Override + public IndexOutput createTempOutput(String prefix, String suffix, IOContext context) + throws IOException { + while (true) { + try { + String name = getTempFileName(prefix, suffix, nextTempFileCounter.getAndIncrement()); + return new HdfsFileWriter(getFileSystem(), new Path(hdfsDirPath, name), name); + } catch (FileAlreadyExistsException faee) { + // Retry with next incremented name + } + } + } + + private String[] getNormalNames(List files) { + int size = files.size(); + for (int i = 0; i < size; i++) { + String str = files.get(i); + files.set(i, toNormalName(str)); + } + return files.toArray(new String[] {}); + } + + private String toNormalName(String name) { + if (name.endsWith(LF_EXT)) { + return name.substring(0, name.length() - 3); + } + return name; + } + + @Override + public IndexInput openInput(String name, IOContext context) throws IOException { + return new HdfsIndexInput(name, getFileSystem(), new Path(hdfsDirPath, name), bufferSize); + } + + @Override + public void deleteFile(String name) throws IOException { + Path path = new Path(hdfsDirPath, name); + LOG.debug("Deleting {}", path); + getFileSystem().delete(path, false); + } + + @Override + public void rename(String source, String dest) throws IOException { + Path sourcePath = new Path(hdfsDirPath, source); + Path destPath = new Path(hdfsDirPath, dest); + fileContext.rename(sourcePath, destPath); + } + + @Override + public void syncMetaData() throws IOException { + // TODO: how? + } + + @Override + public long fileLength(String name) throws IOException { + FileStatus fileStatus = fileSystem.getFileStatus(new Path(hdfsDirPath, name)); + return fileStatus.getLen(); + } + + public long fileModified(String name) throws IOException { + FileStatus fileStatus = getFileSystem().getFileStatus(new Path(hdfsDirPath, name)); + return fileStatus.getModificationTime(); + } + + @Override + public String[] listAll() throws IOException { + FileStatus[] listStatus = getFileSystem().listStatus(hdfsDirPath); + List files = new ArrayList<>(); + if (listStatus == null) { + return new String[] {}; + } + for (FileStatus status : listStatus) { + files.add(status.getPath().getName()); + } + return getNormalNames(files); + } + + public Path getHdfsDirPath() { + return hdfsDirPath; + } + + public FileSystem getFileSystem() { + return fileSystem; + } + + public Configuration getConfiguration() { + return configuration; + } + + @Override + public Set getPendingDeletions() { + return Collections.emptySet(); + } + + public static class HdfsIndexInput extends CustomBufferedIndexInput { + private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); + + private final Path path; + private final FSDataInputStream inputStream; + private final long length; + private boolean clone = false; + + public HdfsIndexInput(String name, FileSystem fileSystem, Path path, int bufferSize) + throws IOException { + super(name, bufferSize); + this.path = path; + LOG.debug("Opening normal index input on {}", path); + FileStatus fileStatus = fileSystem.getFileStatus(path); + length = fileStatus.getLen(); + inputStream = fileSystem.open(path, bufferSize); + } + + @Override + protected void readInternal(byte[] b, int offset, int length) throws IOException { + inputStream.readFully(getFilePointer(), b, offset, length); + } + + @Override + protected void seekInternal(long pos) throws IOException { + } + + @Override + protected void closeInternal() throws IOException { + LOG.debug("Closing normal index input on {}", path); + if (!clone) { + inputStream.close(); + } + } + + @Override + public long length() { + return length; + } + + @Override + public IndexInput clone() { + HdfsIndexInput clone = (HdfsIndexInput) super.clone(); + clone.clone = true; + return clone; + } + } + + @Override + public void sync(Collection names) throws IOException { + if (LOG.isDebugEnabled()) { + LOG.debug("Sync called on {}", Arrays.toString(names.toArray())); + } + } + + @Override + public int hashCode() { + return hdfsDirPath.hashCode(); + } + + @Override + public boolean equals(Object obj) { + if (obj == this) { + return true; + } + if (obj == null) { + return false; + } + if (!(obj instanceof HdfsDirectory)) { + return false; + } + return this.hdfsDirPath.equals(((HdfsDirectory) obj).hdfsDirPath); + } + + @Override + public String toString() { + return this.getClass().getSimpleName() + "@" + hdfsDirPath + " lockFactory=" + lockFactory; + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/secondary/index/lucene/hadoop/HdfsFileWriter.java b/hudi-common/src/main/java/org/apache/hudi/secondary/index/lucene/hadoop/HdfsFileWriter.java new file mode 100644 index 0000000000000..20f676504237b --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/secondary/index/lucene/hadoop/HdfsFileWriter.java @@ -0,0 +1,64 @@ +/* + * 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.secondary.index.lucene.hadoop; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.CreateFlag; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.FsServerDefaults; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.permission.FsPermission; +import org.apache.lucene.store.OutputStreamIndexOutput; + +import java.io.IOException; +import java.io.OutputStream; +import java.util.EnumSet; + +public class HdfsFileWriter extends OutputStreamIndexOutput { + + public static final String HDFS_SYNC_BLOCK = "solr.hdfs.sync.block"; + public static final int BUFFER_SIZE = 16384; + + public HdfsFileWriter(FileSystem fileSystem, Path path, String name) throws IOException { + super("fileSystem=" + fileSystem + " path=" + path, + name, + getOutputStream(fileSystem, path), + BUFFER_SIZE); + } + + private static OutputStream getOutputStream(FileSystem fileSystem, Path path) + throws IOException { + Configuration conf = fileSystem.getConf(); + FsServerDefaults fsDefaults = fileSystem.getServerDefaults(path); + short replication = fileSystem.getDefaultReplication(path); + EnumSet flags = EnumSet.of(CreateFlag.CREATE); + if (Boolean.getBoolean(HDFS_SYNC_BLOCK)) { + flags.add(CreateFlag.SYNC_BLOCK); + } + return fileSystem.create( + path, + FsPermission.getDefault().applyUMask(FsPermission.getUMask(conf)), + flags, + fsDefaults.getFileBufferSize(), + replication, + fsDefaults.getBlockSize(), + null); + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/secondary/index/lucene/hadoop/HdfsLockFactory.java b/hudi-common/src/main/java/org/apache/hudi/secondary/index/lucene/hadoop/HdfsLockFactory.java new file mode 100644 index 0000000000000..f3136831bed4a --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/secondary/index/lucene/hadoop/HdfsLockFactory.java @@ -0,0 +1,131 @@ +/* + * 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.secondary.index.lucene.hadoop; + +import org.apache.hudi.common.util.FileIOUtils; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.ipc.RemoteException; +import org.apache.lucene.store.Directory; +import org.apache.lucene.store.Lock; +import org.apache.lucene.store.LockFactory; +import org.apache.lucene.store.LockObtainFailedException; +import org.apache.lucene.store.LockReleaseFailedException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.lang.invoke.MethodHandles; + +public class HdfsLockFactory extends LockFactory { + private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); + + public static final HdfsLockFactory INSTANCE = new HdfsLockFactory(); + + private HdfsLockFactory() { + } + + @Override + public Lock obtainLock(Directory dir, String lockName) throws IOException { + if (!(dir instanceof HdfsDirectory)) { + throw new UnsupportedOperationException( + "HdfsLockFactory can only be used with HdfsDirectory subclasses, got: " + dir); + } + final HdfsDirectory hdfsDir = (HdfsDirectory) dir; + final Configuration conf = hdfsDir.getConfiguration(); + final Path lockPath = hdfsDir.getHdfsDirPath(); + final Path lockFile = new Path(lockPath, lockName); + + FSDataOutputStream file = null; + final FileSystem fs = FileSystem.get(lockPath.toUri(), conf); + while (true) { + try { + if (!fs.exists(lockPath)) { + boolean success = fs.mkdirs(lockPath); + if (!success) { + throw new RuntimeException("Could not create directory: " + lockPath); + } + } else { + // just to check for safe mode + fs.mkdirs(lockPath); + } + + file = fs.create(lockFile, false); + break; + } catch (RemoteException e) { + if (e.getClassName().equals("org.apache.hadoop.hdfs.server.namenode.SafeModeException")) { + LOG.warn("The NameNode is in SafeMode - Solr will wait 5 seconds and try again."); + try { + Thread.sleep(5000); + } catch (InterruptedException e1) { + Thread.interrupted(); + } + continue; + } + throw new LockObtainFailedException("Cannot obtain lock file: " + lockFile, e); + } catch (IOException e) { + throw new LockObtainFailedException("Cannot obtain lock file: " + lockFile, e); + } finally { + FileIOUtils.closeQuietly(file); + } + } + + return new HdfsLock(conf, lockFile); + } + + private static final class HdfsLock extends Lock { + private final Configuration conf; + private final Path lockFile; + private volatile boolean closed; + + HdfsLock(Configuration conf, Path lockFile) { + this.conf = conf; + this.lockFile = lockFile; + } + + @Override + public void close() throws IOException { + if (closed) { + return; + } + final FileSystem fs = FileSystem.get(lockFile.toUri(), conf); + try { + if (fs.exists(lockFile) && !fs.delete(lockFile, false)) { + throw new LockReleaseFailedException("failed to delete: " + lockFile); + } + } finally { + FileIOUtils.closeQuietly(fs); + } + } + + @Override + public void ensureValid() throws IOException { + // no idea how to implement this on HDFS + } + + @Override + public String toString() { + return "HdfsLock(lockFile=" + lockFile + ")"; + } + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/secondary/index/lucene/hadoop/Store.java b/hudi-common/src/main/java/org/apache/hudi/secondary/index/lucene/hadoop/Store.java new file mode 100644 index 0000000000000..8c12a74c7ce35 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/secondary/index/lucene/hadoop/Store.java @@ -0,0 +1,27 @@ +/* + * 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.secondary.index.lucene.hadoop; + +public interface Store { + + byte[] takeBuffer(int bufferSize); + + void putBuffer(byte[] buffer); +} diff --git a/hudi-examples/hudi-examples-spark/src/main/java/org/apache/hudi/examples/spark/HoodieWriteClientExample.java b/hudi-examples/hudi-examples-spark/src/main/java/org/apache/hudi/examples/spark/HoodieWriteClientExample.java index 299fe992fa00a..ce28b53dfb2f8 100644 --- a/hudi-examples/hudi-examples-spark/src/main/java/org/apache/hudi/examples/spark/HoodieWriteClientExample.java +++ b/hudi-examples/hudi-examples-spark/src/main/java/org/apache/hudi/examples/spark/HoodieWriteClientExample.java @@ -27,6 +27,7 @@ import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieArchivalConfig; @@ -35,11 +36,12 @@ import org.apache.hudi.examples.common.HoodieExampleDataGenerator; import org.apache.hudi.examples.common.HoodieExampleSparkUtils; import org.apache.hudi.index.HoodieIndex; +import org.apache.hudi.secondary.index.HoodieSecondaryIndex; +import org.apache.hudi.secondary.index.SecondaryIndexUtils; +import org.apache.hudi.table.action.HoodieWriteMetadata; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.hudi.table.action.HoodieWriteMetadata; - import org.apache.log4j.LogManager; import org.apache.log4j.Logger; import org.apache.spark.SparkConf; @@ -47,7 +49,10 @@ import org.apache.spark.api.java.JavaSparkContext; import java.util.ArrayList; +import java.util.Collections; +import java.util.LinkedHashMap; import java.util.List; +import java.util.Map; import java.util.stream.Collectors; /** @@ -87,11 +92,23 @@ public static void main(String[] args) throws Exception { Path path = new Path(tablePath); FileSystem fs = FSUtils.getFs(tablePath, jsc.hadoopConfiguration()); if (!fs.exists(path)) { + LinkedHashMap> columns = new LinkedHashMap<>(); + columns.put("rider", Collections.emptyMap()); + HoodieSecondaryIndex secondaryIndex = HoodieSecondaryIndex.builder() + .setIndexName("idx_rider") + .setIndexType("lucene") + .setColumns(columns) + .setOptions(Collections.emptyMap()) + .build(); + + ArrayList secondaryIndexes = new ArrayList<>(); + secondaryIndexes.add(secondaryIndex); HoodieTableMetaClient.withPropertyBuilder() - .setTableType(tableType) - .setTableName(tableName) - .setPayloadClass(HoodieAvroPayload.class) - .initTable(jsc.hadoopConfiguration(), tablePath); + .setTableType(tableType) + .setTableName(tableName) + .setSecondaryIndexesMetadata(SecondaryIndexUtils.toJsonString(secondaryIndexes)) + .setPayloadClass(HoodieAvroPayload.class) + .initTable(jsc.hadoopConfiguration(), tablePath); } // Create the write client to write some records in @@ -111,6 +128,10 @@ public static void main(String[] args) throws Exception { JavaRDD> writeRecords = jsc.parallelize(records, 1); client.insert(writeRecords, newCommitTime); + String instantTime = HoodieActiveTimeline.createNewInstantTime(); + client.scheduleBuildAtInstant(instantTime, Option.empty()); + client.build(instantTime, true); + // updates newCommitTime = client.startCommit(); LOG.info("Starting commit " + newCommitTime); diff --git a/hudi-spark-datasource/hudi-spark/src/main/java/org/apache/hudi/HoodieDataSourceHelpers.java b/hudi-spark-datasource/hudi-spark/src/main/java/org/apache/hudi/HoodieDataSourceHelpers.java index 7ddf348c22bea..adda37901fbec 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/java/org/apache/hudi/HoodieDataSourceHelpers.java +++ b/hudi-spark-datasource/hudi-spark/src/main/java/org/apache/hudi/HoodieDataSourceHelpers.java @@ -18,12 +18,14 @@ package org.apache.hudi; +import org.apache.hudi.avro.model.HoodieBuildPlan; import org.apache.hudi.avro.model.HoodieClusteringPlan; import org.apache.hudi.common.model.HoodieTableType; 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.BuildUtils; import org.apache.hudi.common.util.ClusteringUtils; import org.apache.hudi.common.util.CollectionUtils; import org.apache.hudi.common.util.Option; @@ -97,4 +99,28 @@ public static Option getClusteringPlan(FileSystem fs, Stri return Option.empty(); } } + + @PublicAPIMethod(maturity = ApiMaturityLevel.EVOLVING) + public static HoodieTimeline allBuildCommits(FileSystem fs, String basePath) { + HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder() + .setConf(fs.getConf()) + .setBasePath(basePath) + .setLoadActiveTimelineOnLoad(true) + .build(); + return metaClient.getActiveTimeline() + .getTimelineOfActions(CollectionUtils.createSet(HoodieActiveTimeline.BUILD_ACTION)); + } + + @PublicAPIMethod(maturity = ApiMaturityLevel.EVOLVING) + public static Option getBuildPlan(FileSystem fs, String basePath, String instantTime) { + HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder().setConf(fs.getConf()) + .setBasePath(basePath).setLoadActiveTimelineOnLoad(true).build(); + HoodieInstant hoodieInstant = HoodieTimeline.getBuildRequestedInstant(instantTime); + Option> buildPlan = BuildUtils.getBuildPlan(metaClient, hoodieInstant); + if (buildPlan.isPresent()) { + return Option.of(buildPlan.get().getValue()); + } else { + return Option.empty(); + } + } } diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/HoodieProcedures.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/HoodieProcedures.scala index a53fec33fe9aa..b3dc6b22fad39 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/HoodieProcedures.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/HoodieProcedures.scala @@ -87,6 +87,8 @@ object HoodieProcedures { ,(CopyToTempView.NAME, CopyToTempView.builder) ,(ShowCommitExtraMetadataProcedure.NAME, ShowCommitExtraMetadataProcedure.builder) ,(ShowTablePropertiesProcedure.NAME, ShowTablePropertiesProcedure.builder) + ,(RunBuildProcedure.NAME, RunBuildProcedure.builder) + ,(ShowBuildProcedure.NAME, ShowBuildProcedure.builder) ,(HelpProcedure.NAME, HelpProcedure.builder) ) } diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ProcedurePredicateHelper.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ProcedurePredicateHelper.scala new file mode 100644 index 0000000000000..fe5db6b51fc19 --- /dev/null +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ProcedurePredicateHelper.scala @@ -0,0 +1,54 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.spark.sql.hudi.command.procedures + +import org.apache.hudi.DataSourceReadOptions.{QUERY_TYPE, QUERY_TYPE_SNAPSHOT_OPT_VAL} +import org.apache.hudi.common.table.{HoodieTableMetaClient, TableSchemaResolver} +import org.apache.hudi.common.util.ValidationUtils.checkArgument +import org.apache.hudi.{AvroConversionUtils, HoodieFileIndex} +import org.apache.spark.sql.HoodieCatalystExpressionUtils.{resolveExpr, splitPartitionAndDataPredicates} +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.expressions.PredicateHelper +import org.apache.spark.sql.execution.datasources.FileStatusCache + +trait ProcedurePredicateHelper extends PredicateHelper { + + def prunePartition( + sparkSession: SparkSession, + metaClient: HoodieTableMetaClient, + predicate: String): Seq[String] = { + val options = Map(QUERY_TYPE.key() -> QUERY_TYPE_SNAPSHOT_OPT_VAL, "path" -> metaClient.getBasePath) + val hoodieFileIndex = HoodieFileIndex(sparkSession, metaClient, None, options, + FileStatusCache.getOrCreate(sparkSession)) + + // Resolve partition predicates + val schemaResolver = new TableSchemaResolver(metaClient) + val tableSchema = AvroConversionUtils.convertAvroSchemaToStructType(schemaResolver.getTableAvroSchema) + val condition = resolveExpr(sparkSession, predicate, tableSchema) + val partitionColumns = metaClient.getTableConfig.getPartitionFields.orElse(Array[String]()) + val (partitionPredicates, dataPredicates) = splitPartitionAndDataPredicates( + sparkSession, splitConjunctivePredicates(condition).toArray, partitionColumns) + checkArgument(dataPredicates.isEmpty, "Only partition predicates are allowed") + + // Get all partitions and prune partition by predicates + val prunedPartitions = hoodieFileIndex.getPartitionPaths(partitionPredicates) + prunedPartitions.map(partitionPath => partitionPath.getPath) + } +} diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/RunBuildProcedure.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/RunBuildProcedure.scala new file mode 100644 index 0000000000000..cb4e9ac1b5f64 --- /dev/null +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/RunBuildProcedure.scala @@ -0,0 +1,151 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.spark.sql.hudi.command.procedures + +import org.apache.hudi.HoodieCLIUtils +import org.apache.hudi.common.config.HoodieBuildConfig +import org.apache.hudi.common.table.HoodieTableMetaClient +import org.apache.hudi.common.table.timeline.{HoodieActiveTimeline, HoodieTimeline} +import org.apache.hudi.common.util.{BuildUtils, HoodieTimer, Option => HOption} +import org.apache.spark.internal.Logging +import org.apache.spark.sql.Row +import org.apache.spark.sql.types.{DataTypes, Metadata, StructField, StructType} + +import java.util.function.Supplier +import scala.collection.JavaConverters.{asScalaBufferConverter, asScalaIteratorConverter} + +class RunBuildProcedure extends BaseProcedure + with ProcedureBuilder + with ProcedurePredicateHelper + with Logging { + private val PARAMETERS = Array[ProcedureParameter]( + ProcedureParameter.optional(0, "table", DataTypes.StringType, None), + ProcedureParameter.optional(1, "path", DataTypes.StringType, None), + ProcedureParameter.optional(2, "predicate", DataTypes.StringType, None), + ProcedureParameter.optional(4, "show_involved_partition", DataTypes.BooleanType, false) + ) + + private val OUTPUT_TYPE = new StructType(Array[StructField]( + StructField("timestamp", DataTypes.StringType, nullable = true, Metadata.empty), + StructField("task_num", DataTypes.IntegerType, nullable = true, Metadata.empty), + StructField("state", DataTypes.StringType, nullable = true, Metadata.empty), + StructField("involved_partitions", DataTypes.StringType, nullable = true, Metadata.empty) + )) + + /** + * Returns the input parameters of this procedure. + */ + override def parameters: Array[ProcedureParameter] = PARAMETERS + + /** + * Returns the type of rows produced by this procedure. + */ + override def outputType: StructType = OUTPUT_TYPE + + /** + * Executes this procedure. + *

+ * Spark will align the provided arguments according to the input parameters + * defined in {@link # parameters ( )} either by position or by name before execution. + *

+ * Implementations may provide a summary of execution by returning one or many rows + * as a result. The schema of output rows must match the defined output type + * in {@link # outputType ( )}. + * + * @param args input arguments + * @return the result of executing this procedure with the given arguments + */ + override def call(args: ProcedureArgs): Seq[Row] = { + super.checkArgs(PARAMETERS, args) + + val tableName = getArgValueOrDefault(args, PARAMETERS(0)) + val tablePath = getArgValueOrDefault(args, PARAMETERS(1)) + val predicate = getArgValueOrDefault(args, PARAMETERS(2)) + val showInvolvedPartitions = getArgValueOrDefault(args, PARAMETERS(3)).get.asInstanceOf[Boolean] + + val basePath: String = getBasePath(tableName, tablePath) + val metaClient = HoodieTableMetaClient.builder().setConf(jsc.hadoopConfiguration()).setBasePath(basePath).build() + var conf: Map[String, String] = Map.empty + predicate match { + case Some(p) => + val prunedPartitions = prunePartition(spark, metaClient, p.asInstanceOf[String]) + conf = conf ++ Map( + HoodieBuildConfig.PARTITION_SELECTED.key() -> prunedPartitions.mkString(",") + ) + logInfo(s"Partition predicates: ${p}, partition selected: ${prunedPartitions}") + case _ => + logInfo("No partition predicates") + } + + var pendingBuild = BuildUtils.getAllPendingBuildPlans(metaClient) + .iterator().asScala.map(_.getLeft.getTimestamp).toSeq.sortBy(f => f) + logInfo(s"Pending build instants: ${pendingBuild.mkString(",")}") + + val client = HoodieCLIUtils.createHoodieClientFromPath(sparkSession, basePath, conf) + val instantTime = HoodieActiveTimeline.createNewInstantTime() + if (client.scheduleBuildAtInstant(instantTime, HOption.empty())) { + pendingBuild ++= Seq(instantTime) + } + logInfo(s"Build instants to run: ${pendingBuild.mkString(",")}") + + val timer = new HoodieTimer + timer.startTimer() + pendingBuild.foreach(instant => { + timer.startTimer() + client.build(instant, true) + logInfo(s"Finish build for instant: $instant, time cost: ${timer.endTimer()}ms") + }) + client.close() + logInfo(s"Finish build all instants: ${pendingBuild.mkString(",")}, time cost: ${timer.endTimer()}ms") + + val buildInstants = metaClient.reloadActiveTimeline().getInstants.iterator().asScala + .filter(p => p.getAction == HoodieTimeline.BUILD_ACTION && pendingBuild.contains(p.getTimestamp)) + .toSeq + .sortBy(f => f.getTimestamp) + .reverse + + val buildPlans = buildInstants.map(instant => + BuildUtils.getBuildPlan(metaClient, instant) + ) + + if (showInvolvedPartitions) { + buildPlans.map { p => + Row(p.get().getLeft.getTimestamp, p.get().getRight.getTasks.size(), + p.get().getLeft.getState.name(), + BuildUtils.extractPartitions(p.get().getRight.getTasks).asScala.mkString(",")) + } + } else { + buildPlans.map { p => + Row(p.get().getLeft.getTimestamp, p.get().getRight.getTasks.size(), p.get().getLeft.getState.name(), "*") + } + } + } + + override def build: Procedure = new RunBuildProcedure +} + + +object RunBuildProcedure { + val NAME = "run_build" + + def builder: Supplier[ProcedureBuilder] = new Supplier[ProcedureBuilder] { + override def get() = new RunBuildProcedure + } +} diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/RunClusteringProcedure.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/RunClusteringProcedure.scala index d34c0b0d7b762..25fa691a48d40 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/RunClusteringProcedure.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/RunClusteringProcedure.scala @@ -17,21 +17,16 @@ package org.apache.spark.sql.hudi.command.procedures -import org.apache.hudi.DataSourceReadOptions.{QUERY_TYPE, QUERY_TYPE_SNAPSHOT_OPT_VAL} +import org.apache.hudi.HoodieCLIUtils import org.apache.hudi.client.SparkRDDWriteClient -import org.apache.hudi.common.table.timeline.{HoodieActiveTimeline, HoodieInstant, HoodieTimeline} +import org.apache.hudi.common.table.timeline.{HoodieActiveTimeline, HoodieTimeline} import org.apache.hudi.common.table.{HoodieTableMetaClient, TableSchemaResolver} -import org.apache.hudi.common.util.ValidationUtils.checkArgument import org.apache.hudi.common.util.{ClusteringUtils, StringUtils, Option => HOption} import org.apache.hudi.config.HoodieClusteringConfig import org.apache.hudi.config.HoodieClusteringConfig.{ClusteringOperator, LayoutOptimizationStrategy} import org.apache.hudi.exception.HoodieClusteringException -import org.apache.hudi.{AvroConversionUtils, HoodieCLIUtils, HoodieFileIndex} import org.apache.spark.internal.Logging -import org.apache.spark.sql.HoodieCatalystExpressionUtils.{resolveExpr, splitPartitionAndDataPredicates} import org.apache.spark.sql.Row -import org.apache.spark.sql.catalyst.expressions.PredicateHelper -import org.apache.spark.sql.execution.datasources.FileStatusCache import org.apache.spark.sql.types._ import java.util.Locale @@ -40,7 +35,7 @@ import scala.collection.JavaConverters._ class RunClusteringProcedure extends BaseProcedure with ProcedureBuilder - with PredicateHelper + with ProcedurePredicateHelper with Logging { /** @@ -89,10 +84,10 @@ class RunClusteringProcedure extends BaseProcedure var conf: Map[String, String] = Map.empty predicate match { case Some(p) => - val prunedPartitions = prunePartition(metaClient, p.asInstanceOf[String]) + val prunedPartitions = prunePartition(spark, metaClient, p.asInstanceOf[String]) conf = conf ++ Map( HoodieClusteringConfig.PLAN_PARTITION_FILTER_MODE_NAME.key() -> "SELECTED_PARTITIONS", - HoodieClusteringConfig.PARTITION_SELECTED.key() -> prunedPartitions + HoodieClusteringConfig.PARTITION_SELECTED.key() -> prunedPartitions.mkString(",") ) logInfo(s"Partition predicates: $p, partition selected: $prunedPartitions") case _ => @@ -209,25 +204,6 @@ class RunClusteringProcedure extends BaseProcedure override def build: Procedure = new RunClusteringProcedure() - def prunePartition(metaClient: HoodieTableMetaClient, predicate: String): String = { - val options = Map(QUERY_TYPE.key() -> QUERY_TYPE_SNAPSHOT_OPT_VAL, "path" -> metaClient.getBasePath) - val hoodieFileIndex = HoodieFileIndex(sparkSession, metaClient, None, options, - FileStatusCache.getOrCreate(sparkSession)) - - // Resolve partition predicates - val schemaResolver = new TableSchemaResolver(metaClient) - val tableSchema = AvroConversionUtils.convertAvroSchemaToStructType(schemaResolver.getTableAvroSchema) - val condition = resolveExpr(sparkSession, predicate, tableSchema) - val partitionColumns = metaClient.getTableConfig.getPartitionFields.orElse(Array[String]()) - val (partitionPredicates, dataPredicates) = splitPartitionAndDataPredicates( - sparkSession, splitConjunctivePredicates(condition).toArray, partitionColumns) - checkArgument(dataPredicates.isEmpty, "Only partition predicates are allowed") - - // Get all partitions and prune partition by predicates - val prunedPartitions = hoodieFileIndex.getPartitionPaths(partitionPredicates) - prunedPartitions.map(partitionPath => partitionPath.getPath).toSet.mkString(",") - } - private def validateOrderColumns(orderColumns: String, metaClient: HoodieTableMetaClient): Unit = { if (orderColumns == null) { throw new HoodieClusteringException("Order columns is null") diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/RunCompactionProcedure.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/RunCompactionProcedure.scala index 3c51d7d8b29fe..94ad25b152ea0 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/RunCompactionProcedure.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/RunCompactionProcedure.scala @@ -117,6 +117,7 @@ class RunCompactionProcedure extends BaseProcedure with ProcedureBuilder with Sp } case _ => throw new UnsupportedOperationException(s"Unsupported compaction operation: $operation") } + client.close() val compactionInstants = metaClient.reloadActiveTimeline().getInstantsAsStream.iterator().asScala .filter(instant => willCompactionInstants.contains(instant.getTimestamp)) diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ShowBuildProcedure.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ShowBuildProcedure.scala new file mode 100644 index 0000000000000..cd2d6d98cfcb5 --- /dev/null +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ShowBuildProcedure.scala @@ -0,0 +1,111 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.spark.sql.hudi.command.procedures + +import org.apache.hudi.common.table.HoodieTableMetaClient +import org.apache.hudi.common.table.timeline.HoodieTimeline +import org.apache.hudi.common.util.BuildUtils +import org.apache.spark.sql.Row +import org.apache.spark.sql.types.{DataTypes, Metadata, StructField, StructType} + +import java.util.function.Supplier +import scala.collection.JavaConverters.{asScalaBufferConverter, asScalaIteratorConverter} + +class ShowBuildProcedure extends BaseProcedure with ProcedureBuilder { + private val PARAMETERS = Array[ProcedureParameter]( + ProcedureParameter.optional(0, "table", DataTypes.StringType, None), + ProcedureParameter.optional(1, "path", DataTypes.StringType, None), + ProcedureParameter.optional(2, "limit", DataTypes.IntegerType, 20), + ProcedureParameter.optional(3, "show_involved_partition", DataTypes.BooleanType, false) + ) + + private val OUTPUT_TYPE = new StructType(Array[StructField]( + StructField("timestamp", DataTypes.StringType, nullable = true, Metadata.empty), + StructField("task_num", DataTypes.IntegerType, nullable = true, Metadata.empty), + StructField("state", DataTypes.StringType, nullable = true, Metadata.empty), + StructField("involved_partitions", DataTypes.StringType, nullable = true, Metadata.empty) + )) + + /** + * Returns the input parameters of this procedure. + */ + override def parameters: Array[ProcedureParameter] = PARAMETERS + + /** + * Returns the type of rows produced by this procedure. + */ + override def outputType: StructType = OUTPUT_TYPE + + /** + * Executes this procedure. + *

+ * Spark will align the provided arguments according to the input parameters + * defined in {@link # parameters ( )} either by position or by name before execution. + *

+ * Implementations may provide a summary of execution by returning one or many rows + * as a result. The schema of output rows must match the defined output type + * in {@link # outputType ( )}. + * + * @param args input arguments + * @return the result of executing this procedure with the given arguments + */ + override def call(args: ProcedureArgs): Seq[Row] = { + super.checkArgs(PARAMETERS, args) + + val tableName = getArgValueOrDefault(args, PARAMETERS(0)) + val tablePath = getArgValueOrDefault(args, PARAMETERS(1)) + val limit = getArgValueOrDefault(args, PARAMETERS(2)).get.asInstanceOf[Int] + val showInvolvedPartitions = getArgValueOrDefault(args, PARAMETERS(3)).get.asInstanceOf[Boolean] + + val basePath: String = getBasePath(tableName, tablePath) + val metaClient = HoodieTableMetaClient.builder.setConf(jsc.hadoopConfiguration()).setBasePath(basePath).build + + val buildInstants = metaClient.getActiveTimeline.getInstants.iterator().asScala + .filter(p => p.getAction == HoodieTimeline.BUILD_ACTION) + .toSeq + .sortBy(f => f.getTimestamp) + .reverse + .take(limit) + + val buildPlans = buildInstants.map(instant => BuildUtils.getBuildPlan(metaClient, instant)) + if (showInvolvedPartitions) { + buildPlans.map { p => + Row(p.get().getLeft.getTimestamp, p.get().getRight.getTasks.size(), + p.get().getLeft.getState.name(), + BuildUtils.extractPartitions(p.get().getRight.getTasks).asScala.mkString(",")) + } + } else { + buildPlans.map { p => + Row(p.get().getLeft.getTimestamp, p.get().getRight.getTasks.size(), + p.get().getLeft.getState.name(), "*") + } + } + } + + override def build: Procedure = new ShowBuildProcedure +} + +object ShowBuildProcedure { + val NAME = "show_build" + + def builder: Supplier[ProcedureBuilder] = new Supplier[ProcedureBuilder] { + override def get() = new ShowBuildProcedure + } +} diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/command/index/TestSecondaryIndex.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/command/index/TestSecondaryIndex.scala index eae89099a621c..476bc04e5f3a9 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/command/index/TestSecondaryIndex.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/command/index/TestSecondaryIndex.scala @@ -52,7 +52,7 @@ class TestSecondaryIndex extends HoodieSparkSqlTestBase { checkAnswer(s"create index idx_price on $tableName using lucene (price options(order='desc')) options(block_size=512)")() // Create an index with multiple columns - checkException(s"create index idx_id_ts on $tableName using lucene (id, ts)")("Lucene index only support single column") + checkAnswer(s"create index idx_id_ts on $tableName using lucene (id, ts)")() // Create an index with the occupied name checkException(s"create index idx_price on $tableName using lucene (price)")( @@ -66,6 +66,7 @@ class TestSecondaryIndex extends HoodieSparkSqlTestBase { spark.sql(s"show indexes from $tableName").show() checkAnswer(s"show indexes from $tableName")( + Seq("idx_id_ts", "id,ts", "lucene", "", ""), Seq("idx_name", "name", "lucene", "", "{\"block_size\":\"1024\"}"), Seq("idx_price", "price", "lucene", "{\"price\":{\"order\":\"desc\"}}", "{\"block_size\":\"512\"}") ) @@ -75,11 +76,14 @@ class TestSecondaryIndex extends HoodieSparkSqlTestBase { spark.sql(s"show indexes from $tableName").show() checkAnswer(s"show indexes from $tableName")( + Seq("idx_id_ts", "id,ts", "lucene", "", ""), Seq("idx_price", "price", "lucene", "{\"price\":{\"order\":\"desc\"}}", "{\"block_size\":\"512\"}") ) checkAnswer(s"drop index idx_price on $tableName")() - checkAnswer(s"show indexes from $tableName")() + checkAnswer(s"show indexes from $tableName")( + Seq("idx_id_ts", "id,ts", "lucene", "", "") + ) checkException(s"drop index idx_price on $tableName")("Secondary index not exists: idx_price") diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestBuildProcedure.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestBuildProcedure.scala new file mode 100644 index 0000000000000..c16f2ee9d052c --- /dev/null +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestBuildProcedure.scala @@ -0,0 +1,530 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.spark.sql.hudi.procedure + +import org.apache.hadoop.fs.Path +import org.apache.hudi.common.table.timeline.{HoodieActiveTimeline, HoodieInstant} +import org.apache.hudi.common.util.{Option => HOption} +import org.apache.hudi.{HoodieCLIUtils, HoodieDataSourceHelpers} + +import scala.collection.JavaConverters.asScalaIteratorConverter + +class TestBuildProcedure extends HoodieSparkProcedureTestBase { + test("Test Call run_build Procedure by Table for None-Partitioned Table") { + withTempDir { tmp => + Seq("cow", "mor").foreach { tableType => + val tableName = generateTableName + val basePath = s"${tmp.getCanonicalPath}/$tableName" + spark.sql( + s""" + |create table $tableName ( + | id int, + | name string, + | price double, + | ts long + |) using hudi + | options ( + | primaryKey ='id', + | type = '$tableType', + | preCombineField = 'ts' + | ) + | location '$basePath' + """.stripMargin) + spark.sql(s"insert into $tableName values(1, 'a1', 10, 1000)") + spark.sql(s"insert into $tableName values(2, 'a2', 10, 1001)") + spark.sql(s"insert into $tableName values(3, 'a3', 10, 1002)") + val client = HoodieCLIUtils.createHoodieClientFromPath(spark, basePath, Map.empty) + // Generate the first build plan + val firstScheduledBuild = HoodieActiveTimeline.createNewInstantTime + client.scheduleBuildAtInstant(firstScheduledBuild, HOption.empty()) + + val fs = new Path(basePath).getFileSystem(spark.sessionState.newHadoopConf()) + + // No build triggered for there is no secondary index defined + val scheduledBuild = HoodieDataSourceHelpers.allBuildCommits(fs, basePath) + .getInstants + .iterator().asScala + .toSeq + assertResult(0)(scheduledBuild.size) + + // Add secondary index for this table + checkAnswer(s"create index idx_name on $tableName using lucene (name) options(block_size=1024)")() + + // Generate the secondary build plan, there are three commits, only last + // file slice is effective, so just one build task generated + val secondaryScheduledBuild = HoodieActiveTimeline.createNewInstantTime() + client.scheduleBuildAtInstant(secondaryScheduledBuild, HOption.empty()) + + // Generate the third build plan + spark.sql(s"insert into $tableName values(4, 'a4', 10, 1003)") + val thirdScheduledBuild = HoodieActiveTimeline.createNewInstantTime + client.scheduleBuildAtInstant(thirdScheduledBuild, HOption.empty()) + checkAnswer(s"call show_build('$tableName')")( + Seq(thirdScheduledBuild, 1, HoodieInstant.State.REQUESTED.name(), "*"), + Seq(secondaryScheduledBuild, 1, HoodieInstant.State.REQUESTED.name(), "*") + ) + + // Do build for all build plan generated above, and no new build + // instant will be generated because of there is no commit after the second + // build plan generated + checkAnswer(s"call run_build(table => '$tableName', show_involved_partition => true)")( + Seq(thirdScheduledBuild, 1, HoodieInstant.State.COMPLETED.name(), ""), + Seq(secondaryScheduledBuild, 1, HoodieInstant.State.COMPLETED.name(), "") + ) + + // No new commits + assertResult(false)(HoodieDataSourceHelpers.hasNewCommits(fs, basePath, thirdScheduledBuild)) + + // Check the number of finished build instants + val finishedBuild = HoodieDataSourceHelpers.allBuildCommits(fs, basePath) + .getInstants + .iterator().asScala + .filter(p => p.isCompleted) + .toSeq + assertResult(2)(finishedBuild.size) + + checkAnswer(s"select id, name, price, ts from $tableName order by id")( + Seq(1, "a1", 10.0, 1000), + Seq(2, "a2", 10.0, 1001), + Seq(3, "a3", 10.0, 1002), + Seq(4, "a4", 10.0, 1003) + ) + + // After build there should be no pending build and all build instants should be completed + checkAnswer(s"call show_build(table => '$tableName')")( + Seq(thirdScheduledBuild, 1, HoodieInstant.State.COMPLETED.name(), "*"), + Seq(secondaryScheduledBuild, 1, HoodieInstant.State.COMPLETED.name(), "*") + ) + + // Do build without manual schedule(which will do the schedule if no pending build exists) + spark.sql(s"insert into $tableName values(5, 'a5', 10, 1004)") + spark.sql(s"insert into $tableName values(6, 'a6', 10, 1005)") + spark.sql(s"call run_build(table => '$tableName', show_involved_partition => true)").show() + + val fourthBuildInstant = HoodieDataSourceHelpers.allBuildCommits(fs, basePath) + .findInstantsAfter(thirdScheduledBuild) + .getInstants + .iterator().asScala + .filter(p => p.isCompleted) + .toSeq + // Should have a new replace commit after the second build command. + assertResult(1)(fourthBuildInstant.size) + + checkAnswer(s"select id, name, price, ts from $tableName order by id")( + Seq(1, "a1", 10.0, 1000), + Seq(2, "a2", 10.0, 1001), + Seq(3, "a3", 10.0, 1002), + Seq(4, "a4", 10.0, 1003), + Seq(5, "a5", 10.0, 1004), + Seq(6, "a6", 10.0, 1005) + ) + } + } + } + + test("Test Call run_build Procedure by Table for Partitioned Table") { + withTempDir { tmp => + Seq("cow", "mor").foreach { tableType => + val tableName = generateTableName + val basePath = s"${tmp.getCanonicalPath}/$tableName" + spark.sql( + s""" + |create table $tableName ( + | id int, + | name string, + | price double, + | ts long + |) using hudi + | options ( + | primaryKey ='id', + | type = '$tableType', + | preCombineField = 'ts' + | ) + | partitioned by(ts) + | location '$basePath' + """.stripMargin) + spark.sql(s"insert into $tableName values(1, 'a1', 10, 1000)") + spark.sql(s"insert into $tableName values(2, 'a2', 10, 1001)") + spark.sql(s"insert into $tableName values(3, 'a3', 10, 1002)") + val client = HoodieCLIUtils.createHoodieClientFromPath(spark, basePath, Map.empty) + // Generate the first build plan + val firstScheduledBuild = HoodieActiveTimeline.createNewInstantTime + client.scheduleBuildAtInstant(firstScheduledBuild, HOption.empty()) + + val fs = new Path(basePath).getFileSystem(spark.sessionState.newHadoopConf()) + + // No build triggered for there is no secondary index defined + val scheduledBuild = HoodieDataSourceHelpers.allBuildCommits(fs, basePath) + .getInstants + .iterator().asScala + .toSeq + assertResult(0)(scheduledBuild.size) + + // Add secondary index for this table + checkAnswer(s"create index idx_name on $tableName using lucene (name) options(block_size=1024)")() + + // Generate the secondary build plan + val secondaryScheduledBuild = HoodieActiveTimeline.createNewInstantTime() + client.scheduleBuildAtInstant(secondaryScheduledBuild, HOption.empty()) + + // Generate the third build plan + spark.sql(s"insert into $tableName values(4, 'a4', 10, 1003)") + val thirdScheduledBuild = HoodieActiveTimeline.createNewInstantTime + client.scheduleBuildAtInstant(thirdScheduledBuild, HOption.empty()) + checkAnswer(s"call show_build('$tableName')")( + Seq(thirdScheduledBuild, 1, HoodieInstant.State.REQUESTED.name(), "*"), + Seq(secondaryScheduledBuild, 3, HoodieInstant.State.REQUESTED.name(), "*") + ) + + // Do build for all build plan generated above, and no new build + // instant will be generated because of there is no commit after the second + // build plan generated + checkAnswer(s"call run_build(table => '$tableName', show_involved_partition => true)")( + Seq(thirdScheduledBuild, 1, HoodieInstant.State.COMPLETED.name(), "ts=1003"), + Seq(secondaryScheduledBuild, 3, HoodieInstant.State.COMPLETED.name(), "ts=1000,ts=1001,ts=1002") + ) + + // No new commits + assertResult(false)(HoodieDataSourceHelpers.hasNewCommits(fs, basePath, thirdScheduledBuild)) + + // Check the number of finished build instants + val finishedBuild = HoodieDataSourceHelpers.allBuildCommits(fs, basePath) + .getInstants + .iterator().asScala + .filter(p => p.isCompleted) + .toSeq + assertResult(2)(finishedBuild.size) + + checkAnswer(s"select id, name, price, ts from $tableName order by id")( + Seq(1, "a1", 10.0, 1000), + Seq(2, "a2", 10.0, 1001), + Seq(3, "a3", 10.0, 1002), + Seq(4, "a4", 10.0, 1003) + ) + + // After build there should be no pending build and all build instants should be completed + checkAnswer(s"call show_build(table => '$tableName')")( + Seq(thirdScheduledBuild, 1, HoodieInstant.State.COMPLETED.name(), "*"), + Seq(secondaryScheduledBuild, 3, HoodieInstant.State.COMPLETED.name(), "*") + ) + + // Do build without manual schedule(which will do the schedule if no pending build exists) + spark.sql(s"insert into $tableName values(5, 'a5', 10, 1004)") + spark.sql(s"insert into $tableName values(6, 'a6', 10, 1005)") + spark.sql(s"call run_build(table => '$tableName', show_involved_partition => true)").show() + + val fourthBuildInstant = HoodieDataSourceHelpers.allBuildCommits(fs, basePath) + .findInstantsAfter(thirdScheduledBuild) + .getInstants + .iterator().asScala + .filter(p => p.isCompleted) + .toSeq + // Should have a new replace commit after the second build command. + assertResult(1)(fourthBuildInstant.size) + + checkAnswer(s"select id, name, price, ts from $tableName order by id")( + Seq(1, "a1", 10.0, 1000), + Seq(2, "a2", 10.0, 1001), + Seq(3, "a3", 10.0, 1002), + Seq(4, "a4", 10.0, 1003), + Seq(5, "a5", 10.0, 1004), + Seq(6, "a6", 10.0, 1005) + ) + } + } + } + + test("Test Call run_build Procedure by Path for Partitioned Table") { + withTempDir { tmp => + Seq("cow", "mor").foreach { tableType => + val tableName = generateTableName + val basePath = s"${tmp.getCanonicalPath}/$tableName" + spark.sql( + s""" + |create table $tableName ( + | id int, + | name string, + | price double, + | ts long + |) using hudi + | options ( + | primaryKey ='id', + | type = '$tableType', + | preCombineField = 'ts' + | ) + | partitioned by(ts) + | location '$basePath' + """.stripMargin) + + spark.sql(s"call run_build(path => '$basePath')").show() + checkAnswer(s"call show_build(path => '$basePath')")() + + spark.sql(s"insert into $tableName values(1, 'a1', 10, 1000)") + spark.sql(s"insert into $tableName values(2, 'a2', 10, 1001)") + spark.sql(s"insert into $tableName values(3, 'a3', 10, 1002)") + val client = HoodieCLIUtils.createHoodieClientFromPath(spark, basePath, Map.empty) + // Generate the first build plan + val firstScheduledBuild = HoodieActiveTimeline.createNewInstantTime + client.scheduleBuildAtInstant(firstScheduledBuild, HOption.empty()) + + val fs = new Path(basePath).getFileSystem(spark.sessionState.newHadoopConf()) + + // No build triggered for there is no secondary index defined + val scheduledBuild = HoodieDataSourceHelpers.allBuildCommits(fs, basePath) + .getInstants + .iterator().asScala + .toSeq + assertResult(0)(scheduledBuild.size) + + // Add secondary index for this table + checkAnswer(s"create index idx_name on $tableName using lucene (name) options(block_size=1024)")() + + val secondaryScheduledBuild = HoodieActiveTimeline.createNewInstantTime() + client.scheduleBuildAtInstant(secondaryScheduledBuild, HOption.empty()) + checkAnswer(s"call show_build(path => '$basePath', show_involved_partition => true)")( + Seq(secondaryScheduledBuild, 3, HoodieInstant.State.REQUESTED.name(), "ts=1000,ts=1001,ts=1002") + ) + // Do build for all the build plan + checkAnswer(s"call run_build(path => '$basePath')")( + Seq(secondaryScheduledBuild, 3, HoodieInstant.State.COMPLETED.name(), "*") + ) + + checkAnswer(s"select id, name, price, ts from $tableName order by id")( + Seq(1, "a1", 10.0, 1000), + Seq(2, "a2", 10.0, 1001), + Seq(3, "a3", 10.0, 1002) + ) + + assertResult(false)(HoodieDataSourceHelpers.hasNewCommits(fs, basePath, secondaryScheduledBuild)) + + // Check the number of finished build instants + var finishedBuild = HoodieDataSourceHelpers.allBuildCommits(fs, basePath) + .getInstants + .iterator().asScala + .filter(p => p.isCompleted) + .toSeq + assertResult(1)(finishedBuild.size) + + // Do build without manual schedule(which will do the schedule if no pending build exists) + spark.sql(s"insert into $tableName values(4, 'a4', 10, 1003)") + spark.sql(s"insert into $tableName values(5, 'a5', 10, 1004)") + val resultA = spark.sql(s"call run_build(table => '$tableName', predicate => 'ts <= 1003L', show_involved_partition => true)") + .collect() + .map(row => Seq(row.getString(0), row.getInt(1), row.getString(2), row.getString(3))) + assertResult(1)(resultA.length) + assertResult("ts=1003")(resultA(0)(3)) + + checkAnswer(s"select id, name, price, ts from $tableName order by id")( + Seq(1, "a1", 10.0, 1000), + Seq(2, "a2", 10.0, 1001), + Seq(3, "a3", 10.0, 1002), + Seq(4, "a4", 10.0, 1003), + Seq(5, "a5", 10.0, 1004) + ) + + finishedBuild = HoodieDataSourceHelpers.allBuildCommits(fs, basePath) + .getInstants + .iterator().asScala + .filter(p => p.isCompleted) + .toSeq + assertResult(2)(finishedBuild.size) + } + } + } + + test("Test Call run_build Procedure with Partition Pruning") { + withTempDir { tmp => + Seq("cow", "mor").foreach { tableType => + val tableName = generateTableName + val basePath = s"${tmp.getCanonicalPath}/$tableName" + spark.sql( + s""" + |create table $tableName ( + | id int, + | name string, + | price double, + | ts long + |) using hudi + | options ( + | primaryKey ='id', + | type = '$tableType', + | preCombineField = 'ts' + | ) + | partitioned by(ts) + | location '$basePath' + """.stripMargin) + + // Add secondary index for this table + checkAnswer(s"create index idx_name on $tableName using lucene (name) options(block_size=1024)")() + + val fs = new Path(basePath).getFileSystem(spark.sessionState.newHadoopConf()) + + // Test partition pruning with single predicate + var resultA: Array[Seq[Any]] = Array.empty + + { + spark.sql(s"insert into $tableName values(1, 'a1', 10, 1000)") + spark.sql(s"insert into $tableName values(2, 'a2', 10, 1001)") + spark.sql(s"insert into $tableName values(3, 'a3', 10, 1002)") + + checkException( + s"call run_build(table => '$tableName', predicate => 'ts <= 1001L and id = 10')" + )("Only partition predicates are allowed") + + // Do build table with partition predicate + resultA = spark.sql(s"call run_build(table => '$tableName', predicate => 'ts <= 1001L', show_involved_partition => true)") + .collect() + .map(row => Seq(row.getString(0), row.getInt(1), row.getString(2), row.getString(3))) + assertResult(1)(resultA.length) + assertResult("ts=1000,ts=1001")(resultA(0)(3)) + + // There is 1 completed build instant + val buildInstants = HoodieDataSourceHelpers.allBuildCommits(fs, basePath) + .getInstants + .iterator().asScala + .filter(p => p.isCompleted) + .toSeq + assertResult(1)(buildInstants.size) + + // The latest build should contain 2 build tasks(1000,1001) + val buildInstant = buildInstants.last + val buildPlan = HoodieDataSourceHelpers.getBuildPlan(fs, basePath, buildInstant.getTimestamp) + assertResult(true)(buildPlan.isPresent) + assertResult(2)(buildPlan.get().getTasks.size()) + + // All build instants are completed + checkAnswer(s"call show_build(table => '$tableName', show_involved_partition => true)")( + Seq(resultA(0).head, resultA(0)(1), HoodieInstant.State.COMPLETED.name(), "ts=1000,ts=1001") + ) + + checkAnswer(s"select id, name, price, ts from $tableName order by id")( + Seq(1, "a1", 10.0, 1000), + Seq(2, "a2", 10.0, 1001), + Seq(3, "a3", 10.0, 1002) + ) + } + + // Test partition pruning with {@code And} predicates + var resultB: Array[Seq[Any]] = Array.empty + + { + spark.sql(s"insert into $tableName values(4, 'a4', 10, 1003)") + spark.sql(s"insert into $tableName values(5, 'a5', 10, 1004)") + spark.sql(s"insert into $tableName values(6, 'a6', 10, 1005)") + + checkException( + s"call run_build(table => '$tableName', predicate => 'ts > 1001L and ts <= 1005L and id = 10')" + )("Only partition predicates are allowed") + + // Do build table with partition predicate + resultB = spark.sql(s"call run_build(table => '$tableName', predicate => 'ts > 1001L and ts <= 1005L', show_involved_partition => true)") + .collect() + .map(row => Seq(row.getString(0), row.getInt(1), row.getString(2), row.getString(3))) + assertResult(1)(resultB.length) + assertResult("ts=1002,ts=1003,ts=1004,ts=1005")(resultB(0)(3)) + + // There are 2 completed build instants + val buildInstants = HoodieDataSourceHelpers.allBuildCommits(fs, basePath) + .getInstants + .iterator().asScala + .filter(p => p.isCompleted) + .toSeq + assertResult(2)(buildInstants.size) + + // The latest build should contain 4 build tasks(1002,1003,1004,1005) + val buildInstant = buildInstants.last + val buildPlan = HoodieDataSourceHelpers.getBuildPlan(fs, basePath, buildInstant.getTimestamp) + assertResult(true)(buildPlan.isPresent) + assertResult(4)(buildPlan.get().getTasks.size()) + + // All build instants are completed + checkAnswer(s"call show_build(table => '$tableName', show_involved_partition => true)")( + Seq(resultA(0).head, resultA(0)(1), HoodieInstant.State.COMPLETED.name(), "ts=1000,ts=1001"), + Seq(resultB(0).head, resultB(0)(1), HoodieInstant.State.COMPLETED.name(), "ts=1002,ts=1003,ts=1004,ts=1005") + ) + + checkAnswer(s"select id, name, price, ts from $tableName order by id")( + Seq(1, "a1", 10.0, 1000), + Seq(2, "a2", 10.0, 1001), + Seq(3, "a3", 10.0, 1002), + Seq(4, "a4", 10.0, 1003), + Seq(5, "a5", 10.0, 1004), + Seq(6, "a6", 10.0, 1005) + ) + } + + // Test partition pruning with {@code And}-{@code Or} predicates + var resultC: Array[Seq[Any]] = Array.empty + + { + spark.sql(s"insert into $tableName values(7, 'a7', 10, 1006)") + spark.sql(s"insert into $tableName values(8, 'a8', 10, 1007)") + spark.sql(s"insert into $tableName values(9, 'a9', 10, 1008)") + spark.sql(s"insert into $tableName values(10, 'a10', 10, 1009)") + + checkException( + s"call run_build(table => '$tableName', predicate => 'ts < 1007L or ts >= 1008L or id = 10')" + )("Only partition predicates are allowed") + + // Do build table with partition predicate + resultC = spark.sql(s"call run_build(table => '$tableName', predicate => '(ts >= 1006L and ts < 1008L) or ts >= 1009L', show_involved_partition => true)") + .collect() + .map(row => Seq(row.getString(0), row.getInt(1), row.getString(2), row.getString(3))) + assertResult(1)(resultC.length) + assertResult("ts=1006,ts=1007,ts=1009")(resultC(0)(3)) + + // There are 3 completed build instants + val buildInstants = HoodieDataSourceHelpers.allBuildCommits(fs, basePath) + .getInstants + .iterator().asScala + .filter(p => p.isCompleted) + .toSeq + assertResult(3)(buildInstants.size) + + // The latest build should contain 3 build tasks(1006,1007,1009) + val buildInstant = buildInstants.last + val buildPlan = HoodieDataSourceHelpers.getBuildPlan(fs, basePath, buildInstant.getTimestamp) + assertResult(true)(buildPlan.isPresent) + assertResult(3)(buildPlan.get().getTasks.size()) + + // All build instants are completed + checkAnswer(s"call show_build(table => '$tableName', show_involved_partition => true)")( + Seq(resultA(0).head, resultA(0)(1), HoodieInstant.State.COMPLETED.name(), "ts=1000,ts=1001"), + Seq(resultB(0).head, resultB(0)(1), HoodieInstant.State.COMPLETED.name(), "ts=1002,ts=1003,ts=1004,ts=1005"), + Seq(resultC(0).head, resultC(0)(1), HoodieInstant.State.COMPLETED.name(), "ts=1006,ts=1007,ts=1009") + ) + + checkAnswer(s"select id, name, price, ts from $tableName order by id")( + Seq(1, "a1", 10.0, 1000), + Seq(2, "a2", 10.0, 1001), + Seq(3, "a3", 10.0, 1002), + Seq(4, "a4", 10.0, 1003), + Seq(5, "a5", 10.0, 1004), + Seq(6, "a6", 10.0, 1005), + Seq(7, "a7", 10.0, 1006), + Seq(8, "a8", 10.0, 1007), + Seq(9, "a9", 10.0, 1008), + Seq(10, "a10", 10.0, 1009) + ) + } + } + } + } +} diff --git a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/RequestHandler.java b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/RequestHandler.java index a295036b6d030..b34d239ba409a 100644 --- a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/RequestHandler.java +++ b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/RequestHandler.java @@ -29,6 +29,7 @@ import org.apache.hudi.common.table.timeline.dto.FileGroupDTO; import org.apache.hudi.common.table.timeline.dto.FileSliceDTO; import org.apache.hudi.common.table.timeline.dto.InstantDTO; +import org.apache.hudi.common.table.timeline.dto.SecondaryIndexBaseFilesDTO; import org.apache.hudi.common.table.timeline.dto.TimelineDTO; import org.apache.hudi.common.table.view.FileSystemViewManager; import org.apache.hudi.common.table.view.RemoteHoodieTableFileSystemView; @@ -430,6 +431,20 @@ private void registerFileSlicesAPI() { ctx.queryParamAsClass(RemoteHoodieTableFileSystemView.BASEPATH_PARAM, String.class).getOrThrow(e -> new HoodieException("Basepath is invalid"))); writeValueAsString(ctx, dtos); }, true)); + + app.get(RemoteHoodieTableFileSystemView.PENDING_SECONDARY_INDEX_BASE_FILES, new ViewHandler(ctx -> { + metricsRegistry.add("PENDING_SECONDARY_INDEX_BASE_FILES", 1); + List dtos = sliceHandler.getPendingSecondaryIndexFiles( + ctx.queryParamAsClass(RemoteHoodieTableFileSystemView.BASEPATH_PARAM, String.class).getOrThrow(e -> new HoodieException("Basepath is invalid"))); + writeValueAsString(ctx, dtos); + }, true)); + + app.get(RemoteHoodieTableFileSystemView.COMPLETED_SECONDARY_INDEX_BASE_FILES, new ViewHandler(ctx -> { + metricsRegistry.add("COMPLETED_SECONDARY_INDEX_BASE_FILES", 1); + List dtos = sliceHandler.getSecondaryIndexFiles( + ctx.queryParamAsClass(RemoteHoodieTableFileSystemView.BASEPATH_PARAM, String.class).getOrThrow(e -> new HoodieException("Basepath is invalid"))); + writeValueAsString(ctx, dtos); + }, true)); } private void registerMarkerAPI() { diff --git a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/FileSliceHandler.java b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/FileSliceHandler.java index caf1e3c93997c..27a2ea8da63bb 100644 --- a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/FileSliceHandler.java +++ b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/FileSliceHandler.java @@ -18,16 +18,17 @@ package org.apache.hudi.timeline.service.handlers; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileSystem; - import org.apache.hudi.common.table.timeline.dto.ClusteringOpDTO; import org.apache.hudi.common.table.timeline.dto.CompactionOpDTO; import org.apache.hudi.common.table.timeline.dto.FileGroupDTO; import org.apache.hudi.common.table.timeline.dto.FileSliceDTO; +import org.apache.hudi.common.table.timeline.dto.SecondaryIndexBaseFilesDTO; import org.apache.hudi.common.table.view.FileSystemViewManager; import org.apache.hudi.timeline.service.TimelineService; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; + import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; @@ -108,7 +109,7 @@ public List getReplacedFileGroupsBefore(String basePath, String ma return viewManager.getFileSystemView(basePath).getReplacedFileGroupsBefore(maxCommitTime, partitionPath).map(FileGroupDTO::fromFileGroup) .collect(Collectors.toList()); } - + public List getAllReplacedFileGroups(String basePath, String partitionPath) { return viewManager.getFileSystemView(basePath).getAllReplacedFileGroups(partitionPath).map(FileGroupDTO::fromFileGroup) .collect(Collectors.toList()); @@ -120,6 +121,18 @@ public List getFileGroupsInPendingClustering(String basePath) { .collect(Collectors.toList()); } + public List getPendingSecondaryIndexFiles(String basePath) { + return viewManager.getFileSystemView(basePath).getPendingSecondaryIndexBaseFiles() + .map(p -> SecondaryIndexBaseFilesDTO.fromSecondaryIndexBaseFiles(p.getLeft(), p.getRight())) + .collect(Collectors.toList()); + } + + public List getSecondaryIndexFiles(String basePath) { + return viewManager.getFileSystemView(basePath).getSecondaryIndexBaseFiles() + .map(p -> SecondaryIndexBaseFilesDTO.fromSecondaryIndexBaseFiles(p.getLeft(), p.getRight())) + .collect(Collectors.toList()); + } + public boolean refreshTable(String basePath) { viewManager.clearFileSystemView(basePath); return true;