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 e309dd07d4cd0..c14d864de8f40 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 @@ -21,6 +21,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; @@ -41,6 +42,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; @@ -141,6 +143,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 */ @@ -1353,6 +1378,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); } @@ -1383,9 +1413,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 90a468368f1ae..bcec168ec4afa 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; @@ -2142,6 +2143,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 d13110feef228..f25eb6574cf84 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 @@ -43,6 +43,7 @@ public class HoodieMetrics { public String finalizeTimerName = null; public String compactionTimerName = null; public String indexTimerName = null; + public String buildTimerName = null; private HoodieWriteConfig config; private String tableName; private Timer rollbackTimer = null; @@ -53,6 +54,7 @@ public class HoodieMetrics { private Timer compactionTimer = null; private Timer clusteringTimer = null; private Timer indexTimer = null; + private Timer buildTimer = null; public HoodieMetrics(HoodieWriteConfig config) { this.config = config; @@ -67,6 +69,7 @@ public HoodieMetrics(HoodieWriteConfig config) { this.finalizeTimerName = getMetricsName("timer", "finalize"); this.compactionTimerName = getMetricsName("timer", HoodieTimeline.COMPACTION_ACTION); this.indexTimerName = getMetricsName("timer", "index"); + this.buildTimerName = getMetricsName("timer", HoodieTimeline.BUILD_ACTION); } } @@ -130,6 +133,13 @@ public Timer.Context getIndexCtx() { return indexTimer == null ? null : indexTimer.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. @@ -234,6 +244,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 5ca3aee764afe..b4d9139c5663b 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; @@ -533,8 +535,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()); @@ -564,6 +587,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..ac7c3d85f2e40 --- /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.ISecondaryIndexBuilder; +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(ISecondaryIndexBuilder::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 ISecondaryIndexBuilder 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 495cef117a8c8..d96c2d81985a4 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 @@ -156,7 +156,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 4add51886fe3a..c62d9bd17e42d 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 53a5799508470..a0d8c54c898f1 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; @@ -490,6 +491,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 2c8a3c4e49c3a..9703834228108 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; @@ -419,4 +421,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 88921334980ed..e54e8f74bcb82 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; @@ -317,4 +322,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 a142fd80d4bf8..4199ca4e56410 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 @@ -25,6 +25,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; @@ -40,6 +41,7 @@ import org.apache.hudi.common.util.Option; 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.index.HoodieIndex; import org.apache.hudi.index.SparkHoodieIndexFactory; @@ -62,6 +64,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; @@ -403,6 +406,55 @@ private void completeClustering(HoodieReplaceCommitMetadata metadata, LOG.info("Clustering successfully on commit " + clusteringCommitTime); } + @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()); @@ -450,6 +502,9 @@ private void completeTableService(TableServiceType tableServiceType, HoodieCommi case COMPACT: completeCompaction(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 a88ca65c35a94..7071f30da6f72 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; @@ -305,4 +310,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 d3f37864e3ee3..ae78101eaf259 100644 --- a/hudi-common/pom.xml +++ b/hudi-common/pom.xml @@ -27,6 +27,8 @@ ${project.parent.basedir} + 8.11.1 + 0.9.28 @@ -92,6 +94,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 @@ -107,6 +111,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..7d22df181bc43 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieBuildTaskConfig.java @@ -0,0 +1,143 @@ +/* + * 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.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(); + } + + 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 6d4236b048d2d..61016e96345af 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 @@ -18,11 +18,12 @@ package org.apache.hudi.common.config; -import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.ReflectionUtils; import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.exception.HoodieException; + +import org.apache.hadoop.fs.FSDataInputStream; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; @@ -192,6 +193,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..2be9f75b558f4 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieSecondaryIndexConfig.java @@ -0,0 +1,25 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.hudi.common.config; + +public class HoodieSecondaryIndexConfig { + public static final String HOODIE_SECONDARY_INDEX_DATA = "hoodie.secondary.index.data"; + public static final String HOODIE_SECONDARY_INDEX_FILTER = "hoodie.secondary.index.filter"; +} 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 69dd30782ff77..04961ce2da09a 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; + ARCHIVE, COMPACT, CLUSTER, CLEAN, BUILD; public String getAction() { switch (this) { @@ -37,6 +37,8 @@ public String getAction() { return HoodieTimeline.CLEAN_ACTION; case CLUSTER: return HoodieTimeline.REPLACE_COMMIT_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 f2f3809cf5c3a..5afd3ea3aa6a9 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"), @@ -94,6 +97,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/HoodieTableConfig.java b/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableConfig.java index 6b64ec4897b7c..c69707cf9712e 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableConfig.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableConfig.java @@ -235,6 +235,11 @@ public class HoodieTableConfig extends HoodieConfig { .withDocumentation("Comma-separated list of metadata partitions that have been completely built and in-sync with data table. " + "These partitions are ready for use by the readers"); + public static final ConfigProperty SECONDARY_INDEXES_METADATA = ConfigProperty + .key("hoodie.table.secondary.indexes.metadata") + .noDefaultValue() + .withDocumentation("The metadata of secondary indexes"); + private static final String TABLE_CHECKSUM_FORMAT = "%s.%s"; // . public HoodieTableConfig(FileSystem fs, String metaPath, String payloadClassName) { @@ -499,6 +504,14 @@ public Option getPartitionFields() { return Option.empty(); } + public Option getSecondaryIndexesMetadata() { + if (contains(SECONDARY_INDEXES_METADATA)) { + return Option.of(getString(SECONDARY_INDEXES_METADATA)); + } + + return Option.empty(); + } + /** * @returns the partition field prop. */ 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 16dd373486f61..dd7142db38dd3 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 @@ -276,6 +277,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 */ @@ -742,6 +752,7 @@ public static class PropertyBuilder { private Boolean shouldDropPartitionColumns; private String metadataPartitions; private String inflightMetadataPartitions; + private String secondaryIndexesMetadata; /** * Persist the configs that is written at the first time, and should not be changed. @@ -876,6 +887,11 @@ public PropertyBuilder setInflightMetadataPartitions(String partitions) { return this; } + public PropertyBuilder setSecondaryIndexesMetadata(String secondaryIndexesMetadata) { + this.secondaryIndexesMetadata = secondaryIndexesMetadata; + return this; + } + private void set(String key, Object value) { if (HoodieTableConfig.PERSISTED_CONFIG_LIST.contains(key)) { this.others.put(key, value); @@ -883,7 +899,7 @@ private void set(String key, Object value) { } public PropertyBuilder set(Map props) { - for (String key: HoodieTableConfig.PERSISTED_CONFIG_LIST) { + for (String key : HoodieTableConfig.PERSISTED_CONFIG_LIST) { Object value = props.get(key); if (value != null) { set(key, value); @@ -982,6 +998,9 @@ public PropertyBuilder fromProperties(Properties properties) { if (hoodieConfig.contains(HoodieTableConfig.TABLE_METADATA_PARTITIONS_INFLIGHT)) { setInflightMetadataPartitions(hoodieConfig.getString(HoodieTableConfig.TABLE_METADATA_PARTITIONS_INFLIGHT)); } + if (hoodieConfig.contains(HoodieTableConfig.SECONDARY_INDEXES_METADATA)) { + setSecondaryIndexesMetadata(hoodieConfig.getString(HoodieTableConfig.SECONDARY_INDEXES_METADATA)); + } return this; } @@ -1072,6 +1091,9 @@ public Properties build() { if (null != inflightMetadataPartitions) { tableConfig.setValue(HoodieTableConfig.TABLE_METADATA_PARTITIONS_INFLIGHT, inflightMetadataPartitions); } + if (null != secondaryIndexesMetadata) { + tableConfig.setValue(HoodieTableConfig.SECONDARY_INDEXES_METADATA, secondaryIndexesMetadata); + } return tableConfig.getProps(); } 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 2b27d3ab5e568..c13728d108132 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 @@ -73,6 +73,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) {{ @@ -554,6 +555,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); } @@ -741,6 +774,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 4df30b115e0ea..9dc6890ae6cc7 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 @@ -363,7 +363,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, REPLACE_COMMIT_ACTION); + Set validActions = CollectionUtils.createSet(COMMIT_ACTION, DELTA_COMMIT_ACTION, COMPACTION_ACTION, REPLACE_COMMIT_ACTION, BUILD_ACTION); return new HoodieDefaultTimeline(getInstants().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 7324421894c0d..8aae71b860463 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 @@ -110,7 +110,7 @@ public HoodieTimeline filterCompletedAndCompactionInstants() { @Override public HoodieDefaultTimeline getWriteTimeline() { - Set validActions = CollectionUtils.createSet(COMMIT_ACTION, DELTA_COMMIT_ACTION, COMPACTION_ACTION, REPLACE_COMMIT_ACTION); + Set validActions = CollectionUtils.createSet(COMMIT_ACTION, DELTA_COMMIT_ACTION, COMPACTION_ACTION, REPLACE_COMMIT_ACTION, BUILD_ACTION); return new HoodieDefaultTimeline(instants.stream().filter(s -> validActions.contains(s.getAction())), details); } @@ -203,6 +203,18 @@ public HoodieTimeline filterCompletedIndexTimeline() { return new HoodieDefaultTimeline(instants.stream().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 8b1cb875c09f6..697d9a5dad4e5 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 @@ -19,6 +19,7 @@ package org.apache.hudi.common.table.timeline; import org.apache.hudi.common.util.CollectionUtils; + import org.apache.hadoop.fs.FileStatus; import java.io.Serializable; @@ -180,6 +181,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 e52a2795969ab..199063d7bde01 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 @@ -56,6 +56,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"; @@ -90,6 +91,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; @@ -142,7 +146,7 @@ public interface HoodieTimeline extends Serializable { HoodieTimeline filterCompletedAndCompactionInstants(); /** - * Timeline to just include commits (commit/deltacommit), compaction and replace actions. + * Timeline to just include commits (commit/deltacommit), compaction, replace and build actions. * * @return */ @@ -226,6 +230,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. * @@ -386,10 +400,19 @@ 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). - * @param instant Hoodie Instant + * + * @param instant Hoodie Instant * @param tableType Hoodie Table Type * @return Inflight Hoodie Instant */ @@ -512,6 +535,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 b50846b8780bf..49397d9d864dd 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; @@ -147,6 +148,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); @@ -198,6 +203,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 5818636caef2c..bf2597afb4f9b 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; @@ -106,12 +111,16 @@ 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())))); resetBootstrapBaseFileMapping(Stream.empty()); resetFileGroupsInPendingClustering(ClusteringUtils.getAllFileGroupsInPendingClusteringPlans(metaClient)); + + // Load completed/pending secondary instants + resetSecondaryIndexBaseFiles(visibleCommitsAndCompactionTimeline); + resetPendingSecondaryIndexBaseFiles(BuildUtils.getAllPendingBuildSecondaryIndexes(metaClient)); } /** @@ -245,6 +254,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 { @@ -774,6 +839,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 /** @@ -893,7 +978,6 @@ protected abstract Option> getPendingCompactio */ abstract Stream fetchBootstrapBaseFiles(); - /** * Checks if partition is pre-loaded and available in store. * @@ -944,6 +1028,26 @@ protected abstract Option> getPendingCompactio */ 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 9dac36081384b..eba2267b76329 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; @@ -77,6 +78,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. */ @@ -158,11 +169,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); } @@ -360,6 +381,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. * @@ -381,6 +422,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 ff44c7cef017b..bd65ce3326e01 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; /** @@ -229,6 +231,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 bd18ba22a25d6..b6bc722ba8f69 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; @@ -99,9 +101,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"); @@ -441,13 +448,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 02a406e7e0763..66c55e1e0d552 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 @@ -18,8 +18,6 @@ package org.apache.hudi.common.table.view; -import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.fs.Path; import org.apache.hudi.common.model.BootstrapBaseFileMapping; import org.apache.hudi.common.model.CompactionOperation; import org.apache.hudi.common.model.FileSlice; @@ -35,6 +33,10 @@ 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; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; @@ -485,6 +487,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 c32e2cabb1012..e90a58a36e50f 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; /** @@ -185,4 +187,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..249c064c657eb --- /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.getInstants() + .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 e9377a6674383..7d0a1cde56f13 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 { @@ -49,6 +52,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("/", "_"); @@ -57,11 +62,14 @@ 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(), getColFamilyForBootstrapBaseFile(), - getColFamilyForStoredPartitions(), getColFamilyForReplacedFileGroups(), getColFamilyForFileGroupsInPendingClustering()); + getColFamilyForStoredPartitions(), getColFamilyForReplacedFileGroups(), getColFamilyForFileGroupsInPendingClustering(), + getColFamilyForCompletedSecondaryIndexBaseFiles(), getColFamilyForPendingSecondaryIndexBaseFiles()); } public String getKeyForPartitionLookup(String partition) { @@ -122,6 +130,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; } @@ -145,4 +157,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/exception/HoodieIOException.java b/hudi-common/src/main/java/org/apache/hudi/exception/HoodieIOException.java index f0cabd6620cba..6ce57a5fc781f 100644 --- a/hudi-common/src/main/java/org/apache/hudi/exception/HoodieIOException.java +++ b/hudi-common/src/main/java/org/apache/hudi/exception/HoodieIOException.java @@ -34,6 +34,10 @@ public HoodieIOException(String msg, IOException t) { this.ioException = t; } + public HoodieIOException(IOException t) { + super(t); + } + public HoodieIOException(String msg) { super(msg); } diff --git a/hudi-common/src/main/java/org/apache/hudi/exception/HoodieSecondaryIndexException.java b/hudi-common/src/main/java/org/apache/hudi/exception/HoodieSecondaryIndexException.java new file mode 100644 index 0000000000000..361416c9090a5 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/exception/HoodieSecondaryIndexException.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 HoodieSecondaryIndexException extends HoodieException { + public HoodieSecondaryIndexException(String message) { + super(message); + } + + public HoodieSecondaryIndexException(String message, Throwable t) { + super(message, t); + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/internal/schema/convert/AvroInternalSchemaConverter.java b/hudi-common/src/main/java/org/apache/hudi/internal/schema/convert/AvroInternalSchemaConverter.java index 360134f92803b..b6136335dd26b 100644 --- a/hudi-common/src/main/java/org/apache/hudi/internal/schema/convert/AvroInternalSchemaConverter.java +++ b/hudi-common/src/main/java/org/apache/hudi/internal/schema/convert/AvroInternalSchemaConverter.java @@ -18,15 +18,18 @@ package org.apache.hudi.internal.schema.convert; -import org.apache.avro.JsonProperties; -import org.apache.avro.LogicalType; -import org.apache.avro.LogicalTypes; -import org.apache.avro.Schema; import org.apache.hudi.internal.schema.HoodieSchemaException; import org.apache.hudi.internal.schema.InternalSchema; import org.apache.hudi.internal.schema.Type; import org.apache.hudi.internal.schema.Types; +import org.apache.avro.JsonProperties; +import org.apache.avro.LogicalType; +import org.apache.avro.LogicalTypes; +import org.apache.avro.Schema; +import org.apache.parquet.avro.AvroSchemaConverter; +import org.apache.parquet.schema.MessageType; + import java.util.ArrayList; import java.util.Deque; import java.util.HashMap; @@ -76,18 +79,32 @@ public static Schema convert(Type type, String name) { return buildAvroSchemaFromType(type, name); } - /** Convert an avro schema into internal type. */ + /** + * Convert an avro schema into internal type. + */ public static Type convertToField(Schema schema) { return buildTypeFromAvroSchema(schema); } - /** Convert an avro schema into internalSchema. */ + /** + * Convert an avro schema into internalSchema. + */ public static InternalSchema convert(Schema schema) { List fields = ((Types.RecordType) convertToField(schema)).fields(); return new InternalSchema(fields); } - /** Check whether current avro schema is optional?. */ + /** + * Convert parquet schema into internalSchema. + */ + public static InternalSchema convert(MessageType schema) { + Schema avroSchema = new AvroSchemaConverter().convert(schema); + return convert(avroSchema); + } + + /** + * Check whether current avro schema is optional?. + */ public static boolean isOptional(Schema schema) { if (schema.getType() == UNION && schema.getTypes().size() == 2) { return schema.getTypes().get(0).getType() == Schema.Type.NULL || schema.getTypes().get(1).getType() == Schema.Type.NULL; @@ -95,7 +112,9 @@ public static boolean isOptional(Schema schema) { return false; } - /** Returns schema with nullable true. */ + /** + * Returns schema with nullable true. + */ public static Schema nullableSchema(Schema schema) { if (schema.getType() == UNION) { if (!isOptional(schema)) { diff --git a/hudi-common/src/main/java/org/apache/hudi/secondary/index/AllRowIdSet.java b/hudi-common/src/main/java/org/apache/hudi/secondary/index/AllRowIdSet.java new file mode 100644 index 0000000000000..7dc40a0dc1566 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/secondary/index/AllRowIdSet.java @@ -0,0 +1,29 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.hudi.secondary.index; + +public class AllRowIdSet extends RowIdSet { + + public AllRowIdSet(int maxRow) { + super(); + // Row id belongs to [0, maxRow) + rowIdSet.add(0L, maxRow); + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/secondary/index/EmptyRowIdSet.java b/hudi-common/src/main/java/org/apache/hudi/secondary/index/EmptyRowIdSet.java new file mode 100644 index 0000000000000..1302defab765d --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/secondary/index/EmptyRowIdSet.java @@ -0,0 +1,40 @@ +/* + * 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.exception.HoodieNotSupportedException; + +public class EmptyRowIdSet extends RowIdSet { + + @Override + public void add(int rowId) { + throw new HoodieNotSupportedException("Not supported for empty row id set"); + } + + @Override + public boolean get(int rowId) { + return false; + } + + @Override + public int cardinality() { + return 0; + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/index/HoodieIndex.java b/hudi-common/src/main/java/org/apache/hudi/secondary/index/HoodieSecondaryIndex.java similarity index 51% rename from hudi-common/src/main/java/org/apache/hudi/common/index/HoodieIndex.java rename to hudi-common/src/main/java/org/apache/hudi/secondary/index/HoodieSecondaryIndex.java index 6dabb1a41f8cd..209fce979af2a 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/index/HoodieIndex.java +++ b/hudi-common/src/main/java/org/apache/hudi/secondary/index/HoodieSecondaryIndex.java @@ -17,48 +17,48 @@ * under the License. */ -package org.apache.hudi.common.index; +package org.apache.hudi.secondary.index; -import java.util.Arrays; +import java.io.Serializable; +import java.util.Comparator; +import java.util.LinkedHashMap; import java.util.Map; +import java.util.Objects; -public class HoodieIndex { +public class HoodieSecondaryIndex implements Serializable { private String indexName; - private String[] colNames; - private HoodieIndexType indexType; - private Map> colOptions; + private SecondaryIndexType indexType; + + // The index fields need to be in order + private LinkedHashMap> columns; private Map options; - public HoodieIndex() { + public HoodieSecondaryIndex() { } - public HoodieIndex( + public HoodieSecondaryIndex( String indexName, - String[] colNames, - HoodieIndexType indexType, - Map> colOptions, + SecondaryIndexType indexType, + LinkedHashMap> columns, Map options) { this.indexName = indexName; - this.colNames = colNames; this.indexType = indexType; - this.colOptions = colOptions; + this.columns = columns; this.options = options; + + validate(); } public String getIndexName() { return indexName; } - public String[] getColNames() { - return colNames; - } - - public HoodieIndexType getIndexType() { + public SecondaryIndexType getIndexType() { return indexType; } - public Map> getColOptions() { - return colOptions; + public LinkedHashMap> getColumns() { + return columns; } public Map getOptions() { @@ -69,22 +69,40 @@ public static Builder builder() { return new Builder(); } + private void validate() { + } + + @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 public String toString() { return "HoodieIndex{" + "indexName='" + indexName + '\'' - + ", colNames='" + Arrays.toString(colNames) + '\'' + ", indexType=" + indexType - + ", colOptions=" + colOptions + + ", columns=" + columns + ", options=" + options + '}'; } public static class Builder { private String indexName; - private String[] colNames; - private HoodieIndexType indexType; - private Map> colOptions; + private SecondaryIndexType indexType; + private LinkedHashMap> columns; private Map options; public Builder setIndexName(String indexName) { @@ -92,18 +110,13 @@ public Builder setIndexName(String indexName) { return this; } - public Builder setColNames(String[] colNames) { - this.colNames = colNames; - return this; - } - public Builder setIndexType(String indexType) { - this.indexType = HoodieIndexType.of(indexType); + this.indexType = SecondaryIndexType.of(indexType); return this; } - public Builder setColOptions(Map> colOptions) { - this.colOptions = colOptions; + public Builder setColumns(LinkedHashMap> columns) { + this.columns = columns; return this; } @@ -112,8 +125,15 @@ public Builder setOptions(Map options) { return this; } - public HoodieIndex build() { - return new HoodieIndex(indexName, colNames, indexType, colOptions, options); + public HoodieSecondaryIndex build() { + return new HoodieSecondaryIndex(indexName, indexType, columns, options); + } + } + + public static class HoodieIndexCompactor implements Comparator { + @Override + public int compare(HoodieSecondaryIndex o1, HoodieSecondaryIndex o2) { + return o1.indexName.compareTo(o2.indexName); } } } diff --git a/hudi-common/src/main/java/org/apache/hudi/secondary/index/IRowIdSet.java b/hudi-common/src/main/java/org/apache/hudi/secondary/index/IRowIdSet.java new file mode 100644 index 0000000000000..3814fefd14102 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/secondary/index/IRowIdSet.java @@ -0,0 +1,40 @@ +/* + * 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 java.io.Serializable; + +public interface IRowIdSet extends Serializable { + void add(int rowId); + + boolean get(int rowId); + + IRowIdSetIterator iterator(); + + int cardinality(); + + Object getContainer(); + + IRowIdSet and(IRowIdSet other); + + IRowIdSet or(IRowIdSet other); + + IRowIdSet not(IRowIdSet other); +} diff --git a/hudi-common/src/main/java/org/apache/hudi/secondary/index/IRowIdSetIterator.java b/hudi-common/src/main/java/org/apache/hudi/secondary/index/IRowIdSetIterator.java new file mode 100644 index 0000000000000..233f0349d6fd4 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/secondary/index/IRowIdSetIterator.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; + +public interface IRowIdSetIterator { + static final int NO_MORE_ROWS = Integer.MAX_VALUE; + + boolean hasNext(); + + int nextRowId(); + + void advance(int target); +} diff --git a/hudi-common/src/main/java/org/apache/hudi/secondary/index/ISecondaryIndexBuilder.java b/hudi-common/src/main/java/org/apache/hudi/secondary/index/ISecondaryIndexBuilder.java new file mode 100644 index 0000000000000..43d6b68909448 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/secondary/index/ISecondaryIndexBuilder.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 ISecondaryIndexBuilder { + 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/ISecondaryIndexReader.java b/hudi-common/src/main/java/org/apache/hudi/secondary/index/ISecondaryIndexReader.java new file mode 100644 index 0000000000000..799c36c2af2f1 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/secondary/index/ISecondaryIndexReader.java @@ -0,0 +1,47 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.hudi.secondary.index; + +import org.apache.hudi.internal.schema.Types.Field; + +import java.io.IOException; +import java.util.List; + +public interface ISecondaryIndexReader { + IRowIdSet emptyRows() throws IOException; + + IRowIdSet allRows() throws IOException; + + IRowIdSet queryNullTerm(Field field) throws IOException; + + IRowIdSet queryNotNullTerm(Field field) throws IOException; + + IRowIdSet queryTerm(Field field, Object value) throws IOException; + + IRowIdSet queryTermList(Field field, List values) throws IOException; + + IRowIdSet queryPrefix(Field field, String prefixValue) throws IOException; + + IRowIdSet queryRegex(Field field, String regex) throws IOException; + + IRowIdSet queryRange(Field field, Object min, Object max, boolean includeMin, boolean includeMax) throws IOException; + + void close(); +} diff --git a/hudi-common/src/main/java/org/apache/hudi/secondary/index/IndexConstants.java b/hudi-common/src/main/java/org/apache/hudi/secondary/index/IndexConstants.java new file mode 100644 index 0000000000000..3220677d75f68 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/secondary/index/IndexConstants.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; + +public class IndexConstants { + public static final String NULL_FIELD = "_hoodie_null_field"; + public static final String NOT_NULL_FIELD = "_hoodie_notnull_field"; + + public static final int NO_MORE_ROWS = Integer.MAX_VALUE; +} diff --git a/hudi-common/src/main/java/org/apache/hudi/secondary/index/RowIdSet.java b/hudi-common/src/main/java/org/apache/hudi/secondary/index/RowIdSet.java new file mode 100644 index 0000000000000..473a1e2fb9e5e --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/secondary/index/RowIdSet.java @@ -0,0 +1,74 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.hudi.secondary.index; + +import org.roaringbitmap.RoaringBitmap; + +public class RowIdSet implements IRowIdSet { + protected final RoaringBitmap rowIdSet; + + public RowIdSet() { + this.rowIdSet = new RoaringBitmap(); + } + + public RowIdSet(RoaringBitmap rowIdSet) { + this.rowIdSet = rowIdSet; + } + + @Override + public void add(int rowId) { + rowIdSet.add(rowId); + } + + @Override + public boolean get(int rowId) { + return rowIdSet.contains(rowId); + } + + @Override + public IRowIdSetIterator iterator() { + return new RowIdSetIterator(rowIdSet.getIntIterator()); + } + + @Override + public int cardinality() { + return rowIdSet.getCardinality(); + } + + @Override + public Object getContainer() { + return rowIdSet; + } + + @Override + public IRowIdSet and(IRowIdSet other) { + return new RowIdSet(RoaringBitmap.and(rowIdSet, (RoaringBitmap) other.getContainer())); + } + + @Override + public IRowIdSet or(IRowIdSet other) { + return new RowIdSet(RoaringBitmap.or(rowIdSet, (RoaringBitmap) other.getContainer())); + } + + @Override + public IRowIdSet not(IRowIdSet other) { + return new RowIdSet(RoaringBitmap.andNot(rowIdSet, (RoaringBitmap) other.getContainer())); + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/secondary/index/RowIdSetIterator.java b/hudi-common/src/main/java/org/apache/hudi/secondary/index/RowIdSetIterator.java new file mode 100644 index 0000000000000..368b986feee01 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/secondary/index/RowIdSetIterator.java @@ -0,0 +1,45 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.hudi.secondary.index; + +import org.roaringbitmap.PeekableIntIterator; + +public class RowIdSetIterator implements IRowIdSetIterator { + private final PeekableIntIterator iterator; + + public RowIdSetIterator(PeekableIntIterator iterator) { + this.iterator = iterator; + } + + @Override + public boolean hasNext() { + return iterator.hasNext(); + } + + @Override + public int nextRowId() { + return iterator.next(); + } + + @Override + public void advance(int target) { + iterator.advanceIfNeeded(target); + } +} 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..5d515c5b9d34d --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/secondary/index/SecondaryIndexFactory.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.secondary.index; + +import org.apache.hudi.common.config.HoodieBuildTaskConfig; +import org.apache.hudi.exception.HoodieSecondaryIndexException; +import org.apache.hudi.secondary.index.lucene.LuceneIndexBuilder; +import org.apache.hudi.secondary.index.lucene.LuceneIndexReader; + +import org.apache.hadoop.conf.Configuration; + +public class SecondaryIndexFactory { + public static ISecondaryIndexBuilder getIndexBuilder(HoodieBuildTaskConfig indexConfig) { + switch (indexConfig.getIndexType()) { + case LUCENE: + return new LuceneIndexBuilder(indexConfig); + default: + throw new HoodieSecondaryIndexException( + "Unknown hoodie secondary index type: " + indexConfig.getIndexType()); + } + } + + public static ISecondaryIndexReader getIndexReader( + String indexDir, SecondaryIndexType indexType, Configuration conf) { + switch (indexType) { + case LUCENE: + return new LuceneIndexReader(indexDir, conf); + default: + throw new HoodieSecondaryIndexException( + "Unknown hoodie secondary index type:" + indexType); + } + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/secondary/index/SecondaryIndexManager.java b/hudi-common/src/main/java/org/apache/hudi/secondary/index/SecondaryIndexManager.java new file mode 100644 index 0000000000000..61bd7d76219b7 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/secondary/index/SecondaryIndexManager.java @@ -0,0 +1,221 @@ +/* + * 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.table.HoodieTableConfig; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.TableSchemaResolver; +import org.apache.hudi.common.util.CollectionUtils; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.exception.HoodieSecondaryIndexException; + +import org.apache.avro.Schema; +import org.apache.hadoop.fs.Path; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Collections; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.Set; +import java.util.stream.Collectors; + +import static org.apache.hudi.secondary.index.SecondaryIndexUtils.getSecondaryIndexes; + +public class SecondaryIndexManager { + private static final Logger LOG = LoggerFactory.getLogger(SecondaryIndexManager.class); + + private static volatile SecondaryIndexManager _instance; + + private SecondaryIndexManager() { + } + + public static SecondaryIndexManager getInstance() { + if (_instance == null) { + synchronized (SecondaryIndexManager.class) { + if (_instance == null) { + _instance = new SecondaryIndexManager(); + } + } + } + + return _instance; + } + + /** + * Create a secondary index for hoodie table, two steps will be performed: + * 1. Add secondary index metadata to hoodie.properties + * 2. Trigger build secondary index + * + * @param metaClient Hoodie table meta client + * @param indexName The unique secondary index name + * @param indexType Index type + * @param ignoreIfExists Whether ignore the creation if the specific secondary index exists + * @param columns The columns referenced by this secondary index, each column + * has its own options + * @param options Options for this secondary index + */ + public void create( + HoodieTableMetaClient metaClient, + String indexName, + String indexType, + boolean ignoreIfExists, + LinkedHashMap> columns, + Map options) { + Option> secondaryIndexes = getSecondaryIndexes(metaClient); + Set colNames = columns.keySet(); + Schema avroSchema; + try { + avroSchema = new TableSchemaResolver(metaClient).getTableAvroSchema(false); + } catch (Exception e) { + throw new HoodieSecondaryIndexException( + "Failed to get table avro schema: " + metaClient.getTableConfig().getTableName()); + } + + for (String col : colNames) { + if (avroSchema.getField(col) == null) { + throw new HoodieSecondaryIndexException("Field not exists: " + col); + } + } + + if (indexExists(secondaryIndexes, indexName, Option.of(indexType), Option.of(colNames))) { + if (ignoreIfExists) { + return; + } else { + throw new HoodieSecondaryIndexException("Secondary index already exists: " + indexName); + } + } + + HoodieSecondaryIndex secondaryIndexToAdd = HoodieSecondaryIndex.builder() + .setIndexName(indexName) + .setIndexType(indexType) + .setColumns(columns) + .setOptions(options) + .build(); + + List newSecondaryIndexes = secondaryIndexes.map(h -> { + h.add(secondaryIndexToAdd); + return h; + }).orElse(Collections.singletonList(secondaryIndexToAdd)); + newSecondaryIndexes.sort(new HoodieSecondaryIndex.HoodieIndexCompactor()); + + // Persistence secondary indexes' metadata to hoodie.properties file + Properties updatedProps = new Properties(); + updatedProps.put(HoodieTableConfig.SECONDARY_INDEXES_METADATA.key(), + SecondaryIndexUtils.toJsonString(newSecondaryIndexes)); + HoodieTableConfig.update(metaClient.getFs(), new Path(metaClient.getMetaPath()), updatedProps); + + LOG.info("Success to add secondary index metadata: {}", secondaryIndexToAdd); + + // TODO: build index + } + + /** + * Drop a secondary index by index name + * + * @param metaClient Hoodie table meta client + * @param indexName The unique secondary index name + * @param ignoreIfNotExists Whether ignore drop if the specific secondary index no exists + */ + public void drop(HoodieTableMetaClient metaClient, String indexName, boolean ignoreIfNotExists) { + Option> secondaryIndexes = getSecondaryIndexes(metaClient); + if (!indexExists(secondaryIndexes, indexName, Option.empty(), Option.empty())) { + if (ignoreIfNotExists) { + return; + } else { + throw new HoodieSecondaryIndexException("Secondary index not exists: " + indexName); + } + } + + List secondaryIndexesToKeep = secondaryIndexes.get().stream() + .filter(i -> !i.getIndexName().equals(indexName)) + .sorted(new HoodieSecondaryIndex.HoodieIndexCompactor()) + .collect(Collectors.toList()); + if (CollectionUtils.nonEmpty(secondaryIndexesToKeep)) { + Properties updatedProps = new Properties(); + updatedProps.put(HoodieTableConfig.SECONDARY_INDEXES_METADATA.key(), + SecondaryIndexUtils.toJsonString(secondaryIndexesToKeep)); + HoodieTableConfig.update(metaClient.getFs(), new Path(metaClient.getMetaPath()), updatedProps); + } else { + HoodieTableConfig.delete(metaClient.getFs(), new Path(metaClient.getMetaPath()), + CollectionUtils.createSet(HoodieTableConfig.SECONDARY_INDEXES_METADATA.key())); + } + + LOG.info("Success to delete secondary index metadata: {}", indexName); + + // TODO: drop index data + } + + /** + * Show secondary indexes from hoodie table + * + * @param metaClient Hoodie table meta client + * @return Indexes in this table + */ + public Option> show(HoodieTableMetaClient metaClient) { + return getSecondaryIndexes(metaClient); + } + + /** + * Refresh the specific secondary index + * + * @param metaClient Hoodie table meta client + * @param indexName The target secondary index name + */ + public void refresh(HoodieTableMetaClient metaClient, String indexName) { + // TODO + } + + /** + * Check if the specific secondary index exists. When drop a secondary index, + * only check index name, but for adding a secondary index, we should also + * check the index type and columns when index name is different. + * + * @param secondaryIndexes Current secondary indexes in this table + * @param indexName The index name of target secondary index + * @param indexType The index type of target secondary index + * @param colNames The column names of target secondary index + * @return true if secondary index exists + */ + private boolean indexExists( + Option> secondaryIndexes, + String indexName, + Option indexType, + Option> colNames) { + return secondaryIndexes.map(indexes -> + indexes.stream().anyMatch(index -> { + if (index.getIndexName().equals(indexName)) { + return true; + } else if (indexType.isPresent() && colNames.isPresent()) { + // When secondary index names are different, we should check index type + // and index columns to avoid repeatedly creating the same index. + // For example: + // create index idx_name on test using lucene (name); + // create index idx_name_1 on test using lucene (name); + return index.getIndexType().name().equalsIgnoreCase(indexType.get()) + && CollectionUtils.diff(index.getColumns().keySet(), colNames.get()).isEmpty(); + } + + return false; + })).orElse(false); + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/index/HoodieIndexType.java b/hudi-common/src/main/java/org/apache/hudi/secondary/index/SecondaryIndexType.java similarity index 80% rename from hudi-common/src/main/java/org/apache/hudi/common/index/HoodieIndexType.java rename to hudi-common/src/main/java/org/apache/hudi/secondary/index/SecondaryIndexType.java index 03618a767906d..108d2effce077 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/index/HoodieIndexType.java +++ b/hudi-common/src/main/java/org/apache/hudi/secondary/index/SecondaryIndexType.java @@ -17,18 +17,18 @@ * under the License. */ -package org.apache.hudi.common.index; +package org.apache.hudi.secondary.index; import org.apache.hudi.exception.HoodieIndexException; import java.util.Arrays; -public enum HoodieIndexType { +public enum SecondaryIndexType { LUCENE((byte) 1); private final byte type; - HoodieIndexType(byte type) { + SecondaryIndexType(byte type) { this.type = type; } @@ -36,16 +36,16 @@ public byte getValue() { return type; } - public static HoodieIndexType of(byte indexType) { - return Arrays.stream(HoodieIndexType.values()) + public static SecondaryIndexType of(byte indexType) { + return Arrays.stream(SecondaryIndexType.values()) .filter(t -> t.type == indexType) .findAny() .orElseThrow(() -> new HoodieIndexException("Unknown hoodie index type:" + indexType)); } - public static HoodieIndexType of(String indexType) { - return Arrays.stream(HoodieIndexType.values()) + public static SecondaryIndexType of(String indexType) { + return Arrays.stream(SecondaryIndexType.values()) .filter(t -> t.name().equals(indexType.toUpperCase())) .findAny() .orElseThrow(() -> 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 new file mode 100644 index 0000000000000..81f066e5ec495 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/secondary/index/SecondaryIndexUtils.java @@ -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.hudi.secondary.index; + +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.exception.HoodieIndexException; +import org.apache.hudi.exception.HoodieSecondaryIndexException; + +import com.fasterxml.jackson.annotation.JsonAutoDetect; +import com.fasterxml.jackson.annotation.PropertyAccessor; +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.DeserializationFeature; +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.hadoop.conf.Configuration; +import org.apache.parquet.hadoop.util.SerializationUtil; +import org.roaringbitmap.RoaringBitmap; + +import java.io.IOException; +import java.util.List; + +public class SecondaryIndexUtils { + /** + * key to configure the specific row id set to read + */ + public static final String SPECIFIC_ROW_ID_SET = "parquet.read.specific.row.id.set"; + + public static void setSpecificRowIdSet(Configuration configuration, RoaringBitmap rowIdSet) { + try { + SerializationUtil.writeObjectToConfAsBase64(SPECIFIC_ROW_ID_SET, rowIdSet, configuration); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + public static RoaringBitmap getSpecificRowIdSet(Configuration configuration) { + try { + return SerializationUtil.readObjectFromConfAsBase64(SPECIFIC_ROW_ID_SET, configuration); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + /** + * Get secondary index metadata for this table + * + * @param metaClient HoodieTableMetaClient + * @return HoodieSecondaryIndex List + */ + public static Option> getSecondaryIndexes(HoodieTableMetaClient metaClient) { + Option indexesMetadata = metaClient.getTableConfig().getSecondaryIndexesMetadata(); + return indexesMetadata.map(SecondaryIndexUtils::fromJsonString); + } + + /** + * Parse secondary index str to List + * + * @param jsonStr Secondary indexes with json format + * @return List + */ + public static List fromJsonString(String jsonStr) { + return SecondaryIndexUtils.fromJsonString(jsonStr, + new TypeReference>() { + }); + } + + public static String toJsonString(Object value) { + try { + return getObjectMapper().writeValueAsString(value); + } catch (JsonProcessingException e) { + throw new HoodieIndexException("Fail to convert object to json string", e); + } + } + + public static T fromJsonString(String jsonStr, TypeReference type) { + if (jsonStr == null || jsonStr.isEmpty()) { + return null; + } + + try { + return getObjectMapper().readValue(jsonStr, type); + } catch (IOException e) { + throw new HoodieSecondaryIndexException("Fail to parse json string"); + } + } + + public static ObjectMapper getObjectMapper() { + ObjectMapper mapper = new ObjectMapper(); + mapper.disable(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES); + mapper.setVisibility(PropertyAccessor.FIELD, JsonAutoDetect.Visibility.ANY); + return mapper; + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/secondary/index/filter/AllRowFilter.java b/hudi-common/src/main/java/org/apache/hudi/secondary/index/filter/AllRowFilter.java new file mode 100644 index 0000000000000..3a45a0e8d27f1 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/secondary/index/filter/AllRowFilter.java @@ -0,0 +1,41 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.hudi.secondary.index.filter; + +import org.apache.hudi.internal.schema.Types.Field; +import org.apache.hudi.secondary.index.IRowIdSet; + +import java.io.IOException; + +public class AllRowFilter extends IndexFilter { + public AllRowFilter(Field field) { + super(null, field); + } + + @Override + public IRowIdSet getRowIdSet() throws IOException { + return indexReader.allRows(); + } + + @Override + public String toString() { + return "all(" + getField().name() + ")"; + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/secondary/index/filter/AndFilter.java b/hudi-common/src/main/java/org/apache/hudi/secondary/index/filter/AndFilter.java new file mode 100644 index 0000000000000..af441eafee3ca --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/secondary/index/filter/AndFilter.java @@ -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.hudi.secondary.index.filter; + +import org.apache.hudi.secondary.index.IRowIdSet; + +import java.io.IOException; + +public class AndFilter extends IndexFilter { + private final IndexFilter leftFilter; + private final IndexFilter rightFilter; + + public AndFilter(IndexFilter leftFilter, IndexFilter rightFilter) { + super(leftFilter.getFields(), rightFilter.getFields()); + this.leftFilter = leftFilter; + this.rightFilter = rightFilter; + } + + @Override + public IRowIdSet getRowIdSet() throws IOException { + IRowIdSet rowIdSet = leftFilter.getRowIdSet(); + return rowIdSet.and(rightFilter.getRowIdSet()); + } + + public IndexFilter getLeft() { + return leftFilter; + } + + public IndexFilter getRight() { + return rightFilter; + } + + @Override + public String toString() { + return "and(" + leftFilter.toString() + ", " + rightFilter.toString() + ")"; + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/secondary/index/filter/EmptyRowFilter.java b/hudi-common/src/main/java/org/apache/hudi/secondary/index/filter/EmptyRowFilter.java new file mode 100644 index 0000000000000..99e260856bdef --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/secondary/index/filter/EmptyRowFilter.java @@ -0,0 +1,41 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.hudi.secondary.index.filter; + +import org.apache.hudi.internal.schema.Types.Field; +import org.apache.hudi.secondary.index.IRowIdSet; + +import java.io.IOException; + +public class EmptyRowFilter extends IndexFilter { + public EmptyRowFilter(Field field) { + super(null, field); + } + + @Override + public IRowIdSet getRowIdSet() throws IOException { + return indexReader.emptyRows(); + } + + @Override + public String toString() { + return "empty(" + getField().name() + ")"; + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/secondary/index/filter/IFilter.java b/hudi-common/src/main/java/org/apache/hudi/secondary/index/filter/IFilter.java new file mode 100644 index 0000000000000..f34e5bfb1b19f --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/secondary/index/filter/IFilter.java @@ -0,0 +1,28 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.hudi.secondary.index.filter; + +import org.apache.hudi.secondary.index.IRowIdSet; + +import java.io.IOException; + +public interface IFilter { + IRowIdSet getRowIdSet() throws IOException; +} diff --git a/hudi-common/src/main/java/org/apache/hudi/secondary/index/filter/IndexFilter.java b/hudi-common/src/main/java/org/apache/hudi/secondary/index/filter/IndexFilter.java new file mode 100644 index 0000000000000..693606c3940c6 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/secondary/index/filter/IndexFilter.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.filter; + +import org.apache.hudi.internal.schema.Types.Field; +import org.apache.hudi.secondary.index.ISecondaryIndexReader; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; + +public abstract class IndexFilter implements IFilter { + protected ISecondaryIndexReader indexReader; + protected Field[] fields; + + public IndexFilter() { + } + + public IndexFilter(ISecondaryIndexReader indexReader, Field field) { + this.indexReader = indexReader; + this.fields = new Field[] {field}; + } + + public IndexFilter(Field[]... fields) { + List tmpFields = new ArrayList<>(); + for (Field[] fs : fields) { + tmpFields.addAll(Arrays.asList(fs)); + } + this.fields = tmpFields.toArray(new Field[0]); + } + + public Field getField() { + return fields[0]; + } + + public Field[] getFields() { + return fields; + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/secondary/index/filter/NotFilter.java b/hudi-common/src/main/java/org/apache/hudi/secondary/index/filter/NotFilter.java new file mode 100644 index 0000000000000..a66742f99d62b --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/secondary/index/filter/NotFilter.java @@ -0,0 +1,42 @@ +/* + * 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.filter; + +import org.apache.hudi.secondary.index.IRowIdSet; + +import java.io.IOException; + +public class NotFilter extends IndexFilter { + private final IndexFilter indexFilter; + + public NotFilter(IndexFilter indexFilter) { + this.indexFilter = indexFilter; + } + + @Override + public IRowIdSet getRowIdSet() throws IOException { + return indexReader.allRows().not(indexFilter.getRowIdSet()); + } + + @Override + public String toString() { + return "not(" + indexFilter.toString() + ")"; + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/secondary/index/filter/NotNullFilter.java b/hudi-common/src/main/java/org/apache/hudi/secondary/index/filter/NotNullFilter.java new file mode 100644 index 0000000000000..47f8db351bf03 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/secondary/index/filter/NotNullFilter.java @@ -0,0 +1,42 @@ +/* + * 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.filter; + +import org.apache.hudi.internal.schema.Types.Field; +import org.apache.hudi.secondary.index.IRowIdSet; +import org.apache.hudi.secondary.index.ISecondaryIndexReader; + +import java.io.IOException; + +public class NotNullFilter extends IndexFilter { + public NotNullFilter(ISecondaryIndexReader indexReader, Field field) { + super(indexReader, field); + } + + @Override + public IRowIdSet getRowIdSet() throws IOException { + return indexReader.queryNotNullTerm(getField()); + } + + @Override + public String toString() { + return "notnull(" + getField().name() + ")"; + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/secondary/index/filter/NullFilter.java b/hudi-common/src/main/java/org/apache/hudi/secondary/index/filter/NullFilter.java new file mode 100644 index 0000000000000..cadfcc058e549 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/secondary/index/filter/NullFilter.java @@ -0,0 +1,42 @@ +/* + * 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.filter; + +import org.apache.hudi.internal.schema.Types.Field; +import org.apache.hudi.secondary.index.IRowIdSet; +import org.apache.hudi.secondary.index.ISecondaryIndexReader; + +import java.io.IOException; + +public class NullFilter extends IndexFilter { + public NullFilter(ISecondaryIndexReader indexReader, Field field) { + super(indexReader, field); + } + + @Override + public IRowIdSet getRowIdSet() throws IOException { + return indexReader.queryNullTerm(getField()); + } + + @Override + public String toString() { + return "isnull(" + getField().name() + ")"; + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/secondary/index/filter/OrFilter.java b/hudi-common/src/main/java/org/apache/hudi/secondary/index/filter/OrFilter.java new file mode 100644 index 0000000000000..f94209ee22a41 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/secondary/index/filter/OrFilter.java @@ -0,0 +1,52 @@ +/* + * 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.filter; + +import org.apache.hudi.internal.schema.Types.Field; +import org.apache.hudi.secondary.index.IRowIdSet; + +import java.io.IOException; + +public class OrFilter extends IndexFilter { + private final IndexFilter leftFilter; + private final IndexFilter rightFilter; + + public OrFilter(IndexFilter leftFilter, IndexFilter rightFilter) { + this.leftFilter = leftFilter; + this.rightFilter = rightFilter; + + Field[] leftFields = leftFilter.getFields(); + Field[] rightFields = rightFilter.getFields(); + fields = new Field[leftFields.length + rightFields.length]; + System.arraycopy(leftFields, 0, fields, 0, leftFields.length); + System.arraycopy(rightFields, 0, fields, leftFields.length, rightFields.length); + } + + @Override + public IRowIdSet getRowIdSet() throws IOException { + IRowIdSet rowIdSet = leftFilter.getRowIdSet(); + return rowIdSet.or(rightFilter.getRowIdSet()); + } + + @Override + public String toString() { + return "or(" + leftFilter.toString() + ", " + rightFilter.toString() + ")"; + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/secondary/index/filter/PrefixFilter.java b/hudi-common/src/main/java/org/apache/hudi/secondary/index/filter/PrefixFilter.java new file mode 100644 index 0000000000000..932eba505431a --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/secondary/index/filter/PrefixFilter.java @@ -0,0 +1,45 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.hudi.secondary.index.filter; + +import org.apache.hudi.internal.schema.Types.Field; +import org.apache.hudi.secondary.index.IRowIdSet; +import org.apache.hudi.secondary.index.ISecondaryIndexReader; + +import java.io.IOException; + +public class PrefixFilter extends IndexFilter { + private final String prefix; + + public PrefixFilter(ISecondaryIndexReader indexReader, Field field, String prefix) { + super(indexReader, field); + this.prefix = prefix; + } + + @Override + public IRowIdSet getRowIdSet() throws IOException { + return indexReader.queryPrefix(getField(), prefix); + } + + @Override + public String toString() { + return "prefix(" + getField().name() + ", " + prefix + ")"; + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/secondary/index/filter/RangeFilter.java b/hudi-common/src/main/java/org/apache/hudi/secondary/index/filter/RangeFilter.java new file mode 100644 index 0000000000000..7b1627a2b2bbd --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/secondary/index/filter/RangeFilter.java @@ -0,0 +1,51 @@ +/* + * 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.filter; + +import org.apache.hudi.internal.schema.Types.Field; +import org.apache.hudi.secondary.index.IRowIdSet; +import org.apache.hudi.secondary.index.ISecondaryIndexReader; + +import java.io.IOException; + +public class RangeFilter extends IndexFilter { + private final Object min; + private final Object max; + private final boolean includeMin; + private final boolean includeMax; + + public RangeFilter(ISecondaryIndexReader indexReader, Field field, Object min, Object max, boolean includeMin, boolean includeMax) { + super(indexReader, field); + this.min = min; + this.max = max; + this.includeMin = includeMin; + this.includeMax = includeMax; + } + + @Override + public IRowIdSet getRowIdSet() throws IOException { + return indexReader.queryRange(getField(), min, max, includeMin, includeMax); + } + + @Override + public String toString() { + return "range(" + getField().name() + ", " + min + ", " + max + ", " + includeMin + ", " + includeMax + ")"; + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/secondary/index/filter/RegexFilter.java b/hudi-common/src/main/java/org/apache/hudi/secondary/index/filter/RegexFilter.java new file mode 100644 index 0000000000000..3d934d6dbf02c --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/secondary/index/filter/RegexFilter.java @@ -0,0 +1,45 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.hudi.secondary.index.filter; + +import org.apache.hudi.internal.schema.Types.Field; +import org.apache.hudi.secondary.index.IRowIdSet; +import org.apache.hudi.secondary.index.ISecondaryIndexReader; + +import java.io.IOException; + +public class RegexFilter extends IndexFilter { + private final String regex; + + public RegexFilter(ISecondaryIndexReader indexReader, Field field, String regex) { + super(indexReader, field); + this.regex = regex; + } + + @Override + public IRowIdSet getRowIdSet() throws IOException { + return indexReader.queryRegex(getField(), regex); + } + + @Override + public String toString() { + return "regex(" + getField().name() + ", " + regex + ")"; + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/secondary/index/filter/TermFilter.java b/hudi-common/src/main/java/org/apache/hudi/secondary/index/filter/TermFilter.java new file mode 100644 index 0000000000000..62422b447fb67 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/secondary/index/filter/TermFilter.java @@ -0,0 +1,45 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.hudi.secondary.index.filter; + +import org.apache.hudi.internal.schema.Types.Field; +import org.apache.hudi.secondary.index.IRowIdSet; +import org.apache.hudi.secondary.index.ISecondaryIndexReader; + +import java.io.IOException; + +public class TermFilter extends IndexFilter { + private final Object value; + + public TermFilter(ISecondaryIndexReader indexReader, Field field, Object value) { + super(indexReader, field); + this.value = value; + } + + @Override + public IRowIdSet getRowIdSet() throws IOException { + return indexReader.queryTerm(getField(), value); + } + + @Override + public String toString() { + return "eq(" + getField().name() + ", " + value + ")"; + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/secondary/index/filter/TermListFilter.java b/hudi-common/src/main/java/org/apache/hudi/secondary/index/filter/TermListFilter.java new file mode 100644 index 0000000000000..93a90e4f2a08a --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/secondary/index/filter/TermListFilter.java @@ -0,0 +1,57 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.hudi.secondary.index.filter; + +import org.apache.hudi.internal.schema.Types.Field; +import org.apache.hudi.secondary.index.IRowIdSet; +import org.apache.hudi.secondary.index.ISecondaryIndexReader; + +import java.io.IOException; +import java.util.List; + +public class TermListFilter extends IndexFilter { + private final List values; + + public TermListFilter(ISecondaryIndexReader indexReader, Field field, List values) { + super(indexReader, field); + this.values = values; + } + + @Override + public IRowIdSet getRowIdSet() throws IOException { + return indexReader.queryTermList(getField(), values); + } + + @Override + public String toString() { + boolean first = true; + StringBuilder builder = new StringBuilder(); + for (Object value : values) { + if (first) { + first = false; + } else { + builder.append(","); + } + + builder.append(value); + } + return "in(" + getField().name() + ", list(" + builder.toString() + "))"; + } +} 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..e385559d6e742 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/secondary/index/lucene/LuceneIndexBuilder.java @@ -0,0 +1,226 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.hudi.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.common.util.ValidationUtils; +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.ISecondaryIndexBuilder; +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.ArrayList; +import java.util.LinkedList; +import java.util.List; +import java.util.Objects; +import java.util.stream.IntStream; + +import static org.apache.hudi.secondary.index.IndexConstants.NOT_NULL_FIELD; +import static org.apache.hudi.secondary.index.IndexConstants.NULL_FIELD; + +public class LuceneIndexBuilder implements ISecondaryIndexBuilder { + private static final Logger LOG = LoggerFactory.getLogger(LuceneIndexBuilder.class); + + private final String name; + private final LinkedList indexFields; + private final Configuration conf; + 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.indexFields = indexConfig.getIndexFields(); + 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); + } + + List fieldNames = new ArrayList<>(); + 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.add(field.name()); + }); + LOG.info("Init lucene index builder ok, name: {}, indexFields: {}", name, fieldNames); + } + + @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(); + IntStream.range(0, indexFields.size()).forEach(i -> { + String fieldName = indexFields.get(i).name(); + Object fieldValue = record.get(fieldName); + if (Objects.isNull(fieldValue)) { + reusedDoc.add(new StringField(NULL_FIELD, fieldName, Field.Store.NO)); + } else { + reusedDoc.add(createField(fieldName, fieldTypes[i], fieldValue)); + reusedDoc.add(new StringField(NOT_NULL_FIELD, fieldName, Field.Store.NO)); + } + }); + } + + /** + * Build lucene field from given field name, field type, and field value + * + * @param fieldName Field name + * @param typeID Data type + * @param value Field value + * @return Lucene field + */ + private Field createField(String fieldName, Type.TypeID typeID, Object value) { + ValidationUtils.checkArgument(value != null); + switch (typeID) { + case BOOLEAN: + return new IntPoint(fieldName, (Boolean) value ? 1 : 0); + case INT: + return new IntPoint(fieldName, (Integer) value); + case LONG: + case DATE: + case TIME: + case TIMESTAMP: + return new LongPoint(fieldName, (Long) value); + case FLOAT: + return new FloatPoint(fieldName, (Float) value); + case DOUBLE: + return new DoublePoint(fieldName, (Double) value); + case STRING: + case BINARY: + case UUID: + case DECIMAL: + byte[] bytes = ((Utf8) value).getBytes(); + return new StringField(fieldName, + new BytesRef(bytes, 0, bytes.length), Field.Store.NO); + default: + throw new HoodieSecondaryIndexException("Unsupported field type: " + typeID.getName()); + } + } + + /** + * 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..0deccd1f8bcbb --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/secondary/index/lucene/LuceneIndexReader.java @@ -0,0 +1,351 @@ +/* + * 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.HoodieNotSupportedException; +import org.apache.hudi.exception.HoodieSecondaryIndexException; +import org.apache.hudi.internal.schema.Types.Field; +import org.apache.hudi.secondary.index.AllRowIdSet; +import org.apache.hudi.secondary.index.EmptyRowIdSet; +import org.apache.hudi.secondary.index.IRowIdSet; +import org.apache.hudi.secondary.index.ISecondaryIndexReader; +import org.apache.hudi.secondary.index.RowIdSet; +import org.apache.hudi.secondary.index.lucene.hadoop.HdfsDirectory; + +import org.apache.avro.util.Utf8; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.lucene.document.DoublePoint; +import org.apache.lucene.document.FloatPoint; +import org.apache.lucene.document.IntPoint; +import org.apache.lucene.document.LongPoint; +import org.apache.lucene.index.DirectoryReader; +import org.apache.lucene.index.Term; +import org.apache.lucene.search.BooleanClause; +import org.apache.lucene.search.BooleanQuery; +import org.apache.lucene.search.IndexSearcher; +import org.apache.lucene.search.PrefixQuery; +import org.apache.lucene.search.Query; +import org.apache.lucene.search.RegexpQuery; +import org.apache.lucene.search.TermQuery; +import org.apache.lucene.search.TermRangeQuery; +import org.apache.lucene.store.Directory; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.math.BigDecimal; +import java.util.List; + +import static org.apache.hudi.secondary.index.IndexConstants.NOT_NULL_FIELD; +import static org.apache.hudi.secondary.index.IndexConstants.NULL_FIELD; + +public class LuceneIndexReader implements ISecondaryIndexReader { + 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 IRowIdSet emptyRows() throws IOException { + return new EmptyRowIdSet(); + } + + @Override + public IRowIdSet allRows() throws IOException { + try { + reader.incRef(); + + // RowId belongs to [0, maxDoc) + return new AllRowIdSet(reader.maxDoc()); + } finally { + reader.decRef(); + } + } + + @Override + public IRowIdSet queryNullTerm(Field field) throws IOException { + try { + reader.incRef(); + + TermQuery query = new TermQuery(new Term(NULL_FIELD, field.name())); + return search(query); + } finally { + reader.decRef(); + } + } + + @Override + public IRowIdSet queryNotNullTerm(Field field) throws IOException { + try { + reader.incRef(); + + TermQuery query = new TermQuery(new Term(NOT_NULL_FIELD, field.name())); + return search(query); + } finally { + reader.decRef(); + } + } + + @Override + public IRowIdSet queryTerm(Field field, Object value) throws IOException { + try { + reader.incRef(); + + Query query = buildTermQuery(field, value); + return search(query); + } finally { + reader.decRef(); + } + } + + @Override + public IRowIdSet queryTermList(Field field, List values) throws IOException { + try { + reader.incRef(); + + BooleanQuery.Builder queryBuilder = new BooleanQuery.Builder(); + for (Object value : values) { + queryBuilder.add(buildTermQuery(field, value), BooleanClause.Occur.SHOULD); + } + return search(queryBuilder.build()); + } finally { + reader.decRef(); + } + } + + @Override + public IRowIdSet queryPrefix(Field field, String prefixValue) throws IOException { + try { + reader.incRef(); + + PrefixQuery query = new PrefixQuery(new Term(field.name(), prefixValue)); + return search(query); + } finally { + reader.decRef(); + } + } + + @Override + public IRowIdSet queryRegex(Field field, String regex) throws IOException { + try { + reader.incRef(); + + RegexpQuery query = new RegexpQuery(new Term(field.name(), regex)); + return search(query); + } finally { + reader.decRef(); + } + } + + @Override + public IRowIdSet queryRange( + Field field, Object min, Object max, boolean includeMin, boolean includeMax) throws IOException { + switch (field.type().typeId()) { + case INT: + return queryRangeInt(field, (Integer) min, (Integer) max, includeMin, includeMax); + case LONG: + return queryRangeLong(field, (Long) min, (Long) max, includeMin, includeMax); + case FLOAT: + return queryRangeFloat(field, (Float) min, (Float) max, includeMin, includeMax); + case DOUBLE: + return queryRangeDouble(field, (Double) min, (Double) max, includeMin, includeMax); + case DECIMAL: + return queryRangeDecimal(field, (BigDecimal) min, (BigDecimal) max, includeMin, includeMax); + case STRING: + return queryRangeText(field, (String) min, (String) max, includeMin, includeMax); + default: + throw new HoodieNotSupportedException("Unsupported type for range query: " + field.type().typeId().getName()); + } + } + + private IRowIdSet queryRangeInt( + Field field, Integer min, Integer max, boolean includeMin, boolean includeMax) throws IOException { + try { + reader.incRef(); + + int minBound = min == null ? Integer.MIN_VALUE : min; + int maxBound = max == null ? Integer.MAX_VALUE : max; + if (!includeMin) { + minBound = Math.addExact(minBound, 1); + } + if (!includeMax) { + maxBound = Math.addExact(maxBound, -1); + } + + Query query = IntPoint.newRangeQuery(field.name(), minBound, maxBound); + return search(query); + } finally { + reader.decRef(); + } + } + + private IRowIdSet queryRangeLong( + Field field, Long min, Long max, boolean includeMin, boolean includeMax) throws IOException { + try { + reader.incRef(); + + long minBound = min == null ? Long.MIN_VALUE : min; + long maxBound = max == null ? Long.MAX_VALUE : max; + if (!includeMin) { + minBound = Math.addExact(minBound, 1); + } + if (!includeMax) { + maxBound = Math.addExact(maxBound, -1); + } + + Query query = LongPoint.newRangeQuery(field.name(), minBound, maxBound); + return search(query); + } finally { + reader.decRef(); + } + } + + private IRowIdSet queryRangeFloat( + Field field, Float min, Float max, boolean includeMin, boolean includeMax) throws IOException { + try { + reader.incRef(); + float minBound = min == null ? Float.NEGATIVE_INFINITY : min; + float maxBound = max == null ? Float.POSITIVE_INFINITY : max; + if (!includeMin) { + minBound = FloatPoint.nextUp(minBound); + } + if (!includeMax) { + maxBound = FloatPoint.nextDown(maxBound); + } + + Query query = FloatPoint.newRangeQuery(field.name(), minBound, maxBound); + return search(query); + } finally { + reader.decRef(); + } + } + + private IRowIdSet queryRangeDouble( + Field field, Double min, Double max, boolean includeMin, boolean includeMax) throws IOException { + try { + reader.incRef(); + + double minBound = min == null ? Double.NEGATIVE_INFINITY : min; + double maxBound = max == null ? Double.POSITIVE_INFINITY : max; + if (!includeMin) { + minBound = DoublePoint.nextUp(minBound); + } + if (!includeMax) { + maxBound = DoublePoint.nextDown(maxBound); + } + + Query query = DoublePoint.newRangeQuery(field.name(), minBound, maxBound); + return search(query); + } finally { + reader.decRef(); + } + + } + + private IRowIdSet queryRangeDecimal( + Field field, BigDecimal min, BigDecimal max, boolean includeMin, boolean includeMax) throws IOException { + throw new HoodieNotSupportedException("Not supported for decimal range query"); + } + + private IRowIdSet queryRangeText( + Field field, String min, String max, boolean includeMin, boolean includeMax) throws IOException { + try { + reader.incRef(); + + TermRangeQuery query = TermRangeQuery.newStringRange(field.name(), min, max, includeMin, includeMax); + return search(query); + } finally { + reader.decRef(); + } + } + + private IRowIdSet search(Query query) throws IOException { + IRowIdSet rowIdSet = new RowIdSet(); + searcher.search(query, new LuceneRowIdCollector(rowIdSet)); + return rowIdSet; + } + + private Query buildTermQuery(Field field, Object value) { + Query query = null; + switch (field.type().typeId()) { + case STRING: + query = new TermQuery(new Term(field.name(), (String) value)); + break; + case BINARY: + case UUID: + byte[] bytes = ((Utf8) value).getBytes(); + query = new TermQuery(new Term(field.name(), new String(bytes))); + break; + case BOOLEAN: + case INT: + query = IntPoint.newExactQuery(field.name(), (Integer) value); + break; + case LONG: + case DATE: + case TIME: + case TIMESTAMP: + query = LongPoint.newExactQuery(field.name(), (Long) value); + break; + case FLOAT: + query = FloatPoint.newExactQuery(field.name(), (Float) value); + break; + case DOUBLE: + query = DoublePoint.newExactQuery(field.name(), (Double) value); + break; + case DECIMAL: + default: + throw new HoodieSecondaryIndexException("Unsupported field type: " + field.type().typeId().getName()); + } + + return query; + } + + @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/LuceneRowIdCollector.java b/hudi-common/src/main/java/org/apache/hudi/secondary/index/lucene/LuceneRowIdCollector.java new file mode 100644 index 0000000000000..7c6804540d609 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/secondary/index/lucene/LuceneRowIdCollector.java @@ -0,0 +1,58 @@ +/* + * 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.secondary.index.IRowIdSet; + +import org.apache.lucene.index.LeafReaderContext; +import org.apache.lucene.search.Collector; +import org.apache.lucene.search.LeafCollector; +import org.apache.lucene.search.Scorable; +import org.apache.lucene.search.ScoreMode; + +import java.io.IOException; + +public class LuceneRowIdCollector implements Collector { + private final IRowIdSet rowIdSet; + + public LuceneRowIdCollector(IRowIdSet rowIdSet) { + this.rowIdSet = rowIdSet; + } + + @Override + public LeafCollector getLeafCollector(LeafReaderContext context) throws IOException { + return new LeafCollector() { + @Override + public void setScorer(Scorable scorer) throws IOException { + + } + + @Override + public void collect(int doc) throws IOException { + rowIdSet.add(context.docBase + doc); + } + }; + } + + @Override + public ScoreMode scoreMode() { + return ScoreMode.COMPLETE_NO_SCORES; + } +} 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-common/src/main/scala/org/apache/hudi/DefaultSource.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DefaultSource.scala index 348a6056c8883..ad30ac9c4b19b 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DefaultSource.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DefaultSource.scala @@ -20,14 +20,19 @@ package org.apache.hudi import org.apache.hadoop.fs.Path import org.apache.hudi.DataSourceReadOptions._ import org.apache.hudi.DataSourceWriteOptions.{BOOTSTRAP_OPERATION_OPT_VAL, OPERATION} +import org.apache.hudi.client.utils.SparkInternalSchemaConverter +import org.apache.hudi.common.config.HoodieSecondaryIndexConfig import org.apache.hudi.common.fs.FSUtils import org.apache.hudi.common.model.HoodieRecord import org.apache.hudi.common.model.HoodieTableType.{COPY_ON_WRITE, MERGE_ON_READ} import org.apache.hudi.common.table.timeline.HoodieInstant -import org.apache.hudi.common.table.{HoodieTableMetaClient, TableSchemaResolver} +import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient, TableSchemaResolver} +import org.apache.hudi.common.util.BuildUtils import org.apache.hudi.exception.HoodieException +import org.apache.hudi.secondary.index.SecondaryIndexUtils import org.apache.hudi.util.PathUtils import org.apache.log4j.LogManager +import org.apache.parquet.hadoop.util.SerializationUtil import org.apache.spark.sql.execution.streaming.{Sink, Source} import org.apache.spark.sql.hudi.HoodieSqlCommonUtils.isUsingHiveCatalog import org.apache.spark.sql.hudi.streaming.HoodieStreamSource @@ -125,6 +130,8 @@ class DefaultSource extends RelationProvider if (metaClient.getCommitsTimeline.filterCompletedInstants.countInstants() == 0) { new EmptyRelation(sqlContext, metaClient) } else { + injectSecondaryIndexData(SparkSession.getActiveSession.get, metaClient) + (tableType, queryType, isBootstrappedTable) match { case (COPY_ON_WRITE, QUERY_TYPE_SNAPSHOT_OPT_VAL, false) | (COPY_ON_WRITE, QUERY_TYPE_READ_OPTIMIZED_OPT_VAL, false) | @@ -150,6 +157,20 @@ class DefaultSource extends RelationProvider } } + def injectSecondaryIndexData(spark: SparkSession, metaClient: HoodieTableMetaClient): Unit = { + val indexMeta = SecondaryIndexUtils.getSecondaryIndexes(metaClient) + val indexedBaseFiles = BuildUtils.getBaseFileIndexInfo(metaClient) + if (indexMeta.isPresent && !indexMeta.get().isEmpty && !indexedBaseFiles.isEmpty) { + spark.sparkContext.hadoopConfiguration.set( + SparkInternalSchemaConverter.HOODIE_TABLE_PATH, metaClient.getBasePathV2.toString) + spark.sparkContext.hadoopConfiguration.set( + HoodieTableConfig.SECONDARY_INDEXES_METADATA.key(), SecondaryIndexUtils.toJsonString(indexMeta.get())) + SerializationUtil.writeObjectToConfAsBase64( + HoodieSecondaryIndexConfig.HOODIE_SECONDARY_INDEX_DATA, indexedBaseFiles, spark.sparkContext.hadoopConfiguration) + log.info("Write secondary index data to hadoop config") + } + } + def getValidCommits(metaClient: HoodieTableMetaClient): String = { metaClient .getCommitsAndCompactionTimeline.filterCompletedInstants.getInstants.toArray().map(_.asInstanceOf[HoodieInstant].getFileName).mkString(",") 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 9491e43e21fe8..37e2add58278b 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/catalyst/plans/logical/Index.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/Index.scala index 12ee2e8058343..1cc8c99728448 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/Index.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/Index.scala @@ -31,7 +31,7 @@ case class CreateIndex( indexType: String, ignoreIfExists: Boolean, columns: Seq[(Attribute, Map[String, String])], - properties: Map[String, String], + options: Map[String, String], override val output: Seq[Attribute] = CreateIndex.getOutputAttrs) extends Command { override def children: Seq[LogicalPlan] = Seq(table) diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/analysis/HoodieAnalysis.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/analysis/HoodieAnalysis.scala index ecc5d4a901c58..af78c7944409f 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/analysis/HoodieAnalysis.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/analysis/HoodieAnalysis.scala @@ -191,10 +191,10 @@ case class HoodieAnalysis(sparkSession: SparkSession) extends Rule[LogicalPlan] } // Convert to CreateIndexCommand - case CreateIndex(table, indexName, indexType, ignoreIfExists, columns, properties, output) + case CreateIndex(table, indexName, indexType, ignoreIfExists, columns, options, output) if table.resolved && sparkAdapter.isHoodieTable(table, sparkSession) => CreateIndexCommand( - getTableIdentifier(table), indexName, indexType, ignoreIfExists, columns, properties, output) + getTableIdentifier(table), indexName, indexType, ignoreIfExists, columns, options, output) // Convert to DropIndexCommand case DropIndex(table, indexName, ignoreIfNotExists, output) diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/IndexCommands.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/IndexCommands.scala index 5d73af31a9497..8a3b5630b67bb 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/IndexCommands.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/IndexCommands.scala @@ -19,23 +19,45 @@ package org.apache.spark.sql.hudi.command -import org.apache.hudi.common.index.HoodieIndex +import com.fasterxml.jackson.annotation.{JsonAutoDetect, PropertyAccessor} +import com.fasterxml.jackson.databind.{DeserializationFeature, ObjectMapper} +import org.apache.hudi.HoodieConversionUtils.toScalaOption +import org.apache.hudi.common.table.HoodieTableMetaClient +import org.apache.hudi.secondary.index.SecondaryIndexManager import org.apache.spark.internal.Logging -import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.catalyst.{QualifiedTableName, TableIdentifier} +import org.apache.spark.sql.hudi.HoodieSqlCommonUtils.getTableLocation import org.apache.spark.sql.{Row, SparkSession} +import java.util + +import scala.collection.JavaConverters.{collectionAsScalaIterableConverter, mapAsJavaMapConverter} + case class CreateIndexCommand( tableId: TableIdentifier, indexName: String, indexType: String, ignoreIfExists: Boolean, columns: Seq[(Attribute, Map[String, String])], - properties: Map[String, String], + options: Map[String, String], override val output: Seq[Attribute]) extends IndexBaseCommand { override def run(sparkSession: SparkSession): Seq[Row] = { - // The implementation for different index type + val metaClient = createHoodieTableMetaClient(tableId, sparkSession) + val columnsMap: java.util.LinkedHashMap[String, java.util.Map[String, String]] = + new util.LinkedHashMap[String, java.util.Map[String, String]]() + columns.map(c => columnsMap.put(c._1.name, c._2.asJava)) + + SecondaryIndexManager.getInstance().create( + metaClient, indexName, indexType, ignoreIfExists, columnsMap, options.asJava) + + // Invalidate cached table for queries do not access related table + // through {@code DefaultSource} + val qualifiedTableName = QualifiedTableName( + tableId.database.getOrElse(sparkSession.sessionState.catalog.getCurrentDatabase), + tableId.table) + sparkSession.sessionState.catalog.invalidateCachedTable(qualifiedTableName) Seq.empty } } @@ -47,7 +69,15 @@ case class DropIndexCommand( override val output: Seq[Attribute]) extends IndexBaseCommand { override def run(sparkSession: SparkSession): Seq[Row] = { - // The implementation for different index type + val metaClient = createHoodieTableMetaClient(tableId, sparkSession) + SecondaryIndexManager.getInstance().drop(metaClient, indexName, ignoreIfNotExists) + + // Invalidate cached table for queries do not access related table + // through {@code DefaultSource} + val qualifiedTableName = QualifiedTableName( + tableId.database.getOrElse(sparkSession.sessionState.catalog.getCurrentDatabase), + tableId.table) + sparkSession.sessionState.catalog.invalidateCachedTable(qualifiedTableName) Seq.empty } } @@ -57,8 +87,25 @@ case class ShowIndexesCommand( override val output: Seq[Attribute]) extends IndexBaseCommand { override def run(sparkSession: SparkSession): Seq[Row] = { - // The implementation for different index type - Seq.empty + val metaClient = createHoodieTableMetaClient(tableId, sparkSession) + val secondaryIndexes = SecondaryIndexManager.getInstance().show(metaClient) + + val mapper = getObjectMapper + toScalaOption(secondaryIndexes).map(x => + x.asScala.map(i => { + val colOptions = + if (i.getColumns.values().asScala.forall(_.isEmpty)) "" else mapper.writeValueAsString(i.getColumns) + val options = if (i.getOptions.isEmpty) "" else mapper.writeValueAsString(i.getOptions) + Row(i.getIndexName, i.getColumns.keySet().asScala.mkString(","), + i.getIndexType.name().toLowerCase, colOptions, options) + }).toSeq).getOrElse(Seq.empty[Row]) + } + + protected def getObjectMapper: ObjectMapper = { + val mapper = new ObjectMapper + mapper.disable(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES) + mapper.setVisibility(PropertyAccessor.FIELD, JsonAutoDetect.Visibility.ANY) + mapper } } @@ -68,7 +115,8 @@ case class RefreshIndexCommand( override val output: Seq[Attribute]) extends IndexBaseCommand { override def run(sparkSession: SparkSession): Seq[Row] = { - // The implementation for different index type + val metaClient = createHoodieTableMetaClient(tableId, sparkSession) + SecondaryIndexManager.getInstance().refresh(metaClient, indexName) Seq.empty } } @@ -76,26 +124,21 @@ case class RefreshIndexCommand( abstract class IndexBaseCommand extends HoodieLeafRunnableCommand with Logging { /** - * Check hoodie index exists. In a hoodie table, hoodie index name - * must be unique, so the index name will be checked firstly, + * Create hoodie table meta client according to given table identifier and + * spark session * - * @param secondaryIndexes Current hoodie indexes - * @param indexName The index name to be checked - * @param colNames The column names to be checked - * @return true if the index exists + * @param tableId The table identifier + * @param sparkSession The spark session + * @return The hoodie table meta client */ - def indexExists( - secondaryIndexes: Option[Array[HoodieIndex]], - indexName: String, - indexType: Option[String] = None, - colNames: Option[Array[String]] = None): Boolean = { - secondaryIndexes.exists(i => { - i.exists(_.getIndexName.equals(indexName)) || - // Index type and column name need to be checked if present - indexType.exists(t => - colNames.exists(c => - i.exists(index => - index.getIndexType.name().equalsIgnoreCase(t) && index.getColNames.sameElements(c)))) - }) + def createHoodieTableMetaClient( + tableId: TableIdentifier, + sparkSession: SparkSession): HoodieTableMetaClient = { + val catalogTable = sparkSession.sessionState.catalog.getTableMetadata(tableId) + val basePath = getTableLocation(catalogTable, sparkSession) + HoodieTableMetaClient.builder() + .setConf(sparkSession.sqlContext.sparkContext.hadoopConfiguration) + .setBasePath(basePath) + .build() } } 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 2eee942ff0689..df0b331449b3b 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 @@ -79,6 +79,8 @@ object HoodieProcedures { ,(RunCleanProcedure.NAME, RunCleanProcedure.builder) ,(ValidateHoodieSyncProcedure.NAME, ValidateHoodieSyncProcedure.builder) ,(ShowInvalidParquetProcedure.NAME, ShowInvalidParquetProcedure.builder) + ,(RunBuildProcedure.NAME, RunBuildProcedure.builder) + ,(ShowBuildProcedure.NAME, ShowBuildProcedure.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 18ea636c05719..5b88cfb7a0c9f 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,27 +17,23 @@ 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.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, Option => HOption} import org.apache.hudi.config.HoodieClusteringConfig 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.function.Supplier + import scala.collection.JavaConverters._ class RunClusteringProcedure extends BaseProcedure with ProcedureBuilder - with PredicateHelper + with ProcedurePredicateHelper with Logging { /** @@ -77,10 +73,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 _ => @@ -113,6 +109,7 @@ class RunClusteringProcedure extends BaseProcedure val startTs = System.currentTimeMillis() pendingClustering.foreach(client.cluster(_, true)) + client.close() logInfo(s"Finish clustering all the instants: ${pendingClustering.mkString(",")}," + s" time cost: ${System.currentTimeMillis() - startTs}ms.") @@ -140,25 +137,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") @@ -173,7 +151,6 @@ class RunClusteringProcedure extends BaseProcedure } }) } - } object RunClusteringProcedure { 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 3e5a7e29e4022..b932a408638ac 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 @@ -119,6 +119,7 @@ class RunCompactionProcedure extends BaseProcedure with ProcedureBuilder with Sp } case _ => throw new UnsupportedOperationException(s"Unsupported compaction operation: $operation") } + client.close() val compactionInstants = metaClient.reloadActiveTimeline().getInstants.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/TestIndexSyntax.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/command/index/TestIndexSyntax.scala index 3536ae9e0ac7f..537d3ad6a305a 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/command/index/TestIndexSyntax.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/command/index/TestIndexSyntax.scala @@ -64,7 +64,7 @@ class TestIndexSyntax extends HoodieSparkSqlTestBase { assertResult("idx_name")(resolvedLogicalPlan.asInstanceOf[CreateIndexCommand].indexName) assertResult("lucene")(resolvedLogicalPlan.asInstanceOf[CreateIndexCommand].indexType) assertResult(false)(resolvedLogicalPlan.asInstanceOf[CreateIndexCommand].ignoreIfExists) - assertResult(Map("block_size" -> "1024"))(resolvedLogicalPlan.asInstanceOf[CreateIndexCommand].properties) + assertResult(Map("block_size" -> "1024"))(resolvedLogicalPlan.asInstanceOf[CreateIndexCommand].options) logicalPlan = sqlParser.parsePlan(s"create index if not exists idx_price on $tableName using lucene (price options(order='desc')) options(block_size=512)") resolvedLogicalPlan = analyzer.execute(logicalPlan) @@ -72,7 +72,7 @@ class TestIndexSyntax extends HoodieSparkSqlTestBase { assertResult("idx_price")(resolvedLogicalPlan.asInstanceOf[CreateIndexCommand].indexName) assertResult("lucene")(resolvedLogicalPlan.asInstanceOf[CreateIndexCommand].indexType) assertResult(Map("order" -> "desc"))(resolvedLogicalPlan.asInstanceOf[CreateIndexCommand].columns.head._2) - assertResult(Map("block_size" -> "512"))(resolvedLogicalPlan.asInstanceOf[CreateIndexCommand].properties) + assertResult(Map("block_size" -> "512"))(resolvedLogicalPlan.asInstanceOf[CreateIndexCommand].options) logicalPlan = sqlParser.parsePlan(s"drop index if exists idx_name on $tableName") resolvedLogicalPlan = analyzer.execute(logicalPlan) 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 new file mode 100644 index 0000000000000..bed252311b847 --- /dev/null +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/command/index/TestSecondaryIndex.scala @@ -0,0 +1,125 @@ +/* + * 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.index + +import org.apache.spark.sql.hudi.HoodieSparkSqlTestBase + +class TestSecondaryIndex extends HoodieSparkSqlTestBase { + test("Test Create/Show/Drop Secondary Index") { + 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', 3.2d, 1000)") + spark.sql(s"insert into $tableName values(2, 'a2', 5.5d, 1001)") + spark.sql(s"insert into $tableName values(3, 'a3', 1.25d, 1002)") + + // Check query result + val selectResult1 = spark.sql(s"select id, name, price, ts from $tableName where id = 1").collect() + assertResult(Seq(1, "a1", 3.2, 1000))(selectResult1(0).toSeq) + + val selectResult2 = spark.sql(s"select id, name, price, ts from $tableName order by id").collect() + assertResult(3)(selectResult2.length) + assertResult(Seq(1, "a1", 3.2, 1000), + Seq(2, "a2", 5.5, 1001), + Seq(3, "a3", 1.25, 1002) + )(selectResult2(0).toSeq, selectResult2(1).toSeq, selectResult2(2).toSeq) + + val selectResult3 = spark.sql(s"select id, name, price, ts from $tableName where name = 'a2'").collect() + assertResult(Seq(2, "a2", 5.5, 1001))(selectResult3(0).toSeq) + + val selectResult4 = spark.sql(s"select id, name, price, ts from $tableName where price = 1.25").collect() + assertResult(Seq(3, "a3", 1.25, 1002))(selectResult4(0).toSeq) + + checkAnswer(s"show indexes from default.$tableName")() + + checkAnswer(s"create index idx_name on $tableName using lucene (name) options(block_size=1024)")() + checkAnswer(s"create index idx_price on $tableName using lucene (price options(order='desc')) options(block_size=512)")() + + // Create an index with multiple columns + 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)")( + "Secondary index already exists: idx_price" + ) + + // Create indexes repeatedly on columns(index name is different, but the index type and involved column is same) + checkException(s"create index idx_price_1 on $tableName using lucene (price)")( + "Secondary index already exists: idx_price_1" + ) + + 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\"}") + ) + + // Build secondary index for this table + spark.sql(s"call run_build(table => '$tableName')").show() + spark.sql(s"call show_build(table => '$tableName')").show() + + // Check query result again + spark.sql(s"select id, name, price, ts from $tableName order by id").show() + assertResult(selectResult1)(spark.sql(s"select id, name, price, ts from $tableName where id = 1").collect()) + assertResult(selectResult2)(spark.sql(s"select id, name, price, ts from $tableName order by id").collect()) + assertResult(selectResult3)(spark.sql(s"select id, name, price, ts from $tableName where name = 'a2'").collect()) + assertResult(selectResult4)(spark.sql(s"select id, name, price, ts from $tableName where price = 1.25").collect()) + + checkAnswer(s"drop index idx_name on $tableName")() + checkException(s"drop index idx_name on $tableName")("Secondary index not exists: idx_name") + + 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")( + Seq("idx_id_ts", "id,ts", "lucene", "", "") + ) + + checkException(s"drop index idx_price on $tableName")("Secondary index not exists: idx_price") + + checkException(s"create index idx_price_1 on $tableName using lucene (field_not_exist)")( + "Field not exists: field_not_exist" + ) + } + } + } +} 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-spark-datasource/hudi-spark3.2.x/pom.xml b/hudi-spark-datasource/hudi-spark3.2.x/pom.xml index 7eebeb2200016..a10ef69d6c4cf 100644 --- a/hudi-spark-datasource/hudi-spark3.2.x/pom.xml +++ b/hudi-spark-datasource/hudi-spark3.2.x/pom.xml @@ -172,7 +172,6 @@ - org.apache.spark spark-sql_2.12 diff --git a/hudi-spark-datasource/hudi-spark3.2.x/src/main/java/org/apache/parquet/filter2/bloomfilterlevel/HoodieBloomFilterImpl.java b/hudi-spark-datasource/hudi-spark3.2.x/src/main/java/org/apache/parquet/filter2/bloomfilterlevel/HoodieBloomFilterImpl.java new file mode 100644 index 0000000000000..bac38ef0cb1eb --- /dev/null +++ b/hudi-spark-datasource/hudi-spark3.2.x/src/main/java/org/apache/parquet/filter2/bloomfilterlevel/HoodieBloomFilterImpl.java @@ -0,0 +1,150 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.parquet.filter2.bloomfilterlevel; + +import org.apache.parquet.column.values.bloomfilter.BloomFilter; +import org.apache.parquet.filter2.predicate.FilterPredicate; +import org.apache.parquet.filter2.predicate.Operators; +import org.apache.parquet.filter2.predicate.UserDefinedPredicate; +import org.apache.parquet.hadoop.HoodieBloomFilterReader; +import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData; +import org.apache.parquet.hadoop.metadata.ColumnPath; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.apache.parquet.Preconditions.checkNotNull; + +public class HoodieBloomFilterImpl implements FilterPredicate.Visitor { + private static final Logger LOG = LoggerFactory.getLogger(HoodieBloomFilterImpl.class); + private static final boolean BLOCK_MIGHT_MATCH = false; + private static final boolean BLOCK_CANNOT_MATCH = true; + + private final Map columns = new HashMap(); + + public static boolean canDrop(FilterPredicate pred, List columns, HoodieBloomFilterReader bloomFilterReader) { + checkNotNull(pred, "pred"); + checkNotNull(columns, "columns"); + return pred.accept(new HoodieBloomFilterImpl(columns, bloomFilterReader)); + } + + private HoodieBloomFilterImpl(List columnsList, HoodieBloomFilterReader bloomFilterReader) { + for (ColumnChunkMetaData chunk : columnsList) { + columns.put(chunk.getPath(), chunk); + } + + this.bloomFilterReader = bloomFilterReader; + } + + private HoodieBloomFilterReader bloomFilterReader; + + private ColumnChunkMetaData getColumnChunk(ColumnPath columnPath) { + return columns.get(columnPath); + } + + @Override + public > Boolean visit(Operators.Eq eq) { + T value = eq.getValue(); + + if (value == null) { + // the bloom filter bitset contains only non-null values so isn't helpful. this + // could check the column stats, but the StatisticsFilter is responsible + return BLOCK_MIGHT_MATCH; + } + + Operators.Column filterColumn = eq.getColumn(); + ColumnChunkMetaData meta = getColumnChunk(filterColumn.getColumnPath()); + if (meta == null) { + // the column isn't in this file so all values are null, but the value + // must be non-null because of the above check. + return BLOCK_CANNOT_MATCH; + } + + try { + BloomFilter bloomFilter = bloomFilterReader.readBloomFilter(meta); + if (bloomFilter != null && !bloomFilter.findHash(bloomFilter.hash(value))) { + return BLOCK_CANNOT_MATCH; + } + } catch (RuntimeException e) { + LOG.warn(e.getMessage()); + return BLOCK_MIGHT_MATCH; + } + + return BLOCK_MIGHT_MATCH; + } + + @Override + public > Boolean visit(Operators.NotEq notEq) { + return BLOCK_MIGHT_MATCH; + } + + @Override + public > Boolean visit(Operators.Lt lt) { + return BLOCK_MIGHT_MATCH; + } + + @Override + public > Boolean visit(Operators.LtEq ltEq) { + return BLOCK_MIGHT_MATCH; + } + + @Override + public > Boolean visit(Operators.Gt gt) { + return BLOCK_MIGHT_MATCH; + } + + @Override + public > Boolean visit(Operators.GtEq gtEq) { + return BLOCK_MIGHT_MATCH; + } + + @Override + public Boolean visit(Operators.And and) { + return and.getLeft().accept(this) || and.getRight().accept(this); + } + + @Override + public Boolean visit(Operators.Or or) { + return or.getLeft().accept(this) && or.getRight().accept(this); + } + + @Override + public Boolean visit(Operators.Not not) { + throw new IllegalArgumentException( + "This predicate contains a not! Did you forget to run this predicate through LogicalInverseRewriter? " + not); + } + + private , U extends UserDefinedPredicate> Boolean visit(Operators.UserDefined ud, boolean inverted) { + return BLOCK_MIGHT_MATCH; + } + + @Override + public , U extends UserDefinedPredicate> Boolean visit(Operators.UserDefined udp) { + return visit(udp, false); + } + + @Override + public , U extends UserDefinedPredicate> Boolean visit(Operators.LogicalNotUserDefined udp) { + return visit(udp.getUserDefined(), true); + } +} diff --git a/hudi-spark-datasource/hudi-spark3.2.x/src/main/java/org/apache/parquet/filter2/compat/HoodieRowGroupFilter.java b/hudi-spark-datasource/hudi-spark3.2.x/src/main/java/org/apache/parquet/filter2/compat/HoodieRowGroupFilter.java new file mode 100644 index 0000000000000..b9fc46abbbd55 --- /dev/null +++ b/hudi-spark-datasource/hudi-spark3.2.x/src/main/java/org/apache/parquet/filter2/compat/HoodieRowGroupFilter.java @@ -0,0 +1,130 @@ +/* + * 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.parquet.filter2.compat; + +import org.apache.parquet.filter2.bloomfilterlevel.HoodieBloomFilterImpl; +import org.apache.parquet.filter2.compat.FilterCompat.Filter; +import org.apache.parquet.filter2.compat.FilterCompat.NoOpFilter; +import org.apache.parquet.filter2.compat.FilterCompat.Visitor; +import org.apache.parquet.filter2.dictionarylevel.DictionaryFilter; +import org.apache.parquet.filter2.predicate.FilterPredicate; +import org.apache.parquet.filter2.predicate.SchemaCompatibilityValidator; +import org.apache.parquet.filter2.statisticslevel.StatisticsFilter; +import org.apache.parquet.hadoop.HoodieParquetFileReader; +import org.apache.parquet.hadoop.metadata.BlockMetaData; +import org.apache.parquet.schema.MessageType; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Objects; + +/** + * Given a {@link Filter} applies it to a list of BlockMetaData (row groups) + * If the Filter is an {@link org.apache.parquet.filter.UnboundRecordFilter} or the no op filter, + * no filtering will be performed. + */ +public class HoodieRowGroupFilter implements Visitor> { + private final List blocks; + private final MessageType schema; + private final List levels; + private final HoodieParquetFileReader reader; + + public enum FilterLevel { + STATISTICS, + DICTIONARY, + BLOOMFILTER + } + + /** + * @param filter a filter + * @param blocks a list of block metadata to filter + * @param schema the file schema + * @return a filtered list of block metadata + * @deprecated will be removed in 2.0.0. + */ + @Deprecated + public static List filterRowGroups(Filter filter, List blocks, MessageType schema) { + Objects.requireNonNull(filter, "filter cannot be null"); + return filter.accept(new HoodieRowGroupFilter(blocks, schema)); + } + + public static List filterRowGroups(List levels, Filter filter, List blocks, HoodieParquetFileReader reader) { + Objects.requireNonNull(filter, "filter cannot be null"); + return filter.accept(new HoodieRowGroupFilter(levels, blocks, reader)); + } + + @Deprecated + private HoodieRowGroupFilter(List blocks, MessageType schema) { + this.blocks = Objects.requireNonNull(blocks, "blocks cannnot be null"); + this.schema = Objects.requireNonNull(schema, "schema cannnot be null"); + this.levels = Collections.singletonList(FilterLevel.STATISTICS); + this.reader = null; + } + + private HoodieRowGroupFilter(List levels, List blocks, HoodieParquetFileReader reader) { + this.blocks = Objects.requireNonNull(blocks, "blocks cannnot be null"); + this.reader = Objects.requireNonNull(reader, "reader cannnot be null"); + this.schema = reader.getFileMetaData().getSchema(); + this.levels = levels; + } + + @Override + public List visit(FilterCompat.FilterPredicateCompat filterPredicateCompat) { + FilterPredicate filterPredicate = filterPredicateCompat.getFilterPredicate(); + + // check that the schema of the filter matches the schema of the file + SchemaCompatibilityValidator.validate(filterPredicate, schema); + + List filteredBlocks = new ArrayList(); + + for (BlockMetaData block : blocks) { + boolean drop = false; + + if (levels.contains(FilterLevel.STATISTICS)) { + drop = StatisticsFilter.canDrop(filterPredicate, block.getColumns()); + } + + if (!drop && levels.contains(FilterLevel.DICTIONARY)) { + drop = DictionaryFilter.canDrop(filterPredicate, block.getColumns(), reader.getDictionaryReader(block)); + } + + if (!drop && levels.contains(FilterLevel.BLOOMFILTER)) { + drop = HoodieBloomFilterImpl.canDrop(filterPredicate, block.getColumns(), reader.getBloomFilterDataReader(block)); + } + + if (!drop) { + filteredBlocks.add(block); + } + } + + return filteredBlocks; + } + + @Override + public List visit(FilterCompat.UnboundRecordFilterCompat unboundRecordFilterCompat) { + return blocks; + } + + @Override + public List visit(NoOpFilter noOpFilter) { + return blocks; + } +} diff --git a/hudi-spark-datasource/hudi-spark3.2.x/src/main/java/org/apache/parquet/hadoop/HoodieBloomFilterReader.java b/hudi-spark-datasource/hudi-spark3.2.x/src/main/java/org/apache/parquet/hadoop/HoodieBloomFilterReader.java new file mode 100644 index 0000000000000..d006423184718 --- /dev/null +++ b/hudi-spark-datasource/hudi-spark3.2.x/src/main/java/org/apache/parquet/hadoop/HoodieBloomFilterReader.java @@ -0,0 +1,70 @@ +/* + * 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.parquet.hadoop; + +import org.apache.parquet.column.values.bloomfilter.BloomFilter; +import org.apache.parquet.hadoop.metadata.BlockMetaData; +import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData; +import org.apache.parquet.hadoop.metadata.ColumnPath; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; + +/** + * Bloom filter reader that reads Bloom filter data from an open {@link HoodieParquetFileReader}. + */ +public class HoodieBloomFilterReader { + private final HoodieParquetFileReader reader; + private final Map columns; + private final Map cache = new HashMap<>(); + private Logger logger = LoggerFactory.getLogger(HoodieBloomFilterReader.class); + + public HoodieBloomFilterReader(HoodieParquetFileReader fileReader, BlockMetaData block) { + this.reader = fileReader; + this.columns = new HashMap<>(); + for (ColumnChunkMetaData column : block.getColumns()) { + columns.put(column.getPath(), column); + } + } + + public BloomFilter readBloomFilter(ColumnChunkMetaData meta) { + if (cache.containsKey(meta.getPath())) { + return cache.get(meta.getPath()); + } + try { + if (!cache.containsKey(meta.getPath())) { + BloomFilter bloomFilter = reader.readBloomFilter(meta); + if (bloomFilter == null) { + return null; + } + + cache.put(meta.getPath(), bloomFilter); + } + return cache.get(meta.getPath()); + } catch (IOException e) { + logger.error("Failed to read Bloom filter data", e); + } + + return null; + } +} diff --git a/hudi-spark-datasource/hudi-spark3.2.x/src/main/java/org/apache/parquet/hadoop/HoodieColumnChunkPageReadStore.java b/hudi-spark-datasource/hudi-spark3.2.x/src/main/java/org/apache/parquet/hadoop/HoodieColumnChunkPageReadStore.java new file mode 100644 index 0000000000000..098fbdb5db0ae --- /dev/null +++ b/hudi-spark-datasource/hudi-spark3.2.x/src/main/java/org/apache/parquet/hadoop/HoodieColumnChunkPageReadStore.java @@ -0,0 +1,293 @@ +/* + * 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.parquet.hadoop; + +import org.apache.parquet.bytes.BytesInput; +import org.apache.parquet.column.ColumnDescriptor; +import org.apache.parquet.column.page.DataPage; +import org.apache.parquet.column.page.DataPageV1; +import org.apache.parquet.column.page.DataPageV2; +import org.apache.parquet.column.page.DictionaryPage; +import org.apache.parquet.column.page.DictionaryPageReadStore; +import org.apache.parquet.column.page.PageReadStore; +import org.apache.parquet.column.page.PageReader; +import org.apache.parquet.compression.CompressionCodecFactory.BytesInputDecompressor; +import org.apache.parquet.crypto.AesCipher; +import org.apache.parquet.crypto.ModuleCipherFactory.ModuleType; +import org.apache.parquet.format.BlockCipher; +import org.apache.parquet.internal.column.columnindex.OffsetIndex; +import org.apache.parquet.internal.filter2.columnindex.HoodieRowRanges; +import org.apache.parquet.io.ParquetDecodingException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.ArrayDeque; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.PrimitiveIterator; +import java.util.Queue; + +/** + * TODO: should this actually be called RowGroupImpl or something? + * The name is kind of confusing since it references three different "entities" + * in our format: columns, chunks, and pages + * + */ +class HoodieColumnChunkPageReadStore implements PageReadStore, DictionaryPageReadStore { + private static final Logger LOG = LoggerFactory.getLogger(HoodieColumnChunkPageReadStore.class); + + /** + * PageReader for a single column chunk. A column chunk contains + * several pages, which are yielded one by one in order. + * + * This implementation is provided with a list of pages, each of which + * is decompressed and passed through. + */ + static final class ColumnChunkPageReader implements PageReader { + + private final BytesInputDecompressor decompressor; + private final long valueCount; + private final Queue compressedPages; + private final DictionaryPage compressedDictionaryPage; + // null means no page synchronization is required; firstRowIndex will not be returned by the pages + private final OffsetIndex offsetIndex; + private final long rowCount; + private int pageIndex = 0; + + private final BlockCipher.Decryptor blockDecryptor; + private final byte[] dataPageAAD; + private final byte[] dictionaryPageAAD; + + ColumnChunkPageReader(BytesInputDecompressor decompressor, List compressedPages, + DictionaryPage compressedDictionaryPage, OffsetIndex offsetIndex, long rowCount, + BlockCipher.Decryptor blockDecryptor, byte[] fileAAD, + int rowGroupOrdinal, int columnOrdinal) { + this.decompressor = decompressor; + this.compressedPages = new ArrayDeque(compressedPages); + this.compressedDictionaryPage = compressedDictionaryPage; + long count = 0; + for (DataPage p : compressedPages) { + count += p.getValueCount(); + } + this.valueCount = count; + this.offsetIndex = offsetIndex; + this.rowCount = rowCount; + + this.blockDecryptor = blockDecryptor; + + if (null != blockDecryptor) { + dataPageAAD = AesCipher.createModuleAAD(fileAAD, ModuleType.DataPage, rowGroupOrdinal, columnOrdinal, 0); + dictionaryPageAAD = AesCipher.createModuleAAD(fileAAD, ModuleType.DictionaryPage, rowGroupOrdinal, columnOrdinal, -1); + } else { + dataPageAAD = null; + dictionaryPageAAD = null; + } + } + + private int getPageOrdinal(int currentPageIndex) { + if (null == offsetIndex) { + return currentPageIndex; + } + + return offsetIndex.getPageOrdinal(currentPageIndex); + } + + @Override + public long getTotalValueCount() { + return valueCount; + } + + @Override + public DataPage readPage() { + final DataPage compressedPage = compressedPages.poll(); + if (compressedPage == null) { + return null; + } + final int currentPageIndex = pageIndex++; + + if (null != blockDecryptor) { + AesCipher.quickUpdatePageAAD(dataPageAAD, getPageOrdinal(currentPageIndex)); + } + + return compressedPage.accept(new DataPage.Visitor() { + @Override + public DataPage visit(DataPageV1 dataPageV1) { + try { + BytesInput bytes = dataPageV1.getBytes(); + if (null != blockDecryptor) { + bytes = BytesInput.from(blockDecryptor.decrypt(bytes.toByteArray(), dataPageAAD)); + } + BytesInput decompressed = decompressor.decompress(bytes, dataPageV1.getUncompressedSize()); + + final DataPageV1 decompressedPage; + if (offsetIndex == null) { + decompressedPage = new DataPageV1( + decompressed, + dataPageV1.getValueCount(), + dataPageV1.getUncompressedSize(), + dataPageV1.getStatistics(), + dataPageV1.getRlEncoding(), + dataPageV1.getDlEncoding(), + dataPageV1.getValueEncoding()); + } else { + long firstRowIndex = offsetIndex.getFirstRowIndex(currentPageIndex); + decompressedPage = new DataPageV1( + decompressed, + dataPageV1.getValueCount(), + dataPageV1.getUncompressedSize(), + firstRowIndex, + Math.toIntExact(offsetIndex.getLastRowIndex(currentPageIndex, rowCount) - firstRowIndex + 1), + dataPageV1.getStatistics(), + dataPageV1.getRlEncoding(), + dataPageV1.getDlEncoding(), + dataPageV1.getValueEncoding()); + } + if (dataPageV1.getCrc().isPresent()) { + decompressedPage.setCrc(dataPageV1.getCrc().getAsInt()); + } + return decompressedPage; + } catch (IOException e) { + throw new ParquetDecodingException("could not decompress page", e); + } + } + + @Override + public DataPage visit(DataPageV2 dataPageV2) { + if (!dataPageV2.isCompressed() && offsetIndex == null && null == blockDecryptor) { + return dataPageV2; + } + BytesInput pageBytes = dataPageV2.getData(); + + if (null != blockDecryptor) { + try { + pageBytes = BytesInput.from(blockDecryptor.decrypt(pageBytes.toByteArray(), dataPageAAD)); + } catch (IOException e) { + throw new ParquetDecodingException("could not convert page ByteInput to byte array", e); + } + } + if (dataPageV2.isCompressed()) { + int uncompressedSize = Math.toIntExact( + dataPageV2.getUncompressedSize() + - dataPageV2.getDefinitionLevels().size() + - dataPageV2.getRepetitionLevels().size()); + try { + pageBytes = decompressor.decompress(pageBytes, uncompressedSize); + } catch (IOException e) { + throw new ParquetDecodingException("could not decompress page", e); + } + } + + if (offsetIndex == null) { + return DataPageV2.uncompressed( + dataPageV2.getRowCount(), + dataPageV2.getNullCount(), + dataPageV2.getValueCount(), + dataPageV2.getRepetitionLevels(), + dataPageV2.getDefinitionLevels(), + dataPageV2.getDataEncoding(), + pageBytes, + dataPageV2.getStatistics()); + } else { + return DataPageV2.uncompressed( + dataPageV2.getRowCount(), + dataPageV2.getNullCount(), + dataPageV2.getValueCount(), + offsetIndex.getFirstRowIndex(currentPageIndex), + dataPageV2.getRepetitionLevels(), + dataPageV2.getDefinitionLevels(), + dataPageV2.getDataEncoding(), + pageBytes, + dataPageV2.getStatistics()); + } + } + }); + } + + @Override + public DictionaryPage readDictionaryPage() { + if (compressedDictionaryPage == null) { + return null; + } + try { + BytesInput bytes = compressedDictionaryPage.getBytes(); + if (null != blockDecryptor) { + bytes = BytesInput.from(blockDecryptor.decrypt(bytes.toByteArray(), dictionaryPageAAD)); + } + DictionaryPage decompressedPage = new DictionaryPage( + decompressor.decompress(bytes, compressedDictionaryPage.getUncompressedSize()), + compressedDictionaryPage.getDictionarySize(), + compressedDictionaryPage.getEncoding()); + if (compressedDictionaryPage.getCrc().isPresent()) { + decompressedPage.setCrc(compressedDictionaryPage.getCrc().getAsInt()); + } + return decompressedPage; + } catch (IOException e) { + throw new ParquetDecodingException("Could not decompress dictionary page", e); + } + } + } + + private final Map readers = new HashMap(); + private final long rowCount; + private final HoodieRowRanges rowRanges; + + public HoodieColumnChunkPageReadStore(long rowCount) { + this.rowCount = rowCount; + rowRanges = null; + } + + HoodieColumnChunkPageReadStore(HoodieRowRanges rowRanges) { + this.rowRanges = rowRanges; + rowCount = rowRanges.rowCount(); + } + + @Override + public long getRowCount() { + return rowCount; + } + + @Override + public PageReader getPageReader(ColumnDescriptor path) { + final PageReader pageReader = readers.get(path); + if (pageReader == null) { + throw new IllegalArgumentException(path + " is not in the store: " + readers.keySet() + " " + rowCount); + } + return pageReader; + } + + @Override + public DictionaryPage readDictionaryPage(ColumnDescriptor descriptor) { + return readers.get(descriptor).readDictionaryPage(); + } + + @Override + public Optional getRowIndexes() { + return rowRanges == null ? Optional.empty() : Optional.of(rowRanges.iterator()); + } + + void addColumn(ColumnDescriptor path, ColumnChunkPageReader reader) { + if (readers.put(path, reader) != null) { + throw new RuntimeException(path + " was added twice"); + } + } +} diff --git a/hudi-spark-datasource/hudi-spark3.2.x/src/main/java/org/apache/parquet/hadoop/HoodieColumnIndexFilterUtils.java b/hudi-spark-datasource/hudi-spark3.2.x/src/main/java/org/apache/parquet/hadoop/HoodieColumnIndexFilterUtils.java new file mode 100644 index 0000000000000..570a18b3f4421 --- /dev/null +++ b/hudi-spark-datasource/hudi-spark3.2.x/src/main/java/org/apache/parquet/hadoop/HoodieColumnIndexFilterUtils.java @@ -0,0 +1,162 @@ +/* + * 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.parquet.hadoop; + +import it.unimi.dsi.fastutil.ints.IntArrayList; +import it.unimi.dsi.fastutil.ints.IntList; +import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData; +import org.apache.parquet.internal.column.columnindex.OffsetIndex; +import org.apache.parquet.internal.filter2.columnindex.HoodieRowRanges; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Formatter; +import java.util.List; + +/** + * Internal utility class to help at column index based filtering. + */ +class HoodieColumnIndexFilterUtils { + static class OffsetRange { + private final long offset; + private long length; + + private OffsetRange(long offset, int length) { + this.offset = offset; + this.length = length; + } + + long getOffset() { + return offset; + } + + long getLength() { + return length; + } + + private boolean extend(long offset, int length) { + if (this.offset + this.length == offset) { + this.length += length; + return true; + } else { + return false; + } + } + } + + private static class FilteredOffsetIndex implements OffsetIndex { + private final OffsetIndex offsetIndex; + private final int[] indexMap; + + private FilteredOffsetIndex(OffsetIndex offsetIndex, int[] indexMap) { + this.offsetIndex = offsetIndex; + this.indexMap = indexMap; + } + + @Override + public int getPageOrdinal(int pageIndex) { + return indexMap[pageIndex]; + } + + @Override + public int getPageCount() { + return indexMap.length; + } + + @Override + public long getOffset(int pageIndex) { + return offsetIndex.getOffset(indexMap[pageIndex]); + } + + @Override + public int getCompressedPageSize(int pageIndex) { + return offsetIndex.getCompressedPageSize(indexMap[pageIndex]); + } + + @Override + public long getFirstRowIndex(int pageIndex) { + return offsetIndex.getFirstRowIndex(indexMap[pageIndex]); + } + + @Override + public long getLastRowIndex(int pageIndex, long totalRowCount) { + int nextIndex = indexMap[pageIndex] + 1; + return (nextIndex >= offsetIndex.getPageCount() ? totalRowCount : offsetIndex.getFirstRowIndex(nextIndex)) - 1; + } + + @Override + public String toString() { + try (Formatter formatter = new Formatter()) { + formatter.format("%-12s %20s %16s %20s\n", "", "offset", "compressed size", "first row index"); + for (int i = 0, n = offsetIndex.getPageCount(); i < n; ++i) { + int index = Arrays.binarySearch(indexMap, i); + boolean isHidden = index < 0; + formatter.format("%spage-%-5d %20d %16d %20d\n", + isHidden ? "- " : " ", + isHidden ? i : index, + offsetIndex.getOffset(i), + offsetIndex.getCompressedPageSize(i), + offsetIndex.getFirstRowIndex(i)); + } + return formatter.toString(); + } + } + } + + /* + * Returns the filtered offset index containing only the pages which are overlapping with rowRanges. + */ + static OffsetIndex filterOffsetIndex(OffsetIndex offsetIndex, HoodieRowRanges rowRanges, long totalRowCount) { + IntList indexMap = new IntArrayList(); + for (int i = 0, n = offsetIndex.getPageCount(); i < n; ++i) { + long from = offsetIndex.getFirstRowIndex(i); + if (rowRanges.isOverlapping(from, offsetIndex.getLastRowIndex(i, totalRowCount))) { + indexMap.add(i); + } + } + return new FilteredOffsetIndex(offsetIndex, indexMap.toIntArray()); + } + + static List calculateOffsetRanges(OffsetIndex offsetIndex, ColumnChunkMetaData cm, + long firstPageOffset) { + List ranges = new ArrayList<>(); + int n = offsetIndex.getPageCount(); + if (n > 0) { + OffsetRange currentRange = null; + + // Add a range for the dictionary page if required + long rowGroupOffset = cm.getStartingPos(); + if (rowGroupOffset < firstPageOffset) { + currentRange = new OffsetRange(rowGroupOffset, (int) (firstPageOffset - rowGroupOffset)); + ranges.add(currentRange); + } + + for (int i = 0; i < n; ++i) { + long offset = offsetIndex.getOffset(i); + int length = offsetIndex.getCompressedPageSize(i); + if (currentRange == null || !currentRange.extend(offset, length)) { + currentRange = new OffsetRange(offset, length); + ranges.add(currentRange); + } + } + } + return ranges; + } +} diff --git a/hudi-spark-datasource/hudi-spark3.2.x/src/main/java/org/apache/parquet/hadoop/HoodieColumnIndexStoreImpl.java b/hudi-spark-datasource/hudi-spark3.2.x/src/main/java/org/apache/parquet/hadoop/HoodieColumnIndexStoreImpl.java new file mode 100644 index 0000000000000..36400b4d16172 --- /dev/null +++ b/hudi-spark-datasource/hudi-spark3.2.x/src/main/java/org/apache/parquet/hadoop/HoodieColumnIndexStoreImpl.java @@ -0,0 +1,156 @@ +/* + * 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.parquet.hadoop; + +import org.apache.parquet.hadoop.metadata.BlockMetaData; +import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData; +import org.apache.parquet.hadoop.metadata.ColumnPath; +import org.apache.parquet.internal.column.columnindex.ColumnIndex; +import org.apache.parquet.internal.column.columnindex.OffsetIndex; +import org.apache.parquet.internal.filter2.columnindex.ColumnIndexStore; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; +import java.util.Set; + +import static java.util.Collections.emptySet; + +/** + * Internal implementation of {@link ColumnIndexStore}. + */ +class HoodieColumnIndexStoreImpl implements ColumnIndexStore { + + private interface IndexStore { + ColumnIndex getColumnIndex(); + + OffsetIndex getOffsetIndex(); + } + + private class IndexStoreImpl implements IndexStore { + private final ColumnChunkMetaData meta; + private ColumnIndex columnIndex; + private boolean columnIndexRead; + private final OffsetIndex offsetIndex; + + IndexStoreImpl(ColumnChunkMetaData meta) { + this.meta = meta; + OffsetIndex oi; + try { + oi = reader.readOffsetIndex(meta); + } catch (IOException e) { + // If the I/O issue still stands it will fail the reading later; + // otherwise we fail the filtering only with a missing offset index. + LOGGER.warn("Unable to read offset index for column {}", meta.getPath(), e); + oi = null; + } + if (oi == null) { + throw new MissingOffsetIndexException(meta.getPath()); + } + offsetIndex = oi; + } + + @Override + public ColumnIndex getColumnIndex() { + if (!columnIndexRead) { + try { + columnIndex = reader.readColumnIndex(meta); + } catch (IOException e) { + // If the I/O issue still stands it will fail the reading later; + // otherwise we fail the filtering only with a missing column index. + LOGGER.warn("Unable to read column index for column {}", meta.getPath(), e); + } + columnIndexRead = true; + } + return columnIndex; + } + + @Override + public OffsetIndex getOffsetIndex() { + return offsetIndex; + } + } + + private static final Logger LOGGER = LoggerFactory.getLogger(HoodieColumnIndexStoreImpl.class); + // Used for columns are not in this parquet file + private static final IndexStore MISSING_INDEX_STORE = new IndexStore() { + @Override + public ColumnIndex getColumnIndex() { + return null; + } + + @Override + public OffsetIndex getOffsetIndex() { + return null; + } + }; + private static final HoodieColumnIndexStoreImpl EMPTY = new HoodieColumnIndexStoreImpl(null, new BlockMetaData(), emptySet()) { + @Override + public ColumnIndex getColumnIndex(ColumnPath column) { + return null; + } + + @Override + public OffsetIndex getOffsetIndex(ColumnPath column) { + throw new MissingOffsetIndexException(column); + } + }; + + private final HoodieParquetFileReader reader; + private final Map store; + + /* + * Creates a column index store which lazily reads column/offset indexes for the columns in paths. (paths are the set + * of columns used for the projection) + */ + static ColumnIndexStore create(HoodieParquetFileReader reader, BlockMetaData block, Set paths) { + try { + return new HoodieColumnIndexStoreImpl(reader, block, paths); + } catch (MissingOffsetIndexException e) { + return EMPTY; + } + } + + private HoodieColumnIndexStoreImpl(HoodieParquetFileReader reader, BlockMetaData block, Set paths) { + // TODO[GS]: Offset index for every paths will be required; pre-read the consecutive ones at once? + // TODO[GS]: Pre-read column index based on filter? + this.reader = reader; + Map store = new HashMap<>(); + for (ColumnChunkMetaData column : block.getColumns()) { + ColumnPath path = column.getPath(); + if (paths.contains(path)) { + store.put(path, new IndexStoreImpl(column)); + } + } + this.store = store; + } + + @Override + public ColumnIndex getColumnIndex(ColumnPath column) { + return store.getOrDefault(column, MISSING_INDEX_STORE).getColumnIndex(); + } + + @Override + public OffsetIndex getOffsetIndex(ColumnPath column) { + return store.getOrDefault(column, MISSING_INDEX_STORE).getOffsetIndex(); + } +} diff --git a/hudi-spark-datasource/hudi-spark3.2.x/src/main/java/org/apache/parquet/hadoop/HoodieDictionaryPageReader.java b/hudi-spark-datasource/hudi-spark3.2.x/src/main/java/org/apache/parquet/hadoop/HoodieDictionaryPageReader.java new file mode 100644 index 0000000000000..315efc4f3bcc0 --- /dev/null +++ b/hudi-spark-datasource/hudi-spark3.2.x/src/main/java/org/apache/parquet/hadoop/HoodieDictionaryPageReader.java @@ -0,0 +1,118 @@ +/* + * 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.parquet.hadoop; + +import org.apache.hudi.common.util.Option; + +import org.apache.parquet.bytes.BytesInput; +import org.apache.parquet.column.ColumnDescriptor; +import org.apache.parquet.column.page.DictionaryPage; +import org.apache.parquet.column.page.DictionaryPageReadStore; +import org.apache.parquet.hadoop.metadata.BlockMetaData; +import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData; +import org.apache.parquet.io.ParquetDecodingException; + +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; +import java.util.Objects; +import java.util.concurrent.ConcurrentHashMap; + +/** + * A {@link DictionaryPageReadStore} implementation that reads dictionaries from + * an open {@link ParquetFileReader}. + * + * This implementation will delegate dictionary reads to a + * {@link HoodieColumnChunkPageReadStore} to avoid extra reads after a row group has + * been loaded into memory. + */ +class HoodieDictionaryPageReader implements DictionaryPageReadStore { + + private final HoodieParquetFileReader reader; + private final Map columns; + private final Map> dictionaryPageCache; + private HoodieColumnChunkPageReadStore rowGroup = null; + + /** + * Instantiate a new HoodieDictionaryPageReader. + * + * @param reader The target ParquetFileReader + * @param block The target BlockMetaData + * + * @throws NullPointerException if {@code reader} or {@code block} is + * {@code null} + */ + HoodieDictionaryPageReader(HoodieParquetFileReader reader, BlockMetaData block) { + this.reader = Objects.requireNonNull(reader); + this.columns = new HashMap<>(); + this.dictionaryPageCache = new ConcurrentHashMap<>(); + + for (ColumnChunkMetaData column : block.getColumns()) { + columns.put(column.getPath().toDotString(), column); + } + } + + /** + * Sets this reader's row group's page store. When a row group is set, this + * reader will delegate to that row group to return dictionary pages. This + * avoids seeking and re-reading dictionary bytes after this reader's row + * group is loaded into memory. + * + * @param rowGroup a HoodieColumnChunkPageReadStore for this reader's row group + */ + void setRowGroup(HoodieColumnChunkPageReadStore rowGroup) { + this.rowGroup = rowGroup; + } + + @Override + public DictionaryPage readDictionaryPage(ColumnDescriptor descriptor) { + if (rowGroup != null) { + // if the row group has already been read, use that dictionary + return rowGroup.readDictionaryPage(descriptor); + } + + String dotPath = String.join(".", descriptor.getPath()); + ColumnChunkMetaData column = columns.get(dotPath); + if (column == null) { + throw new ParquetDecodingException( + "Failed to load dictionary, unknown column: " + dotPath); + } + + return dictionaryPageCache.computeIfAbsent(dotPath, key -> { + try { + final DictionaryPage dict = + column.hasDictionaryPage() ? reader.readDictionary(column) : null; + + // Copy the dictionary to ensure it can be reused if it is returned + // more than once. This can happen when a DictionaryFilter has two or + // more predicates for the same column. Cache misses as well. + return (dict != null) ? Option.of(reusableCopy(dict)) : Option.empty(); + } catch (IOException e) { + throw new ParquetDecodingException("Failed to read dictionary", e); + } + }).orElse(null); + } + + private static DictionaryPage reusableCopy(DictionaryPage dict) + throws IOException { + return new DictionaryPage(BytesInput.from(dict.getBytes().toByteArray()), + dict.getDictionarySize(), dict.getEncoding()); + } +} diff --git a/hudi-spark-datasource/hudi-spark3.2.x/src/main/java/org/apache/parquet/hadoop/HoodieParquetFileReader.java b/hudi-spark-datasource/hudi-spark3.2.x/src/main/java/org/apache/parquet/hadoop/HoodieParquetFileReader.java new file mode 100644 index 0000000000000..6873cea4a64a7 --- /dev/null +++ b/hudi-spark-datasource/hudi-spark3.2.x/src/main/java/org/apache/parquet/hadoop/HoodieParquetFileReader.java @@ -0,0 +1,1789 @@ +/* + * 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.parquet.hadoop; + +import org.apache.hudi.secondary.index.SecondaryIndexUtils; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.parquet.HadoopReadOptions; +import org.apache.parquet.ParquetReadOptions; +import org.apache.parquet.bytes.ByteBufferInputStream; +import org.apache.parquet.bytes.BytesInput; +import org.apache.parquet.column.ColumnDescriptor; +import org.apache.parquet.column.page.DataPage; +import org.apache.parquet.column.page.DataPageV1; +import org.apache.parquet.column.page.DataPageV2; +import org.apache.parquet.column.page.DictionaryPage; +import org.apache.parquet.column.page.DictionaryPageReadStore; +import org.apache.parquet.column.page.PageReadStore; +import org.apache.parquet.column.values.bloomfilter.BlockSplitBloomFilter; +import org.apache.parquet.column.values.bloomfilter.BloomFilter; +import org.apache.parquet.compression.CompressionCodecFactory.BytesInputDecompressor; +import org.apache.parquet.crypto.AesCipher; +import org.apache.parquet.crypto.FileDecryptionProperties; +import org.apache.parquet.crypto.InternalColumnDecryptionSetup; +import org.apache.parquet.crypto.InternalFileDecryptor; +import org.apache.parquet.crypto.ModuleCipherFactory.ModuleType; +import org.apache.parquet.crypto.ParquetCryptoRuntimeException; +import org.apache.parquet.filter2.compat.FilterCompat; +import org.apache.parquet.filter2.compat.HoodieRowGroupFilter; +import org.apache.parquet.format.BlockCipher; +import org.apache.parquet.format.BloomFilterHeader; +import org.apache.parquet.format.DataPageHeader; +import org.apache.parquet.format.DataPageHeaderV2; +import org.apache.parquet.format.DictionaryPageHeader; +import org.apache.parquet.format.FileCryptoMetaData; +import org.apache.parquet.format.PageHeader; +import org.apache.parquet.format.Util; +import org.apache.parquet.format.converter.ParquetMetadataConverter; +import org.apache.parquet.format.converter.ParquetMetadataConverter.MetadataFilter; +import org.apache.parquet.hadoop.HoodieColumnChunkPageReadStore.ColumnChunkPageReader; +import org.apache.parquet.hadoop.HoodieColumnIndexFilterUtils.OffsetRange; +import org.apache.parquet.hadoop.metadata.BlockMetaData; +import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData; +import org.apache.parquet.hadoop.metadata.ColumnPath; +import org.apache.parquet.hadoop.metadata.FileMetaData; +import org.apache.parquet.hadoop.metadata.ParquetMetadata; +import org.apache.parquet.hadoop.util.HadoopInputFile; +import org.apache.parquet.hadoop.util.HiddenFileFilter; +import org.apache.parquet.hadoop.util.counters.BenchmarkCounter; +import org.apache.parquet.internal.column.columnindex.ColumnIndex; +import org.apache.parquet.internal.column.columnindex.OffsetIndex; +import org.apache.parquet.internal.filter2.columnindex.ColumnIndexStore; +import org.apache.parquet.internal.filter2.columnindex.HoodieColumnIndexFilter; +import org.apache.parquet.internal.filter2.columnindex.HoodieRowRanges; +import org.apache.parquet.internal.hadoop.metadata.IndexReference; +import org.apache.parquet.io.InputFile; +import org.apache.parquet.io.ParquetDecodingException; +import org.apache.parquet.io.SeekableInputStream; +import org.apache.parquet.schema.MessageType; +import org.apache.parquet.schema.PrimitiveType; +import org.apache.yetus.audience.InterfaceAudience.Private; +import org.roaringbitmap.PeekableIntIterator; +import org.roaringbitmap.RoaringBitmap; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.Closeable; +import java.io.IOException; +import java.io.InputStream; +import java.io.SequenceInputStream; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Set; +import java.util.concurrent.Callable; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; +import java.util.stream.Collectors; +import java.util.stream.Stream; +import java.util.zip.CRC32; + +import static org.apache.parquet.bytes.BytesUtils.readIntLittleEndian; +import static org.apache.parquet.filter2.compat.HoodieRowGroupFilter.FilterLevel.BLOOMFILTER; +import static org.apache.parquet.filter2.compat.HoodieRowGroupFilter.FilterLevel.DICTIONARY; +import static org.apache.parquet.filter2.compat.HoodieRowGroupFilter.FilterLevel.STATISTICS; +import static org.apache.parquet.format.Util.readFileCryptoMetaData; +import static org.apache.parquet.format.converter.ParquetMetadataConverter.NO_FILTER; +import static org.apache.parquet.format.converter.ParquetMetadataConverter.SKIP_ROW_GROUPS; +import static org.apache.parquet.hadoop.HoodieColumnIndexFilterUtils.calculateOffsetRanges; +import static org.apache.parquet.hadoop.HoodieColumnIndexFilterUtils.filterOffsetIndex; +import static org.apache.parquet.hadoop.ParquetFileWriter.EFMAGIC; +import static org.apache.parquet.hadoop.ParquetFileWriter.MAGIC; +import static org.apache.parquet.hadoop.ParquetFileWriter.PARQUET_COMMON_METADATA_FILE; +import static org.apache.parquet.hadoop.ParquetFileWriter.PARQUET_METADATA_FILE; + +/** + * Internal implementation of the Parquet file reader as a block container + */ +public class HoodieParquetFileReader implements Closeable { + + private static final Logger LOG = LoggerFactory.getLogger(HoodieParquetFileReader.class); + + public static String PARQUET_READ_PARALLELISM = "parquet.metadata.read.parallelism"; + + private final ParquetMetadataConverter converter; + + private final CRC32 crc; + + /** + * for files provided, check if there's a summary file. + * If a summary file is found it is used otherwise the file footer is used. + * @param configuration the hadoop conf to connect to the file system; + * @param partFiles the part files to read + * @return the footers for those files using the summary file if possible. + * @throws IOException if there is an exception while reading footers + * @deprecated metadata files are not recommended and will be removed in 2.0.0 + */ + @Deprecated + public static List