diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieWriteClient.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieWriteClient.java index a197febea..b65060fa2 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieWriteClient.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieWriteClient.java @@ -101,7 +101,7 @@ public abstract class AbstractHoodieWriteClient index; + private final transient HoodieIndex index; protected transient Timer.Context writeTimer = null; protected transient Timer.Context compactionTimer; @@ -138,7 +138,7 @@ public abstract class AbstractHoodieWriteClient createIndex(HoodieWriteConfig writeConfig); + protected abstract HoodieIndex createIndex(HoodieWriteConfig writeConfig); public void setOperationType(WriteOperationType operationType) { this.operationType = operationType; @@ -1006,7 +1006,7 @@ public abstract class AbstractHoodieWriteClient getIndex() { + public HoodieIndex getIndex() { return index; } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/HoodieIndex.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/HoodieIndex.java index 2e1915ff2..974f4d546 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/HoodieIndex.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/HoodieIndex.java @@ -21,12 +21,16 @@ package org.apache.hudi.index; import org.apache.hudi.ApiMaturityLevel; import org.apache.hudi.PublicAPIClass; import org.apache.hudi.PublicAPIMethod; +import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.FileSlice; import org.apache.hudi.common.model.HoodieKey; +import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieIndexException; +import org.apache.hudi.exception.HoodieNotSupportedException; import org.apache.hudi.table.HoodieTable; import java.io.Serializable; @@ -35,9 +39,9 @@ import java.io.Serializable; * Base class for different types of indexes to determine the mapping from uuid. * * @param Sub type of HoodieRecordPayload - * @param Type of inputs - * @param Type of keys - * @param Type of outputs + * @param Type of inputs for deprecated APIs + * @param Type of keys for deprecated APIs + * @param Type of outputs for deprecated APIs */ @PublicAPIClass(maturity = ApiMaturityLevel.EVOLVING) public abstract class HoodieIndex implements Serializable { @@ -52,18 +56,39 @@ public abstract class HoodieIndex implem * Looks up the index and tags each incoming record with a location of a file that contains the row (if it is actually * present). */ - @PublicAPIMethod(maturity = ApiMaturityLevel.STABLE) - public abstract I tagLocation(I records, HoodieEngineContext context, - HoodieTable hoodieTable) throws HoodieIndexException; + @Deprecated + @PublicAPIMethod(maturity = ApiMaturityLevel.DEPRECATED) + public I tagLocation(I records, HoodieEngineContext context, + HoodieTable hoodieTable) throws HoodieIndexException { + throw new HoodieNotSupportedException("Deprecated API should not be called"); + } /** * Extracts the location of written records, and updates the index. - *

- * TODO(vc): We may need to propagate the record as well in a WriteStatus class */ - @PublicAPIMethod(maturity = ApiMaturityLevel.STABLE) - public abstract O updateLocation(O writeStatuses, HoodieEngineContext context, - HoodieTable hoodieTable) throws HoodieIndexException; + @Deprecated + @PublicAPIMethod(maturity = ApiMaturityLevel.DEPRECATED) + public O updateLocation(O writeStatuses, HoodieEngineContext context, + HoodieTable hoodieTable) throws HoodieIndexException { + throw new HoodieNotSupportedException("Deprecated API should not be called"); + } + + /** + * Looks up the index and tags each incoming record with a location of a file that contains + * the row (if it is actually present). + */ + @PublicAPIMethod(maturity = ApiMaturityLevel.EVOLVING) + public abstract HoodieData> tagLocation( + HoodieData> records, HoodieEngineContext context, + HoodieTable hoodieTable) throws HoodieIndexException; + + /** + * Extracts the location of written records, and updates the index. + */ + @PublicAPIMethod(maturity = ApiMaturityLevel.EVOLVING) + public abstract HoodieData updateLocation( + HoodieData writeStatuses, HoodieEngineContext context, + HoodieTable hoodieTable) throws HoodieIndexException; /** * Rollback the effects of the commit made at instantTime. diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bloom/BaseHoodieBloomIndexHelper.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bloom/BaseHoodieBloomIndexHelper.java new file mode 100644 index 000000000..9f0e81563 --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bloom/BaseHoodieBloomIndexHelper.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.index.bloom; + +import org.apache.hudi.common.data.HoodieData; +import org.apache.hudi.common.data.HoodiePairData; +import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.model.HoodieKey; +import org.apache.hudi.common.model.HoodieRecordLocation; +import org.apache.hudi.common.util.collection.ImmutablePair; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.table.HoodieTable; + +import java.io.Serializable; +import java.util.List; +import java.util.Map; + +/** + * Helper for {@link HoodieBloomIndex} containing engine-specific logic. + */ +public abstract class BaseHoodieBloomIndexHelper implements Serializable { + /** + * Find out pair. + * + * @param config Write config. + * @param context {@link HoodieEngineContext} instance to use. + * @param hoodieTable {@link HoodieTable} instance to use. + * @param partitionRecordKeyPairs Pairs of partition path and record key. + * @param fileComparisonPairs Pairs of filename and record key based on file comparisons. + * @param partitionToFileInfo Partition path to {@link BloomIndexFileInfo} map. + * @param recordsPerPartition Number of records per partition in a map. + * @return {@link HoodiePairData} of {@link HoodieKey} and {@link HoodieRecordLocation} pairs. + */ + public abstract HoodiePairData findMatchingFilesForRecordKeys( + HoodieWriteConfig config, HoodieEngineContext context, HoodieTable hoodieTable, + HoodiePairData partitionRecordKeyPairs, + HoodieData> fileComparisonPairs, + Map> partitionToFileInfo, + Map recordsPerPartition); +} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bloom/HoodieBaseBloomIndex.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bloom/HoodieBaseBloomIndex.java deleted file mode 100644 index 75ab693d1..000000000 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bloom/HoodieBaseBloomIndex.java +++ /dev/null @@ -1,261 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hudi.index.bloom; - -import com.beust.jcommander.internal.Lists; -import org.apache.hudi.client.WriteStatus; -import org.apache.hudi.common.engine.HoodieEngineContext; -import org.apache.hudi.common.model.HoodieKey; -import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordLocation; -import org.apache.hudi.common.model.HoodieRecordPayload; -import org.apache.hudi.common.util.Option; -import org.apache.hudi.common.util.collection.Pair; -import org.apache.hudi.config.HoodieWriteConfig; -import org.apache.hudi.exception.MetadataNotFoundException; -import org.apache.hudi.index.HoodieIndex; -import org.apache.hudi.index.HoodieIndexUtils; -import org.apache.hudi.io.HoodieKeyLookupHandle; -import org.apache.hudi.io.HoodieRangeInfoHandle; -import org.apache.hudi.table.HoodieTable; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; - -import java.util.ArrayList; -import java.util.HashMap; -import java.util.Iterator; -import java.util.List; -import java.util.Map; - -import static java.util.stream.Collectors.groupingBy; -import static java.util.stream.Collectors.mapping; -import static java.util.stream.Collectors.toList; -import static org.apache.hudi.index.HoodieIndexUtils.getLatestBaseFilesForAllPartitions; - -@SuppressWarnings("checkstyle:LineLength") -public class HoodieBaseBloomIndex extends HoodieIndex>, List, List> { - - private static final Logger LOG = LogManager.getLogger(HoodieBaseBloomIndex.class); - - public HoodieBaseBloomIndex(HoodieWriteConfig config) { - super(config); - } - - @Override - public List> tagLocation(List> records, HoodieEngineContext context, - HoodieTable>, List, List> hoodieTable) { - // Step 1: Extract out thinner Map of (partitionPath, recordKey) - Map> partitionRecordKeyMap = new HashMap<>(); - records.forEach(record -> { - if (partitionRecordKeyMap.containsKey(record.getPartitionPath())) { - partitionRecordKeyMap.get(record.getPartitionPath()).add(record.getRecordKey()); - } else { - List recordKeys = Lists.newArrayList(); - recordKeys.add(record.getRecordKey()); - partitionRecordKeyMap.put(record.getPartitionPath(), recordKeys); - } - }); - - // Step 2: Lookup indexes for all the partition/recordkey pair - Map keyFilenamePairMap = - lookupIndex(partitionRecordKeyMap, context, hoodieTable); - - if (LOG.isDebugEnabled()) { - long totalTaggedRecords = keyFilenamePairMap.values().size(); - LOG.debug("Number of update records (ones tagged with a fileID): " + totalTaggedRecords); - } - - // Step 3: Tag the incoming records, as inserts or updates, by joining with existing record keys - List> taggedRecords = tagLocationBacktoRecords(keyFilenamePairMap, records); - - return taggedRecords; - } - - /** - * Lookup the location for each record key and return the pair for all record keys already - * present and drop the record keys if not present. - */ - private Map lookupIndex( - Map> partitionRecordKeyMap, final HoodieEngineContext context, - final HoodieTable hoodieTable) { - // Obtain records per partition, in the incoming records - Map recordsPerPartition = new HashMap<>(); - partitionRecordKeyMap.keySet().forEach(k -> recordsPerPartition.put(k, Long.valueOf(partitionRecordKeyMap.get(k).size()))); - List affectedPartitionPathList = new ArrayList<>(recordsPerPartition.keySet()); - - // Step 2: Load all involved files as pairs - List> fileInfoList = - loadInvolvedFiles(affectedPartitionPathList, context, hoodieTable); - final Map> partitionToFileInfo = - fileInfoList.stream().collect(groupingBy(Pair::getLeft, mapping(Pair::getRight, toList()))); - - // Step 3: Obtain a List, for each incoming record, that already exists, with the file id, - // that contains it. - List> fileComparisons = - explodeRecordsWithFileComparisons(partitionToFileInfo, partitionRecordKeyMap); - return findMatchingFilesForRecordKeys(fileComparisons, hoodieTable); - } - - /** - * Load all involved files as pair List. - */ - //TODO duplicate code with spark, we can optimize this method later - List> loadInvolvedFiles(List partitions, final HoodieEngineContext context, - final HoodieTable hoodieTable) { - // Obtain the latest data files from all the partitions. - List> partitionPathFileIDList = getLatestBaseFilesForAllPartitions(partitions, context, hoodieTable).stream() - .map(pair -> Pair.of(pair.getKey(), pair.getValue().getFileId())) - .collect(toList()); - - if (config.getBloomIndexPruneByRanges()) { - // also obtain file ranges, if range pruning is enabled - context.setJobStatus(this.getClass().getName(), "Obtain key ranges for file slices (range pruning=on)"); - return context.map(partitionPathFileIDList, pf -> { - try { - HoodieRangeInfoHandle rangeInfoHandle = new HoodieRangeInfoHandle(config, hoodieTable, pf); - String[] minMaxKeys = rangeInfoHandle.getMinMaxKeys(); - return Pair.of(pf.getKey(), new BloomIndexFileInfo(pf.getValue(), minMaxKeys[0], minMaxKeys[1])); - } catch (MetadataNotFoundException me) { - LOG.warn("Unable to find range metadata in file :" + pf); - return Pair.of(pf.getKey(), new BloomIndexFileInfo(pf.getValue())); - } - }, Math.max(partitionPathFileIDList.size(), 1)); - } else { - return partitionPathFileIDList.stream() - .map(pf -> Pair.of(pf.getKey(), new BloomIndexFileInfo(pf.getValue()))).collect(toList()); - } - } - - @Override - public boolean rollbackCommit(String instantTime) { - // Nope, don't need to do anything. - return true; - } - - /** - * This is not global, since we depend on the partitionPath to do the lookup. - */ - @Override - public boolean isGlobal() { - return false; - } - - /** - * No indexes into log files yet. - */ - @Override - public boolean canIndexLogFiles() { - return false; - } - - /** - * Bloom filters are stored, into the same data files. - */ - @Override - public boolean isImplicitWithStorage() { - return true; - } - - /** - * For each incoming record, produce N output records, 1 each for each file against which the record's key needs to be - * checked. For tables, where the keys have a definite insert order (e.g: timestamp as prefix), the number of files - * to be compared gets cut down a lot from range pruning. - *

- * Sub-partition to ensure the records can be looked up against files & also prune file<=>record comparisons based on - * recordKey ranges in the index info. - */ - List> explodeRecordsWithFileComparisons( - final Map> partitionToFileIndexInfo, - Map> partitionRecordKeyMap) { - IndexFileFilter indexFileFilter = - config.useBloomIndexTreebasedFilter() ? new IntervalTreeBasedIndexFileFilter(partitionToFileIndexInfo) - : new ListBasedIndexFileFilter(partitionToFileIndexInfo); - - List> fileRecordPairs = new ArrayList<>(); - partitionRecordKeyMap.keySet().forEach(partitionPath -> { - List hoodieRecordKeys = partitionRecordKeyMap.get(partitionPath); - hoodieRecordKeys.forEach(hoodieRecordKey -> { - indexFileFilter.getMatchingFilesAndPartition(partitionPath, hoodieRecordKey).forEach(partitionFileIdPair -> { - fileRecordPairs.add(Pair.of(partitionFileIdPair.getRight(), - new HoodieKey(hoodieRecordKey, partitionPath))); - }); - }); - }); - return fileRecordPairs; - } - - /** - * Find out pair. - */ - Map findMatchingFilesForRecordKeys( - List> fileComparisons, - HoodieTable hoodieTable) { - - fileComparisons = fileComparisons.stream().sorted((o1, o2) -> o1.getLeft().compareTo(o2.getLeft())).collect(toList()); - - List keyLookupResults = new ArrayList<>(); - - Iterator> iterator = new HoodieBaseBloomIndexCheckFunction(hoodieTable, config).apply(fileComparisons.iterator()); - while (iterator.hasNext()) { - keyLookupResults.addAll(iterator.next()); - } - - Map hoodieRecordLocationMap = new HashMap<>(); - - keyLookupResults = keyLookupResults.stream().filter(lr -> lr.getMatchingRecordKeys().size() > 0).collect(toList()); - keyLookupResults.forEach(lookupResult -> { - lookupResult.getMatchingRecordKeys().forEach(r -> { - hoodieRecordLocationMap.put(new HoodieKey(r, lookupResult.getPartitionPath()), new HoodieRecordLocation(lookupResult.getBaseInstantTime(), lookupResult.getFileId())); - }); - }); - - return hoodieRecordLocationMap; - } - - - /** - * Tag the back to the original HoodieRecord List. - */ - protected List> tagLocationBacktoRecords( - Map keyFilenamePair, List> records) { - Map> keyRecordPairMap = new HashMap<>(); - records.forEach(r -> keyRecordPairMap.put(r.getKey(), r)); - // Here as the record might have more data than rowKey (some rowKeys' fileId is null), - // so we do left outer join. - List, HoodieRecordLocation>> newList = new ArrayList<>(); - keyRecordPairMap.keySet().forEach(k -> { - if (keyFilenamePair.containsKey(k)) { - newList.add(Pair.of(keyRecordPairMap.get(k), keyFilenamePair.get(k))); - } else { - newList.add(Pair.of(keyRecordPairMap.get(k), null)); - } - }); - List> res = Lists.newArrayList(); - for (Pair, HoodieRecordLocation> v : newList) { - res.add(HoodieIndexUtils.getTaggedRecord(v.getLeft(), Option.ofNullable(v.getRight()))); - } - return res; - } - - @Override - public List updateLocation(List writeStatusList, HoodieEngineContext context, - HoodieTable>, List, List> hoodieTable) { - return writeStatusList; - } -} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bloom/HoodieBloomIndex.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bloom/HoodieBloomIndex.java new file mode 100644 index 000000000..a223d695c --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bloom/HoodieBloomIndex.java @@ -0,0 +1,238 @@ +/* + * 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.index.bloom; + +import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.common.config.HoodieConfig; +import org.apache.hudi.common.data.HoodieData; +import org.apache.hudi.common.data.HoodiePairData; +import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.model.HoodieKey; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecordLocation; +import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.collection.ImmutablePair; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.config.HoodieIndexConfig; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.exception.MetadataNotFoundException; +import org.apache.hudi.index.HoodieIndex; +import org.apache.hudi.index.HoodieIndexUtils; +import org.apache.hudi.io.HoodieRangeInfoHandle; +import org.apache.hudi.table.HoodieTable; + +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +import static java.util.stream.Collectors.groupingBy; +import static java.util.stream.Collectors.mapping; +import static java.util.stream.Collectors.toList; +import static org.apache.hudi.index.HoodieIndexUtils.getLatestBaseFilesForAllPartitions; + +/** + * Indexing mechanism based on bloom filter. Each parquet file includes its row_key bloom filter in its metadata. + */ +public class HoodieBloomIndex> + extends HoodieIndex { + private static final Logger LOG = LogManager.getLogger(HoodieBloomIndex.class); + + private final BaseHoodieBloomIndexHelper bloomIndexHelper; + + public HoodieBloomIndex(HoodieWriteConfig config, BaseHoodieBloomIndexHelper bloomIndexHelper) { + super(config); + this.bloomIndexHelper = bloomIndexHelper; + } + + @Override + public HoodieData> tagLocation( + HoodieData> records, HoodieEngineContext context, + HoodieTable hoodieTable) { + // Step 0: cache the input records if needed + if (config.getBloomIndexUseCaching()) { + records.persist(new HoodieConfig(config.getProps()) + .getString(HoodieIndexConfig.BLOOM_INDEX_INPUT_STORAGE_LEVEL_VALUE)); + } + + // Step 1: Extract out thinner pairs of (partitionPath, recordKey) + HoodiePairData partitionRecordKeyPairs = records.mapToPair( + record -> new ImmutablePair<>(record.getPartitionPath(), record.getRecordKey())); + + // Step 2: Lookup indexes for all the partition/recordkey pair + HoodiePairData keyFilenamePairs = + lookupIndex(partitionRecordKeyPairs, context, hoodieTable); + + // Cache the result, for subsequent stages. + if (config.getBloomIndexUseCaching()) { + keyFilenamePairs.persist("MEMORY_AND_DISK_SER"); + } + if (LOG.isDebugEnabled()) { + long totalTaggedRecords = keyFilenamePairs.count(); + LOG.debug("Number of update records (ones tagged with a fileID): " + totalTaggedRecords); + } + + // Step 3: Tag the incoming records, as inserts or updates, by joining with existing record keys + HoodieData> taggedRecords = tagLocationBacktoRecords(keyFilenamePairs, records); + + if (config.getBloomIndexUseCaching()) { + records.unpersist(); + keyFilenamePairs.unpersist(); + } + + return taggedRecords; + } + + /** + * Lookup the location for each record key and return the pair for all record keys already + * present and drop the record keys if not present. + */ + private HoodiePairData lookupIndex( + HoodiePairData partitionRecordKeyPairs, final HoodieEngineContext context, + final HoodieTable hoodieTable) { + // Obtain records per partition, in the incoming records + Map recordsPerPartition = partitionRecordKeyPairs.countByKey(); + List affectedPartitionPathList = new ArrayList<>(recordsPerPartition.keySet()); + + // Step 2: Load all involved files as pairs + List> fileInfoList = + loadInvolvedFiles(affectedPartitionPathList, context, hoodieTable); + final Map> partitionToFileInfo = + fileInfoList.stream().collect(groupingBy(Pair::getLeft, mapping(Pair::getRight, toList()))); + + // Step 3: Obtain a HoodieData, for each incoming record, that already exists, with the file id, + // that contains it. + HoodieData> fileComparisonPairs = + explodeRecordsWithFileComparisons(partitionToFileInfo, partitionRecordKeyPairs); + + return bloomIndexHelper.findMatchingFilesForRecordKeys(config, context, hoodieTable, + partitionRecordKeyPairs, fileComparisonPairs, partitionToFileInfo, recordsPerPartition); + } + + /** + * Load all involved files as pair List. + */ + List> loadInvolvedFiles( + List partitions, final HoodieEngineContext context, final HoodieTable hoodieTable) { + // Obtain the latest data files from all the partitions. + List> partitionPathFileIDList = getLatestBaseFilesForAllPartitions(partitions, context, hoodieTable).stream() + .map(pair -> Pair.of(pair.getKey(), pair.getValue().getFileId())) + .collect(toList()); + + if (config.getBloomIndexPruneByRanges()) { + // also obtain file ranges, if range pruning is enabled + context.setJobStatus(this.getClass().getName(), "Obtain key ranges for file slices (range pruning=on)"); + return context.map(partitionPathFileIDList, pf -> { + try { + HoodieRangeInfoHandle rangeInfoHandle = new HoodieRangeInfoHandle(config, hoodieTable, pf); + String[] minMaxKeys = rangeInfoHandle.getMinMaxKeys(); + return Pair.of(pf.getKey(), new BloomIndexFileInfo(pf.getValue(), minMaxKeys[0], minMaxKeys[1])); + } catch (MetadataNotFoundException me) { + LOG.warn("Unable to find range metadata in file :" + pf); + return Pair.of(pf.getKey(), new BloomIndexFileInfo(pf.getValue())); + } + }, Math.max(partitionPathFileIDList.size(), 1)); + } else { + return partitionPathFileIDList.stream() + .map(pf -> Pair.of(pf.getKey(), new BloomIndexFileInfo(pf.getValue()))).collect(toList()); + } + } + + @Override + public boolean rollbackCommit(String instantTime) { + // Nope, don't need to do anything. + return true; + } + + /** + * This is not global, since we depend on the partitionPath to do the lookup. + */ + @Override + public boolean isGlobal() { + return false; + } + + /** + * No indexes into log files yet. + */ + @Override + public boolean canIndexLogFiles() { + return false; + } + + /** + * Bloom filters are stored, into the same data files. + */ + @Override + public boolean isImplicitWithStorage() { + return true; + } + + /** + * For each incoming record, produce N output records, 1 each for each file against which the record's key needs to be + * checked. For tables, where the keys have a definite insert order (e.g: timestamp as prefix), the number of files + * to be compared gets cut down a lot from range pruning. + *

+ * Sub-partition to ensure the records can be looked up against files & also prune file<=>record comparisons based on + * recordKey ranges in the index info. + */ + HoodieData> explodeRecordsWithFileComparisons( + final Map> partitionToFileIndexInfo, + HoodiePairData partitionRecordKeyPairs) { + IndexFileFilter indexFileFilter = + config.useBloomIndexTreebasedFilter() ? new IntervalTreeBasedIndexFileFilter(partitionToFileIndexInfo) + : new ListBasedIndexFileFilter(partitionToFileIndexInfo); + + return partitionRecordKeyPairs.map(partitionRecordKeyPair -> { + String recordKey = partitionRecordKeyPair.getRight(); + String partitionPath = partitionRecordKeyPair.getLeft(); + + return indexFileFilter.getMatchingFilesAndPartition(partitionPath, recordKey).stream() + .map(partitionFileIdPair -> new ImmutablePair<>(partitionFileIdPair.getRight(), + new HoodieKey(recordKey, partitionPath))) + .collect(Collectors.toList()); + }).flatMap(List::iterator); + } + + /** + * Tag the back to the original HoodieRecord List. + */ + protected HoodieData> tagLocationBacktoRecords( + HoodiePairData keyFilenamePair, + HoodieData> records) { + HoodiePairData> keyRecordPairs = + records.mapToPair(record -> new ImmutablePair<>(record.getKey(), record)); + // Here as the records might have more data than keyFilenamePairs (some row keys' fileId is null), + // so we do left outer join. + return keyRecordPairs.leftOuterJoin(keyFilenamePair).values() + .map(v -> HoodieIndexUtils.getTaggedRecord(v.getLeft(), Option.ofNullable(v.getRight().orElse(null)))); + } + + @Override + public HoodieData updateLocation( + HoodieData writeStatusData, HoodieEngineContext context, + HoodieTable hoodieTable) { + return writeStatusData; + } +} diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/bloom/SparkHoodieGlobalBloomIndex.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bloom/HoodieGlobalBloomIndex.java similarity index 57% rename from hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/bloom/SparkHoodieGlobalBloomIndex.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bloom/HoodieGlobalBloomIndex.java index 21b9d4060..39fa72a32 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/bloom/SparkHoodieGlobalBloomIndex.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bloom/HoodieGlobalBloomIndex.java @@ -7,17 +7,20 @@ * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. */ package org.apache.hudi.index.bloom; +import org.apache.hudi.common.data.HoodieData; +import org.apache.hudi.common.data.HoodiePairData; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.EmptyHoodieRecordPayload; @@ -27,38 +30,33 @@ import org.apache.hudi.common.model.HoodieRecordLocation; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.collection.ImmutablePair; +import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.index.HoodieIndexUtils; import org.apache.hudi.table.HoodieTable; -import org.apache.spark.api.java.JavaPairRDD; -import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.Optional; - import java.util.Arrays; import java.util.Collections; import java.util.List; import java.util.Map; import java.util.stream.Collectors; -import scala.Tuple2; - /** - * This filter will only work with hoodie table since it will only load partitions with .hoodie_partition_metadata - * file in it. + * This filter will only work with hoodie table since it will only load partitions + * with .hoodie_partition_metadata file in it. */ -public class SparkHoodieGlobalBloomIndex extends SparkHoodieBloomIndex { - - public SparkHoodieGlobalBloomIndex(HoodieWriteConfig config) { - super(config); +public class HoodieGlobalBloomIndex> extends HoodieBloomIndex { + public HoodieGlobalBloomIndex(HoodieWriteConfig config, BaseHoodieBloomIndexHelper bloomIndexHelper) { + super(config, bloomIndexHelper); } /** - * Load all involved files as pair RDD from all partitions in the table. + * Load all involved files as pairs from all partitions in the table. */ @Override - List> loadInvolvedFiles(List partitions, final HoodieEngineContext context, - final HoodieTable hoodieTable) { + List> loadInvolvedFiles(List partitions, final HoodieEngineContext context, + final HoodieTable hoodieTable) { HoodieTableMetaClient metaClient = hoodieTable.getMetaClient(); List allPartitionPaths = FSUtils.getAllPartitionPaths(context, config.getMetadataConfig(), metaClient.getBasePath()); return super.loadInvolvedFiles(allPartitionPaths, context, hoodieTable); @@ -70,25 +68,25 @@ public class SparkHoodieGlobalBloomIndex extends * to be compared gets cut down a lot from range pruning. *

* Sub-partition to ensure the records can be looked up against files & also prune file<=>record comparisons based on - * recordKey ranges in the index info. the partition path of the incoming record (partitionRecordKeyPairRDD._2()) will + * recordKey ranges in the index info. the partition path of the incoming record (partitionRecordKeyPairs._2()) will * be ignored since the search scope should be bigger than that */ @Override - JavaRDD> explodeRecordRDDWithFileComparisons( + HoodieData> explodeRecordsWithFileComparisons( final Map> partitionToFileIndexInfo, - JavaPairRDD partitionRecordKeyPairRDD) { + HoodiePairData partitionRecordKeyPairs) { IndexFileFilter indexFileFilter = config.useBloomIndexTreebasedFilter() ? new IntervalTreeBasedGlobalIndexFileFilter(partitionToFileIndexInfo) : new ListBasedGlobalIndexFileFilter(partitionToFileIndexInfo); - return partitionRecordKeyPairRDD.map(partitionRecordKeyPair -> { - String recordKey = partitionRecordKeyPair._2(); - String partitionPath = partitionRecordKeyPair._1(); + return partitionRecordKeyPairs.map(partitionRecordKeyPair -> { + String recordKey = partitionRecordKeyPair.getRight(); + String partitionPath = partitionRecordKeyPair.getLeft(); return indexFileFilter.getMatchingFilesAndPartition(partitionPath, recordKey).stream() - .map(partitionFileIdPair -> new Tuple2<>(partitionFileIdPair.getRight(), + .map(partitionFileIdPair -> new ImmutablePair<>(partitionFileIdPair.getRight(), new HoodieKey(recordKey, partitionFileIdPair.getLeft()))) .collect(Collectors.toList()); }).flatMap(List::iterator); @@ -98,27 +96,29 @@ public class SparkHoodieGlobalBloomIndex extends * Tagging for global index should only consider the record key. */ @Override - protected JavaRDD> tagLocationBacktoRecords( - JavaPairRDD keyLocationPairRDD, JavaRDD> recordRDD) { + protected HoodieData> tagLocationBacktoRecords( + HoodiePairData keyLocationPairs, + HoodieData> records) { - JavaPairRDD> incomingRowKeyRecordPairRDD = - recordRDD.mapToPair(record -> new Tuple2<>(record.getRecordKey(), record)); + HoodiePairData> incomingRowKeyRecordPairs = + records.mapToPair(record -> new ImmutablePair<>(record.getRecordKey(), record)); - JavaPairRDD> existingRecordKeyToRecordLocationHoodieKeyMap = - keyLocationPairRDD.mapToPair(p -> new Tuple2<>(p._1.getRecordKey(), new Tuple2<>(p._2, p._1))); + HoodiePairData> existingRecordKeyToRecordLocationHoodieKeyMap = + keyLocationPairs.mapToPair(p -> new ImmutablePair<>( + p.getKey().getRecordKey(), new ImmutablePair<>(p.getValue(), p.getKey()))); - // Here as the recordRDD might have more data than rowKeyRDD (some rowKeys' fileId is null), so we do left outer join. - return incomingRowKeyRecordPairRDD.leftOuterJoin(existingRecordKeyToRecordLocationHoodieKeyMap).values().flatMap(record -> { - final HoodieRecord hoodieRecord = record._1; - final Optional> recordLocationHoodieKeyPair = record._2; + // Here as the records might have more data than rowKeys (some rowKeys' fileId is null), so we do left outer join. + return incomingRowKeyRecordPairs.leftOuterJoin(existingRecordKeyToRecordLocationHoodieKeyMap).values().flatMap(record -> { + final HoodieRecord hoodieRecord = record.getLeft(); + final Option> recordLocationHoodieKeyPair = record.getRight(); if (recordLocationHoodieKeyPair.isPresent()) { // Record key matched to file if (config.getBloomIndexUpdatePartitionPath() - && !recordLocationHoodieKeyPair.get()._2.getPartitionPath().equals(hoodieRecord.getPartitionPath())) { + && !recordLocationHoodieKeyPair.get().getRight().getPartitionPath().equals(hoodieRecord.getPartitionPath())) { // Create an empty record to delete the record in the old partition - HoodieRecord deleteRecord = new HoodieRecord(recordLocationHoodieKeyPair.get()._2, + HoodieRecord deleteRecord = new HoodieRecord(recordLocationHoodieKeyPair.get().getRight(), new EmptyHoodieRecordPayload()); - deleteRecord.setCurrentLocation(recordLocationHoodieKeyPair.get()._1()); + deleteRecord.setCurrentLocation(recordLocationHoodieKeyPair.get().getLeft()); deleteRecord.seal(); // Tag the incoming record for inserting to the new partition HoodieRecord insertRecord = HoodieIndexUtils.getTaggedRecord(hoodieRecord, Option.empty()); @@ -127,8 +127,8 @@ public class SparkHoodieGlobalBloomIndex extends // Ignore the incoming record's partition, regardless of whether it differs from its old partition or not. // When it differs, the record will still be updated at its old partition. return Collections.singletonList( - (HoodieRecord) HoodieIndexUtils.getTaggedRecord(new HoodieRecord<>(recordLocationHoodieKeyPair.get()._2, hoodieRecord.getData()), - Option.ofNullable(recordLocationHoodieKeyPair.get()._1))).iterator(); + (HoodieRecord) HoodieIndexUtils.getTaggedRecord(new HoodieRecord<>(recordLocationHoodieKeyPair.get().getRight(), hoodieRecord.getData()), + Option.ofNullable(recordLocationHoodieKeyPair.get().getLeft()))).iterator(); } } else { return Collections.singletonList((HoodieRecord) HoodieIndexUtils.getTaggedRecord(hoodieRecord, Option.empty())).iterator(); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bloom/ListBasedHoodieBloomIndexHelper.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bloom/ListBasedHoodieBloomIndexHelper.java new file mode 100644 index 000000000..74191df52 --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bloom/ListBasedHoodieBloomIndexHelper.java @@ -0,0 +1,87 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.hudi.index.bloom; + +import org.apache.hudi.common.data.HoodieData; +import org.apache.hudi.common.data.HoodieList; +import org.apache.hudi.common.data.HoodiePairData; +import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.model.HoodieKey; +import org.apache.hudi.common.model.HoodieRecordLocation; +import org.apache.hudi.common.util.collection.ImmutablePair; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.io.HoodieKeyLookupHandle; +import org.apache.hudi.table.HoodieTable; + +import java.util.ArrayList; +import java.util.Comparator; +import java.util.Iterator; +import java.util.List; +import java.util.Map; + +import static java.util.stream.Collectors.toList; + +/** + * Helper for {@link HoodieBloomIndex} containing Java {@link List}-based logic. + */ +public class ListBasedHoodieBloomIndexHelper extends BaseHoodieBloomIndexHelper { + + private static final ListBasedHoodieBloomIndexHelper SINGLETON_INSTANCE = new ListBasedHoodieBloomIndexHelper(); + + protected ListBasedHoodieBloomIndexHelper() { + } + + public static ListBasedHoodieBloomIndexHelper getInstance() { + return SINGLETON_INSTANCE; + } + + @Override + public HoodiePairData findMatchingFilesForRecordKeys( + HoodieWriteConfig config, HoodieEngineContext context, HoodieTable hoodieTable, + HoodiePairData partitionRecordKeyPairs, + HoodieData> fileComparisonPairs, + Map> partitionToFileInfo, Map recordsPerPartition) { + List> fileComparisonPairList = + HoodieList.getList(fileComparisonPairs).stream() + .sorted(Comparator.comparing(ImmutablePair::getLeft)).collect(toList()); + + List keyLookupResults = new ArrayList<>(); + + Iterator> iterator = new HoodieBaseBloomIndexCheckFunction( + hoodieTable, config).apply(fileComparisonPairList.iterator()); + while (iterator.hasNext()) { + keyLookupResults.addAll(iterator.next()); + } + + keyLookupResults = keyLookupResults.stream().filter( + lr -> lr.getMatchingRecordKeys().size() > 0).collect(toList()); + return context.parallelize(keyLookupResults).flatMap(lookupResult -> + lookupResult.getMatchingRecordKeys().stream() + .map(recordKey -> new ImmutablePair<>(lookupResult, recordKey)).iterator() + ).mapToPair(pair -> { + HoodieKeyLookupHandle.KeyLookupResult lookupResult = pair.getLeft(); + String recordKey = pair.getRight(); + return new ImmutablePair<>( + new HoodieKey(recordKey, lookupResult.getPartitionPath()), + new HoodieRecordLocation(lookupResult.getBaseInstantTime(), lookupResult.getFileId())); + }); + } +} diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/index/JavaInMemoryHashIndex.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/inmemory/HoodieInMemoryHashIndex.java similarity index 59% rename from hudi-client/hudi-java-client/src/main/java/org/apache/hudi/index/JavaInMemoryHashIndex.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/inmemory/HoodieInMemoryHashIndex.java index 8a7197759..bec675c10 100644 --- a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/index/JavaInMemoryHashIndex.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/inmemory/HoodieInMemoryHashIndex.java @@ -7,18 +7,20 @@ * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. */ -package org.apache.hudi.index; +package org.apache.hudi.index.inmemory; import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; @@ -26,27 +28,27 @@ import org.apache.hudi.common.model.HoodieRecordLocation; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.index.HoodieIndex; import org.apache.hudi.table.HoodieTable; import java.util.ArrayList; import java.util.List; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; -import java.util.stream.Collectors; /** * Hoodie Index implementation backed by an in-memory Hash map. *

* ONLY USE FOR LOCAL TESTING */ -@SuppressWarnings("checkstyle:LineLength") -public class JavaInMemoryHashIndex extends JavaHoodieIndex { +public class HoodieInMemoryHashIndex> + extends HoodieIndex { private static ConcurrentMap recordLocationMap; - public JavaInMemoryHashIndex(HoodieWriteConfig config) { + public HoodieInMemoryHashIndex(HoodieWriteConfig config) { super(config); - synchronized (JavaInMemoryHashIndex.class) { + synchronized (HoodieInMemoryHashIndex.class) { if (recordLocationMap == null) { recordLocationMap = new ConcurrentHashMap<>(); } @@ -54,25 +56,29 @@ public class JavaInMemoryHashIndex extends JavaHo } @Override - public List> tagLocation(List> records, HoodieEngineContext context, - HoodieTable>, List, List> hoodieTable) { - List> taggedRecords = new ArrayList<>(); - records.stream().forEach(record -> { - if (recordLocationMap.containsKey(record.getKey())) { - record.unseal(); - record.setCurrentLocation(recordLocationMap.get(record.getKey())); - record.seal(); + public HoodieData> tagLocation( + HoodieData> records, HoodieEngineContext context, + HoodieTable hoodieTable) { + return records.mapPartitions(hoodieRecordIterator -> { + List> taggedRecords = new ArrayList<>(); + while (hoodieRecordIterator.hasNext()) { + HoodieRecord record = hoodieRecordIterator.next(); + if (recordLocationMap.containsKey(record.getKey())) { + record.unseal(); + record.setCurrentLocation(recordLocationMap.get(record.getKey())); + record.seal(); + } + taggedRecords.add(record); } - taggedRecords.add(record); - }); - return taggedRecords; + return taggedRecords.iterator(); + }, true); } @Override - public List updateLocation(List writeStatusList, - HoodieEngineContext context, - HoodieTable>, List, List> hoodieTable) { - return writeStatusList.stream().map(writeStatus -> { + public HoodieData updateLocation( + HoodieData writeStatuses, HoodieEngineContext context, + HoodieTable hoodieTable) { + return writeStatuses.map(writeStatus -> { for (HoodieRecord record : writeStatus.getWrittenRecords()) { if (!writeStatus.isErrored(record.getKey())) { HoodieKey key = record.getKey(); @@ -86,7 +92,7 @@ public class JavaInMemoryHashIndex extends JavaHo } } return writeStatus; - }).collect(Collectors.toList()); + }); } @Override diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/simple/SparkHoodieGlobalSimpleIndex.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/simple/HoodieGlobalSimpleIndex.java similarity index 58% rename from hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/simple/SparkHoodieGlobalSimpleIndex.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/simple/HoodieGlobalSimpleIndex.java index 63e13c463..8935fcb02 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/simple/SparkHoodieGlobalSimpleIndex.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/simple/HoodieGlobalSimpleIndex.java @@ -7,18 +7,20 @@ * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. */ package org.apache.hudi.index.simple; -import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.common.data.HoodieData; +import org.apache.hudi.common.data.HoodiePairData; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.EmptyHoodieRecordPayload; @@ -29,13 +31,12 @@ import org.apache.hudi.common.model.HoodieRecordLocation; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.collection.ImmutablePair; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.index.HoodieIndexUtils; +import org.apache.hudi.keygen.BaseKeyGenerator; import org.apache.hudi.table.HoodieTable; -import org.apache.spark.api.java.JavaPairRDD; -import org.apache.spark.api.java.JavaRDD; -import scala.Tuple2; import java.util.Arrays; import java.util.Collections; @@ -49,57 +50,57 @@ import static org.apache.hudi.index.HoodieIndexUtils.getLatestBaseFilesForAllPar * * @param */ -@SuppressWarnings("checkstyle:LineLength") -public class SparkHoodieGlobalSimpleIndex extends SparkHoodieSimpleIndex { - - public SparkHoodieGlobalSimpleIndex(HoodieWriteConfig config) { - super(config); +public class HoodieGlobalSimpleIndex> extends HoodieSimpleIndex { + public HoodieGlobalSimpleIndex(HoodieWriteConfig config, Option keyGeneratorOpt) { + super(config, keyGeneratorOpt); } @Override - public JavaRDD> tagLocation(JavaRDD> recordRDD, HoodieEngineContext context, - HoodieTable>, JavaRDD, JavaRDD> hoodieTable) { - return tagLocationInternal(recordRDD, context, hoodieTable); + public HoodieData> tagLocation( + HoodieData> records, HoodieEngineContext context, + HoodieTable hoodieTable) { + return tagLocationInternal(records, context, hoodieTable); } /** * Tags records location for incoming records. * - * @param inputRecordRDD {@link JavaRDD} of incoming records - * @param context instance of {@link HoodieEngineContext} to use - * @param hoodieTable instance of {@link HoodieTable} to use - * @return {@link JavaRDD} of records with record locations set + * @param inputRecords {@link HoodieData} of incoming records + * @param context instance of {@link HoodieEngineContext} to use + * @param hoodieTable instance of {@link HoodieTable} to use + * @return {@link HoodieData} of records with record locations set */ @Override - protected JavaRDD> tagLocationInternal(JavaRDD> inputRecordRDD, HoodieEngineContext context, - HoodieTable>, JavaRDD, JavaRDD> hoodieTable) { + protected HoodieData> tagLocationInternal( + HoodieData> inputRecords, HoodieEngineContext context, + HoodieTable hoodieTable) { - JavaPairRDD> keyedInputRecordRDD = inputRecordRDD.mapToPair(entry -> new Tuple2<>(entry.getRecordKey(), entry)); - JavaPairRDD allRecordLocationsInTable = fetchAllRecordLocations(context, hoodieTable, - config.getGlobalSimpleIndexParallelism()); - return getTaggedRecords(keyedInputRecordRDD, allRecordLocationsInTable); + HoodiePairData> keyedInputRecords = + inputRecords.mapToPair(entry -> new ImmutablePair<>(entry.getRecordKey(), entry)); + HoodiePairData allRecordLocationsInTable = + fetchAllRecordLocations(context, hoodieTable, config.getGlobalSimpleIndexParallelism()); + return getTaggedRecords(keyedInputRecords, allRecordLocationsInTable); } /** * Fetch record locations for passed in {@link HoodieKey}s. * - * @param context instance of {@link HoodieEngineContext} to use + * @param context instance of {@link HoodieEngineContext} to use * @param hoodieTable instance of {@link HoodieTable} of interest * @param parallelism parallelism to use - * @return {@link JavaPairRDD} of {@link HoodieKey} and {@link HoodieRecordLocation} + * @return {@link HoodiePairData} of {@link HoodieKey} and {@link HoodieRecordLocation} */ - protected JavaPairRDD fetchAllRecordLocations(HoodieEngineContext context, - HoodieTable>, JavaRDD, JavaRDD> hoodieTable, - int parallelism) { + protected HoodiePairData fetchAllRecordLocations( + HoodieEngineContext context, HoodieTable hoodieTable, int parallelism) { List> latestBaseFiles = getAllBaseFilesInTable(context, hoodieTable); return fetchRecordLocations(context, hoodieTable, parallelism, latestBaseFiles); } /** - * Load all files for all partitions as pair RDD. + * Load all files for all partitions as pair data. */ - protected List> getAllBaseFilesInTable(final HoodieEngineContext context, - final HoodieTable>, JavaRDD, JavaRDD> hoodieTable) { + protected List> getAllBaseFilesInTable( + final HoodieEngineContext context, final HoodieTable hoodieTable) { HoodieTableMetaClient metaClient = hoodieTable.getMetaClient(); List allPartitionPaths = FSUtils.getAllPartitionPaths(context, config.getMetadataConfig(), metaClient.getBasePath()); // Obtain the latest data files from all the partitions. @@ -111,16 +112,20 @@ public class SparkHoodieGlobalSimpleIndex extends * * @param incomingRecords incoming {@link HoodieRecord}s * @param existingRecords existing records with {@link HoodieRecordLocation}s - * @return {@link JavaRDD} of {@link HoodieRecord}s with tagged {@link HoodieRecordLocation}s + * @return {@link HoodieData} of {@link HoodieRecord}s with tagged {@link HoodieRecordLocation}s */ - private JavaRDD> getTaggedRecords(JavaPairRDD> incomingRecords, JavaPairRDD existingRecords) { - JavaPairRDD> existingRecordByRecordKey = existingRecords - .mapToPair(entry -> new Tuple2<>(entry._1.getRecordKey(), Pair.of(entry._1.getPartitionPath(), entry._2))); + private HoodieData> getTaggedRecords( + HoodiePairData> incomingRecords, + HoodiePairData existingRecords) { + HoodiePairData> existingRecordByRecordKey = + existingRecords.mapToPair( + entry -> new ImmutablePair<>(entry.getLeft().getRecordKey(), + Pair.of(entry.getLeft().getPartitionPath(), entry.getRight()))); return incomingRecords.leftOuterJoin(existingRecordByRecordKey).values() .flatMap(entry -> { - HoodieRecord inputRecord = entry._1; - Option> partitionPathLocationPair = Option.ofNullable(entry._2.orNull()); + HoodieRecord inputRecord = entry.getLeft(); + Option> partitionPathLocationPair = Option.ofNullable(entry.getRight().orElse(null)); List> taggedRecords; if (partitionPathLocationPair.isPresent()) { diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/simple/HoodieSimpleIndex.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/simple/HoodieSimpleIndex.java new file mode 100644 index 000000000..dfefe5ada --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/simple/HoodieSimpleIndex.java @@ -0,0 +1,161 @@ +/* + * 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.index.simple; + +import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.common.config.HoodieConfig; +import org.apache.hudi.common.data.HoodieData; +import org.apache.hudi.common.data.HoodiePairData; +import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.model.HoodieBaseFile; +import org.apache.hudi.common.model.HoodieKey; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecordLocation; +import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.collection.ImmutablePair; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.config.HoodieIndexConfig; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.index.HoodieIndex; +import org.apache.hudi.index.HoodieIndexUtils; +import org.apache.hudi.io.HoodieKeyLocationFetchHandle; +import org.apache.hudi.keygen.BaseKeyGenerator; +import org.apache.hudi.table.HoodieTable; + +import java.util.List; + +import static org.apache.hudi.index.HoodieIndexUtils.getLatestBaseFilesForAllPartitions; + +/** + * A simple index which reads interested fields(record key and partition path) from base files and + * joins with incoming records to find the tagged location. + * + * @param type of {@link HoodieRecordPayload} + */ +public class HoodieSimpleIndex> + extends HoodieIndex { + + private final Option keyGeneratorOpt; + + public HoodieSimpleIndex(HoodieWriteConfig config, Option keyGeneratorOpt) { + super(config); + this.keyGeneratorOpt = keyGeneratorOpt; + } + + @Override + public HoodieData updateLocation( + HoodieData writeStatuses, HoodieEngineContext context, + HoodieTable hoodieTable) { + return writeStatuses; + } + + @Override + public boolean rollbackCommit(String commitTime) { + return true; + } + + @Override + public boolean isGlobal() { + return false; + } + + @Override + public boolean canIndexLogFiles() { + return false; + } + + @Override + public boolean isImplicitWithStorage() { + return true; + } + + @Override + public HoodieData> tagLocation( + HoodieData> records, HoodieEngineContext context, + HoodieTable hoodieTable) { + return tagLocationInternal(records, context, hoodieTable); + } + + /** + * Tags records location for incoming records. + * + * @param inputRecords {@link HoodieData} of incoming records + * @param context instance of {@link HoodieEngineContext} to use + * @param hoodieTable instance of {@link HoodieTable} to use + * @return {@link HoodieData} of records with record locations set + */ + protected HoodieData> tagLocationInternal( + HoodieData> inputRecords, HoodieEngineContext context, + HoodieTable hoodieTable) { + if (config.getSimpleIndexUseCaching()) { + inputRecords.persist(new HoodieConfig(config.getProps()) + .getString(HoodieIndexConfig.SIMPLE_INDEX_INPUT_STORAGE_LEVEL_VALUE)); + } + + HoodiePairData> keyedInputRecords = + inputRecords.mapToPair(record -> new ImmutablePair<>(record.getKey(), record)); + HoodiePairData existingLocationsOnTable = + fetchRecordLocationsForAffectedPartitions(keyedInputRecords.keys(), context, hoodieTable, + config.getSimpleIndexParallelism()); + + HoodieData> taggedRecords = + keyedInputRecords.leftOuterJoin(existingLocationsOnTable).map(entry -> { + final HoodieRecord untaggedRecord = entry.getRight().getLeft(); + final Option location = Option.ofNullable(entry.getRight().getRight().orElse(null)); + return HoodieIndexUtils.getTaggedRecord(untaggedRecord, location); + }); + + if (config.getSimpleIndexUseCaching()) { + inputRecords.unpersist(); + } + return taggedRecords; + } + + /** + * Fetch record locations for passed in {@link HoodieKey}s. + * + * @param hoodieKeys {@link HoodieData} of {@link HoodieKey}s for which locations are fetched + * @param context instance of {@link HoodieEngineContext} to use + * @param hoodieTable instance of {@link HoodieTable} of interest + * @param parallelism parallelism to use + * @return {@link HoodiePairData} of {@link HoodieKey} and {@link HoodieRecordLocation} + */ + protected HoodiePairData fetchRecordLocationsForAffectedPartitions( + HoodieData hoodieKeys, HoodieEngineContext context, HoodieTable hoodieTable, + int parallelism) { + List affectedPartitionPathList = + hoodieKeys.map(HoodieKey::getPartitionPath).distinct().collectAsList(); + List> latestBaseFiles = + getLatestBaseFilesForAllPartitions(affectedPartitionPathList, context, hoodieTable); + return fetchRecordLocations(context, hoodieTable, parallelism, latestBaseFiles); + } + + protected HoodiePairData fetchRecordLocations( + HoodieEngineContext context, HoodieTable hoodieTable, int parallelism, + List> baseFiles) { + int fetchParallelism = Math.max(1, Math.max(baseFiles.size(), parallelism)); + + return context.parallelize(baseFiles, fetchParallelism) + .flatMap(partitionPathBaseFile -> new HoodieKeyLocationFetchHandle(config, hoodieTable, partitionPathBaseFile, keyGeneratorOpt) + .locations().iterator()) + .mapToPair(e -> (Pair) e); + } +} 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 2efd1b149..6de40a7b5 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 @@ -101,7 +101,7 @@ public abstract class HoodieTable implem protected final HoodieWriteConfig config; protected final HoodieTableMetaClient metaClient; - protected final HoodieIndex index; + protected final HoodieIndex index; private SerializableConfiguration hadoopConfiguration; protected final TaskContextSupplier taskContextSupplier; private final HoodieTableMetadata metadata; @@ -125,7 +125,7 @@ public abstract class HoodieTable implem this.taskContextSupplier = context.getTaskContextSupplier(); } - protected abstract HoodieIndex getIndex(HoodieWriteConfig config, HoodieEngineContext context); + protected abstract HoodieIndex getIndex(HoodieWriteConfig config, HoodieEngineContext context); private synchronized FileSystemViewManager getViewManager() { if (null == viewManager) { @@ -347,7 +347,7 @@ public abstract class HoodieTable implem /** * Return the index. */ - public HoodieIndex getIndex() { + public HoodieIndex getIndex() { return index; } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/AbstractWriteHelper.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/AbstractWriteHelper.java index 455952ae5..0d9cdc0aa 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/AbstractWriteHelper.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/AbstractWriteHelper.java @@ -63,11 +63,8 @@ public abstract class AbstractWriteHelper table) { - // perform index loop up to get existing location of records - return table.getIndex().tagLocation(dedupedRecords, context, table); - } + protected abstract I tag( + I dedupedRecords, HoodieEngineContext context, HoodieTable table); public I combineOnCondition( boolean condition, I records, int parallelism, HoodieTable table) { @@ -87,5 +84,5 @@ public abstract class AbstractWriteHelper index, int parallelism); + I records, HoodieIndex index, int parallelism); } 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 7c60a4ad9..c73de656a 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 @@ -42,7 +42,7 @@ import org.apache.hudi.exception.HoodieCommitException; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.exception.HoodieNotSupportedException; -import org.apache.hudi.index.FlinkHoodieIndex; +import org.apache.hudi.index.FlinkHoodieIndexFactory; import org.apache.hudi.index.HoodieIndex; import org.apache.hudi.io.FlinkAppendHandle; import org.apache.hudi.io.FlinkConcatAndReplaceHandle; @@ -104,8 +104,8 @@ public class HoodieFlinkWriteClient extends * Complete changes performed at the given instantTime marker with specified action. */ @Override - protected HoodieIndex>, List, List> createIndex(HoodieWriteConfig writeConfig) { - return FlinkHoodieIndex.createIndex((HoodieFlinkEngineContext) context, config); + protected HoodieIndex createIndex(HoodieWriteConfig writeConfig) { + return FlinkHoodieIndexFactory.createIndex((HoodieFlinkEngineContext) context, config); } @Override @@ -125,7 +125,8 @@ public class HoodieFlinkWriteClient extends // Create a Hoodie table which encapsulated the commits and files visible HoodieFlinkTable table = getHoodieTable(); Timer.Context indexTimer = metrics.getIndexCtx(); - List> recordsWithLocation = getIndex().tagLocation(hoodieRecords, context, table); + List> recordsWithLocation = HoodieList.getList( + getIndex().tagLocation(HoodieList.of(hoodieRecords), context, table)); metrics.updateIndexMetrics(LOOKUP_STR, metrics.getDurationInMs(indexTimer == null ? 0L : indexTimer.stop())); return recordsWithLocation.stream().filter(v1 -> !v1.isCurrentLocationKnown()).collect(Collectors.toList()); } diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/common/HoodieFlinkEngineContext.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/common/HoodieFlinkEngineContext.java index c0bbd0878..62f8d4fa0 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/common/HoodieFlinkEngineContext.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/common/HoodieFlinkEngineContext.java @@ -88,7 +88,7 @@ public class HoodieFlinkEngineContext extends HoodieEngineContext { } @Override - public HoodieData parallelize(List data) { + public HoodieData parallelize(List data, int parallelism) { return HoodieList.of(data); } diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/index/FlinkHoodieIndex.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/index/FlinkHoodieIndex.java index 272da8c6c..847a2183a 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/index/FlinkHoodieIndex.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/index/FlinkHoodieIndex.java @@ -7,32 +7,29 @@ * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. */ package org.apache.hudi.index; import org.apache.hudi.ApiMaturityLevel; +import org.apache.hudi.PublicAPIMethod; import org.apache.hudi.client.WriteStatus; -import org.apache.hudi.client.common.HoodieFlinkEngineContext; +import org.apache.hudi.common.data.HoodieData; +import org.apache.hudi.common.data.HoodieList; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; -import org.apache.hudi.common.util.ReflectionUtils; -import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieIndexException; -import org.apache.hudi.index.simple.FlinkHoodieSimpleIndex; -import org.apache.hudi.index.bloom.FlinkHoodieBloomIndex; -import org.apache.hudi.index.state.FlinkInMemoryStateIndex; -import org.apache.hudi.PublicAPIMethod; import org.apache.hudi.table.HoodieTable; import java.util.List; @@ -46,38 +43,33 @@ public abstract class FlinkHoodieIndex extends Ho super(config); } - public static HoodieIndex createIndex(HoodieFlinkEngineContext context, HoodieWriteConfig config) { - // first use index class config to create index. - if (!StringUtils.isNullOrEmpty(config.getIndexClass())) { - Object instance = ReflectionUtils.loadClass(config.getIndexClass(), config); - if (!(instance instanceof HoodieIndex)) { - throw new HoodieIndexException(config.getIndexClass() + " is not a subclass of HoodieIndex"); - } - return (FlinkHoodieIndex) instance; - } + @Override + @Deprecated + @PublicAPIMethod(maturity = ApiMaturityLevel.DEPRECATED) + public abstract List updateLocation(List writeStatuses, + HoodieEngineContext context, + HoodieTable>, List, List> hoodieTable) throws HoodieIndexException; - // TODO more indexes to be added - switch (config.getIndexType()) { - case INMEMORY: - return new FlinkInMemoryStateIndex<>(context, config); - case BLOOM: - return new FlinkHoodieBloomIndex(config); - case SIMPLE: - return new FlinkHoodieSimpleIndex<>(config); - default: - throw new HoodieIndexException("Unsupported index type " + config.getIndexType()); - } + @Override + @Deprecated + @PublicAPIMethod(maturity = ApiMaturityLevel.DEPRECATED) + public abstract List> tagLocation(List> records, + HoodieEngineContext context, + HoodieTable>, List, List> hoodieTable) throws HoodieIndexException; + + @Override + @PublicAPIMethod(maturity = ApiMaturityLevel.EVOLVING) + public HoodieData> tagLocation( + HoodieData> records, HoodieEngineContext context, + HoodieTable hoodieTable) throws HoodieIndexException { + return HoodieList.of(tagLocation(HoodieList.getList(records), context, hoodieTable)); } @Override - @PublicAPIMethod(maturity = ApiMaturityLevel.STABLE) - public abstract List updateLocation(List writeStatuses, - HoodieEngineContext context, - HoodieTable>, List, List> hoodieTable) throws HoodieIndexException; - - @Override - @PublicAPIMethod(maturity = ApiMaturityLevel.STABLE) - public abstract List> tagLocation(List> records, - HoodieEngineContext context, - HoodieTable>, List, List> hoodieTable) throws HoodieIndexException; + @PublicAPIMethod(maturity = ApiMaturityLevel.EVOLVING) + public HoodieData updateLocation( + HoodieData writeStatuses, HoodieEngineContext context, + HoodieTable hoodieTable) throws HoodieIndexException { + return HoodieList.of(updateLocation(HoodieList.getList(writeStatuses), context, hoodieTable)); + } } diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/index/FlinkHoodieIndexFactory.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/index/FlinkHoodieIndexFactory.java new file mode 100644 index 000000000..a9196ca9a --- /dev/null +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/index/FlinkHoodieIndexFactory.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.index; + +import org.apache.hudi.client.common.HoodieFlinkEngineContext; +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.config.HoodieWriteConfig; +import org.apache.hudi.exception.HoodieIndexException; +import org.apache.hudi.index.bloom.HoodieBloomIndex; +import org.apache.hudi.index.bloom.ListBasedHoodieBloomIndexHelper; +import org.apache.hudi.index.simple.HoodieSimpleIndex; +import org.apache.hudi.index.state.FlinkInMemoryStateIndex; + +/** + * A factory to generate Flink {@link HoodieIndex}. + */ +public final class FlinkHoodieIndexFactory { + public static HoodieIndex createIndex(HoodieFlinkEngineContext context, HoodieWriteConfig config) { + // first use index class config to create index. + if (!StringUtils.isNullOrEmpty(config.getIndexClass())) { + Object instance = ReflectionUtils.loadClass(config.getIndexClass(), config); + if (!(instance instanceof HoodieIndex)) { + throw new HoodieIndexException(config.getIndexClass() + " is not a subclass of HoodieIndex"); + } + return (HoodieIndex) instance; + } + + // TODO more indexes to be added + switch (config.getIndexType()) { + case INMEMORY: + return new FlinkInMemoryStateIndex<>(context, config); + case BLOOM: + return new HoodieBloomIndex<>(config, ListBasedHoodieBloomIndexHelper.getInstance()); + case SIMPLE: + return new HoodieSimpleIndex<>(config, Option.empty()); + default: + throw new HoodieIndexException("Unsupported index type " + config.getIndexType()); + } + } +} diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/index/bloom/FlinkHoodieBloomIndex.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/index/bloom/FlinkHoodieBloomIndex.java deleted file mode 100644 index 355dced71..000000000 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/index/bloom/FlinkHoodieBloomIndex.java +++ /dev/null @@ -1,32 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hudi.index.bloom; - -import org.apache.hudi.common.model.HoodieRecordPayload; -import org.apache.hudi.config.HoodieWriteConfig; - -/** - * Indexing mechanism based on bloom filter. Each parquet file includes its row_key bloom filter in its metadata. - */ -@SuppressWarnings("checkstyle:LineLength") -public class FlinkHoodieBloomIndex extends HoodieBaseBloomIndex { - public FlinkHoodieBloomIndex(HoodieWriteConfig config) { - super(config); - } -} diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/index/simple/FlinkHoodieSimpleIndex.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/index/simple/FlinkHoodieSimpleIndex.java deleted file mode 100644 index 530bdbf16..000000000 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/index/simple/FlinkHoodieSimpleIndex.java +++ /dev/null @@ -1,140 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hudi.index.simple; - -import org.apache.hudi.client.WriteStatus; -import org.apache.hudi.common.engine.HoodieEngineContext; -import org.apache.hudi.common.model.HoodieBaseFile; -import org.apache.hudi.common.model.HoodieKey; -import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordLocation; -import org.apache.hudi.common.model.HoodieRecordPayload; -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.HoodieIndexException; -import org.apache.hudi.index.FlinkHoodieIndex; -import org.apache.hudi.index.HoodieIndexUtils; -import org.apache.hudi.io.HoodieKeyLocationFetchHandle; -import org.apache.hudi.table.HoodieTable; - -import java.util.HashMap; -import java.util.LinkedList; -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.stream.Collectors; - -import static org.apache.hudi.index.HoodieIndexUtils.getLatestBaseFilesForAllPartitions; - -/** - * A simple index which reads interested fields(record key and partition path) from base files and - * compares with incoming records to find the tagged location. - * - * @param type of payload - */ -public class FlinkHoodieSimpleIndex extends FlinkHoodieIndex { - - public FlinkHoodieSimpleIndex(HoodieWriteConfig config) { - super(config); - } - - @Override - public List updateLocation(List writeStatuses, HoodieEngineContext context, - HoodieTable>, List, List> hoodieTable) throws HoodieIndexException { - return writeStatuses; - } - - @Override - public boolean rollbackCommit(String instantTime) { - return true; - } - - @Override - public boolean isGlobal() { - return false; - } - - @Override - public boolean canIndexLogFiles() { - return false; - } - - @Override - public boolean isImplicitWithStorage() { - return true; - } - - @Override - public List> tagLocation(List> hoodieRecords, HoodieEngineContext context, - HoodieTable>, List, List> hoodieTable) throws HoodieIndexException { - return tagLocationInternal(hoodieRecords, context, hoodieTable); - } - - /** - * Tags records location for incoming records. - */ - private List> tagLocationInternal(List> hoodieRecords, HoodieEngineContext context, - HoodieTable>, List, List> hoodieTable) { - Map> keyedInputRecords = context.mapToPair(hoodieRecords, record -> Pair.of(record.getKey(), record), 0); - Map existingLocationsOnTable = fetchRecordLocationsForAffectedPartitions(keyedInputRecords.keySet(), context, hoodieTable, config.getSimpleIndexParallelism()); - List> taggedRecords = new LinkedList<>(); - - for (Map.Entry> hoodieKeyHoodieRecordEntry : keyedInputRecords.entrySet()) { - HoodieKey key = hoodieKeyHoodieRecordEntry.getKey(); - HoodieRecord record = hoodieKeyHoodieRecordEntry.getValue(); - if (existingLocationsOnTable.containsKey(key)) { - taggedRecords.add(HoodieIndexUtils.getTaggedRecord(record, Option.ofNullable(existingLocationsOnTable.get(key)))); - } - } - return taggedRecords; - } - - /** - * Fetch record locations for passed in {@link HoodieKey}s. - * - * @param keySet {@link HoodieKey}s for which locations are fetched - * @param context instance of {@link HoodieEngineContext} to use - * @param hoodieTable instance of {@link HoodieTable} of interest - * @param parallelism parallelism to use - * @return {@link Map} of {@link HoodieKey} and {@link HoodieRecordLocation} - */ - private Map fetchRecordLocationsForAffectedPartitions(Set keySet, - HoodieEngineContext context, - HoodieTable>, List, List> hoodieTable, - int parallelism) { - List affectedPartitionPathList = keySet.stream().map(HoodieKey::getPartitionPath).distinct().collect(Collectors.toList()); - List> latestBaseFiles = getLatestBaseFilesForAllPartitions(affectedPartitionPathList, context, hoodieTable); - return fetchRecordLocations(context, hoodieTable, parallelism, latestBaseFiles); - } - - private Map fetchRecordLocations(HoodieEngineContext context, - HoodieTable>, List, List> hoodieTable, - int parallelism, - List> latestBaseFiles) { - - List>, List, List>> hoodieKeyLocationFetchHandles = - context.map(latestBaseFiles, partitionPathBaseFile -> new HoodieKeyLocationFetchHandle<>(config, hoodieTable, partitionPathBaseFile, Option.empty()), parallelism); - Map recordLocations = new HashMap<>(); - hoodieKeyLocationFetchHandles.stream() - .flatMap(handle -> handle.locations()) - .forEach(x -> recordLocations.put(x.getKey(), x.getRight())); - return recordLocations; - } -} diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/index/state/FlinkInMemoryStateIndex.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/index/state/FlinkInMemoryStateIndex.java index 118a0e729..aa779c425 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/index/state/FlinkInMemoryStateIndex.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/index/state/FlinkInMemoryStateIndex.java @@ -20,13 +20,14 @@ package org.apache.hudi.index.state; import org.apache.hudi.client.WriteStatus; import org.apache.hudi.client.common.HoodieFlinkEngineContext; +import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieIndexException; -import org.apache.hudi.index.FlinkHoodieIndex; +import org.apache.hudi.index.HoodieIndex; import org.apache.hudi.table.HoodieTable; import org.apache.log4j.LogManager; @@ -39,7 +40,8 @@ import java.util.List; * * @param type of payload */ -public class FlinkInMemoryStateIndex extends FlinkHoodieIndex { +public class FlinkInMemoryStateIndex> + extends HoodieIndex>, List, List> { private static final Logger LOG = LogManager.getLogger(FlinkInMemoryStateIndex.class); @@ -48,16 +50,16 @@ public class FlinkInMemoryStateIndex extends Flin } @Override - public List> tagLocation(List> records, - HoodieEngineContext context, - HoodieTable>, List, List> hoodieTable) throws HoodieIndexException { + public HoodieData> tagLocation( + HoodieData> records, HoodieEngineContext context, + HoodieTable hoodieTable) throws HoodieIndexException { throw new UnsupportedOperationException("No need to tag location for FlinkInMemoryStateIndex"); } @Override - public List updateLocation(List writeStatuses, - HoodieEngineContext context, - HoodieTable>, List, List> hoodieTable) throws HoodieIndexException { + public HoodieData updateLocation( + HoodieData writeStatuses, HoodieEngineContext context, + HoodieTable hoodieTable) throws HoodieIndexException { throw new UnsupportedOperationException("No need to update location for FlinkInMemoryStateIndex"); } diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkTable.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkTable.java index fdae2558e..f00781f8f 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkTable.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkTable.java @@ -32,7 +32,7 @@ import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.exception.HoodieMetadataException; -import org.apache.hudi.index.FlinkHoodieIndex; +import org.apache.hudi.index.FlinkHoodieIndexFactory; import org.apache.hudi.index.HoodieIndex; import org.apache.hudi.metadata.FlinkHoodieBackedTableMetadataWriter; import org.apache.hudi.metadata.HoodieTableMetadata; @@ -98,8 +98,8 @@ public abstract class HoodieFlinkTable } @Override - protected HoodieIndex>, List, List> getIndex(HoodieWriteConfig config, HoodieEngineContext context) { - return FlinkHoodieIndex.createIndex((HoodieFlinkEngineContext) context, config); + protected HoodieIndex getIndex(HoodieWriteConfig config, HoodieEngineContext context) { + return FlinkHoodieIndexFactory.createIndex((HoodieFlinkEngineContext) context, config); } /** diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkDeleteHelper.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkDeleteHelper.java index 185482642..05ac93725 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkDeleteHelper.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkDeleteHelper.java @@ -19,6 +19,7 @@ package org.apache.hudi.table.action.commit; import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.common.data.HoodieList; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.EmptyHoodieRecordPayload; import org.apache.hudi.common.model.HoodieKey; @@ -95,8 +96,8 @@ public class FlinkDeleteHelper extends dedupedKeys.stream().map(key -> new HoodieRecord<>(key, new EmptyHoodieRecordPayload())).collect(Collectors.toList()); Instant beginTag = Instant.now(); // perform index look up to get existing location of records - List> taggedRecords = - table.getIndex().tagLocation(dedupedRecords, context, table); + List> taggedRecords = HoodieList.getList( + table.getIndex().tagLocation(HoodieList.of(dedupedRecords), context, table)); Duration tagLocationDuration = Duration.between(beginTag, Instant.now()); // filter out non existent keys/records diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkWriteHelper.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkWriteHelper.java index 5cb1b80a5..0863ad8e4 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkWriteHelper.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkWriteHelper.java @@ -19,6 +19,7 @@ package org.apache.hudi.table.action.commit; import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.common.data.HoodieList; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieOperation; @@ -80,9 +81,14 @@ public class FlinkWriteHelper extends Abstract } @Override - public List> deduplicateRecords(List> records, - HoodieIndex>, List, List> index, - int parallelism) { + protected List> tag(List> dedupedRecords, HoodieEngineContext context, HoodieTable>, List, List> table) { + return HoodieList.getList( + table.getIndex().tagLocation(HoodieList.of(dedupedRecords), context, table)); + } + + @Override + public List> deduplicateRecords( + List> records, HoodieIndex index, int parallelism) { Map>>> keyedRecords = records.stream().map(record -> { // If index used is global, then records are expected to differ in their partitionPath final Object key = record.getKey().getRecordKey(); diff --git a/hudi-client/hudi-flink-client/src/test/java/org/apache/hudi/index/bloom/TestFlinkHoodieBloomIndex.java b/hudi-client/hudi-flink-client/src/test/java/org/apache/hudi/index/bloom/TestFlinkHoodieBloomIndex.java index a41e8c835..7b4e3b675 100644 --- a/hudi-client/hudi-flink-client/src/test/java/org/apache/hudi/index/bloom/TestFlinkHoodieBloomIndex.java +++ b/hudi-client/hudi-flink-client/src/test/java/org/apache/hudi/index/bloom/TestFlinkHoodieBloomIndex.java @@ -21,6 +21,8 @@ package org.apache.hudi.index.bloom; import org.apache.hudi.common.bloom.BloomFilter; import org.apache.hudi.common.bloom.BloomFilterFactory; import org.apache.hudi.common.bloom.BloomFilterTypeCode; +import org.apache.hudi.common.data.HoodieList; +import org.apache.hudi.common.data.HoodieMapPair; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.table.HoodieTableMetaClient; @@ -100,7 +102,7 @@ public class TestFlinkHoodieBloomIndex extends HoodieFlinkClientTestHarness { @MethodSource("configParams") public void testLoadInvolvedFiles(boolean rangePruning, boolean treeFiltering, boolean bucketizedChecking) throws Exception { HoodieWriteConfig config = makeConfig(rangePruning, treeFiltering, bucketizedChecking); - FlinkHoodieBloomIndex index = new FlinkHoodieBloomIndex(config); + HoodieBloomIndex index = new HoodieBloomIndex(config, ListBasedHoodieBloomIndexHelper.getInstance()); HoodieTable hoodieTable = HoodieFlinkTable.create(config, context, metaClient, false); HoodieFlinkWriteableTestTable testTable = HoodieFlinkWriteableTestTable.of(hoodieTable, SCHEMA); @@ -165,7 +167,7 @@ public class TestFlinkHoodieBloomIndex extends HoodieFlinkClientTestHarness { @MethodSource("configParams") public void testRangePruning(boolean rangePruning, boolean treeFiltering, boolean bucketizedChecking) { HoodieWriteConfig config = makeConfig(rangePruning, treeFiltering, bucketizedChecking); - FlinkHoodieBloomIndex index = new FlinkHoodieBloomIndex(config); + HoodieBloomIndex index = new HoodieBloomIndex(config, ListBasedHoodieBloomIndexHelper.getInstance()); final Map> partitionToFileIndexInfo = new HashMap<>(); partitionToFileIndexInfo.put("2017/10/22", @@ -176,14 +178,14 @@ public class TestFlinkHoodieBloomIndex extends HoodieFlinkClientTestHarness { Map> partitionRecordKeyMap = new HashMap<>(); asList(Pair.of("2017/10/22", "003"), Pair.of("2017/10/22", "002"), Pair.of("2017/10/22", "005"), Pair.of("2017/10/22", "004")) - .forEach(t -> { - List recordKeyList = partitionRecordKeyMap.getOrDefault(t.getLeft(), new ArrayList<>()); - recordKeyList.add(t.getRight()); - partitionRecordKeyMap.put(t.getLeft(), recordKeyList); - }); + .forEach(t -> { + List recordKeyList = partitionRecordKeyMap.getOrDefault(t.getLeft(), new ArrayList<>()); + recordKeyList.add(t.getRight()); + partitionRecordKeyMap.put(t.getLeft(), recordKeyList); + }); - List> comparisonKeyList = - index.explodeRecordsWithFileComparisons(partitionToFileIndexInfo, partitionRecordKeyMap); + List> comparisonKeyList = HoodieList.getList( + index.explodeRecordsWithFileComparisons(partitionToFileIndexInfo, HoodieMapPair.of(partitionRecordKeyMap))); assertEquals(10, comparisonKeyList.size()); java.util.Map> recordKeyToFileComps = comparisonKeyList.stream() @@ -264,10 +266,10 @@ public class TestFlinkHoodieBloomIndex extends HoodieFlinkClientTestHarness { HoodieFlinkTable table = HoodieFlinkTable.create(config, context, metaClient); // Let's tag - FlinkHoodieBloomIndex bloomIndex = new FlinkHoodieBloomIndex(config); + HoodieBloomIndex bloomIndex = new HoodieBloomIndex(config, ListBasedHoodieBloomIndexHelper.getInstance()); assertDoesNotThrow(() -> { - bloomIndex.tagLocation(records, context, table); + tagLocation(bloomIndex, records, table); }, "EmptyList should not result in IllegalArgumentException: Positive number of slices required"); } @@ -303,8 +305,8 @@ public class TestFlinkHoodieBloomIndex extends HoodieFlinkClientTestHarness { HoodieFlinkWriteableTestTable testTable = HoodieFlinkWriteableTestTable.of(hoodieTable, SCHEMA); // Let's tag - FlinkHoodieBloomIndex bloomIndex = new FlinkHoodieBloomIndex(config); - List taggedRecords = bloomIndex.tagLocation(records, context, hoodieTable); + HoodieBloomIndex bloomIndex = new HoodieBloomIndex(config, ListBasedHoodieBloomIndexHelper.getInstance()); + List taggedRecords = tagLocation(bloomIndex, records, hoodieTable); // Should not find any files for (HoodieRecord record : taggedRecords) { @@ -319,7 +321,7 @@ public class TestFlinkHoodieBloomIndex extends HoodieFlinkClientTestHarness { metaClient.reloadActiveTimeline(); // We do the tag again - taggedRecords = bloomIndex.tagLocation(records, context, HoodieFlinkTable.create(config, context, metaClient)); + taggedRecords = tagLocation(bloomIndex, records, HoodieFlinkTable.create(config, context, metaClient)); // Check results for (HoodieRecord record : taggedRecords) { @@ -370,10 +372,10 @@ public class TestFlinkHoodieBloomIndex extends HoodieFlinkClientTestHarness { HoodieFlinkWriteableTestTable testTable = HoodieFlinkWriteableTestTable.of(hoodieTable, SCHEMA); // Let's tag - FlinkHoodieBloomIndex bloomIndex = new FlinkHoodieBloomIndex(config); + HoodieBloomIndex bloomIndex = new HoodieBloomIndex(config, ListBasedHoodieBloomIndexHelper.getInstance()); List toTagRecords = new ArrayList<>(); toTagRecords.add(new HoodieRecord(record4.getKey(), null)); - List taggedRecords = bloomIndex.tagLocation(toTagRecords, context, hoodieTable); + List taggedRecords = tagLocation(bloomIndex, toTagRecords, hoodieTable); Map>> recordLocations = new HashMap<>(); for (HoodieRecord taggedRecord : taggedRecords) { recordLocations.put(taggedRecord.getKey(), taggedRecord.isCurrentLocationKnown() @@ -398,7 +400,7 @@ public class TestFlinkHoodieBloomIndex extends HoodieFlinkClientTestHarness { taggedRecords.add(new HoodieRecord(key, null)); } - taggedRecords = bloomIndex.tagLocation(toTagRecords1, context, hoodieTable); + taggedRecords = tagLocation(bloomIndex, toTagRecords1, hoodieTable); recordLocations.clear(); for (HoodieRecord taggedRecord : taggedRecords) { recordLocations.put(taggedRecord.getKey(), taggedRecord.isCurrentLocationKnown() @@ -452,8 +454,8 @@ public class TestFlinkHoodieBloomIndex extends HoodieFlinkClientTestHarness { metaClient = HoodieTableMetaClient.reload(metaClient); HoodieTable table = HoodieFlinkTable.create(config, context, metaClient); - FlinkHoodieBloomIndex bloomIndex = new FlinkHoodieBloomIndex(config); - List taggedRecords = bloomIndex.tagLocation(records, context, table); + HoodieBloomIndex bloomIndex = new HoodieBloomIndex(config, ListBasedHoodieBloomIndexHelper.getInstance()); + List taggedRecords = tagLocation(bloomIndex, records, table); // Check results for (HoodieRecord record : taggedRecords) { diff --git a/hudi-client/hudi-flink-client/src/test/java/org/apache/hudi/testutils/HoodieFlinkClientTestHarness.java b/hudi-client/hudi-flink-client/src/test/java/org/apache/hudi/testutils/HoodieFlinkClientTestHarness.java index 171bab9fb..054a36316 100644 --- a/hudi-client/hudi-flink-client/src/test/java/org/apache/hudi/testutils/HoodieFlinkClientTestHarness.java +++ b/hudi-client/hudi-flink-client/src/test/java/org/apache/hudi/testutils/HoodieFlinkClientTestHarness.java @@ -21,6 +21,7 @@ package org.apache.hudi.testutils; import org.apache.hudi.client.FlinkTaskContextSupplier; import org.apache.hudi.client.HoodieFlinkWriteClient; import org.apache.hudi.client.common.HoodieFlinkEngineContext; +import org.apache.hudi.common.data.HoodieList; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieTableType; @@ -29,7 +30,9 @@ import org.apache.hudi.common.table.view.HoodieTableFileSystemView; import org.apache.hudi.common.testutils.HoodieCommonTestHarness; import org.apache.hudi.common.testutils.HoodieTestUtils; import org.apache.hudi.common.testutils.minicluster.HdfsTestService; +import org.apache.hudi.index.HoodieIndex; import org.apache.hudi.index.bloom.TestFlinkHoodieBloomIndex; +import org.apache.hudi.table.HoodieTable; import org.apache.hadoop.hdfs.DistributedFileSystem; import org.apache.hadoop.hdfs.MiniDFSCluster; @@ -128,6 +131,10 @@ public class HoodieFlinkClientTestHarness extends HoodieCommonTestHarness implem metaClient = HoodieTestUtils.init(hadoopConf, basePath, tableType); } + protected List tagLocation( + HoodieIndex index, List records, HoodieTable table) { + return HoodieList.getList(index.tagLocation(HoodieList.of(records), context, table)); + } /** * Cleanups file system. 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 a699d4437..212187b2d 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 @@ -20,6 +20,7 @@ package org.apache.hudi.client; import org.apache.hudi.client.common.HoodieJavaEngineContext; import org.apache.hudi.client.embedded.EmbeddedTimelineService; +import org.apache.hudi.common.data.HoodieList; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.model.HoodieKey; @@ -34,7 +35,7 @@ import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieNotSupportedException; import org.apache.hudi.index.HoodieIndex; -import org.apache.hudi.index.JavaHoodieIndex; +import org.apache.hudi.index.JavaHoodieIndexFactory; import org.apache.hudi.table.BulkInsertPartitioner; import org.apache.hudi.table.HoodieJavaTable; import org.apache.hudi.table.HoodieTable; @@ -67,14 +68,15 @@ public class HoodieJavaWriteClient extends // Create a Hoodie table which encapsulated the commits and files visible HoodieJavaTable table = HoodieJavaTable.create(config, (HoodieJavaEngineContext) context); Timer.Context indexTimer = metrics.getIndexCtx(); - List> recordsWithLocation = getIndex().tagLocation(hoodieRecords, context, table); + List> recordsWithLocation = HoodieList.getList( + getIndex().tagLocation(HoodieList.of(hoodieRecords), context, table)); metrics.updateIndexMetrics(LOOKUP_STR, metrics.getDurationInMs(indexTimer == null ? 0L : indexTimer.stop())); return recordsWithLocation.stream().filter(v1 -> !v1.isCurrentLocationKnown()).collect(Collectors.toList()); } @Override - protected HoodieIndex>, List, List> createIndex(HoodieWriteConfig writeConfig) { - return JavaHoodieIndex.createIndex(config); + protected HoodieIndex createIndex(HoodieWriteConfig writeConfig) { + return JavaHoodieIndexFactory.createIndex(config); } @Override diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/client/common/HoodieJavaEngineContext.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/client/common/HoodieJavaEngineContext.java index 2fdd86e0f..22d4ccabc 100644 --- a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/client/common/HoodieJavaEngineContext.java +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/client/common/HoodieJavaEngineContext.java @@ -78,7 +78,7 @@ public class HoodieJavaEngineContext extends HoodieEngineContext { } @Override - public HoodieData parallelize(List data) { + public HoodieData parallelize(List data, int parallelism) { return HoodieList.of(data); } diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/index/JavaHoodieBloomIndex.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/index/JavaHoodieBloomIndex.java deleted file mode 100644 index 47d47c847..000000000 --- a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/index/JavaHoodieBloomIndex.java +++ /dev/null @@ -1,32 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hudi.index; - -import org.apache.hudi.common.model.HoodieRecordPayload; -import org.apache.hudi.config.HoodieWriteConfig; -import org.apache.hudi.index.bloom.HoodieBaseBloomIndex; - -/** - * Indexing mechanism based on bloom filter. Each parquet file includes its row_key bloom filter in its metadata. - */ -public class JavaHoodieBloomIndex extends HoodieBaseBloomIndex { - public JavaHoodieBloomIndex(HoodieWriteConfig config) { - super(config); - } -} diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/index/JavaHoodieIndex.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/index/JavaHoodieIndex.java index fc7a451dc..7f8b83f5c 100644 --- a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/index/JavaHoodieIndex.java +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/index/JavaHoodieIndex.java @@ -7,13 +7,14 @@ * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. */ package org.apache.hudi.index; @@ -21,12 +22,12 @@ package org.apache.hudi.index; import org.apache.hudi.ApiMaturityLevel; import org.apache.hudi.PublicAPIMethod; import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.common.data.HoodieData; +import org.apache.hudi.common.data.HoodieList; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; -import org.apache.hudi.common.util.ReflectionUtils; -import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieIndexException; import org.apache.hudi.table.HoodieTable; @@ -38,36 +39,33 @@ public abstract class JavaHoodieIndex extends Hoo super(config); } - public static HoodieIndex createIndex(HoodieWriteConfig config) { - // first use index class config to create index. - if (!StringUtils.isNullOrEmpty(config.getIndexClass())) { - Object instance = ReflectionUtils.loadClass(config.getIndexClass(), config); - if (!(instance instanceof HoodieIndex)) { - throw new HoodieIndexException(config.getIndexClass() + " is not a subclass of HoodieIndex"); - } - return (JavaHoodieIndex) instance; - } - - // TODO more indexes to be added - switch (config.getIndexType()) { - case INMEMORY: - return new JavaInMemoryHashIndex(config); - case BLOOM: - return new JavaHoodieBloomIndex(config); - default: - throw new HoodieIndexException("Unsupported index type " + config.getIndexType()); - } - } - @Override - @PublicAPIMethod(maturity = ApiMaturityLevel.STABLE) + @Deprecated + @PublicAPIMethod(maturity = ApiMaturityLevel.DEPRECATED) public abstract List updateLocation(List writeStatuses, HoodieEngineContext context, HoodieTable>, List, List> hoodieTable) throws HoodieIndexException; @Override - @PublicAPIMethod(maturity = ApiMaturityLevel.STABLE) + @Deprecated + @PublicAPIMethod(maturity = ApiMaturityLevel.DEPRECATED) public abstract List> tagLocation(List> records, HoodieEngineContext context, HoodieTable>, List, List> hoodieTable) throws HoodieIndexException; + + @Override + @PublicAPIMethod(maturity = ApiMaturityLevel.EVOLVING) + public HoodieData> tagLocation( + HoodieData> records, HoodieEngineContext context, + HoodieTable hoodieTable) throws HoodieIndexException { + return HoodieList.of(tagLocation(HoodieList.getList(records), context, hoodieTable)); + } + + @Override + @PublicAPIMethod(maturity = ApiMaturityLevel.EVOLVING) + public HoodieData updateLocation( + HoodieData writeStatuses, HoodieEngineContext context, + HoodieTable hoodieTable) throws HoodieIndexException { + return HoodieList.of(updateLocation(HoodieList.getList(writeStatuses), context, hoodieTable)); + } } diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/index/JavaHoodieIndexFactory.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/index/JavaHoodieIndexFactory.java new file mode 100644 index 000000000..f6135fb13 --- /dev/null +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/index/JavaHoodieIndexFactory.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.index; + +import org.apache.hudi.common.util.ReflectionUtils; +import org.apache.hudi.common.util.StringUtils; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.exception.HoodieIndexException; +import org.apache.hudi.index.bloom.HoodieBloomIndex; +import org.apache.hudi.index.bloom.ListBasedHoodieBloomIndexHelper; +import org.apache.hudi.index.inmemory.HoodieInMemoryHashIndex; + +/** + * A factory to generate Java {@link HoodieIndex}. + */ +public final class JavaHoodieIndexFactory { + + public static HoodieIndex createIndex(HoodieWriteConfig config) { + // first use index class config to create index. + if (!StringUtils.isNullOrEmpty(config.getIndexClass())) { + Object instance = ReflectionUtils.loadClass(config.getIndexClass(), config); + if (!(instance instanceof HoodieIndex)) { + throw new HoodieIndexException(config.getIndexClass() + " is not a subclass of HoodieIndex"); + } + return (HoodieIndex) instance; + } + + // TODO more indexes to be added + switch (config.getIndexType()) { + case INMEMORY: + return new HoodieInMemoryHashIndex<>(config); + case BLOOM: + return new HoodieBloomIndex<>(config, ListBasedHoodieBloomIndexHelper.getInstance()); + default: + throw new HoodieIndexException("Unsupported index type " + config.getIndexType()); + } + } +} diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/HoodieJavaTable.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/HoodieJavaTable.java index 9cf9a6dc1..8b0a7a95e 100644 --- a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/HoodieJavaTable.java +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/HoodieJavaTable.java @@ -30,7 +30,7 @@ import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.index.HoodieIndex; -import org.apache.hudi.index.JavaHoodieIndex; +import org.apache.hudi.index.JavaHoodieIndexFactory; import java.util.List; @@ -62,7 +62,7 @@ public abstract class HoodieJavaTable } @Override - protected HoodieIndex>, List, List> getIndex(HoodieWriteConfig config, HoodieEngineContext context) { - return JavaHoodieIndex.createIndex(config); + protected HoodieIndex getIndex(HoodieWriteConfig config, HoodieEngineContext context) { + return JavaHoodieIndexFactory.createIndex(config); } } diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/BaseJavaCommitActionExecutor.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/BaseJavaCommitActionExecutor.java index e9abf277f..79aad595f 100644 --- a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/BaseJavaCommitActionExecutor.java +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/BaseJavaCommitActionExecutor.java @@ -19,6 +19,7 @@ package org.apache.hudi.table.action.commit; import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.common.data.HoodieList; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodieCommitMetadata; @@ -128,7 +129,8 @@ public abstract class BaseJavaCommitActionExecutor writeStatuses, HoodieWriteMetadata> result) { Instant indexStartTime = Instant.now(); // Update the index back - List statuses = table.getIndex().updateLocation(writeStatuses, context, table); + List statuses = HoodieList.getList( + table.getIndex().updateLocation(HoodieList.of(writeStatuses), context, table)); result.setIndexUpdateDuration(Duration.between(indexStartTime, Instant.now())); result.setWriteStatuses(statuses); } @@ -329,7 +331,8 @@ public abstract class BaseJavaCommitActionExecutor writeStatuses, HoodieWriteMetadata result) { Instant indexStartTime = Instant.now(); // Update the index back - List statuses = table.getIndex().updateLocation(writeStatuses, context, table); + List statuses = HoodieList.getList( + table.getIndex().updateLocation(HoodieList.of(writeStatuses), context, table)); result.setIndexUpdateDuration(Duration.between(indexStartTime, Instant.now())); result.setWriteStatuses(statuses); result.setPartitionToReplaceFileIds(getPartitionToReplacedFileIds(result)); diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaDeleteHelper.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaDeleteHelper.java index a907c9fd9..fc81b787f 100644 --- a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaDeleteHelper.java +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaDeleteHelper.java @@ -19,6 +19,7 @@ package org.apache.hudi.table.action.commit; import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.common.data.HoodieList; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.EmptyHoodieRecordPayload; import org.apache.hudi.common.model.HoodieKey; @@ -97,8 +98,8 @@ public class JavaDeleteHelper extends dedupedKeys.stream().map(key -> new HoodieRecord<>(key, new EmptyHoodieRecordPayload())).collect(Collectors.toList()); Instant beginTag = Instant.now(); // perform index look up to get existing location of records - List> taggedRecords = - table.getIndex().tagLocation(dedupedRecords, context, table); + List> taggedRecords = HoodieList.getList( + table.getIndex().tagLocation(HoodieList.of(dedupedRecords), context, table)); Duration tagLocationDuration = Duration.between(beginTag, Instant.now()); // filter out non existent keys/records diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaWriteHelper.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaWriteHelper.java index ec7ea1641..8af7707ea 100644 --- a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaWriteHelper.java +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaWriteHelper.java @@ -19,11 +19,14 @@ package org.apache.hudi.table.action.commit; import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.common.data.HoodieList; +import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.index.HoodieIndex; +import org.apache.hudi.table.HoodieTable; import java.util.List; import java.util.Map; @@ -45,9 +48,14 @@ public class JavaWriteHelper extends AbstractWr } @Override - public List> deduplicateRecords(List> records, - HoodieIndex>, List, List> index, - int parallelism) { + protected List> tag(List> dedupedRecords, HoodieEngineContext context, HoodieTable>, List, List> table) { + return HoodieList.getList( + table.getIndex().tagLocation(HoodieList.of(dedupedRecords), context, table)); + } + + @Override + public List> deduplicateRecords( + List> records, HoodieIndex index, int parallelism) { boolean isIndexingGlobal = index.isGlobal(); Map>>> keyedRecords = records.stream().map(record -> { HoodieKey hoodieKey = record.getKey(); diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/HoodieReadClient.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/HoodieReadClient.java index fc615e0f8..84040f906 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/HoodieReadClient.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/HoodieReadClient.java @@ -32,9 +32,10 @@ import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.config.HoodieIndexConfig; import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.data.HoodieJavaRDD; import org.apache.hudi.exception.HoodieIndexException; import org.apache.hudi.index.HoodieIndex; -import org.apache.hudi.index.SparkHoodieIndex; +import org.apache.hudi.index.SparkHoodieIndexFactory; import org.apache.hudi.table.HoodieSparkTable; import org.apache.hudi.table.HoodieTable; @@ -57,7 +58,7 @@ import scala.Tuple2; /** * Provides an RDD based API for accessing/filtering Hoodie tables, based on keys. */ -public class HoodieReadClient implements Serializable { +public class HoodieReadClient> implements Serializable { private static final long serialVersionUID = 1L; @@ -65,7 +66,7 @@ public class HoodieReadClient implements Serializ * TODO: We need to persist the index type into hoodie.properties and be able to access the index just with a simple * basepath pointing to the table. Until, then just always assume a BloomIndex */ - private final transient HoodieIndex>, JavaRDD, JavaRDD> index; + private final transient HoodieIndex index; private HoodieTable>, JavaRDD, JavaRDD> hoodieTable; private transient Option sqlContextOpt; private final transient HoodieSparkEngineContext context; @@ -100,7 +101,7 @@ public class HoodieReadClient implements Serializ // Create a Hoodie table which encapsulated the commits and files visible HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder().setConf(hadoopConf).setBasePath(basePath).setLoadActiveTimelineOnLoad(true).build(); this.hoodieTable = HoodieSparkTable.create(clientConfig, context, metaClient); - this.index = SparkHoodieIndex.createIndex(clientConfig); + this.index = SparkHoodieIndexFactory.createIndex(clientConfig); this.sqlContextOpt = Option.empty(); } @@ -170,7 +171,9 @@ public class HoodieReadClient implements Serializ * component (without scheme) of the URI underlying file */ public JavaPairRDD>> checkExists(JavaRDD hoodieKeys) { - return index.tagLocation(hoodieKeys.map(k -> new HoodieRecord<>(k, null)), context, hoodieTable) + return HoodieJavaRDD.getJavaRDD( + index.tagLocation(HoodieJavaRDD.of(hoodieKeys.map(k -> new HoodieRecord<>(k, null))), + context, hoodieTable)) .mapToPair(hr -> new Tuple2<>(hr.getKey(), hr.isCurrentLocationKnown() ? Option.of(Pair.of(hr.getPartitionPath(), hr.getCurrentLocation().getFileId())) : Option.empty()) @@ -196,7 +199,8 @@ public class HoodieReadClient implements Serializ * @return Tagged RDD of Hoodie records */ public JavaRDD> tagLocation(JavaRDD> hoodieRecords) throws HoodieIndexException { - return index.tagLocation(hoodieRecords, context, hoodieTable); + return HoodieJavaRDD.getJavaRDD( + index.tagLocation(HoodieJavaRDD.of(hoodieRecords), context, hoodieTable)); } /** 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 6c71d7548..a1a5c8552 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 @@ -44,7 +44,7 @@ import org.apache.hudi.data.HoodieJavaRDD; import org.apache.hudi.exception.HoodieClusteringException; import org.apache.hudi.exception.HoodieCommitException; import org.apache.hudi.index.HoodieIndex; -import org.apache.hudi.index.SparkHoodieIndex; +import org.apache.hudi.index.SparkHoodieIndexFactory; import org.apache.hudi.metadata.HoodieTableMetadataWriter; import org.apache.hudi.metadata.SparkHoodieBackedTableMetadataWriter; import org.apache.hudi.metrics.DistributedRegistry; @@ -115,8 +115,8 @@ public class SparkRDDWriteClient extends } @Override - protected HoodieIndex>, JavaRDD, JavaRDD> createIndex(HoodieWriteConfig writeConfig) { - return SparkHoodieIndex.createIndex(config); + protected HoodieIndex createIndex(HoodieWriteConfig writeConfig) { + return SparkHoodieIndexFactory.createIndex(config); } /** @@ -141,7 +141,8 @@ public class SparkRDDWriteClient extends // Create a Hoodie table which encapsulated the commits and files visible HoodieSparkTable table = HoodieSparkTable.create(config, context); Timer.Context indexTimer = metrics.getIndexCtx(); - JavaRDD> recordsWithLocation = getIndex().tagLocation(hoodieRecords, context, table); + JavaRDD> recordsWithLocation = HoodieJavaRDD.getJavaRDD( + getIndex().tagLocation(HoodieJavaRDD.of(hoodieRecords), context, table)); metrics.updateIndexMetrics(LOOKUP_STR, metrics.getDurationInMs(indexTimer == null ? 0L : indexTimer.stop())); return recordsWithLocation.filter(v1 -> !v1.isCurrentLocationKnown()); } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/common/HoodieSparkEngineContext.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/common/HoodieSparkEngineContext.java index 1c7f1c8e2..cc29ef70f 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/common/HoodieSparkEngineContext.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/common/HoodieSparkEngineContext.java @@ -91,8 +91,8 @@ public class HoodieSparkEngineContext extends HoodieEngineContext { } @Override - public HoodieData parallelize(List data) { - return HoodieJavaRDD.of(javaSparkContext.parallelize(data, data.size())); + public HoodieData parallelize(List data, int parallelism) { + return HoodieJavaRDD.of(javaSparkContext.parallelize(data, parallelism)); } @Override diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/utils/SparkMemoryUtils.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/utils/SparkMemoryUtils.java index 58d822a23..9cb127f39 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/utils/SparkMemoryUtils.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/utils/SparkMemoryUtils.java @@ -19,7 +19,6 @@ package org.apache.hudi.client.utils; import org.apache.hudi.common.config.HoodieConfig; -import org.apache.hudi.config.HoodieIndexConfig; import org.apache.spark.storage.StorageLevel; @@ -34,12 +33,4 @@ public class SparkMemoryUtils { public static StorageLevel getWriteStatusStorageLevel(Properties properties) { return StorageLevel.fromString(new HoodieConfig(properties).getString(WRITE_STATUS_STORAGE_LEVEL_VALUE)); } - - public static StorageLevel getBloomIndexInputStorageLevel(Properties properties) { - return StorageLevel.fromString(new HoodieConfig(properties).getString(HoodieIndexConfig.BLOOM_INDEX_INPUT_STORAGE_LEVEL_VALUE)); - } - - public static StorageLevel getSimpleIndexInputStorageLevel(Properties properties) { - return StorageLevel.fromString(new HoodieConfig(properties).getString(HoodieIndexConfig.SIMPLE_INDEX_INPUT_STORAGE_LEVEL_VALUE)); - } } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/data/HoodieJavaPairRDD.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/data/HoodieJavaPairRDD.java new file mode 100644 index 000000000..ffa1a3565 --- /dev/null +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/data/HoodieJavaPairRDD.java @@ -0,0 +1,127 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.hudi.data; + +import org.apache.hudi.common.data.HoodieData; +import org.apache.hudi.common.data.HoodiePairData; +import org.apache.hudi.common.function.SerializableFunction; +import org.apache.hudi.common.function.SerializablePairFunction; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.collection.ImmutablePair; +import org.apache.hudi.common.util.collection.Pair; + +import org.apache.spark.api.java.JavaPairRDD; +import org.apache.spark.storage.StorageLevel; + +import java.util.Map; + +import scala.Tuple2; + +/** + * Implementation of {@link HoodiePairData} using Spark {@link JavaPairRDD}. + * + * @param type of key. + * @param type of value. + */ +public class HoodieJavaPairRDD extends HoodiePairData { + + private final JavaPairRDD pairRDDData; + + private HoodieJavaPairRDD(JavaPairRDD pairRDDData) { + this.pairRDDData = pairRDDData; + } + + /** + * @param pairRDDData a {@link JavaPairRDD} of pairs. + * @param type of key. + * @param type of value. + * @return a new instance containing the {@link JavaPairRDD} reference. + */ + public static HoodieJavaPairRDD of(JavaPairRDD pairRDDData) { + return new HoodieJavaPairRDD<>(pairRDDData); + } + + /** + * @param hoodiePairData {@link HoodieJavaPairRDD } instance containing the {@link JavaPairRDD} of pairs. + * @param type of key. + * @param type of value. + * @return the {@link JavaPairRDD} of pairs. + */ + public static JavaPairRDD getJavaPairRDD(HoodiePairData hoodiePairData) { + return ((HoodieJavaPairRDD) hoodiePairData).get(); + } + + @Override + public JavaPairRDD get() { + return pairRDDData; + } + + @Override + public void persist(String storageLevel) { + pairRDDData.persist(StorageLevel.fromString(storageLevel)); + } + + @Override + public void unpersist() { + pairRDDData.unpersist(); + } + + @Override + public HoodieData keys() { + return HoodieJavaRDD.of(pairRDDData.keys()); + } + + @Override + public HoodieData values() { + return HoodieJavaRDD.of(pairRDDData.values()); + } + + @Override + public long count() { + return pairRDDData.count(); + } + + @Override + public Map countByKey() { + return pairRDDData.countByKey(); + } + + @Override + public HoodieData map(SerializableFunction, O> func) { + return HoodieJavaRDD.of(pairRDDData.map( + tuple -> func.apply(new ImmutablePair<>(tuple._1, tuple._2)))); + } + + @Override + public HoodiePairData mapToPair(SerializablePairFunction, L, W> mapToPairFunc) { + return HoodieJavaPairRDD.of(pairRDDData.mapToPair(pair -> { + Pair newPair = mapToPairFunc.call(new ImmutablePair<>(pair._1, pair._2)); + return new Tuple2<>(newPair.getLeft(), newPair.getRight()); + })); + } + + @Override + public HoodiePairData>> leftOuterJoin(HoodiePairData other) { + return HoodieJavaPairRDD.of(JavaPairRDD.fromJavaRDD( + pairRDDData.leftOuterJoin(HoodieJavaPairRDD.getJavaPairRDD(other)) + .map(tuple -> new Tuple2<>(tuple._1, + new ImmutablePair<>(tuple._2._1, Option.ofNullable(tuple._2._2.orElse(null))))))); + } +} diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/data/HoodieJavaRDD.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/data/HoodieJavaRDD.java index e6defd49f..ceaee4728 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/data/HoodieJavaRDD.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/data/HoodieJavaRDD.java @@ -21,13 +21,19 @@ package org.apache.hudi.data; import org.apache.hudi.client.common.HoodieSparkEngineContext; import org.apache.hudi.common.data.HoodieData; +import org.apache.hudi.common.data.HoodiePairData; import org.apache.hudi.common.function.SerializableFunction; +import org.apache.hudi.common.function.SerializablePairFunction; +import org.apache.hudi.common.util.collection.Pair; import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.storage.StorageLevel; import java.util.Iterator; import java.util.List; +import scala.Tuple2; + /** * Holds a {@link JavaRDD} of objects. * @@ -76,19 +82,52 @@ public class HoodieJavaRDD extends HoodieData { return rddData; } + @Override + public void persist(String storageLevel) { + rddData.persist(StorageLevel.fromString(storageLevel)); + } + + @Override + public void unpersist() { + rddData.unpersist(); + } + @Override public boolean isEmpty() { return rddData.isEmpty(); } + @Override + public long count() { + return rddData.count(); + } + @Override public HoodieData map(SerializableFunction func) { return HoodieJavaRDD.of(rddData.map(func::apply)); } + @Override + public HoodieData mapPartitions(SerializableFunction, Iterator> func, boolean preservesPartitioning) { + return HoodieJavaRDD.of(rddData.mapPartitions(func::apply, preservesPartitioning)); + } + @Override public HoodieData flatMap(SerializableFunction> func) { - return HoodieJavaRDD.of(rddData.flatMap(func::apply)); + return HoodieJavaRDD.of(rddData.flatMap(e -> func.apply(e))); + } + + @Override + public HoodiePairData mapToPair(SerializablePairFunction mapToPairFunc) { + return HoodieJavaPairRDD.of(rddData.mapToPair(input -> { + Pair pair = mapToPairFunc.call(input); + return new Tuple2<>(pair.getLeft(), pair.getRight()); + })); + } + + @Override + public HoodieData distinct() { + return HoodieJavaRDD.of(rddData.distinct()); } @Override diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/SparkHoodieIndex.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/SparkHoodieIndex.java index 45094546b..62bf5c100 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/SparkHoodieIndex.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/SparkHoodieIndex.java @@ -7,13 +7,14 @@ * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. */ package org.apache.hudi.index; @@ -21,89 +22,52 @@ package org.apache.hudi.index; import org.apache.hudi.ApiMaturityLevel; import org.apache.hudi.PublicAPIMethod; import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; -import org.apache.hudi.common.util.ReflectionUtils; -import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.data.HoodieJavaRDD; import org.apache.hudi.exception.HoodieIndexException; -import org.apache.hudi.index.bloom.SparkHoodieBloomIndex; -import org.apache.hudi.index.bloom.SparkHoodieGlobalBloomIndex; -import org.apache.hudi.index.hbase.SparkHoodieHBaseIndex; -import org.apache.hudi.index.simple.SparkHoodieGlobalSimpleIndex; -import org.apache.hudi.index.simple.SparkHoodieSimpleIndex; import org.apache.hudi.table.HoodieTable; import org.apache.spark.api.java.JavaRDD; @SuppressWarnings("checkstyle:LineLength") -public abstract class SparkHoodieIndex extends HoodieIndex>, JavaRDD, JavaRDD> { +public abstract class SparkHoodieIndex> + extends HoodieIndex>, JavaRDD, JavaRDD> { protected SparkHoodieIndex(HoodieWriteConfig config) { super(config); } - public static SparkHoodieIndex createIndex(HoodieWriteConfig config) { - // first use index class config to create index. - if (!StringUtils.isNullOrEmpty(config.getIndexClass())) { - Object instance = ReflectionUtils.loadClass(config.getIndexClass(), config); - if (!(instance instanceof HoodieIndex)) { - throw new HoodieIndexException(config.getIndexClass() + " is not a subclass of HoodieIndex"); - } - return (SparkHoodieIndex) instance; - } - switch (config.getIndexType()) { - case HBASE: - return new SparkHoodieHBaseIndex<>(config); - case INMEMORY: - return new SparkInMemoryHashIndex(config); - case BLOOM: - return new SparkHoodieBloomIndex<>(config); - case GLOBAL_BLOOM: - return new SparkHoodieGlobalBloomIndex<>(config); - case SIMPLE: - return new SparkHoodieSimpleIndex(config); - case GLOBAL_SIMPLE: - return new SparkHoodieGlobalSimpleIndex(config); - default: - throw new HoodieIndexException("Index type unspecified, set " + config.getIndexType()); - } - } - - /** - * Whether index is global or not. - * @param config HoodieWriteConfig to use. - * @return {@code true} if index is a global one. else {@code false}. - */ - public static boolean isGlobalIndex(HoodieWriteConfig config) { - switch (config.getIndexType()) { - case HBASE: - return true; - case INMEMORY: - return true; - case BLOOM: - return false; - case GLOBAL_BLOOM: - return true; - case SIMPLE: - return false; - case GLOBAL_SIMPLE: - return true; - default: - return createIndex(config).isGlobal(); - } - } - @Override - @PublicAPIMethod(maturity = ApiMaturityLevel.STABLE) + @Deprecated + @PublicAPIMethod(maturity = ApiMaturityLevel.DEPRECATED) public abstract JavaRDD updateLocation(JavaRDD writeStatusRDD, HoodieEngineContext context, HoodieTable>, JavaRDD, JavaRDD> hoodieTable) throws HoodieIndexException; @Override - @PublicAPIMethod(maturity = ApiMaturityLevel.STABLE) + @Deprecated + @PublicAPIMethod(maturity = ApiMaturityLevel.DEPRECATED) public abstract JavaRDD> tagLocation(JavaRDD> records, HoodieEngineContext context, HoodieTable>, JavaRDD, JavaRDD> hoodieTable) throws HoodieIndexException; + + @Override + @PublicAPIMethod(maturity = ApiMaturityLevel.EVOLVING) + public HoodieData> tagLocation( + HoodieData> records, HoodieEngineContext context, + HoodieTable hoodieTable) throws HoodieIndexException { + return HoodieJavaRDD.of(tagLocation(HoodieJavaRDD.getJavaRDD(records), context, hoodieTable)); + } + + @Override + @PublicAPIMethod(maturity = ApiMaturityLevel.EVOLVING) + public HoodieData updateLocation( + HoodieData writeStatuses, HoodieEngineContext context, + HoodieTable hoodieTable) throws HoodieIndexException { + return HoodieJavaRDD.of(updateLocation(HoodieJavaRDD.getJavaRDD(writeStatuses), context, hoodieTable)); + } } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/SparkHoodieIndexFactory.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/SparkHoodieIndexFactory.java new file mode 100644 index 000000000..5e686463b --- /dev/null +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/SparkHoodieIndexFactory.java @@ -0,0 +1,103 @@ +/* + * 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.index; + +import org.apache.hudi.common.config.TypedProperties; +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.config.HoodieWriteConfig; +import org.apache.hudi.exception.HoodieIOException; +import org.apache.hudi.exception.HoodieIndexException; +import org.apache.hudi.index.bloom.HoodieBloomIndex; +import org.apache.hudi.index.bloom.HoodieGlobalBloomIndex; +import org.apache.hudi.index.bloom.SparkHoodieBloomIndexHelper; +import org.apache.hudi.index.hbase.SparkHoodieHBaseIndex; +import org.apache.hudi.index.inmemory.HoodieInMemoryHashIndex; +import org.apache.hudi.index.simple.HoodieGlobalSimpleIndex; +import org.apache.hudi.index.simple.HoodieSimpleIndex; +import org.apache.hudi.keygen.BaseKeyGenerator; +import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory; + +import java.io.IOException; + +/** + * A factory to generate Spark {@link HoodieIndex}. + */ +public final class SparkHoodieIndexFactory { + public static HoodieIndex createIndex(HoodieWriteConfig config) { + // first use index class config to create index. + if (!StringUtils.isNullOrEmpty(config.getIndexClass())) { + Object instance = ReflectionUtils.loadClass(config.getIndexClass(), config); + if (!(instance instanceof HoodieIndex)) { + throw new HoodieIndexException(config.getIndexClass() + " is not a subclass of HoodieIndex"); + } + return (HoodieIndex) instance; + } + switch (config.getIndexType()) { + case HBASE: + return new SparkHoodieHBaseIndex<>(config); + case INMEMORY: + return new HoodieInMemoryHashIndex<>(config); + case BLOOM: + return new HoodieBloomIndex<>(config, SparkHoodieBloomIndexHelper.getInstance()); + case GLOBAL_BLOOM: + return new HoodieGlobalBloomIndex<>(config, SparkHoodieBloomIndexHelper.getInstance()); + case SIMPLE: + return new HoodieSimpleIndex<>(config, getKeyGeneratorForSimpleIndex(config)); + case GLOBAL_SIMPLE: + return new HoodieGlobalSimpleIndex<>(config, getKeyGeneratorForSimpleIndex(config)); + default: + throw new HoodieIndexException("Index type unspecified, set " + config.getIndexType()); + } + } + + /** + * Whether index is global or not. + * @param config HoodieWriteConfig to use. + * @return {@code true} if index is a global one. else {@code false}. + */ + public static boolean isGlobalIndex(HoodieWriteConfig config) { + switch (config.getIndexType()) { + case HBASE: + return true; + case INMEMORY: + return true; + case BLOOM: + return false; + case GLOBAL_BLOOM: + return true; + case SIMPLE: + return false; + case GLOBAL_SIMPLE: + return true; + default: + return createIndex(config).isGlobal(); + } + } + + private static Option getKeyGeneratorForSimpleIndex(HoodieWriteConfig config) { + try { + return config.populateMetaFields() ? Option.empty() + : Option.of((BaseKeyGenerator) HoodieSparkKeyGeneratorFactory.createKeyGenerator(new TypedProperties(config.getProps()))); + } catch (IOException e) { + throw new HoodieIOException("KeyGenerator instantiation failed ", e); + } + } +} diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/SparkInMemoryHashIndex.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/SparkInMemoryHashIndex.java deleted file mode 100644 index 0cd839aa8..000000000 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/SparkInMemoryHashIndex.java +++ /dev/null @@ -1,135 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hudi.index; - -import org.apache.hudi.client.WriteStatus; -import org.apache.hudi.common.engine.HoodieEngineContext; -import org.apache.hudi.common.model.HoodieKey; -import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordLocation; -import org.apache.hudi.common.model.HoodieRecordPayload; -import org.apache.hudi.common.util.Option; -import org.apache.hudi.config.HoodieWriteConfig; -import org.apache.hudi.table.HoodieTable; - -import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.function.Function2; - -import java.util.ArrayList; -import java.util.Iterator; -import java.util.List; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ConcurrentMap; - -/** - * Hoodie Index implementation backed by an in-memory Hash map. - *

- * ONLY USE FOR LOCAL TESTING - */ -@SuppressWarnings("checkstyle:LineLength") -public class SparkInMemoryHashIndex extends SparkHoodieIndex { - - private static ConcurrentMap recordLocationMap; - - public SparkInMemoryHashIndex(HoodieWriteConfig config) { - super(config); - synchronized (SparkInMemoryHashIndex.class) { - if (recordLocationMap == null) { - recordLocationMap = new ConcurrentHashMap<>(); - } - } - } - - @Override - public JavaRDD> tagLocation(JavaRDD> recordRDD, HoodieEngineContext context, - HoodieTable>, JavaRDD, JavaRDD> hoodieTable) { - return recordRDD.mapPartitionsWithIndex(this.new LocationTagFunction(), true); - } - - @Override - public JavaRDD updateLocation(JavaRDD writeStatusRDD, - HoodieEngineContext context, - HoodieTable>, JavaRDD, JavaRDD> hoodieTable) { - return writeStatusRDD.map(writeStatus -> { - for (HoodieRecord record : writeStatus.getWrittenRecords()) { - if (!writeStatus.isErrored(record.getKey())) { - HoodieKey key = record.getKey(); - Option newLocation = record.getNewLocation(); - if (newLocation.isPresent()) { - recordLocationMap.put(key, newLocation.get()); - } else { - // Delete existing index for a deleted record - recordLocationMap.remove(key); - } - } - } - return writeStatus; - }); - } - - @Override - public boolean rollbackCommit(String instantTime) { - return true; - } - - /** - * Only looks up by recordKey. - */ - @Override - public boolean isGlobal() { - return true; - } - - /** - * Mapping is available in HBase already. - */ - @Override - public boolean canIndexLogFiles() { - return true; - } - - /** - * Index needs to be explicitly updated after storage write. - */ - @Override - public boolean isImplicitWithStorage() { - return false; - } - - /** - * Function that tags each HoodieRecord with an existing location, if known. - */ - class LocationTagFunction implements Function2>, Iterator>> { - - @Override - public Iterator> call(Integer partitionNum, Iterator> hoodieRecordIterator) { - List> taggedRecords = new ArrayList<>(); - while (hoodieRecordIterator.hasNext()) { - HoodieRecord rec = hoodieRecordIterator.next(); - if (recordLocationMap.containsKey(rec.getKey())) { - rec.unseal(); - rec.setCurrentLocation(recordLocationMap.get(rec.getKey())); - rec.seal(); - } - taggedRecords.add(rec); - } - return taggedRecords.iterator(); - } - } -} diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/bloom/SparkHoodieBloomIndex.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/bloom/SparkHoodieBloomIndex.java deleted file mode 100644 index 2dd485ebc..000000000 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/bloom/SparkHoodieBloomIndex.java +++ /dev/null @@ -1,299 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hudi.index.bloom; - -import org.apache.hudi.client.WriteStatus; -import org.apache.hudi.client.utils.SparkMemoryUtils; -import org.apache.hudi.common.engine.HoodieEngineContext; -import org.apache.hudi.common.model.HoodieKey; -import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordLocation; -import org.apache.hudi.common.model.HoodieRecordPayload; -import org.apache.hudi.common.util.Option; -import org.apache.hudi.common.util.collection.Pair; -import org.apache.hudi.config.HoodieWriteConfig; -import org.apache.hudi.exception.MetadataNotFoundException; -import org.apache.hudi.index.HoodieIndexUtils; -import org.apache.hudi.index.SparkHoodieIndex; -import org.apache.hudi.io.HoodieRangeInfoHandle; -import org.apache.hudi.table.HoodieTable; - -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; -import org.apache.spark.Partitioner; -import org.apache.spark.api.java.JavaPairRDD; -import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.storage.StorageLevel; - -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.stream.Collectors; - -import scala.Tuple2; - -import static java.util.stream.Collectors.groupingBy; -import static java.util.stream.Collectors.mapping; -import static java.util.stream.Collectors.toList; -import static org.apache.hudi.index.HoodieIndexUtils.getLatestBaseFilesForAllPartitions; - -/** - * Indexing mechanism based on bloom filter. Each parquet file includes its row_key bloom filter in its metadata. - */ -@SuppressWarnings("checkstyle:LineLength") -public class SparkHoodieBloomIndex extends SparkHoodieIndex { - - private static final Logger LOG = LogManager.getLogger(SparkHoodieBloomIndex.class); - - public SparkHoodieBloomIndex(HoodieWriteConfig config) { - super(config); - } - - @Override - public JavaRDD> tagLocation(JavaRDD> recordRDD, HoodieEngineContext context, - HoodieTable>, JavaRDD, JavaRDD> hoodieTable) { - - // Step 0: cache the input record RDD - if (config.getBloomIndexUseCaching()) { - recordRDD.persist(SparkMemoryUtils.getBloomIndexInputStorageLevel(config.getProps())); - } - - // Step 1: Extract out thinner JavaPairRDD of (partitionPath, recordKey) - JavaPairRDD partitionRecordKeyPairRDD = - recordRDD.mapToPair(record -> new Tuple2<>(record.getPartitionPath(), record.getRecordKey())); - - // Lookup indexes for all the partition/recordkey pair - JavaPairRDD keyFilenamePairRDD = - lookupIndex(partitionRecordKeyPairRDD, context, hoodieTable); - - // Cache the result, for subsequent stages. - if (config.getBloomIndexUseCaching()) { - keyFilenamePairRDD.persist(StorageLevel.MEMORY_AND_DISK_SER()); - } - if (LOG.isDebugEnabled()) { - long totalTaggedRecords = keyFilenamePairRDD.count(); - LOG.debug("Number of update records (ones tagged with a fileID): " + totalTaggedRecords); - } - - // Step 4: Tag the incoming records, as inserts or updates, by joining with existing record keys - // Cost: 4 sec. - JavaRDD> taggedRecordRDD = tagLocationBacktoRecords(keyFilenamePairRDD, recordRDD); - - if (config.getBloomIndexUseCaching()) { - recordRDD.unpersist(); // unpersist the input Record RDD - keyFilenamePairRDD.unpersist(); - } - return taggedRecordRDD; - } - - /** - * Lookup the location for each record key and return the pair for all record keys already - * present and drop the record keys if not present. - */ - private JavaPairRDD lookupIndex( - JavaPairRDD partitionRecordKeyPairRDD, final HoodieEngineContext context, - final HoodieTable hoodieTable) { - // Obtain records per partition, in the incoming records - Map recordsPerPartition = partitionRecordKeyPairRDD.countByKey(); - List affectedPartitionPathList = new ArrayList<>(recordsPerPartition.keySet()); - - // Step 2: Load all involved files as pairs - List> fileInfoList = - loadInvolvedFiles(affectedPartitionPathList, context, hoodieTable); - final Map> partitionToFileInfo = - fileInfoList.stream().collect(groupingBy(Tuple2::_1, mapping(Tuple2::_2, toList()))); - - // Step 3: Obtain a RDD, for each incoming record, that already exists, with the file id, - // that contains it. - JavaRDD> fileComparisonsRDD = - explodeRecordRDDWithFileComparisons(partitionToFileInfo, partitionRecordKeyPairRDD); - Map comparisonsPerFileGroup = - computeComparisonsPerFileGroup(recordsPerPartition, partitionToFileInfo, fileComparisonsRDD, context); - int inputParallelism = partitionRecordKeyPairRDD.partitions().size(); - int joinParallelism = Math.max(inputParallelism, config.getBloomIndexParallelism()); - LOG.info("InputParallelism: ${" + inputParallelism + "}, IndexParallelism: ${" - + config.getBloomIndexParallelism() + "}"); - return findMatchingFilesForRecordKeys(fileComparisonsRDD, joinParallelism, hoodieTable, - comparisonsPerFileGroup); - } - - /** - * Compute the estimated number of bloom filter comparisons to be performed on each file group. - */ - private Map computeComparisonsPerFileGroup(final Map recordsPerPartition, - final Map> partitionToFileInfo, - final JavaRDD> fileComparisonsRDD, - final HoodieEngineContext context) { - Map fileToComparisons; - if (config.getBloomIndexPruneByRanges()) { - // we will just try exploding the input and then count to determine comparisons - // FIX(vc): Only do sampling here and extrapolate? - context.setJobStatus(this.getClass().getSimpleName(), "Compute all comparisons needed between records and files"); - fileToComparisons = fileComparisonsRDD.mapToPair(t -> t).countByKey(); - } else { - fileToComparisons = new HashMap<>(); - partitionToFileInfo.forEach((key, value) -> { - for (BloomIndexFileInfo fileInfo : value) { - // each file needs to be compared against all the records coming into the partition - fileToComparisons.put(fileInfo.getFileId(), recordsPerPartition.get(key)); - } - }); - } - return fileToComparisons; - } - - /** - * Load all involved files as pair RDD. - */ - List> loadInvolvedFiles(List partitions, final HoodieEngineContext context, - final HoodieTable hoodieTable) { - - // Obtain the latest data files from all the partitions. - List> partitionPathFileIDList = getLatestBaseFilesForAllPartitions(partitions, context, hoodieTable).stream() - .map(pair -> Pair.of(pair.getKey(), pair.getValue().getFileId())) - .collect(toList()); - - if (config.getBloomIndexPruneByRanges()) { - // also obtain file ranges, if range pruning is enabled - context.setJobStatus(this.getClass().getName(), "Obtain key ranges for file slices (range pruning=on)"); - return context.map(partitionPathFileIDList, pf -> { - try { - HoodieRangeInfoHandle rangeInfoHandle = new HoodieRangeInfoHandle(config, hoodieTable, pf); - String[] minMaxKeys = rangeInfoHandle.getMinMaxKeys(); - return new Tuple2<>(pf.getKey(), new BloomIndexFileInfo(pf.getValue(), minMaxKeys[0], minMaxKeys[1])); - } catch (MetadataNotFoundException me) { - LOG.warn("Unable to find range metadata in file :" + pf); - return new Tuple2<>(pf.getKey(), new BloomIndexFileInfo(pf.getValue())); - } - }, Math.max(partitionPathFileIDList.size(), 1)); - } else { - return partitionPathFileIDList.stream() - .map(pf -> new Tuple2<>(pf.getKey(), new BloomIndexFileInfo(pf.getValue()))).collect(toList()); - } - } - - @Override - public boolean rollbackCommit(String instantTime) { - // Nope, don't need to do anything. - return true; - } - - /** - * This is not global, since we depend on the partitionPath to do the lookup. - */ - @Override - public boolean isGlobal() { - return false; - } - - /** - * No indexes into log files yet. - */ - @Override - public boolean canIndexLogFiles() { - return false; - } - - /** - * Bloom filters are stored, into the same data files. - */ - @Override - public boolean isImplicitWithStorage() { - return true; - } - - /** - * For each incoming record, produce N output records, 1 each for each file against which the record's key needs to be - * checked. For tables, where the keys have a definite insert order (e.g: timestamp as prefix), the number of files - * to be compared gets cut down a lot from range pruning. - *

- * Sub-partition to ensure the records can be looked up against files & also prune file<=>record comparisons based on - * recordKey ranges in the index info. - */ - JavaRDD> explodeRecordRDDWithFileComparisons( - final Map> partitionToFileIndexInfo, - JavaPairRDD partitionRecordKeyPairRDD) { - IndexFileFilter indexFileFilter = - config.useBloomIndexTreebasedFilter() ? new IntervalTreeBasedIndexFileFilter(partitionToFileIndexInfo) - : new ListBasedIndexFileFilter(partitionToFileIndexInfo); - - return partitionRecordKeyPairRDD.map(partitionRecordKeyPair -> { - String recordKey = partitionRecordKeyPair._2(); - String partitionPath = partitionRecordKeyPair._1(); - - return indexFileFilter.getMatchingFilesAndPartition(partitionPath, recordKey).stream() - .map(partitionFileIdPair -> new Tuple2<>(partitionFileIdPair.getRight(), - new HoodieKey(recordKey, partitionPath))) - .collect(Collectors.toList()); - }).flatMap(List::iterator); - } - - /** - * Find out pair. All workload grouped by file-level. - *

- * Join PairRDD(PartitionPath, RecordKey) and PairRDD(PartitionPath, File) & then repartition such that each RDD - * partition is a file, then for each file, we do (1) load bloom filter, (2) load rowKeys, (3) Tag rowKey - *

- * Make sure the parallelism is atleast the groupby parallelism for tagging location - */ - JavaPairRDD findMatchingFilesForRecordKeys( - JavaRDD> fileComparisonsRDD, - int shuffleParallelism, - HoodieTable hoodieTable, - Map fileGroupToComparisons) { - - if (config.useBloomIndexBucketizedChecking()) { - Partitioner partitioner = new BucketizedBloomCheckPartitioner(shuffleParallelism, fileGroupToComparisons, - config.getBloomIndexKeysPerBucket()); - - fileComparisonsRDD = fileComparisonsRDD.mapToPair(t -> new Tuple2<>(Pair.of(t._1, t._2.getRecordKey()), t)) - .repartitionAndSortWithinPartitions(partitioner).map(Tuple2::_2); - } else { - fileComparisonsRDD = fileComparisonsRDD.sortBy(Tuple2::_1, true, shuffleParallelism); - } - - return fileComparisonsRDD.mapPartitionsWithIndex(new HoodieBloomIndexCheckFunction(hoodieTable, config), true) - .flatMap(List::iterator).filter(lr -> lr.getMatchingRecordKeys().size() > 0) - .flatMapToPair(lookupResult -> lookupResult.getMatchingRecordKeys().stream() - .map(recordKey -> new Tuple2<>(new HoodieKey(recordKey, lookupResult.getPartitionPath()), - new HoodieRecordLocation(lookupResult.getBaseInstantTime(), lookupResult.getFileId()))) - .collect(Collectors.toList()).iterator()); - } - - - /** - * Tag the back to the original HoodieRecord RDD. - */ - protected JavaRDD> tagLocationBacktoRecords( - JavaPairRDD keyFilenamePairRDD, JavaRDD> recordRDD) { - JavaPairRDD> keyRecordPairRDD = - recordRDD.mapToPair(record -> new Tuple2<>(record.getKey(), record)); - // Here as the recordRDD might have more data than rowKeyRDD (some rowKeys' fileId is null), - // so we do left outer join. - return keyRecordPairRDD.leftOuterJoin(keyFilenamePairRDD).values() - .map(v1 -> HoodieIndexUtils.getTaggedRecord(v1._1, Option.ofNullable(v1._2.orNull()))); - } - - @Override - public JavaRDD updateLocation(JavaRDD writeStatusRDD, HoodieEngineContext context, - HoodieTable>, JavaRDD, JavaRDD> hoodieTable) { - return writeStatusRDD; - } -} diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/bloom/SparkHoodieBloomIndexHelper.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/bloom/SparkHoodieBloomIndexHelper.java new file mode 100644 index 000000000..bbb50d5cf --- /dev/null +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/bloom/SparkHoodieBloomIndexHelper.java @@ -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.hudi.index.bloom; + +import org.apache.hudi.common.data.HoodieData; +import org.apache.hudi.common.data.HoodiePairData; +import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.model.HoodieKey; +import org.apache.hudi.common.model.HoodieRecordLocation; +import org.apache.hudi.common.util.collection.ImmutablePair; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.data.HoodieJavaPairRDD; +import org.apache.hudi.data.HoodieJavaRDD; +import org.apache.hudi.table.HoodieTable; + +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; +import org.apache.spark.Partitioner; +import org.apache.spark.api.java.JavaRDD; + +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +import scala.Tuple2; + +/** + * Helper for {@link HoodieBloomIndex} containing Spark-specific logic. + */ +public class SparkHoodieBloomIndexHelper extends BaseHoodieBloomIndexHelper { + + private static final Logger LOG = LogManager.getLogger(SparkHoodieBloomIndexHelper.class); + + private static final SparkHoodieBloomIndexHelper SINGLETON_INSTANCE = + new SparkHoodieBloomIndexHelper(); + + private SparkHoodieBloomIndexHelper() { + } + + public static SparkHoodieBloomIndexHelper getInstance() { + return SINGLETON_INSTANCE; + } + + @Override + public HoodiePairData findMatchingFilesForRecordKeys( + HoodieWriteConfig config, HoodieEngineContext context, HoodieTable hoodieTable, + HoodiePairData partitionRecordKeyPairs, + HoodieData> fileComparisonPairs, + Map> partitionToFileInfo, + Map recordsPerPartition) { + JavaRDD> fileComparisonsRDD = + HoodieJavaRDD.getJavaRDD(fileComparisonPairs) + .map(pair -> new Tuple2<>(pair.getLeft(), pair.getRight())); + Map comparisonsPerFileGroup = computeComparisonsPerFileGroup( + config, recordsPerPartition, partitionToFileInfo, fileComparisonsRDD, context); + int inputParallelism = + HoodieJavaPairRDD.getJavaPairRDD(partitionRecordKeyPairs).partitions().size(); + int joinParallelism = Math.max(inputParallelism, config.getBloomIndexParallelism()); + LOG.info("InputParallelism: ${" + inputParallelism + "}, IndexParallelism: ${" + + config.getBloomIndexParallelism() + "}"); + + if (config.useBloomIndexBucketizedChecking()) { + Partitioner partitioner = new BucketizedBloomCheckPartitioner(joinParallelism, comparisonsPerFileGroup, + config.getBloomIndexKeysPerBucket()); + + fileComparisonsRDD = fileComparisonsRDD.mapToPair(t -> new Tuple2<>(Pair.of(t._1, t._2.getRecordKey()), t)) + .repartitionAndSortWithinPartitions(partitioner).map(Tuple2::_2); + } else { + fileComparisonsRDD = fileComparisonsRDD.sortBy(Tuple2::_1, true, joinParallelism); + } + + return HoodieJavaPairRDD.of(fileComparisonsRDD.mapPartitionsWithIndex(new HoodieBloomIndexCheckFunction(hoodieTable, config), true) + .flatMap(List::iterator).filter(lr -> lr.getMatchingRecordKeys().size() > 0) + .flatMapToPair(lookupResult -> lookupResult.getMatchingRecordKeys().stream() + .map(recordKey -> new Tuple2<>(new HoodieKey(recordKey, lookupResult.getPartitionPath()), + new HoodieRecordLocation(lookupResult.getBaseInstantTime(), lookupResult.getFileId()))) + .collect(Collectors.toList()).iterator())); + } + + /** + * Compute the estimated number of bloom filter comparisons to be performed on each file group. + */ + private Map computeComparisonsPerFileGroup( + final HoodieWriteConfig config, + final Map recordsPerPartition, + final Map> partitionToFileInfo, + final JavaRDD> fileComparisonsRDD, + final HoodieEngineContext context) { + Map fileToComparisons; + if (config.getBloomIndexPruneByRanges()) { + // we will just try exploding the input and then count to determine comparisons + // FIX(vc): Only do sampling here and extrapolate? + context.setJobStatus(this.getClass().getSimpleName(), "Compute all comparisons needed between records and files"); + fileToComparisons = fileComparisonsRDD.mapToPair(t -> t).countByKey(); + } else { + fileToComparisons = new HashMap<>(); + partitionToFileInfo.forEach((key, value) -> { + for (BloomIndexFileInfo fileInfo : value) { + // each file needs to be compared against all the records coming into the partition + fileToComparisons.put(fileInfo.getFileId(), recordsPerPartition.get(key)); + } + }); + } + return fileToComparisons; + } +} diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/hbase/SparkHoodieHBaseIndex.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/hbase/SparkHoodieHBaseIndex.java index 866d0d397..56dd49515 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/hbase/SparkHoodieHBaseIndex.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/hbase/SparkHoodieHBaseIndex.java @@ -21,6 +21,7 @@ package org.apache.hudi.index.hbase; import org.apache.hudi.client.WriteStatus; import org.apache.hudi.client.common.HoodieSparkEngineContext; import org.apache.hudi.client.utils.SparkMemoryUtils; +import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.EmptyHoodieRecordPayload; import org.apache.hudi.common.model.HoodieKey; @@ -35,9 +36,10 @@ import org.apache.hudi.common.util.RateLimiter; import org.apache.hudi.common.util.ReflectionUtils; import org.apache.hudi.config.HoodieHBaseIndexConfig; import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.data.HoodieJavaRDD; import org.apache.hudi.exception.HoodieDependentSystemUnavailableException; import org.apache.hudi.exception.HoodieIndexException; -import org.apache.hudi.index.SparkHoodieIndex; +import org.apache.hudi.index.HoodieIndex; import org.apache.hudi.table.HoodieTable; import org.apache.hadoop.conf.Configuration; @@ -83,7 +85,8 @@ import scala.Tuple2; /** * Hoodie Index implementation backed by HBase. */ -public class SparkHoodieHBaseIndex extends SparkHoodieIndex { +public class SparkHoodieHBaseIndex> + extends HoodieIndex>, JavaRDD, JavaRDD> { public static final String DEFAULT_SPARK_EXECUTOR_INSTANCES_CONFIG_NAME = "spark.executor.instances"; public static final String DEFAULT_SPARK_DYNAMIC_ALLOCATION_ENABLED_CONFIG_NAME = "spark.dynamicAllocation.enabled"; @@ -291,10 +294,11 @@ public class SparkHoodieHBaseIndex extends SparkH } @Override - public JavaRDD> tagLocation(JavaRDD> recordRDD, - HoodieEngineContext context, - HoodieTable>, JavaRDD, JavaRDD> hoodieTable) { - return recordRDD.mapPartitionsWithIndex(locationTagFunction(hoodieTable.getMetaClient()), true); + public HoodieData> tagLocation( + HoodieData> records, HoodieEngineContext context, + HoodieTable hoodieTable) { + return HoodieJavaRDD.of(HoodieJavaRDD.getJavaRDD(records) + .mapPartitionsWithIndex(locationTagFunction(hoodieTable.getMetaClient()), true)); } private Function2, Iterator> updateLocationFunction() { @@ -395,16 +399,17 @@ public class SparkHoodieHBaseIndex extends SparkH } @Override - public JavaRDD updateLocation(JavaRDD writeStatusRDD, HoodieEngineContext context, - HoodieTable>, JavaRDD, - JavaRDD> hoodieTable) { - final Option desiredQPSFraction = calculateQPSFraction(writeStatusRDD); + public HoodieData updateLocation( + HoodieData writeStatus, HoodieEngineContext context, + HoodieTable hoodieTable) { + JavaRDD writeStatusRDD = HoodieJavaRDD.getJavaRDD(writeStatus); + final Option desiredQPSFraction = calculateQPSFraction(writeStatusRDD); final Map fileIdPartitionMap = mapFileWithInsertsToUniquePartition(writeStatusRDD); JavaRDD partitionedRDD = this.numWriteStatusWithInserts == 0 ? writeStatusRDD : - writeStatusRDD.mapToPair(w -> new Tuple2<>(w.getFileId(), w)) - .partitionBy(new WriteStatusPartitioner(fileIdPartitionMap, - this.numWriteStatusWithInserts)) - .map(w -> w._2()); + writeStatusRDD.mapToPair(w -> new Tuple2<>(w.getFileId(), w)) + .partitionBy(new WriteStatusPartitioner(fileIdPartitionMap, + this.numWriteStatusWithInserts)) + .map(w -> w._2()); JavaSparkContext jsc = HoodieSparkEngineContext.getSparkContext(context); acquireQPSResourcesAndSetBatchSize(desiredQPSFraction, jsc); JavaRDD writeStatusJavaRDD = partitionedRDD.mapPartitionsWithIndex(updateLocationFunction(), @@ -414,7 +419,7 @@ public class SparkHoodieHBaseIndex extends SparkH // force trigger update location(hbase puts) writeStatusJavaRDD.count(); this.hBaseIndexQPSResourceAllocator.releaseQPSResources(); - return writeStatusJavaRDD; + return HoodieJavaRDD.of(writeStatusJavaRDD); } private Option calculateQPSFraction(JavaRDD writeStatusRDD) { diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/simple/SparkHoodieSimpleIndex.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/simple/SparkHoodieSimpleIndex.java deleted file mode 100644 index 5add5825c..000000000 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/simple/SparkHoodieSimpleIndex.java +++ /dev/null @@ -1,165 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hudi.index.simple; - -import org.apache.hudi.client.WriteStatus; -import org.apache.hudi.client.utils.SparkMemoryUtils; -import org.apache.hudi.client.common.HoodieSparkEngineContext; -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.HoodieKey; -import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordLocation; -import org.apache.hudi.common.model.HoodieRecordPayload; -import org.apache.hudi.common.util.Option; -import org.apache.hudi.common.util.collection.Pair; -import org.apache.hudi.config.HoodieWriteConfig; -import org.apache.hudi.exception.HoodieIOException; -import org.apache.hudi.index.HoodieIndexUtils; -import org.apache.hudi.index.SparkHoodieIndex; -import org.apache.hudi.io.HoodieKeyLocationFetchHandle; -import org.apache.hudi.keygen.BaseKeyGenerator; -import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory; -import org.apache.hudi.table.HoodieTable; - -import org.apache.spark.api.java.JavaPairRDD; -import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; - -import java.io.IOException; -import java.util.List; - -import scala.Tuple2; - -import static org.apache.hudi.index.HoodieIndexUtils.getLatestBaseFilesForAllPartitions; - -/** - * A simple index which reads interested fields(record key and partition path) from base files and - * joins with incoming records to find the tagged location. - * - * @param - */ -@SuppressWarnings("checkstyle:LineLength") -public class SparkHoodieSimpleIndex extends SparkHoodieIndex { - - public SparkHoodieSimpleIndex(HoodieWriteConfig config) { - super(config); - } - - @Override - public JavaRDD updateLocation(JavaRDD writeStatusRDD, HoodieEngineContext context, - HoodieTable>, JavaRDD, JavaRDD> hoodieTable) { - return writeStatusRDD; - } - - @Override - public boolean rollbackCommit(String commitTime) { - return true; - } - - @Override - public boolean isGlobal() { - return false; - } - - @Override - public boolean canIndexLogFiles() { - return false; - } - - @Override - public boolean isImplicitWithStorage() { - return true; - } - - @Override - public JavaRDD> tagLocation(JavaRDD> recordRDD, - HoodieEngineContext context, - HoodieTable>, JavaRDD, JavaRDD> hoodieTable) { - return tagLocationInternal(recordRDD, context, hoodieTable); - } - - /** - * Tags records location for incoming records. - * - * @param inputRecordRDD {@link JavaRDD} of incoming records - * @param context instance of {@link HoodieEngineContext} to use - * @param hoodieTable instance of {@link HoodieTable} to use - * @return {@link JavaRDD} of records with record locations set - */ - protected JavaRDD> tagLocationInternal(JavaRDD> inputRecordRDD, HoodieEngineContext context, - HoodieTable>, JavaRDD, JavaRDD> hoodieTable) { - if (config.getSimpleIndexUseCaching()) { - inputRecordRDD.persist(SparkMemoryUtils.getSimpleIndexInputStorageLevel(config.getProps())); - } - - JavaPairRDD> keyedInputRecordRDD = inputRecordRDD.mapToPair(record -> new Tuple2<>(record.getKey(), record)); - JavaPairRDD existingLocationsOnTable = fetchRecordLocationsForAffectedPartitions(keyedInputRecordRDD.keys(), context, hoodieTable, - config.getSimpleIndexParallelism()); - - JavaRDD> taggedRecordRDD = keyedInputRecordRDD.leftOuterJoin(existingLocationsOnTable) - .map(entry -> { - final HoodieRecord untaggedRecord = entry._2._1; - final Option location = Option.ofNullable(entry._2._2.orNull()); - return HoodieIndexUtils.getTaggedRecord(untaggedRecord, location); - }); - - if (config.getSimpleIndexUseCaching()) { - inputRecordRDD.unpersist(); - } - return taggedRecordRDD; - } - - /** - * Fetch record locations for passed in {@link HoodieKey}s. - * - * @param hoodieKeys {@link JavaRDD} of {@link HoodieKey}s for which locations are fetched - * @param context instance of {@link HoodieEngineContext} to use - * @param hoodieTable instance of {@link HoodieTable} of interest - * @param parallelism parallelism to use - * @return {@link JavaPairRDD} of {@link HoodieKey} and {@link HoodieRecordLocation} - */ - protected JavaPairRDD fetchRecordLocationsForAffectedPartitions(JavaRDD hoodieKeys, - HoodieEngineContext context, - HoodieTable>, JavaRDD, JavaRDD> hoodieTable, - int parallelism) { - List affectedPartitionPathList = hoodieKeys.map(HoodieKey::getPartitionPath).distinct().collect(); - List> latestBaseFiles = getLatestBaseFilesForAllPartitions(affectedPartitionPathList, context, hoodieTable); - return fetchRecordLocations(context, hoodieTable, parallelism, latestBaseFiles); - } - - protected JavaPairRDD fetchRecordLocations(HoodieEngineContext context, - HoodieTable>, JavaRDD, JavaRDD> hoodieTable, - int parallelism, - List> baseFiles) { - JavaSparkContext jsc = HoodieSparkEngineContext.getSparkContext(context); - int fetchParallelism = Math.max(1, Math.max(baseFiles.size(), parallelism)); - - try { - Option keyGeneratorOpt = config.populateMetaFields() ? Option.empty() - : Option.of((BaseKeyGenerator) HoodieSparkKeyGeneratorFactory.createKeyGenerator(new TypedProperties(config.getProps()))); - return jsc.parallelize(baseFiles, fetchParallelism) - .flatMapToPair(partitionPathBaseFile -> new HoodieKeyLocationFetchHandle(config, hoodieTable, partitionPathBaseFile, keyGeneratorOpt) - .locations().map(x -> Tuple2.apply(((Pair)x).getLeft(), ((Pair)x).getRight())).iterator()); - } catch (IOException e) { - throw new HoodieIOException("KeyGenerator instantiation failed ", e); - } - } -} diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkTable.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkTable.java index e3e732b47..abbfd3167 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkTable.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkTable.java @@ -33,7 +33,7 @@ import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.exception.HoodieMetadataException; import org.apache.hudi.index.HoodieIndex; -import org.apache.hudi.index.SparkHoodieIndex; +import org.apache.hudi.index.SparkHoodieIndexFactory; import org.apache.hudi.metadata.HoodieTableMetadata; import org.apache.hudi.metadata.HoodieTableMetadataWriter; import org.apache.hudi.metadata.SparkHoodieBackedTableMetadataWriter; @@ -102,8 +102,8 @@ public abstract class HoodieSparkTable } @Override - protected HoodieIndex>, JavaRDD, JavaRDD> getIndex(HoodieWriteConfig config, HoodieEngineContext context) { - return SparkHoodieIndex.createIndex(config); + protected HoodieIndex getIndex(HoodieWriteConfig config, HoodieEngineContext context) { + return SparkHoodieIndexFactory.createIndex(config); } /** diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/SparkBootstrapCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/SparkBootstrapCommitActionExecutor.java index 457fdaee5..bdeb041b3 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/SparkBootstrapCommitActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/SparkBootstrapCommitActionExecutor.java @@ -55,6 +55,7 @@ import org.apache.hudi.common.util.ValidationUtils; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.common.util.queue.BoundedInMemoryExecutor; import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.data.HoodieJavaRDD; import org.apache.hudi.exception.HoodieCommitException; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.exception.HoodieIOException; @@ -182,8 +183,8 @@ public class SparkBootstrapCommitActionExecutor writeStatusRDD = writeStatusRDD.persist(SparkMemoryUtils.getWriteStatusStorageLevel(config.getProps())); Instant indexStartTime = Instant.now(); // Update the index back - JavaRDD statuses = table.getIndex().updateLocation(writeStatusRDD, context, - table); + JavaRDD statuses = HoodieJavaRDD.getJavaRDD( + table.getIndex().updateLocation(HoodieJavaRDD.of(writeStatusRDD), context, table)); result.setIndexUpdateDuration(Duration.between(indexStartTime, Instant.now())); result.setWriteStatuses(statuses); commitOnAutoCommit(result); diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/BaseSparkCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/BaseSparkCommitActionExecutor.java index 18c659373..0b673b890 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/BaseSparkCommitActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/BaseSparkCommitActionExecutor.java @@ -38,6 +38,7 @@ import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.ReflectionUtils; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.data.HoodieJavaRDD; import org.apache.hudi.exception.HoodieCommitException; import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.exception.HoodieUpsertException; @@ -231,7 +232,8 @@ public abstract class BaseSparkCommitActionExecutor statuses = table.getIndex().updateLocation(writeStatusRDD, context, table); + JavaRDD statuses = HoodieJavaRDD.getJavaRDD( + table.getIndex().updateLocation(HoodieJavaRDD.of(writeStatusRDD), context, table)); result.setIndexUpdateDuration(Duration.between(indexStartTime, Instant.now())); result.setWriteStatuses(statuses); return statuses; diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkDeleteHelper.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkDeleteHelper.java index 83ead0508..5c3b4ca22 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkDeleteHelper.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkDeleteHelper.java @@ -27,6 +27,7 @@ import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; 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.HoodieUpsertException; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.WorkloadProfile; @@ -95,8 +96,8 @@ public class SparkDeleteHelper extends dedupedKeys.map(key -> new HoodieRecord(key, new EmptyHoodieRecordPayload())); Instant beginTag = Instant.now(); // perform index loop up to get existing location of records - JavaRDD> taggedRecords = - table.getIndex().tagLocation(dedupedRecords, context, table); + JavaRDD> taggedRecords = HoodieJavaRDD.getJavaRDD( + table.getIndex().tagLocation(HoodieJavaRDD.of(dedupedRecords), context, table)); Duration tagLocationDuration = Duration.between(beginTag, Instant.now()); // filter out non existent keys/records diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkWriteHelper.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkWriteHelper.java index 38820be53..f4eff44a2 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkWriteHelper.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkWriteHelper.java @@ -19,10 +19,13 @@ package org.apache.hudi.table.action.commit; import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.data.HoodieJavaRDD; import org.apache.hudi.index.HoodieIndex; +import org.apache.hudi.table.HoodieTable; import org.apache.spark.api.java.JavaRDD; @@ -47,9 +50,15 @@ public class SparkWriteHelper extends AbstractW } @Override - public JavaRDD> deduplicateRecords(JavaRDD> records, - HoodieIndex>, JavaRDD, JavaRDD> index, - int parallelism) { + protected JavaRDD> tag(JavaRDD> dedupedRecords, HoodieEngineContext context, + HoodieTable>, JavaRDD, JavaRDD> table) { + return HoodieJavaRDD.getJavaRDD( + table.getIndex().tagLocation(HoodieJavaRDD.of(dedupedRecords), context, table)); + } + + @Override + public JavaRDD> deduplicateRecords( + JavaRDD> records, HoodieIndex index, int parallelism) { boolean isIndexingGlobal = index.isGlobal(); return records.mapToPair(record -> { HoodieKey hoodieKey = record.getKey(); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHBaseIndex.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHBaseIndex.java index 0b0f356f3..db17ceae9 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHBaseIndex.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHBaseIndex.java @@ -156,7 +156,7 @@ public class TestHBaseIndex extends SparkClientFunctionalTestHarness { HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient); // Test tagLocation without any entries in index - JavaRDD records1 = index.tagLocation(writeRecords, context(), hoodieTable); + JavaRDD records1 = tagLocation(index, writeRecords, hoodieTable); assertEquals(0, records1.filter(record -> record.isCurrentLocationKnown()).count()); // Insert 200 records @@ -165,7 +165,7 @@ public class TestHBaseIndex extends SparkClientFunctionalTestHarness { assertNoWriteErrors(writeStatues.collect()); // Now tagLocation for these records, hbaseIndex should not tag them since commit never occurred - JavaRDD records2 = index.tagLocation(writeRecords, context(), hoodieTable); + JavaRDD records2 = tagLocation(index, writeRecords, hoodieTable); assertEquals(0, records2.filter(record -> record.isCurrentLocationKnown()).count()); // Now commit this & update location of records inserted and validate no errors @@ -173,7 +173,7 @@ public class TestHBaseIndex extends SparkClientFunctionalTestHarness { // Now tagLocation for these records, hbaseIndex should tag them correctly metaClient = HoodieTableMetaClient.reload(metaClient); hoodieTable = HoodieSparkTable.create(config, context, metaClient); - List records3 = index.tagLocation(writeRecords, context(), hoodieTable).collect(); + List records3 = tagLocation(index, writeRecords, hoodieTable).collect(); assertEquals(numRecords, records3.stream().filter(record -> record.isCurrentLocationKnown()).count()); assertEquals(numRecords, records3.stream().map(record -> record.getKey().getRecordKey()).distinct().count()); assertEquals(numRecords, records3.stream().filter(record -> (record.getCurrentLocation() != null @@ -207,17 +207,17 @@ public class TestHBaseIndex extends SparkClientFunctionalTestHarness { metaClient = HoodieTableMetaClient.reload(metaClient); HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient); - JavaRDD oldHoodieRecord = index.tagLocation(oldWriteRecords, context, hoodieTable); + JavaRDD oldHoodieRecord = tagLocation(index, oldWriteRecords, hoodieTable); assertEquals(0, oldHoodieRecord.filter(record -> record.isCurrentLocationKnown()).count()); writeClient.startCommitWithTime(newCommitTime); JavaRDD writeStatues = writeClient.upsert(oldWriteRecords, newCommitTime); writeClient.commit(newCommitTime, writeStatues); assertNoWriteErrors(writeStatues.collect()); - index.updateLocation(writeStatues, context, hoodieTable); + updateLocation(index, writeStatues, hoodieTable); metaClient = HoodieTableMetaClient.reload(metaClient); hoodieTable = HoodieSparkTable.create(config, context, metaClient); - List taggedRecords = index.tagLocation(newWriteRecords, context, hoodieTable).collect(); + List taggedRecords = tagLocation(index, newWriteRecords, hoodieTable).collect(); assertEquals(numRecords * 2L, taggedRecords.stream().count()); // Verify the number of deleted records assertEquals(numRecords, taggedRecords.stream().filter(record -> record.getKey().getPartitionPath().equals(oldPartitionPath) @@ -227,7 +227,7 @@ public class TestHBaseIndex extends SparkClientFunctionalTestHarness { // not allowed path change test index = new SparkHoodieHBaseIndex<>(getConfig(false, false)); - List notAllowPathChangeRecords = index.tagLocation(newWriteRecords, context, hoodieTable).collect(); + List notAllowPathChangeRecords = tagLocation(index, newWriteRecords, hoodieTable).collect(); assertEquals(numRecords, notAllowPathChangeRecords.stream().count()); assertEquals(numRecords, taggedRecords.stream().filter(hoodieRecord -> hoodieRecord.isCurrentLocationKnown() && hoodieRecord.getKey().getPartitionPath().equals(oldPartitionPath)).count()); @@ -250,7 +250,7 @@ public class TestHBaseIndex extends SparkClientFunctionalTestHarness { HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient); JavaRDD writeStatues = writeClient.upsert(writeRecords, newCommitTime); - index.tagLocation(writeRecords, context(), hoodieTable); + tagLocation(index, writeRecords, hoodieTable); // Duplicate upsert and ensure correctness is maintained // We are trying to approximately imitate the case when the RDD is recomputed. For RDD creating, driver code is not @@ -266,7 +266,7 @@ public class TestHBaseIndex extends SparkClientFunctionalTestHarness { // Now tagLocation for these records, hbaseIndex should tag them correctly metaClient = HoodieTableMetaClient.reload(metaClient); hoodieTable = HoodieSparkTable.create(config, context, metaClient); - List taggedRecords = index.tagLocation(writeRecords, context(), hoodieTable).collect(); + List taggedRecords = tagLocation(index, writeRecords, hoodieTable).collect(); assertEquals(numRecords, taggedRecords.stream().filter(HoodieRecord::isCurrentLocationKnown).count()); assertEquals(numRecords, taggedRecords.stream().map(record -> record.getKey().getRecordKey()).distinct().count()); assertEquals(numRecords, taggedRecords.stream().filter(record -> (record.getCurrentLocation() != null @@ -295,22 +295,22 @@ public class TestHBaseIndex extends SparkClientFunctionalTestHarness { // first commit old record metaClient = HoodieTableMetaClient.reload(metaClient); HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient); - List beforeFirstTaggedRecords = index.tagLocation(oldWriteRecords, context, hoodieTable).collect(); + List beforeFirstTaggedRecords = tagLocation(index, oldWriteRecords, hoodieTable).collect(); JavaRDD oldWriteStatues = writeClient.upsert(oldWriteRecords, firstCommitTime); - index.updateLocation(oldWriteStatues, context, hoodieTable); + updateLocation(index, oldWriteStatues, hoodieTable); writeClient.commit(firstCommitTime, oldWriteStatues); - List afterFirstTaggedRecords = index.tagLocation(oldWriteRecords, context, hoodieTable).collect(); + List afterFirstTaggedRecords = tagLocation(index, oldWriteRecords, hoodieTable).collect(); metaClient = HoodieTableMetaClient.reload(metaClient); hoodieTable = HoodieSparkTable.create(config, context, metaClient); final String secondCommitTime = writeClient.startCommit(); - List beforeSecondTaggedRecords = index.tagLocation(newWriteRecords, context, hoodieTable).collect(); + List beforeSecondTaggedRecords = tagLocation(index, newWriteRecords, hoodieTable).collect(); JavaRDD newWriteStatues = writeClient.upsert(newWriteRecords, secondCommitTime); - index.updateLocation(newWriteStatues, context, hoodieTable); + updateLocation(index, newWriteStatues, hoodieTable); writeClient.commit(secondCommitTime, newWriteStatues); - List afterSecondTaggedRecords = index.tagLocation(newWriteRecords, context, hoodieTable).collect(); + List afterSecondTaggedRecords = tagLocation(index, newWriteRecords, hoodieTable).collect(); writeClient.rollback(secondCommitTime); - List afterRollback = index.tagLocation(newWriteRecords, context, hoodieTable).collect(); + List afterRollback = tagLocation(index, newWriteRecords, hoodieTable).collect(); // Verify the first commit assertEquals(numRecords, beforeFirstTaggedRecords.stream().filter(record -> record.getCurrentLocation() == null).count()); @@ -355,7 +355,7 @@ public class TestHBaseIndex extends SparkClientFunctionalTestHarness { writeClient.commit(newCommitTime, writeStatues); HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient); // Now tagLocation for these records, hbaseIndex should tag them - List records2 = index.tagLocation(writeRecords, context(), hoodieTable).collect(); + List records2 = tagLocation(index, writeRecords, hoodieTable).collect(); assertEquals(numRecords, records2.stream().filter(HoodieRecord::isCurrentLocationKnown).count()); // check tagged records are tagged with correct fileIds @@ -371,7 +371,7 @@ public class TestHBaseIndex extends SparkClientFunctionalTestHarness { hoodieTable = HoodieSparkTable.create(config, context, metaClient); // Now tagLocation for these records, hbaseIndex should not tag them since it was a rolled // back commit - List records3 = index.tagLocation(writeRecords, context(), hoodieTable).collect(); + List records3 = tagLocation(index, writeRecords, hoodieTable).collect(); assertEquals(0, records3.stream().filter(HoodieRecord::isCurrentLocationKnown).count()); assertEquals(0, records3.stream().filter(record -> record.getCurrentLocation() != null).count()); } @@ -397,7 +397,7 @@ public class TestHBaseIndex extends SparkClientFunctionalTestHarness { // verify location is tagged. HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient); - JavaRDD javaRDD0 = index.tagLocation(invalidWriteRecords, context(), hoodieTable); + JavaRDD javaRDD0 = tagLocation(index, invalidWriteRecords, hoodieTable); assert (javaRDD0.collect().size() == 1); // one record present assert (javaRDD0.filter(HoodieRecord::isCurrentLocationKnown).collect().size() == 1); // it is tagged assert (javaRDD0.collect().get(0).getCurrentLocation().getInstantTime().equals(invalidCommit)); @@ -408,11 +408,11 @@ public class TestHBaseIndex extends SparkClientFunctionalTestHarness { // Now tagLocation for the valid records, hbaseIndex should tag them metaClient = HoodieTableMetaClient.reload(metaClient); hoodieTable = HoodieSparkTable.create(config, context, metaClient); - JavaRDD javaRDD1 = index.tagLocation(writeRecords, context(), hoodieTable); + JavaRDD javaRDD1 = tagLocation(index, writeRecords, hoodieTable); assert (javaRDD1.filter(HoodieRecord::isCurrentLocationKnown).collect().size() == 199); // tagLocation for the invalid record - commit is not present in timeline due to rollback. - JavaRDD javaRDD2 = index.tagLocation(invalidWriteRecords, context(), hoodieTable); + JavaRDD javaRDD2 = tagLocation(index, invalidWriteRecords, hoodieTable); assert (javaRDD2.collect().size() == 1); // one record present assert (javaRDD2.filter(HoodieRecord::isCurrentLocationKnown).collect().size() == 0); // it is not tagged } @@ -442,7 +442,7 @@ public class TestHBaseIndex extends SparkClientFunctionalTestHarness { // Now tagLocation for the first set of rolledback records, hbaseIndex should tag them metaClient = HoodieTableMetaClient.reload(metaClient); HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient); - JavaRDD javaRDD1 = index.tagLocation(writeRecords1, context(), hoodieTable); + JavaRDD javaRDD1 = tagLocation(index, writeRecords1, hoodieTable); assert (javaRDD1.filter(HoodieRecord::isCurrentLocationKnown).collect().size() == 20); } @@ -492,7 +492,7 @@ public class TestHBaseIndex extends SparkClientFunctionalTestHarness { // tagLocation for the first set of records (for the archived commit), hbaseIndex should tag them as valid metaClient = HoodieTableMetaClient.reload(metaClient); HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient); - JavaRDD javaRDD1 = index.tagLocation(writeRecords1, context(), hoodieTable); + JavaRDD javaRDD1 = tagLocation(index, writeRecords1, hoodieTable); assertEquals(20, javaRDD1.filter(HoodieRecord::isCurrentLocationKnown).collect().size()); } @@ -524,7 +524,7 @@ public class TestHBaseIndex extends SparkClientFunctionalTestHarness { assertNoWriteErrors(writeStatues.collect()); // Now tagLocation for these records, hbaseIndex should tag them - index.tagLocation(writeRecords, context(), hoodieTable); + tagLocation(index, writeRecords, hoodieTable); // 3 batches should be executed given batchSize = 100 and parallelism = 1 verify(table, times(3)).get((List) any()); @@ -562,7 +562,7 @@ public class TestHBaseIndex extends SparkClientFunctionalTestHarness { // Get all the files generated int numberOfDataFileIds = (int) writeStatues.map(status -> status.getFileId()).distinct().count(); - index.updateLocation(writeStatues, context(), hoodieTable); + updateLocation(index, writeStatues, hoodieTable); // 3 batches should be executed given batchSize = 100 and <=numberOfDataFileIds getting updated, // so each fileId ideally gets updates verify(table, atMost(numberOfDataFileIds)).put((List) any()); @@ -696,7 +696,7 @@ public class TestHBaseIndex extends SparkClientFunctionalTestHarness { HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient); // Test tagLocation without any entries in index - JavaRDD records1 = index.tagLocation(writeRecords, context(), hoodieTable); + JavaRDD records1 = tagLocation(index, writeRecords, hoodieTable); assertEquals(0, records1.filter(record -> record.isCurrentLocationKnown()).count()); // Insert 200 records writeClient.startCommitWithTime(newCommitTime); @@ -705,7 +705,7 @@ public class TestHBaseIndex extends SparkClientFunctionalTestHarness { // Now tagLocation for these records, hbaseIndex should not tag them since it was a failed // commit - JavaRDD records2 = index.tagLocation(writeRecords, context(), hoodieTable); + JavaRDD records2 = tagLocation(index, writeRecords, hoodieTable); assertEquals(0, records2.filter(record -> record.isCurrentLocationKnown()).count()); // Now commit this & update location of records inserted and validate no errors @@ -713,7 +713,7 @@ public class TestHBaseIndex extends SparkClientFunctionalTestHarness { // Now tagLocation for these records, hbaseIndex should tag them correctly metaClient = HoodieTableMetaClient.reload(metaClient); hoodieTable = HoodieSparkTable.create(config, context, metaClient); - List records3 = index.tagLocation(writeRecords, context(), hoodieTable).collect(); + List records3 = tagLocation(index, writeRecords, hoodieTable).collect(); assertEquals(numRecords, records3.stream().filter(record -> record.isCurrentLocationKnown()).count()); assertEquals(numRecords, records3.stream().map(record -> record.getKey().getRecordKey()).distinct().count()); assertEquals(numRecords, records3.stream().filter(record -> (record.getCurrentLocation() != null @@ -736,7 +736,7 @@ public class TestHBaseIndex extends SparkClientFunctionalTestHarness { HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient); // Test tagLocation without any entries in index - JavaRDD records1 = index.tagLocation(writeRecords, context(), hoodieTable); + JavaRDD records1 = tagLocation(index, writeRecords, hoodieTable); assertEquals(0, records1.filter(record -> record.isCurrentLocationKnown()).count()); // Insert records @@ -748,7 +748,7 @@ public class TestHBaseIndex extends SparkClientFunctionalTestHarness { // Now tagLocation for these records, hbaseIndex should tag them correctly metaClient = HoodieTableMetaClient.reload(metaClient); hoodieTable = HoodieSparkTable.create(config, context, metaClient); - List records2 = index.tagLocation(writeRecords, context(), hoodieTable).collect(); + List records2 = tagLocation(index, writeRecords, hoodieTable).collect(); assertEquals(numRecords, records2.stream().filter(record -> record.isCurrentLocationKnown()).count()); assertEquals(numRecords, records2.stream().map(record -> record.getKey().getRecordKey()).distinct().count()); assertEquals(numRecords, records2.stream().filter(record -> (record.getCurrentLocation() != null @@ -766,12 +766,12 @@ public class TestHBaseIndex extends SparkClientFunctionalTestHarness { // if not for this caching, due to RDD chaining/lineage, first time update is called again when subsequent update is called. // So caching here to break the chain and so future update does not re-trigger update of older Rdd. deleteWriteStatues.cache(); - JavaRDD deleteStatus = index.updateLocation(deleteWriteStatues, context(), hoodieTable); + JavaRDD deleteStatus = updateLocation(index, deleteWriteStatues, hoodieTable); assertEquals(deleteStatus.count(), deleteWriteStatues.count()); assertNoWriteErrors(deleteStatus.collect()); // Ensure no records can be tagged - List records3 = index.tagLocation(writeRecords, context(), hoodieTable).collect(); + List records3 = tagLocation(index, writeRecords, hoodieTable).collect(); assertEquals(0, records3.stream().filter(record -> record.isCurrentLocationKnown()).count()); assertEquals(numRecords, records3.stream().map(record -> record.getKey().getRecordKey()).distinct().count()); assertEquals(0, records3.stream().filter(record -> (record.getCurrentLocation() != null diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieIndex.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieIndex.java index 824d74218..5c75bb565 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieIndex.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieIndex.java @@ -135,7 +135,7 @@ public class TestHoodieIndex extends HoodieClientTestHarness { HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient); // Test tagLocation without any entries in index - JavaRDD javaRDD = (JavaRDD) index.tagLocation(writeRecords, context, hoodieTable); + JavaRDD javaRDD = tagLocation(index, writeRecords, hoodieTable); assert (javaRDD.filter(record -> record.isCurrentLocationKnown()).collect().size() == 0); // Insert totalRecords records @@ -145,14 +145,14 @@ public class TestHoodieIndex extends HoodieClientTestHarness { // Now tagLocation for these records, index should not tag them since it was a failed // commit - javaRDD = (JavaRDD) index.tagLocation(writeRecords, context, hoodieTable); + javaRDD = tagLocation(index, writeRecords, hoodieTable); assert (javaRDD.filter(record -> record.isCurrentLocationKnown()).collect().size() == 0); // Now commit this & update location of records inserted and validate no errors writeClient.commit(newCommitTime, writeStatues); // Now tagLocation for these records, index should tag them correctly metaClient = HoodieTableMetaClient.reload(metaClient); hoodieTable = HoodieSparkTable.create(config, context, metaClient); - javaRDD = (JavaRDD) index.tagLocation(writeRecords, context, hoodieTable); + javaRDD = tagLocation(index, writeRecords, hoodieTable); Map recordKeyToPartitionPathMap = new HashMap(); List hoodieRecords = writeRecords.collect(); hoodieRecords.forEach(entry -> recordKeyToPartitionPathMap.put(entry.getRecordKey(), entry.getPartitionPath())); @@ -185,7 +185,7 @@ public class TestHoodieIndex extends HoodieClientTestHarness { writeClient.startCommitWithTime(newCommitTime); JavaRDD writeStatues = writeClient.upsert(writeRecords, newCommitTime); - JavaRDD javaRDD1 = (JavaRDD) index.tagLocation(writeRecords, context, hoodieTable); + JavaRDD javaRDD1 = tagLocation(index, writeRecords, hoodieTable); // Duplicate upsert and ensure correctness is maintained // We are trying to approximately imitate the case when the RDD is recomputed. For RDD creating, driver code is not @@ -201,7 +201,7 @@ public class TestHoodieIndex extends HoodieClientTestHarness { // Now tagLocation for these records, hbaseIndex should tag them correctly metaClient = HoodieTableMetaClient.reload(metaClient); hoodieTable = HoodieSparkTable.create(config, context, metaClient); - JavaRDD javaRDD = (JavaRDD) index.tagLocation(writeRecords, context, hoodieTable); + JavaRDD javaRDD = tagLocation(index, writeRecords, hoodieTable); Map recordKeyToPartitionPathMap = new HashMap(); List hoodieRecords = writeRecords.collect(); @@ -241,7 +241,7 @@ public class TestHoodieIndex extends HoodieClientTestHarness { HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient); // Now tagLocation for these records, hbaseIndex should tag them - JavaRDD javaRDD = (JavaRDD) index.tagLocation(writeRecords, context, hoodieTable); + JavaRDD javaRDD = tagLocation(index, writeRecords, hoodieTable); assert (javaRDD.filter(HoodieRecord::isCurrentLocationKnown).collect().size() == totalRecords); // check tagged records are tagged with correct fileIds @@ -269,7 +269,7 @@ public class TestHoodieIndex extends HoodieClientTestHarness { hoodieTable = HoodieSparkTable.create(config, context, metaClient); // Now tagLocation for these records, hbaseIndex should not tag them since it was a rolled // back commit - javaRDD = (JavaRDD) index.tagLocation(writeRecords, context, hoodieTable); + javaRDD = tagLocation(index, writeRecords, hoodieTable); assert (javaRDD.filter(HoodieRecord::isCurrentLocationKnown).collect().size() == 0); assert (javaRDD.filter(record -> record.getCurrentLocation() != null).collect().size() == 0); } @@ -312,7 +312,7 @@ public class TestHoodieIndex extends HoodieClientTestHarness { HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient); - JavaRDD taggedRecordRDD = (JavaRDD) index.tagLocation(recordRDD, context, hoodieTable); + JavaRDD taggedRecordRDD = tagLocation(index, recordRDD, hoodieTable); // Should not find any files for (HoodieRecord record : taggedRecordRDD.collect()) { @@ -329,7 +329,7 @@ public class TestHoodieIndex extends HoodieClientTestHarness { metaClient = HoodieTableMetaClient.reload(metaClient); hoodieTable = HoodieSparkTable.create(config, context, metaClient); - taggedRecordRDD = (JavaRDD) index.tagLocation(recordRDD, context, hoodieTable); + taggedRecordRDD = tagLocation(index, recordRDD, hoodieTable); // Check results for (HoodieRecord record : taggedRecordRDD.collect()) { @@ -419,7 +419,7 @@ public class TestHoodieIndex extends HoodieClientTestHarness { // test against incoming record with a different partition JavaRDD recordRDD = jsc.parallelize(Collections.singletonList(incomingRecord)); - JavaRDD taggedRecordRDD = (JavaRDD) index.tagLocation(recordRDD, context, hoodieTable); + JavaRDD taggedRecordRDD = tagLocation(index, recordRDD, hoodieTable); assertEquals(2, taggedRecordRDD.count()); for (HoodieRecord record : taggedRecordRDD.collect()) { @@ -440,7 +440,7 @@ public class TestHoodieIndex extends HoodieClientTestHarness { // test against incoming record with the same partition JavaRDD recordRDDSamePartition = jsc .parallelize(Collections.singletonList(incomingRecordSamePartition)); - JavaRDD taggedRecordRDDSamePartition = (JavaRDD) index.tagLocation(recordRDDSamePartition, context, hoodieTable); + JavaRDD taggedRecordRDDSamePartition = tagLocation(index, recordRDDSamePartition, hoodieTable); assertEquals(1, taggedRecordRDDSamePartition.count()); HoodieRecord record = taggedRecordRDDSamePartition.first(); @@ -463,8 +463,8 @@ public class TestHoodieIndex extends HoodieClientTestHarness { } private JavaPairRDD>> getRecordLocations(JavaRDD keyRDD, HoodieTable hoodieTable) { - JavaRDD recordRDD = (JavaRDD) index.tagLocation( - keyRDD.map(k -> new HoodieRecord(k, new EmptyHoodieRecordPayload())), context, hoodieTable); + JavaRDD recordRDD = tagLocation( + index, keyRDD.map(k -> new HoodieRecord(k, new EmptyHoodieRecordPayload())), hoodieTable); return recordRDD.mapToPair(hr -> new Tuple2<>(hr.getKey(), hr.isCurrentLocationKnown() ? Option.of(Pair.of(hr.getPartitionPath(), hr.getCurrentLocation().getFileId())) : Option.empty()) diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/TestHoodieIndexConfigs.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/TestHoodieIndexConfigs.java index 0af28cc8d..2fb364187 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/TestHoodieIndexConfigs.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/TestHoodieIndexConfigs.java @@ -30,10 +30,11 @@ import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.exception.HoodieIndexException; import org.apache.hudi.index.HoodieIndex.IndexType; -import org.apache.hudi.index.bloom.SparkHoodieBloomIndex; -import org.apache.hudi.index.bloom.SparkHoodieGlobalBloomIndex; +import org.apache.hudi.index.bloom.HoodieBloomIndex; +import org.apache.hudi.index.bloom.HoodieGlobalBloomIndex; import org.apache.hudi.index.hbase.SparkHoodieHBaseIndex; -import org.apache.hudi.index.simple.SparkHoodieSimpleIndex; +import org.apache.hudi.index.inmemory.HoodieInMemoryHashIndex; +import org.apache.hudi.index.simple.HoodieSimpleIndex; import org.apache.hudi.table.HoodieTable; import org.apache.spark.api.java.JavaRDD; @@ -68,29 +69,29 @@ public class TestHoodieIndexConfigs { case INMEMORY: config = clientConfigBuilder.withPath(basePath) .withIndexConfig(indexConfigBuilder.withIndexType(HoodieIndex.IndexType.INMEMORY).build()).build(); - assertTrue(SparkHoodieIndex.createIndex(config) instanceof SparkInMemoryHashIndex); + assertTrue(SparkHoodieIndexFactory.createIndex(config) instanceof HoodieInMemoryHashIndex); break; case BLOOM: config = clientConfigBuilder.withPath(basePath) .withIndexConfig(indexConfigBuilder.withIndexType(HoodieIndex.IndexType.BLOOM).build()).build(); - assertTrue(SparkHoodieIndex.createIndex(config) instanceof SparkHoodieBloomIndex); + assertTrue(SparkHoodieIndexFactory.createIndex(config) instanceof HoodieBloomIndex); break; case GLOBAL_BLOOM: config = clientConfigBuilder.withPath(basePath) .withIndexConfig(indexConfigBuilder.withIndexType(IndexType.GLOBAL_BLOOM).build()).build(); - assertTrue(SparkHoodieIndex.createIndex(config) instanceof SparkHoodieGlobalBloomIndex); + assertTrue(SparkHoodieIndexFactory.createIndex(config) instanceof HoodieGlobalBloomIndex); break; case SIMPLE: config = clientConfigBuilder.withPath(basePath) .withIndexConfig(indexConfigBuilder.withIndexType(IndexType.SIMPLE).build()).build(); - assertTrue(SparkHoodieIndex.createIndex(config) instanceof SparkHoodieSimpleIndex); + assertTrue(SparkHoodieIndexFactory.createIndex(config) instanceof HoodieSimpleIndex); break; case HBASE: config = clientConfigBuilder.withPath(basePath) .withIndexConfig(indexConfigBuilder.withIndexType(HoodieIndex.IndexType.HBASE) .withHBaseIndexConfig(new HoodieHBaseIndexConfig.Builder().build()).build()) .build(); - assertTrue(SparkHoodieIndex.createIndex(config) instanceof SparkHoodieHBaseIndex); + assertTrue(SparkHoodieIndexFactory.createIndex(config) instanceof SparkHoodieHBaseIndex); break; default: // no -op. just for checkstyle errors @@ -103,7 +104,7 @@ public class TestHoodieIndexConfigs { HoodieIndexConfig.Builder indexConfigBuilder = HoodieIndexConfig.newBuilder(); HoodieWriteConfig config = clientConfigBuilder.withPath(basePath) .withIndexConfig(indexConfigBuilder.withIndexClass(DummyHoodieIndex.class.getName()).build()).build(); - assertTrue(SparkHoodieIndex.createIndex(config) instanceof DummyHoodieIndex); + assertTrue(SparkHoodieIndexFactory.createIndex(config) instanceof DummyHoodieIndex); } @Test @@ -113,14 +114,14 @@ public class TestHoodieIndexConfigs { final HoodieWriteConfig config1 = clientConfigBuilder.withPath(basePath) .withIndexConfig(indexConfigBuilder.withIndexClass(IndexWithConstructor.class.getName()).build()).build(); final Throwable thrown1 = assertThrows(HoodieException.class, () -> { - SparkHoodieIndex.createIndex(config1); + SparkHoodieIndexFactory.createIndex(config1); }, "exception is expected"); assertTrue(thrown1.getMessage().contains("is not a subclass of HoodieIndex")); final HoodieWriteConfig config2 = clientConfigBuilder.withPath(basePath) .withIndexConfig(indexConfigBuilder.withIndexClass(IndexWithoutConstructor.class.getName()).build()).build(); final Throwable thrown2 = assertThrows(HoodieException.class, () -> { - SparkHoodieIndex.createIndex(config2); + SparkHoodieIndexFactory.createIndex(config2); }, "exception is expected"); assertTrue(thrown2.getMessage().contains("Unable to instantiate class")); } diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/bloom/TestHoodieBloomIndex.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/bloom/TestHoodieBloomIndex.java index b325eb6b1..1334adb20 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/bloom/TestHoodieBloomIndex.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/bloom/TestHoodieBloomIndex.java @@ -26,9 +26,12 @@ import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.testutils.RawTripTestPayload; import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.collection.ImmutablePair; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.config.HoodieIndexConfig; import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.data.HoodieJavaPairRDD; +import org.apache.hudi.data.HoodieJavaRDD; import org.apache.hudi.io.HoodieKeyLookupHandle; import org.apache.hudi.table.HoodieSparkTable; import org.apache.hudi.table.HoodieTable; @@ -103,7 +106,7 @@ public class TestHoodieBloomIndex extends HoodieClientTestHarness { @MethodSource("configParams") public void testLoadInvolvedFiles(boolean rangePruning, boolean treeFiltering, boolean bucketizedChecking) throws Exception { HoodieWriteConfig config = makeConfig(rangePruning, treeFiltering, bucketizedChecking); - SparkHoodieBloomIndex index = new SparkHoodieBloomIndex(config); + HoodieBloomIndex index = new HoodieBloomIndex(config, SparkHoodieBloomIndexHelper.getInstance()); HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient); HoodieSparkWriteableTestTable testTable = HoodieSparkWriteableTestTable.of(hoodieTable, SCHEMA); @@ -131,7 +134,7 @@ public class TestHoodieBloomIndex extends HoodieClientTestHarness { new HoodieRecord(new HoodieKey(rowChange4.getRowKey(), rowChange4.getPartitionPath()), rowChange4); List partitions = Arrays.asList("2016/01/21", "2016/04/01", "2015/03/12"); - List> filesList = index.loadInvolvedFiles(partitions, context, hoodieTable); + List> filesList = index.loadInvolvedFiles(partitions, context, hoodieTable); // Still 0, as no valid commit assertEquals(0, filesList.size()); @@ -145,20 +148,20 @@ public class TestHoodieBloomIndex extends HoodieClientTestHarness { if (rangePruning) { // these files will not have the key ranges - assertNull(filesList.get(0)._2().getMaxRecordKey()); - assertNull(filesList.get(0)._2().getMinRecordKey()); - assertFalse(filesList.get(1)._2().hasKeyRanges()); - assertNotNull(filesList.get(2)._2().getMaxRecordKey()); - assertNotNull(filesList.get(2)._2().getMinRecordKey()); - assertTrue(filesList.get(3)._2().hasKeyRanges()); + assertNull(filesList.get(0).getRight().getMaxRecordKey()); + assertNull(filesList.get(0).getRight().getMinRecordKey()); + assertFalse(filesList.get(1).getRight().hasKeyRanges()); + assertNotNull(filesList.get(2).getRight().getMaxRecordKey()); + assertNotNull(filesList.get(2).getRight().getMinRecordKey()); + assertTrue(filesList.get(3).getRight().hasKeyRanges()); // no longer sorted, but should have same files. - List> expected = - Arrays.asList(new Tuple2<>("2016/04/01", new BloomIndexFileInfo("2")), - new Tuple2<>("2015/03/12", new BloomIndexFileInfo("1")), - new Tuple2<>("2015/03/12", new BloomIndexFileInfo("3", "000", "000")), - new Tuple2<>("2015/03/12", new BloomIndexFileInfo("4", "001", "003"))); + List> expected = + Arrays.asList(new ImmutablePair<>("2016/04/01", new BloomIndexFileInfo("2")), + new ImmutablePair<>("2015/03/12", new BloomIndexFileInfo("1")), + new ImmutablePair<>("2015/03/12", new BloomIndexFileInfo("3", "000", "000")), + new ImmutablePair<>("2015/03/12", new BloomIndexFileInfo("4", "001", "003"))); assertEquals(expected, filesList); } } @@ -167,7 +170,7 @@ public class TestHoodieBloomIndex extends HoodieClientTestHarness { @MethodSource("configParams") public void testRangePruning(boolean rangePruning, boolean treeFiltering, boolean bucketizedChecking) { HoodieWriteConfig config = makeConfig(rangePruning, treeFiltering, bucketizedChecking); - SparkHoodieBloomIndex index = new SparkHoodieBloomIndex(config); + HoodieBloomIndex index = new HoodieBloomIndex(config, SparkHoodieBloomIndexHelper.getInstance()); final Map> partitionToFileIndexInfo = new HashMap<>(); partitionToFileIndexInfo.put("2017/10/22", @@ -179,12 +182,12 @@ public class TestHoodieBloomIndex extends HoodieClientTestHarness { jsc.parallelize(Arrays.asList(new Tuple2<>("2017/10/22", "003"), new Tuple2<>("2017/10/22", "002"), new Tuple2<>("2017/10/22", "005"), new Tuple2<>("2017/10/22", "004"))).mapToPair(t -> t); - List> comparisonKeyList = - index.explodeRecordRDDWithFileComparisons(partitionToFileIndexInfo, partitionRecordKeyPairRDD).collect(); + List> comparisonKeyList = HoodieJavaRDD.getJavaRDD( + index.explodeRecordsWithFileComparisons(partitionToFileIndexInfo, HoodieJavaPairRDD.of(partitionRecordKeyPairRDD))).collect(); assertEquals(10, comparisonKeyList.size()); Map> recordKeyToFileComps = comparisonKeyList.stream() - .collect(Collectors.groupingBy(t -> t._2.getRecordKey(), Collectors.mapping(t -> t._1, Collectors.toList()))); + .collect(Collectors.groupingBy(t -> t.getRight().getRecordKey(), Collectors.mapping(Pair::getLeft, Collectors.toList()))); assertEquals(4, recordKeyToFileComps.size()); assertEquals(new HashSet<>(Arrays.asList("f1", "f3", "f4")), new HashSet<>(recordKeyToFileComps.get("002"))); @@ -262,10 +265,10 @@ public class TestHoodieBloomIndex extends HoodieClientTestHarness { HoodieSparkTable table = HoodieSparkTable.create(config, context, metaClient); // Let's tag - SparkHoodieBloomIndex bloomIndex = new SparkHoodieBloomIndex(config); + HoodieBloomIndex bloomIndex = new HoodieBloomIndex(config, SparkHoodieBloomIndexHelper.getInstance()); assertDoesNotThrow(() -> { - bloomIndex.tagLocation(recordRDD, context, table); + tagLocation(bloomIndex, recordRDD, table); }, "EmptyRDD should not result in IllegalArgumentException: Positive number of slices required"); } @@ -301,8 +304,8 @@ public class TestHoodieBloomIndex extends HoodieClientTestHarness { HoodieSparkWriteableTestTable testTable = HoodieSparkWriteableTestTable.of(hoodieTable, SCHEMA); // Let's tag - SparkHoodieBloomIndex bloomIndex = new SparkHoodieBloomIndex(config); - JavaRDD taggedRecordRDD = bloomIndex.tagLocation(recordRDD, context, hoodieTable); + HoodieBloomIndex bloomIndex = new HoodieBloomIndex(config, SparkHoodieBloomIndexHelper.getInstance()); + JavaRDD taggedRecordRDD = tagLocation(bloomIndex, recordRDD, hoodieTable); // Should not find any files for (HoodieRecord record : taggedRecordRDD.collect()) { @@ -315,7 +318,7 @@ public class TestHoodieBloomIndex extends HoodieClientTestHarness { String fileId3 = testTable.addCommit("003").getFileIdWithInserts("2015/01/31", record4); // We do the tag again - taggedRecordRDD = bloomIndex.tagLocation(recordRDD, context, HoodieSparkTable.create(config, context, metaClient)); + taggedRecordRDD = tagLocation(bloomIndex, recordRDD, HoodieSparkTable.create(config, context, metaClient)); // Check results for (HoodieRecord record : taggedRecordRDD.collect()) { @@ -366,8 +369,9 @@ public class TestHoodieBloomIndex extends HoodieClientTestHarness { HoodieSparkWriteableTestTable testTable = HoodieSparkWriteableTestTable.of(hoodieTable, SCHEMA); // Let's tag - SparkHoodieBloomIndex bloomIndex = new SparkHoodieBloomIndex(config); - JavaRDD taggedRecords = bloomIndex.tagLocation(keysRDD.map(k -> new HoodieRecord(k, null)), context, hoodieTable); + HoodieBloomIndex bloomIndex = new HoodieBloomIndex(config, SparkHoodieBloomIndexHelper.getInstance()); + JavaRDD taggedRecords = tagLocation( + bloomIndex, keysRDD.map(k -> new HoodieRecord(k, null)), hoodieTable); JavaPairRDD>> recordLocationsRDD = taggedRecords .mapToPair(hr -> new Tuple2<>(hr.getKey(), hr.isCurrentLocationKnown() ? Option.of(Pair.of(hr.getPartitionPath(), hr.getCurrentLocation().getFileId())) @@ -387,7 +391,7 @@ public class TestHoodieBloomIndex extends HoodieClientTestHarness { // We do the tag again metaClient = HoodieTableMetaClient.reload(metaClient); hoodieTable = HoodieSparkTable.create(config, context, metaClient); - taggedRecords = bloomIndex.tagLocation(keysRDD.map(k -> new HoodieRecord(k, null)), context, hoodieTable); + taggedRecords = tagLocation(bloomIndex, keysRDD.map(k -> new HoodieRecord(k, null)), hoodieTable); recordLocationsRDD = taggedRecords .mapToPair(hr -> new Tuple2<>(hr.getKey(), hr.isCurrentLocationKnown() ? Option.of(Pair.of(hr.getPartitionPath(), hr.getCurrentLocation().getFileId())) @@ -443,8 +447,8 @@ public class TestHoodieBloomIndex extends HoodieClientTestHarness { metaClient = HoodieTableMetaClient.reload(metaClient); HoodieTable table = HoodieSparkTable.create(config, context, metaClient); - SparkHoodieBloomIndex bloomIndex = new SparkHoodieBloomIndex(config); - JavaRDD taggedRecordRDD = bloomIndex.tagLocation(recordRDD, context, table); + HoodieBloomIndex bloomIndex = new HoodieBloomIndex(config, SparkHoodieBloomIndexHelper.getInstance()); + JavaRDD taggedRecordRDD = tagLocation(bloomIndex, recordRDD, table); // Check results for (HoodieRecord record : taggedRecordRDD.collect()) { diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/bloom/TestHoodieGlobalBloomIndex.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/bloom/TestHoodieGlobalBloomIndex.java index 3970ab250..fa7d586d2 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/bloom/TestHoodieGlobalBloomIndex.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/bloom/TestHoodieGlobalBloomIndex.java @@ -22,8 +22,12 @@ import org.apache.hudi.common.model.EmptyHoodieRecordPayload; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.testutils.RawTripTestPayload; +import org.apache.hudi.common.util.collection.ImmutablePair; +import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.config.HoodieIndexConfig; import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.data.HoodieJavaPairRDD; +import org.apache.hudi.data.HoodieJavaRDD; import org.apache.hudi.table.HoodieSparkTable; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.testutils.HoodieClientTestHarness; @@ -74,7 +78,8 @@ public class TestHoodieGlobalBloomIndex extends HoodieClientTestHarness { @Test public void testLoadInvolvedFiles() throws Exception { HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build(); - SparkHoodieGlobalBloomIndex index = new SparkHoodieGlobalBloomIndex(config); + HoodieGlobalBloomIndex index = + new HoodieGlobalBloomIndex(config, SparkHoodieBloomIndexHelper.getInstance()); HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient); HoodieSparkWriteableTestTable testTable = HoodieSparkWriteableTestTable.of(hoodieTable, SCHEMA); @@ -104,7 +109,7 @@ public class TestHoodieGlobalBloomIndex extends HoodieClientTestHarness { // intentionally missed the partition "2015/03/12" to see if the GlobalBloomIndex can pick it up List partitions = Arrays.asList("2016/01/21", "2016/04/01"); // partitions will NOT be respected by this loadInvolvedFiles(...) call - List> filesList = index.loadInvolvedFiles(partitions, context, hoodieTable); + List> filesList = index.loadInvolvedFiles(partitions, context, hoodieTable); // Still 0, as no valid commit assertEquals(0, filesList.size()); @@ -138,7 +143,8 @@ public class TestHoodieGlobalBloomIndex extends HoodieClientTestHarness { public void testExplodeRecordRDDWithFileComparisons() { HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build(); - SparkHoodieGlobalBloomIndex index = new SparkHoodieGlobalBloomIndex(config); + HoodieGlobalBloomIndex index = + new HoodieGlobalBloomIndex(config, SparkHoodieBloomIndexHelper.getInstance()); final Map> partitionToFileIndexInfo = new HashMap<>(); partitionToFileIndexInfo.put("2017/10/22", Arrays.asList(new BloomIndexFileInfo("f1"), @@ -152,8 +158,9 @@ public class TestHoodieGlobalBloomIndex extends HoodieClientTestHarness { jsc.parallelize(Arrays.asList(new Tuple2<>("2017/10/21", "003"), new Tuple2<>("2017/10/22", "002"), new Tuple2<>("2017/10/22", "005"), new Tuple2<>("2017/10/23", "004"))).mapToPair(t -> t); - List> comparisonKeyList = - index.explodeRecordRDDWithFileComparisons(partitionToFileIndexInfo, partitionRecordKeyPairRDD).collect(); + List> comparisonKeyList = HoodieJavaRDD.getJavaRDD( + index.explodeRecordsWithFileComparisons(partitionToFileIndexInfo, + HoodieJavaPairRDD.of(partitionRecordKeyPairRDD))).collect(); /* * expecting: f4, HoodieKey { recordKey=003 partitionPath=2017/10/23} f1, HoodieKey { recordKey=003 @@ -166,7 +173,7 @@ public class TestHoodieGlobalBloomIndex extends HoodieClientTestHarness { assertEquals(10, comparisonKeyList.size()); Map> recordKeyToFileComps = comparisonKeyList.stream() - .collect(Collectors.groupingBy(t -> t._2.getRecordKey(), Collectors.mapping(Tuple2::_1, Collectors.toList()))); + .collect(Collectors.groupingBy(t -> t.getRight().getRecordKey(), Collectors.mapping(Pair::getKey, Collectors.toList()))); assertEquals(4, recordKeyToFileComps.size()); assertEquals(new HashSet<>(Arrays.asList("f4", "f1", "f3")), new HashSet<>(recordKeyToFileComps.get("002"))); @@ -179,7 +186,8 @@ public class TestHoodieGlobalBloomIndex extends HoodieClientTestHarness { public void testTagLocation() throws Exception { HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath) .withIndexConfig(HoodieIndexConfig.newBuilder().withBloomIndexUpdatePartitionPath(false).build()).build(); - SparkHoodieGlobalBloomIndex index = new SparkHoodieGlobalBloomIndex(config); + HoodieGlobalBloomIndex index = + new HoodieGlobalBloomIndex(config, SparkHoodieBloomIndexHelper.getInstance()); HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient); HoodieSparkWriteableTestTable testTable = HoodieSparkWriteableTestTable.of(hoodieTable, SCHEMA); @@ -224,7 +232,7 @@ public class TestHoodieGlobalBloomIndex extends HoodieClientTestHarness { String fileId4 = testTable.addCommit("4000").getFileIdWithInserts("2015/03/12", record4); // partitions will NOT be respected by this loadInvolvedFiles(...) call - JavaRDD taggedRecordRDD = index.tagLocation(recordRDD, context, hoodieTable); + JavaRDD taggedRecordRDD = tagLocation(index, recordRDD, hoodieTable); for (HoodieRecord record : taggedRecordRDD.collect()) { switch (record.getRecordKey()) { @@ -260,7 +268,8 @@ public class TestHoodieGlobalBloomIndex extends HoodieClientTestHarness { .withPath(basePath) .withIndexConfig(HoodieIndexConfig.newBuilder().withBloomIndexUpdatePartitionPath(true).build()) .build(); - SparkHoodieGlobalBloomIndex index = new SparkHoodieGlobalBloomIndex(config); + HoodieGlobalBloomIndex index = + new HoodieGlobalBloomIndex(config, SparkHoodieBloomIndexHelper.getInstance()); HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient); HoodieSparkWriteableTestTable testTable = HoodieSparkWriteableTestTable.of(hoodieTable, SCHEMA); final String p1 = "2016/01/31"; @@ -304,7 +313,7 @@ public class TestHoodieGlobalBloomIndex extends HoodieClientTestHarness { // test against incoming record with a different partition JavaRDD recordRDD = jsc.parallelize(Collections.singletonList(incomingRecord)); - JavaRDD taggedRecordRDD = index.tagLocation(recordRDD, context, hoodieTable); + JavaRDD taggedRecordRDD = tagLocation(index, recordRDD, hoodieTable); assertEquals(2, taggedRecordRDD.count()); for (HoodieRecord record : taggedRecordRDD.collect()) { @@ -325,7 +334,7 @@ public class TestHoodieGlobalBloomIndex extends HoodieClientTestHarness { // test against incoming record with the same partition JavaRDD recordRDDSamePartition = jsc .parallelize(Collections.singletonList(incomingRecordSamePartition)); - JavaRDD taggedRecordRDDSamePartition = index.tagLocation(recordRDDSamePartition, context, hoodieTable); + JavaRDD taggedRecordRDDSamePartition = tagLocation(index, recordRDDSamePartition, hoodieTable); assertEquals(1, taggedRecordRDDSamePartition.count()); HoodieRecord record = taggedRecordRDDSamePartition.first(); @@ -335,10 +344,10 @@ public class TestHoodieGlobalBloomIndex extends HoodieClientTestHarness { } // convert list to map to avoid sorting order dependencies - private static Map toFileMap(List> filesList) { + private static Map toFileMap(List> filesList) { Map filesMap = new HashMap<>(); - for (Tuple2 t : filesList) { - filesMap.put(t._1() + "/" + t._2().getFileId(), t._2()); + for (Pair t : filesList) { + filesMap.put(t.getKey() + "/" + t.getValue().getFileId(), t.getValue()); } return filesMap; } diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/TestCleaner.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/TestCleaner.java index 8714df026..063b55686 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/TestCleaner.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/TestCleaner.java @@ -76,7 +76,7 @@ import org.apache.hudi.config.HoodieCompactionConfig; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.index.HoodieIndex; -import org.apache.hudi.index.SparkHoodieIndex; +import org.apache.hudi.index.SparkHoodieIndexFactory; import org.apache.hudi.metadata.HoodieTableMetadataWriter; import org.apache.hudi.metadata.SparkHoodieBackedTableMetadataWriter; import org.apache.hudi.table.action.clean.CleanPlanner; @@ -169,8 +169,8 @@ public class TestCleaner extends HoodieClientTestBase { assertTrue(table.getCompletedCleanTimeline().empty()); if (client.getConfig().shouldAutoCommit()) { - HoodieIndex index = SparkHoodieIndex.createIndex(cfg); - List taggedRecords = ((JavaRDD) index.tagLocation(jsc.parallelize(records, 1), context, table)).collect(); + HoodieIndex index = SparkHoodieIndexFactory.createIndex(cfg); + List taggedRecords = tagLocation(index, jsc.parallelize(records, 1), table).collect(); checkTaggedRecords(taggedRecords, newCommitTime); } return Pair.of(newCommitTime, statuses); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/commit/TestDeleteHelper.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/commit/TestDeleteHelper.java index d9dc6ac97..8617c8487 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/commit/TestDeleteHelper.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/commit/TestDeleteHelper.java @@ -23,7 +23,8 @@ import org.apache.hudi.common.model.EmptyHoodieRecordPayload; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.config.HoodieWriteConfig; -import org.apache.hudi.index.bloom.SparkHoodieBloomIndex; +import org.apache.hudi.data.HoodieJavaRDD; +import org.apache.hudi.index.bloom.HoodieBloomIndex; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.action.HoodieWriteMetadata; @@ -64,13 +65,20 @@ public class TestDeleteHelper { private static final boolean WITHOUT_COMBINE = false; private static final int DELETE_PARALLELISM = 200; - @Mock private SparkHoodieBloomIndex index; - @Mock private HoodieTable, JavaRDD, JavaRDD> table; - @Mock private BaseSparkCommitActionExecutor executor; - @Mock private HoodieWriteMetadata metadata; - @Mock private JavaPairRDD keyPairs; - @Mock private JavaSparkContext jsc; - @Mock private HoodieSparkEngineContext context; + @Mock + private HoodieBloomIndex index; + @Mock + private HoodieTable, JavaRDD, JavaRDD> table; + @Mock + private BaseSparkCommitActionExecutor executor; + @Mock + private HoodieWriteMetadata metadata; + @Mock + private JavaPairRDD keyPairs; + @Mock + private JavaSparkContext jsc; + @Mock + private HoodieSparkEngineContext context; private JavaRDD rddToDelete; private HoodieWriteConfig config; @@ -149,7 +157,7 @@ public class TestDeleteHelper { JavaRDD recordsRdd = mock(JavaRDD.class); when(recordsRdd.filter(any())).thenReturn(recordsRdd); when(recordsRdd.isEmpty()).thenReturn(howMany <= 0); - when(index.tagLocation(any(), any(), any())).thenReturn(recordsRdd); + when(index.tagLocation(any(), any(), any())).thenReturn(HoodieJavaRDD.of(recordsRdd)); if (combineMode == CombineTestMode.GlobalIndex) { when(keyPairs.reduceByKey(any(), anyInt())).thenReturn(keyPairs); @@ -175,7 +183,7 @@ public class TestDeleteHelper { doReturn(Collections.emptyList()).when(emptyRdd).partitions(); doReturn(emptyRdd).when(emptyRdd).map(any()); - doReturn(emptyRdd).when(index).tagLocation(any(), any(), any()); + doReturn(HoodieJavaRDD.of(emptyRdd)).when(index).tagLocation(any(), any(), any()); doReturn(emptyRdd).when(emptyRdd).filter(any()); doNothing().when(executor).saveWorkloadProfileMetadataToInflight(any(), anyString()); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/TestHoodieCompactor.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/TestHoodieCompactor.java index 36a70d71b..6b837e317 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/TestHoodieCompactor.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/TestHoodieCompactor.java @@ -41,7 +41,8 @@ import org.apache.hudi.config.HoodieStorageConfig; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieNotSupportedException; import org.apache.hudi.index.HoodieIndex; -import org.apache.hudi.index.bloom.SparkHoodieBloomIndex; +import org.apache.hudi.index.bloom.HoodieBloomIndex; +import org.apache.hudi.index.bloom.SparkHoodieBloomIndexHelper; import org.apache.hudi.table.HoodieSparkTable; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.testutils.HoodieClientTestHarness; @@ -179,8 +180,8 @@ public class TestHoodieCompactor extends HoodieClientTestHarness { List updatedRecords = dataGen.generateUpdates(newCommitTime, records); JavaRDD updatedRecordsRDD = jsc.parallelize(updatedRecords, 1); - HoodieIndex index = new SparkHoodieBloomIndex<>(config); - updatedRecords = ((JavaRDD)index.tagLocation(updatedRecordsRDD, context, table)).collect(); + HoodieIndex index = new HoodieBloomIndex<>(config, SparkHoodieBloomIndexHelper.getInstance()); + updatedRecords = tagLocation(index, updatedRecordsRDD, table).collect(); // Write them to corresponding avro logfiles. Also, set the state transition properly. HoodieSparkWriteableTestTable.of(table, HoodieTestDataGenerator.AVRO_SCHEMA_WITH_METADATA_FIELDS) diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestBase.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestBase.java index aa8814ad6..0a010dde5 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestBase.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestBase.java @@ -41,8 +41,9 @@ import org.apache.hudi.config.HoodieCompactionConfig; import org.apache.hudi.config.HoodieIndexConfig; import org.apache.hudi.config.HoodieStorageConfig; import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.index.HoodieIndex; import org.apache.hudi.index.HoodieIndex.IndexType; -import org.apache.hudi.index.SparkHoodieIndex; +import org.apache.hudi.index.SparkHoodieIndexFactory; import org.apache.hudi.table.HoodieSparkTable; import org.apache.hadoop.fs.FileSystem; @@ -234,11 +235,11 @@ public class HoodieClientTestBase extends HoodieClientTestHarness { private Function2, String, Integer> wrapRecordsGenFunctionForPreppedCalls( final HoodieWriteConfig writeConfig, final Function2, String, Integer> recordGenFunction) { return (commit, numRecords) -> { - final SparkHoodieIndex index = SparkHoodieIndex.createIndex(writeConfig); + final HoodieIndex index = SparkHoodieIndexFactory.createIndex(writeConfig); List records = recordGenFunction.apply(commit, numRecords); final HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder().setConf(hadoopConf).setBasePath(basePath).setLoadActiveTimelineOnLoad(true).build(); HoodieSparkTable table = HoodieSparkTable.create(writeConfig, context, metaClient); - JavaRDD taggedRecords = index.tagLocation(jsc.parallelize(records, 1), context, table); + JavaRDD taggedRecords = tagLocation(index, jsc.parallelize(records, 1), table); return taggedRecords.collect(); }; } @@ -255,13 +256,13 @@ public class HoodieClientTestBase extends HoodieClientTestHarness { private Function> wrapDeleteKeysGenFunctionForPreppedCalls( final HoodieWriteConfig writeConfig, final Function> keyGenFunction) { return (numRecords) -> { - final SparkHoodieIndex index = SparkHoodieIndex.createIndex(writeConfig); + final HoodieIndex index = SparkHoodieIndexFactory.createIndex(writeConfig); List records = keyGenFunction.apply(numRecords); final HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder().setConf(hadoopConf).setBasePath(basePath).setLoadActiveTimelineOnLoad(true).build(); HoodieSparkTable table = HoodieSparkTable.create(writeConfig, context, metaClient); JavaRDD recordsToDelete = jsc.parallelize(records, 1) .map(key -> new HoodieRecord(key, new EmptyHoodieRecordPayload())); - JavaRDD taggedRecords = index.tagLocation(recordsToDelete, context, table); + JavaRDD taggedRecords = tagLocation(index, recordsToDelete, table); return taggedRecords.map(record -> record.getKey()).collect(); }; } diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestHarness.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestHarness.java index 03c68eca4..8c0a3bd7e 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestHarness.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestHarness.java @@ -43,6 +43,7 @@ import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.config.HoodieIndexConfig; import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.data.HoodieJavaRDD; import org.apache.hudi.exception.HoodieMetadataException; import org.apache.hudi.index.HoodieIndex; import org.apache.hudi.keygen.SimpleKeyGenerator; @@ -413,6 +414,12 @@ public abstract class HoodieClientTestHarness extends HoodieCommonTestHarness im return tableView; } + public JavaRDD tagLocation( + HoodieIndex index, JavaRDD records, HoodieTable table) { + return HoodieJavaRDD.getJavaRDD( + index.tagLocation(HoodieJavaRDD.of(records), context, table)); + } + public static Pair, WorkloadStat> buildProfile(JavaRDD inputRecordsRDD) { HashMap partitionPathStatMap = new HashMap<>(); WorkloadStat globalStat = new WorkloadStat(); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/SparkClientFunctionalTestHarness.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/SparkClientFunctionalTestHarness.java index aca1d83d4..79fbdcaad 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/SparkClientFunctionalTestHarness.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/SparkClientFunctionalTestHarness.java @@ -41,6 +41,7 @@ import org.apache.hudi.config.HoodieCompactionConfig; import org.apache.hudi.config.HoodieIndexConfig; import org.apache.hudi.config.HoodieStorageConfig; import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.data.HoodieJavaRDD; import org.apache.hudi.index.HoodieIndex; import org.apache.hudi.keygen.SimpleKeyGenerator; import org.apache.hudi.table.HoodieSparkTable; @@ -190,6 +191,18 @@ public class SparkClientFunctionalTestHarness implements SparkProvider, HoodieMe } } + protected JavaRDD tagLocation( + HoodieIndex index, JavaRDD records, HoodieTable table) { + return HoodieJavaRDD.getJavaRDD( + index.tagLocation(HoodieJavaRDD.of(records), context, table)); + } + + protected JavaRDD updateLocation( + HoodieIndex index, JavaRDD writeStatus, HoodieTable table) { + return HoodieJavaRDD.getJavaRDD( + index.updateLocation(HoodieJavaRDD.of(writeStatus), context, table)); + } + protected void insertRecords(HoodieTableMetaClient metaClient, List records, SparkRDDWriteClient client, HoodieWriteConfig cfg, String commitTime) throws IOException { HoodieTableMetaClient reloadedMetaClient = HoodieTableMetaClient.reload(metaClient); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/data/HoodieData.java b/hudi-common/src/main/java/org/apache/hudi/common/data/HoodieData.java index f26a42035..7ea7e0d64 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/data/HoodieData.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/data/HoodieData.java @@ -20,6 +20,7 @@ package org.apache.hudi.common.data; import org.apache.hudi.common.function.SerializableFunction; +import org.apache.hudi.common.function.SerializablePairFunction; import java.io.Serializable; import java.util.Iterator; @@ -37,11 +38,28 @@ public abstract class HoodieData implements Serializable { */ public abstract Object get(); + /** + * Caches the data. + * + * @param cacheConfig config value for caching. + */ + public abstract void persist(String cacheConfig); + + /** + * Removes the cached data. + */ + public abstract void unpersist(); + /** * @return whether the collection is empty. */ public abstract boolean isEmpty(); + /** + * @return the number of objects. + */ + public abstract long count(); + /** * @param func serializable map function. * @param output object type. @@ -49,6 +67,16 @@ public abstract class HoodieData implements Serializable { */ public abstract HoodieData map(SerializableFunction func); + /** + * @param func serializable map function by taking a partition of objects + * and generating an iterator. + * @param preservesPartitioning whether to preserve partitions in the result. + * @param output object type. + * @return {@link HoodieData} containing the result. Actual execution may be deferred. + */ + public abstract HoodieData mapPartitions( + SerializableFunction, Iterator> func, boolean preservesPartitioning); + /** * @param func serializable flatmap function. * @param output object type. @@ -56,6 +84,19 @@ public abstract class HoodieData implements Serializable { */ public abstract HoodieData flatMap(SerializableFunction> func); + /** + * @param mapToPairFunc serializable map function to generate a pair. + * @param key type of the pair. + * @param value type of the pair. + * @return {@link HoodiePairData} containing the result. Actual execution may be deferred. + */ + public abstract HoodiePairData mapToPair(SerializablePairFunction mapToPairFunc); + + /** + * @return distinct objects in {@link HoodieData}. + */ + public abstract HoodieData distinct(); + /** * @return collected results in {@link List}. */ diff --git a/hudi-common/src/main/java/org/apache/hudi/common/data/HoodieList.java b/hudi-common/src/main/java/org/apache/hudi/common/data/HoodieList.java index 2dd8c2ec5..6c23fdff2 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/data/HoodieList.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/data/HoodieList.java @@ -20,13 +20,19 @@ package org.apache.hudi.common.data; import org.apache.hudi.common.function.SerializableFunction; +import org.apache.hudi.common.function.SerializablePairFunction; +import org.apache.hudi.common.util.collection.Pair; import java.util.ArrayList; +import java.util.HashMap; +import java.util.HashSet; import java.util.Iterator; import java.util.List; +import java.util.Map; import java.util.function.Function; import java.util.stream.Collectors; +import static org.apache.hudi.common.function.FunctionWrapper.throwingMapToPairWrapper; import static org.apache.hudi.common.function.FunctionWrapper.throwingMapWrapper; /** @@ -65,17 +71,39 @@ public class HoodieList extends HoodieData { return listData; } + @Override + public void persist(String cacheConfig) { + // No OP + } + + @Override + public void unpersist() { + // No OP + } + @Override public boolean isEmpty() { return listData.isEmpty(); } + @Override + public long count() { + return listData.size(); + } + @Override public HoodieData map(SerializableFunction func) { return HoodieList.of(listData.stream().parallel() .map(throwingMapWrapper(func)).collect(Collectors.toList())); } + @Override + public HoodieData mapPartitions(SerializableFunction, Iterator> func, boolean preservesPartitioning) { + List result = new ArrayList<>(); + throwingMapWrapper(func).apply(listData.iterator()).forEachRemaining(result::add); + return HoodieList.of(result); + } + @Override public HoodieData flatMap(SerializableFunction> func) { Function> throwableFunc = throwingMapWrapper(func); @@ -87,6 +115,23 @@ public class HoodieList extends HoodieData { }).collect(Collectors.toList())); } + @Override + public HoodiePairData mapToPair(SerializablePairFunction mapToPairFunc) { + Map> mapOfPairs = new HashMap<>(); + Function> throwableMapToPairFunc = throwingMapToPairWrapper(mapToPairFunc); + listData.forEach(data -> { + Pair pair = throwableMapToPairFunc.apply(data); + List list = mapOfPairs.computeIfAbsent(pair.getKey(), k -> new ArrayList<>()); + list.add(pair.getValue()); + }); + return HoodieMapPair.of(mapOfPairs); + } + + @Override + public HoodieData distinct() { + return HoodieList.of(new ArrayList<>(new HashSet<>(listData))); + } + @Override public List collectAsList() { return listData; diff --git a/hudi-common/src/main/java/org/apache/hudi/common/data/HoodieMapPair.java b/hudi-common/src/main/java/org/apache/hudi/common/data/HoodieMapPair.java new file mode 100644 index 000000000..c941231e6 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/data/HoodieMapPair.java @@ -0,0 +1,157 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.hudi.common.data; + +import org.apache.hudi.common.function.FunctionWrapper; +import org.apache.hudi.common.function.SerializableFunction; +import org.apache.hudi.common.function.SerializablePairFunction; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.collection.ImmutablePair; +import org.apache.hudi.common.util.collection.Pair; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.function.Function; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static org.apache.hudi.common.function.FunctionWrapper.throwingMapWrapper; + +/** + * Implementation of {@link HoodiePairData} using Java {@link Map}. + * The pairs are organized by the key in the Map and values for the same key + * are stored in a list as the value corresponding to the key in the Map. + * + * @param type of key. + * @param type of value. + */ +public class HoodieMapPair extends HoodiePairData { + + private final Map> mapPairData; + + private HoodieMapPair(Map> mapPairData) { + this.mapPairData = mapPairData; + } + + /** + * @param mapPairData a {@link Map} of pairs. + * @param type of key. + * @param type of value. + * @return a new instance containing the {@link Map>} reference. + */ + public static HoodieMapPair of(Map> mapPairData) { + return new HoodieMapPair<>(mapPairData); + } + + /** + * @param hoodiePairData {@link HoodieMapPair } instance containing the {@link Map} of pairs. + * @param type of key. + * @param type of value. + * @return the {@link Map} of pairs. + */ + public static Map> getMapPair(HoodiePairData hoodiePairData) { + return ((HoodieMapPair) hoodiePairData).get(); + } + + @Override + public Map> get() { + return mapPairData; + } + + @Override + public void persist(String cacheConfig) { + // No OP + } + + @Override + public void unpersist() { + // No OP + } + + @Override + public HoodieData keys() { + return HoodieList.of(new ArrayList<>(mapPairData.keySet())); + } + + @Override + public HoodieData values() { + return HoodieList.of( + mapPairData.values().stream().flatMap(List::stream).collect(Collectors.toList())); + } + + @Override + public long count() { + return mapPairData.values().stream().map( + list -> (long) list.size()).reduce(Long::sum).orElse(0L); + } + + @Override + public Map countByKey() { + return mapPairData.entrySet().stream().collect( + Collectors.toMap(Map.Entry::getKey, entry -> (long) entry.getValue().size())); + } + + @Override + public HoodieData map(SerializableFunction, O> func) { + Function, O> throwableFunc = throwingMapWrapper(func); + return HoodieList.of( + streamAllPairs().map(throwableFunc).collect(Collectors.toList())); + } + + @Override + public HoodiePairData mapToPair(SerializablePairFunction, L, W> mapToPairFunc) { + Map> newMap = new HashMap<>(); + Function, Pair> throwableMapToPairFunc = + FunctionWrapper.throwingMapToPairWrapper(mapToPairFunc); + streamAllPairs().map(pair -> throwableMapToPairFunc.apply(pair)).forEach(newPair -> { + List list = newMap.computeIfAbsent(newPair.getKey(), k -> new ArrayList<>()); + list.add(newPair.getValue()); + }); + return HoodieMapPair.of(newMap); + } + + @Override + public HoodiePairData>> leftOuterJoin(HoodiePairData other) { + Map> otherMapPairData = HoodieMapPair.getMapPair(other); + Stream>>>> pairs = streamAllPairs() + .map(pair -> new ImmutablePair<>(pair.getKey(), new ImmutablePair<>( + pair.getValue(), Option.ofNullable(otherMapPairData.get(pair.getKey()))))); + Map>>> resultMap = new HashMap<>(); + pairs.forEach(pair -> { + K key = pair.getKey(); + ImmutablePair>> valuePair = pair.getValue(); + List>> resultList = resultMap.computeIfAbsent(key, k -> new ArrayList<>()); + if (!valuePair.getRight().isPresent()) { + resultList.add(new ImmutablePair<>(valuePair.getLeft(), Option.empty())); + } else { + resultList.addAll(valuePair.getRight().get().stream().map( + w -> new ImmutablePair<>(valuePair.getLeft(), Option.of(w))).collect(Collectors.toList())); + } + }); + return HoodieMapPair.of(resultMap); + } + + private Stream> streamAllPairs() { + return mapPairData.entrySet().stream().flatMap( + entry -> entry.getValue().stream().map(e -> new ImmutablePair<>(entry.getKey(), e))); + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/data/HoodiePairData.java b/hudi-common/src/main/java/org/apache/hudi/common/data/HoodiePairData.java new file mode 100644 index 000000000..b9bdcb3d9 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/data/HoodiePairData.java @@ -0,0 +1,102 @@ +/* + * 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.data; + +import org.apache.hudi.common.function.SerializableFunction; +import org.apache.hudi.common.function.SerializablePairFunction; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.collection.Pair; + +import java.io.Serializable; +import java.util.Map; + +/** + * An abstraction for pairs of key in type K and value in type V to store the reference + * and do transformation. + * + * @param type of key. + * @param type of value. + */ +public abstract class HoodiePairData implements Serializable { + /** + * @return the collection of pairs. + */ + public abstract Object get(); + + /** + * Caches the data. + * + * @param cacheConfig config value for caching. + */ + public abstract void persist(String cacheConfig); + + /** + * Removes the cached data. + */ + public abstract void unpersist(); + + /** + * @return all keys in {@link HoodieData}. + */ + public abstract HoodieData keys(); + + /** + * @return all values in {@link HoodieData}. + */ + public abstract HoodieData values(); + + /** + * @return the number of pairs. + */ + public abstract long count(); + + /** + * @return the number of pairs per key in a {@link Map}. + */ + public abstract Map countByKey(); + + /** + * @param func serializable map function. + * @param output object type. + * @return {@link HoodieData} containing the result. Actual execution may be deferred. + */ + public abstract HoodieData map(SerializableFunction, O> func); + + /** + * @param mapToPairFunc serializable map function to generate another pair. + * @param new key type. + * @param new value type. + * @return {@link HoodiePairData} containing the result. Actual execution may be deferred. + */ + public abstract HoodiePairData mapToPair( + SerializablePairFunction, L, W> mapToPairFunc); + + /** + * Performs a left outer join of this and other. For each element (k, v) in this, + * the resulting HoodiePairData will either contain all pairs (k, (v, Some(w))) for w in other, + * or the pair (k, (v, None)) if no elements in other have key k. + * + * @param other the other {@link HoodiePairData} + * @param value type of the other {@link HoodiePairData} + * @return {@link HoodiePairData>>} containing the left outer join result. + * Actual execution may be deferred. + */ + public abstract HoodiePairData>> leftOuterJoin(HoodiePairData other); +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/engine/HoodieEngineContext.java b/hudi-common/src/main/java/org/apache/hudi/common/engine/HoodieEngineContext.java index 4e5120ab1..d400a10f6 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/engine/HoodieEngineContext.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/engine/HoodieEngineContext.java @@ -65,7 +65,11 @@ public abstract class HoodieEngineContext { public abstract HoodieData emptyHoodieData(); - public abstract HoodieData parallelize(List data); + public HoodieData parallelize(List data) { + return parallelize(data, data.size()); + } + + public abstract HoodieData parallelize(List data, int parallelism); public abstract List map(List data, SerializableFunction func, int parallelism); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/engine/HoodieLocalEngineContext.java b/hudi-common/src/main/java/org/apache/hudi/common/engine/HoodieLocalEngineContext.java index 61cbaed02..c99430e28 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/engine/HoodieLocalEngineContext.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/engine/HoodieLocalEngineContext.java @@ -75,7 +75,7 @@ public final class HoodieLocalEngineContext extends HoodieEngineContext { } @Override - public HoodieData parallelize(List data) { + public HoodieData parallelize(List data, int parallelism) { return HoodieList.of(data); } diff --git a/hudi-common/src/test/java/org/apache/hudi/common/data/TestHoodieMapPair.java b/hudi-common/src/test/java/org/apache/hudi/common/data/TestHoodieMapPair.java new file mode 100644 index 000000000..86b1a213b --- /dev/null +++ b/hudi-common/src/test/java/org/apache/hudi/common/data/TestHoodieMapPair.java @@ -0,0 +1,216 @@ +/* + * 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.data; + +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.collection.ImmutablePair; +import org.apache.hudi.common.util.collection.Pair; + +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Comparator; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +import static org.junit.jupiter.api.Assertions.assertEquals; + +public class TestHoodieMapPair { + + private static final String KEY1 = "key1"; + private static final String KEY2 = "key2"; + private static final String KEY3 = "key3"; + private static final String KEY4 = "key4"; + private static final String KEY5 = "key5"; + + private static final String STRING_VALUE1 = "value1"; + private static final String STRING_VALUE2 = "value2"; + private static final String STRING_VALUE3 = "value3"; + private static final String STRING_VALUE4 = "value4"; + private static final String STRING_VALUE5 = "value5"; + private static final String STRING_VALUE6 = "value6"; + + private static final int INTEGER_VALUE1 = 1; + private static final int INTEGER_VALUE2 = 2; + private static final int INTEGER_VALUE3 = 3; + private static final int INTEGER_VALUE4 = 4; + private static final int INTEGER_VALUE5 = 5; + + private static List> TEST_PAIRS; + private static HoodiePairData TEST_HOODIE_MAP_PAIR; + + @BeforeAll + public static void setup() { + TEST_PAIRS = constructPairs(); + TEST_HOODIE_MAP_PAIR = constructTestMapPairData(TEST_PAIRS); + } + + @Test + public void testKeys() { + assertHoodieDataEquals(Arrays.asList(KEY1, KEY2, KEY3, KEY4), TEST_HOODIE_MAP_PAIR.keys()); + } + + @Test + public void testValues() { + assertHoodieDataEquals(Arrays.asList( + STRING_VALUE1, STRING_VALUE2, STRING_VALUE3, STRING_VALUE4, STRING_VALUE5, STRING_VALUE6), + TEST_HOODIE_MAP_PAIR.values()); + } + + @Test + public void testCount() { + assertEquals(6, TEST_HOODIE_MAP_PAIR.count()); + } + + @Test + public void testCountByKey() { + Map expectedResultMap = new HashMap<>(); + expectedResultMap.put(KEY1, 2L); + expectedResultMap.put(KEY2, 2L); + expectedResultMap.put(KEY3, 1L); + expectedResultMap.put(KEY4, 1L); + + assertEquals(expectedResultMap, TEST_HOODIE_MAP_PAIR.countByKey()); + } + + @Test + public void testMap() { + assertHoodieDataEquals(Arrays.asList( + "key1,value1", "key1,value2", "key2,value3", "key2,value4", "key3,value5", "key4,value6"), + TEST_HOODIE_MAP_PAIR.map(pair -> pair.getKey() + "," + pair.getValue())); + } + + @Test + public void testMapToPair() { + Map> expectedResultMap = new HashMap<>(); + expectedResultMap.put("key10", Arrays.asList(1, 2)); + expectedResultMap.put("key20", Arrays.asList(3, 4)); + expectedResultMap.put("key30", Arrays.asList(5)); + expectedResultMap.put("key40", Arrays.asList(6)); + assertEquals(expectedResultMap, HoodieMapPair.getMapPair( + TEST_HOODIE_MAP_PAIR.mapToPair( + pair -> { + String value = pair.getValue(); + return new ImmutablePair<>(pair.getKey() + "0", + Integer.parseInt(String.valueOf(value.charAt(value.length() - 1)))); + }))); + } + + @Test + public void testLeftOuterJoinSingleValuePerKey() { + HoodiePairData pairData1 = constructTestMapPairData(Arrays.asList( + ImmutablePair.of(KEY1, STRING_VALUE1), + ImmutablePair.of(KEY2, STRING_VALUE2), + ImmutablePair.of(KEY3, STRING_VALUE3), + ImmutablePair.of(KEY4, STRING_VALUE4) + )); + + HoodiePairData pairData2 = constructTestMapPairData(Arrays.asList( + ImmutablePair.of(KEY1, INTEGER_VALUE1), + ImmutablePair.of(KEY2, INTEGER_VALUE2), + ImmutablePair.of(KEY5, INTEGER_VALUE3) + )); + + Map>>> expectedResultMap = new HashMap<>(); + expectedResultMap.put(KEY1, Arrays.asList( + ImmutablePair.of(STRING_VALUE1, Option.of(INTEGER_VALUE1)))); + expectedResultMap.put(KEY2, Arrays.asList( + ImmutablePair.of(STRING_VALUE2, Option.of(INTEGER_VALUE2)))); + expectedResultMap.put(KEY3, Arrays.asList( + ImmutablePair.of(STRING_VALUE3, Option.empty()))); + expectedResultMap.put(KEY4, Arrays.asList( + ImmutablePair.of(STRING_VALUE4, Option.empty()))); + + assertEquals(expectedResultMap, + HoodieMapPair.getMapPair(pairData1.leftOuterJoin(pairData2))); + } + + @Test + public void testLeftOuterJoinMultipleValuesPerKey() { + HoodiePairData otherPairData = constructTestMapPairData(Arrays.asList( + ImmutablePair.of(KEY1, INTEGER_VALUE1), + ImmutablePair.of(KEY2, INTEGER_VALUE2), + ImmutablePair.of(KEY2, INTEGER_VALUE3), + ImmutablePair.of(KEY3, INTEGER_VALUE4), + ImmutablePair.of(KEY5, INTEGER_VALUE5) + )); + + Map>>> expectedResultMap = new HashMap<>(); + expectedResultMap.put(KEY1, Arrays.asList( + ImmutablePair.of(STRING_VALUE1, Option.of(INTEGER_VALUE1)), + ImmutablePair.of(STRING_VALUE2, Option.of(INTEGER_VALUE1)))); + expectedResultMap.put(KEY2, Arrays.asList( + ImmutablePair.of(STRING_VALUE3, Option.of(INTEGER_VALUE2)), + ImmutablePair.of(STRING_VALUE3, Option.of(INTEGER_VALUE3)), + ImmutablePair.of(STRING_VALUE4, Option.of(INTEGER_VALUE2)), + ImmutablePair.of(STRING_VALUE4, Option.of(INTEGER_VALUE3)))); + expectedResultMap.put(KEY3, Arrays.asList( + ImmutablePair.of(STRING_VALUE5, Option.of(INTEGER_VALUE4)))); + expectedResultMap.put(KEY4, Arrays.asList( + ImmutablePair.of(STRING_VALUE6, Option.empty()))); + + assertEquals(expectedResultMap, + HoodieMapPair.getMapPair(TEST_HOODIE_MAP_PAIR.leftOuterJoin(otherPairData))); + } + + private static List> constructPairs() { + return Arrays.asList( + ImmutablePair.of(KEY1, STRING_VALUE1), + ImmutablePair.of(KEY1, STRING_VALUE2), + ImmutablePair.of(KEY2, STRING_VALUE3), + ImmutablePair.of(KEY2, STRING_VALUE4), + ImmutablePair.of(KEY3, STRING_VALUE5), + ImmutablePair.of(KEY4, STRING_VALUE6) + ); + } + + private static HoodiePairData constructTestMapPairData( + final List> pairs) { + Map> map = new HashMap<>(); + addPairsToMap(map, pairs); + return HoodieMapPair.of(map); + } + + private static void addPairsToMap( + Map> map, final List> pairs) { + for (Pair pair : pairs) { + String key = pair.getKey(); + V value = pair.getValue(); + List list = map.computeIfAbsent(key, k -> new ArrayList<>()); + list.add(value); + } + } + + private void assertHoodieDataEquals( + List expectedList, HoodieData hoodieData) { + assertHoodieDataEquals(expectedList, hoodieData, Comparator.naturalOrder()); + } + + private void assertHoodieDataEquals( + List expectedList, HoodieData hoodieData, Comparator comparator) { + assertEquals(expectedList, + hoodieData.collectAsList().stream().sorted(comparator).collect(Collectors.toList()) + ); + } +} diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala index 9726be894..e5c1a7a14 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala @@ -37,7 +37,6 @@ import org.apache.hudi.config.{HoodieInternalConfig, HoodieWriteConfig} import org.apache.hudi.exception.HoodieException import org.apache.hudi.execution.bulkinsert.{BulkInsertInternalPartitionerWithRowsFactory, NonSortPartitionerWithRows} import org.apache.hudi.hive.{HiveSyncConfig, HiveSyncTool} -import org.apache.hudi.index.SparkHoodieIndex import org.apache.hudi.internal.DataSourceInternalWriterHelper import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory import org.apache.hudi.sync.common.AbstractSyncTool @@ -49,9 +48,11 @@ import org.apache.spark.sql.internal.{SQLConf, StaticSQLConf} import org.apache.spark.sql.types.StructType import org.apache.spark.sql.{DataFrame, Dataset, Row, SQLContext, SaveMode, SparkSession} import org.apache.spark.{SPARK_VERSION, SparkContext} - import java.util import java.util.Properties + +import org.apache.hudi.index.SparkHoodieIndexFactory + import scala.collection.JavaConversions._ import scala.collection.mutable.ListBuffer @@ -439,7 +440,7 @@ object HoodieSparkSqlWriter { val arePartitionRecordsSorted = bulkInsertPartitionerRows.arePartitionRecordsSorted(); parameters.updated(HoodieInternalConfig.BULKINSERT_ARE_PARTITIONER_RECORDS_SORTED, arePartitionRecordsSorted.toString) val isGlobalIndex = if (populateMetaFields) { - SparkHoodieIndex.isGlobalIndex(writeConfig) + SparkHoodieIndexFactory.isGlobalIndex(writeConfig) } else { false }