[HUDI-2502] Refactor index in hudi-client module (#3778)
- Refactor Index to reduce Line of Code and re-use across engines.
This commit is contained in:
@@ -101,7 +101,7 @@ public abstract class AbstractHoodieWriteClient<T extends HoodieRecordPayload, I
|
||||
private static final Logger LOG = LogManager.getLogger(AbstractHoodieWriteClient.class);
|
||||
|
||||
protected final transient HoodieMetrics metrics;
|
||||
private final transient HoodieIndex<T, I, K, O> index;
|
||||
private final transient HoodieIndex<T, ?, ?, ?> index;
|
||||
|
||||
protected transient Timer.Context writeTimer = null;
|
||||
protected transient Timer.Context compactionTimer;
|
||||
@@ -138,7 +138,7 @@ public abstract class AbstractHoodieWriteClient<T extends HoodieRecordPayload, I
|
||||
this.txnManager = new TransactionManager(config, fs);
|
||||
}
|
||||
|
||||
protected abstract HoodieIndex<T, I, K, O> createIndex(HoodieWriteConfig writeConfig);
|
||||
protected abstract HoodieIndex<T, ?, ?, ?> createIndex(HoodieWriteConfig writeConfig);
|
||||
|
||||
public void setOperationType(WriteOperationType operationType) {
|
||||
this.operationType = operationType;
|
||||
@@ -1006,7 +1006,7 @@ public abstract class AbstractHoodieWriteClient<T extends HoodieRecordPayload, I
|
||||
return metrics;
|
||||
}
|
||||
|
||||
public HoodieIndex<T, I, K, O> getIndex() {
|
||||
public HoodieIndex<T, ?, ?, ?> getIndex() {
|
||||
return index;
|
||||
}
|
||||
|
||||
|
||||
@@ -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 <T> Sub type of HoodieRecordPayload
|
||||
* @param <I> Type of inputs
|
||||
* @param <K> Type of keys
|
||||
* @param <O> Type of outputs
|
||||
* @param <I> Type of inputs for deprecated APIs
|
||||
* @param <K> Type of keys for deprecated APIs
|
||||
* @param <O> Type of outputs for deprecated APIs
|
||||
*/
|
||||
@PublicAPIClass(maturity = ApiMaturityLevel.EVOLVING)
|
||||
public abstract class HoodieIndex<T extends HoodieRecordPayload, I, K, O> implements Serializable {
|
||||
@@ -52,18 +56,39 @@ public abstract class HoodieIndex<T extends HoodieRecordPayload, I, K, O> 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<T, I, K, O> hoodieTable) throws HoodieIndexException;
|
||||
@Deprecated
|
||||
@PublicAPIMethod(maturity = ApiMaturityLevel.DEPRECATED)
|
||||
public I tagLocation(I records, HoodieEngineContext context,
|
||||
HoodieTable<T, I, K, O> hoodieTable) throws HoodieIndexException {
|
||||
throw new HoodieNotSupportedException("Deprecated API should not be called");
|
||||
}
|
||||
|
||||
/**
|
||||
* Extracts the location of written records, and updates the index.
|
||||
* <p>
|
||||
* 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<T, I, K, O> hoodieTable) throws HoodieIndexException;
|
||||
@Deprecated
|
||||
@PublicAPIMethod(maturity = ApiMaturityLevel.DEPRECATED)
|
||||
public O updateLocation(O writeStatuses, HoodieEngineContext context,
|
||||
HoodieTable<T, I, K, O> 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<HoodieRecord<T>> tagLocation(
|
||||
HoodieData<HoodieRecord<T>> records, HoodieEngineContext context,
|
||||
HoodieTable hoodieTable) throws HoodieIndexException;
|
||||
|
||||
/**
|
||||
* Extracts the location of written records, and updates the index.
|
||||
*/
|
||||
@PublicAPIMethod(maturity = ApiMaturityLevel.EVOLVING)
|
||||
public abstract HoodieData<WriteStatus> updateLocation(
|
||||
HoodieData<WriteStatus> writeStatuses, HoodieEngineContext context,
|
||||
HoodieTable hoodieTable) throws HoodieIndexException;
|
||||
|
||||
/**
|
||||
* Rollback the effects of the commit made at instantTime.
|
||||
|
||||
@@ -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 <RowKey, filename> 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<HoodieKey, HoodieRecordLocation> findMatchingFilesForRecordKeys(
|
||||
HoodieWriteConfig config, HoodieEngineContext context, HoodieTable hoodieTable,
|
||||
HoodiePairData<String, String> partitionRecordKeyPairs,
|
||||
HoodieData<ImmutablePair<String, HoodieKey>> fileComparisonPairs,
|
||||
Map<String, List<BloomIndexFileInfo>> partitionToFileInfo,
|
||||
Map<String, Long> recordsPerPartition);
|
||||
}
|
||||
@@ -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<T extends HoodieRecordPayload> extends HoodieIndex<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>> {
|
||||
|
||||
private static final Logger LOG = LogManager.getLogger(HoodieBaseBloomIndex.class);
|
||||
|
||||
public HoodieBaseBloomIndex(HoodieWriteConfig config) {
|
||||
super(config);
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<HoodieRecord<T>> tagLocation(List<HoodieRecord<T>> records, HoodieEngineContext context,
|
||||
HoodieTable<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>> hoodieTable) {
|
||||
// Step 1: Extract out thinner Map of (partitionPath, recordKey)
|
||||
Map<String, List<String>> partitionRecordKeyMap = new HashMap<>();
|
||||
records.forEach(record -> {
|
||||
if (partitionRecordKeyMap.containsKey(record.getPartitionPath())) {
|
||||
partitionRecordKeyMap.get(record.getPartitionPath()).add(record.getRecordKey());
|
||||
} else {
|
||||
List<String> recordKeys = Lists.newArrayList();
|
||||
recordKeys.add(record.getRecordKey());
|
||||
partitionRecordKeyMap.put(record.getPartitionPath(), recordKeys);
|
||||
}
|
||||
});
|
||||
|
||||
// Step 2: Lookup indexes for all the partition/recordkey pair
|
||||
Map<HoodieKey, HoodieRecordLocation> 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<HoodieRecord<T>> taggedRecords = tagLocationBacktoRecords(keyFilenamePairMap, records);
|
||||
|
||||
return taggedRecords;
|
||||
}
|
||||
|
||||
/**
|
||||
* Lookup the location for each record key and return the pair<record_key,location> for all record keys already
|
||||
* present and drop the record keys if not present.
|
||||
*/
|
||||
private Map<HoodieKey, HoodieRecordLocation> lookupIndex(
|
||||
Map<String, List<String>> partitionRecordKeyMap, final HoodieEngineContext context,
|
||||
final HoodieTable hoodieTable) {
|
||||
// Obtain records per partition, in the incoming records
|
||||
Map<String, Long> recordsPerPartition = new HashMap<>();
|
||||
partitionRecordKeyMap.keySet().forEach(k -> recordsPerPartition.put(k, Long.valueOf(partitionRecordKeyMap.get(k).size())));
|
||||
List<String> affectedPartitionPathList = new ArrayList<>(recordsPerPartition.keySet());
|
||||
|
||||
// Step 2: Load all involved files as <Partition, filename> pairs
|
||||
List<Pair<String, BloomIndexFileInfo>> fileInfoList =
|
||||
loadInvolvedFiles(affectedPartitionPathList, context, hoodieTable);
|
||||
final Map<String, List<BloomIndexFileInfo>> 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<Pair<String, HoodieKey>> fileComparisons =
|
||||
explodeRecordsWithFileComparisons(partitionToFileInfo, partitionRecordKeyMap);
|
||||
return findMatchingFilesForRecordKeys(fileComparisons, hoodieTable);
|
||||
}
|
||||
|
||||
/**
|
||||
* Load all involved files as <Partition, filename> pair List.
|
||||
*/
|
||||
//TODO duplicate code with spark, we can optimize this method later
|
||||
List<Pair<String, BloomIndexFileInfo>> loadInvolvedFiles(List<String> partitions, final HoodieEngineContext context,
|
||||
final HoodieTable hoodieTable) {
|
||||
// Obtain the latest data files from all the partitions.
|
||||
List<Pair<String, String>> 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.
|
||||
* <p>
|
||||
* 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<Pair<String, HoodieKey>> explodeRecordsWithFileComparisons(
|
||||
final Map<String, List<BloomIndexFileInfo>> partitionToFileIndexInfo,
|
||||
Map<String, List<String>> partitionRecordKeyMap) {
|
||||
IndexFileFilter indexFileFilter =
|
||||
config.useBloomIndexTreebasedFilter() ? new IntervalTreeBasedIndexFileFilter(partitionToFileIndexInfo)
|
||||
: new ListBasedIndexFileFilter(partitionToFileIndexInfo);
|
||||
|
||||
List<Pair<String, HoodieKey>> fileRecordPairs = new ArrayList<>();
|
||||
partitionRecordKeyMap.keySet().forEach(partitionPath -> {
|
||||
List<String> 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 <RowKey, filename> pair.
|
||||
*/
|
||||
Map<HoodieKey, HoodieRecordLocation> findMatchingFilesForRecordKeys(
|
||||
List<Pair<String, HoodieKey>> fileComparisons,
|
||||
HoodieTable hoodieTable) {
|
||||
|
||||
fileComparisons = fileComparisons.stream().sorted((o1, o2) -> o1.getLeft().compareTo(o2.getLeft())).collect(toList());
|
||||
|
||||
List<HoodieKeyLookupHandle.KeyLookupResult> keyLookupResults = new ArrayList<>();
|
||||
|
||||
Iterator<List<HoodieKeyLookupHandle.KeyLookupResult>> iterator = new HoodieBaseBloomIndexCheckFunction(hoodieTable, config).apply(fileComparisons.iterator());
|
||||
while (iterator.hasNext()) {
|
||||
keyLookupResults.addAll(iterator.next());
|
||||
}
|
||||
|
||||
Map<HoodieKey, HoodieRecordLocation> 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 <rowKey, filename> back to the original HoodieRecord List.
|
||||
*/
|
||||
protected List<HoodieRecord<T>> tagLocationBacktoRecords(
|
||||
Map<HoodieKey, HoodieRecordLocation> keyFilenamePair, List<HoodieRecord<T>> records) {
|
||||
Map<HoodieKey, HoodieRecord<T>> 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<Pair<HoodieRecord<T>, 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<HoodieRecord<T>> res = Lists.newArrayList();
|
||||
for (Pair<HoodieRecord<T>, HoodieRecordLocation> v : newList) {
|
||||
res.add(HoodieIndexUtils.getTaggedRecord(v.getLeft(), Option.ofNullable(v.getRight())));
|
||||
}
|
||||
return res;
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<WriteStatus> updateLocation(List<WriteStatus> writeStatusList, HoodieEngineContext context,
|
||||
HoodieTable<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>> hoodieTable) {
|
||||
return writeStatusList;
|
||||
}
|
||||
}
|
||||
@@ -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<T extends HoodieRecordPayload<T>>
|
||||
extends HoodieIndex<T, Object, Object, Object> {
|
||||
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<HoodieRecord<T>> tagLocation(
|
||||
HoodieData<HoodieRecord<T>> 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<String, String> partitionRecordKeyPairs = records.mapToPair(
|
||||
record -> new ImmutablePair<>(record.getPartitionPath(), record.getRecordKey()));
|
||||
|
||||
// Step 2: Lookup indexes for all the partition/recordkey pair
|
||||
HoodiePairData<HoodieKey, HoodieRecordLocation> 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<HoodieRecord<T>> taggedRecords = tagLocationBacktoRecords(keyFilenamePairs, records);
|
||||
|
||||
if (config.getBloomIndexUseCaching()) {
|
||||
records.unpersist();
|
||||
keyFilenamePairs.unpersist();
|
||||
}
|
||||
|
||||
return taggedRecords;
|
||||
}
|
||||
|
||||
/**
|
||||
* Lookup the location for each record key and return the pair<record_key,location> for all record keys already
|
||||
* present and drop the record keys if not present.
|
||||
*/
|
||||
private HoodiePairData<HoodieKey, HoodieRecordLocation> lookupIndex(
|
||||
HoodiePairData<String, String> partitionRecordKeyPairs, final HoodieEngineContext context,
|
||||
final HoodieTable hoodieTable) {
|
||||
// Obtain records per partition, in the incoming records
|
||||
Map<String, Long> recordsPerPartition = partitionRecordKeyPairs.countByKey();
|
||||
List<String> affectedPartitionPathList = new ArrayList<>(recordsPerPartition.keySet());
|
||||
|
||||
// Step 2: Load all involved files as <Partition, filename> pairs
|
||||
List<Pair<String, BloomIndexFileInfo>> fileInfoList =
|
||||
loadInvolvedFiles(affectedPartitionPathList, context, hoodieTable);
|
||||
final Map<String, List<BloomIndexFileInfo>> 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<ImmutablePair<String, HoodieKey>> fileComparisonPairs =
|
||||
explodeRecordsWithFileComparisons(partitionToFileInfo, partitionRecordKeyPairs);
|
||||
|
||||
return bloomIndexHelper.findMatchingFilesForRecordKeys(config, context, hoodieTable,
|
||||
partitionRecordKeyPairs, fileComparisonPairs, partitionToFileInfo, recordsPerPartition);
|
||||
}
|
||||
|
||||
/**
|
||||
* Load all involved files as <Partition, filename> pair List.
|
||||
*/
|
||||
List<Pair<String, BloomIndexFileInfo>> loadInvolvedFiles(
|
||||
List<String> partitions, final HoodieEngineContext context, final HoodieTable hoodieTable) {
|
||||
// Obtain the latest data files from all the partitions.
|
||||
List<Pair<String, String>> 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.
|
||||
* <p>
|
||||
* 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<ImmutablePair<String, HoodieKey>> explodeRecordsWithFileComparisons(
|
||||
final Map<String, List<BloomIndexFileInfo>> partitionToFileIndexInfo,
|
||||
HoodiePairData<String, String> 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 <rowKey, filename> back to the original HoodieRecord List.
|
||||
*/
|
||||
protected HoodieData<HoodieRecord<T>> tagLocationBacktoRecords(
|
||||
HoodiePairData<HoodieKey, HoodieRecordLocation> keyFilenamePair,
|
||||
HoodieData<HoodieRecord<T>> records) {
|
||||
HoodiePairData<HoodieKey, HoodieRecord<T>> 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<WriteStatus> updateLocation(
|
||||
HoodieData<WriteStatus> writeStatusData, HoodieEngineContext context,
|
||||
HoodieTable hoodieTable) {
|
||||
return writeStatusData;
|
||||
}
|
||||
}
|
||||
@@ -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<T extends HoodieRecordPayload> extends SparkHoodieBloomIndex<T> {
|
||||
|
||||
public SparkHoodieGlobalBloomIndex(HoodieWriteConfig config) {
|
||||
super(config);
|
||||
public class HoodieGlobalBloomIndex<T extends HoodieRecordPayload<T>> extends HoodieBloomIndex<T> {
|
||||
public HoodieGlobalBloomIndex(HoodieWriteConfig config, BaseHoodieBloomIndexHelper bloomIndexHelper) {
|
||||
super(config, bloomIndexHelper);
|
||||
}
|
||||
|
||||
/**
|
||||
* Load all involved files as <Partition, filename> pair RDD from all partitions in the table.
|
||||
* Load all involved files as <Partition, filename> pairs from all partitions in the table.
|
||||
*/
|
||||
@Override
|
||||
List<Tuple2<String, BloomIndexFileInfo>> loadInvolvedFiles(List<String> partitions, final HoodieEngineContext context,
|
||||
final HoodieTable hoodieTable) {
|
||||
List<Pair<String, BloomIndexFileInfo>> loadInvolvedFiles(List<String> partitions, final HoodieEngineContext context,
|
||||
final HoodieTable hoodieTable) {
|
||||
HoodieTableMetaClient metaClient = hoodieTable.getMetaClient();
|
||||
List<String> allPartitionPaths = FSUtils.getAllPartitionPaths(context, config.getMetadataConfig(), metaClient.getBasePath());
|
||||
return super.loadInvolvedFiles(allPartitionPaths, context, hoodieTable);
|
||||
@@ -70,25 +68,25 @@ public class SparkHoodieGlobalBloomIndex<T extends HoodieRecordPayload> extends
|
||||
* to be compared gets cut down a lot from range pruning.
|
||||
* <p>
|
||||
* 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<Tuple2<String, HoodieKey>> explodeRecordRDDWithFileComparisons(
|
||||
HoodieData<ImmutablePair<String, HoodieKey>> explodeRecordsWithFileComparisons(
|
||||
final Map<String, List<BloomIndexFileInfo>> partitionToFileIndexInfo,
|
||||
JavaPairRDD<String, String> partitionRecordKeyPairRDD) {
|
||||
HoodiePairData<String, String> 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<T extends HoodieRecordPayload> extends
|
||||
* Tagging for global index should only consider the record key.
|
||||
*/
|
||||
@Override
|
||||
protected JavaRDD<HoodieRecord<T>> tagLocationBacktoRecords(
|
||||
JavaPairRDD<HoodieKey, HoodieRecordLocation> keyLocationPairRDD, JavaRDD<HoodieRecord<T>> recordRDD) {
|
||||
protected HoodieData<HoodieRecord<T>> tagLocationBacktoRecords(
|
||||
HoodiePairData<HoodieKey, HoodieRecordLocation> keyLocationPairs,
|
||||
HoodieData<HoodieRecord<T>> records) {
|
||||
|
||||
JavaPairRDD<String, HoodieRecord<T>> incomingRowKeyRecordPairRDD =
|
||||
recordRDD.mapToPair(record -> new Tuple2<>(record.getRecordKey(), record));
|
||||
HoodiePairData<String, HoodieRecord<T>> incomingRowKeyRecordPairs =
|
||||
records.mapToPair(record -> new ImmutablePair<>(record.getRecordKey(), record));
|
||||
|
||||
JavaPairRDD<String, Tuple2<HoodieRecordLocation, HoodieKey>> existingRecordKeyToRecordLocationHoodieKeyMap =
|
||||
keyLocationPairRDD.mapToPair(p -> new Tuple2<>(p._1.getRecordKey(), new Tuple2<>(p._2, p._1)));
|
||||
HoodiePairData<String, Pair<HoodieRecordLocation, HoodieKey>> 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<T> hoodieRecord = record._1;
|
||||
final Optional<Tuple2<HoodieRecordLocation, HoodieKey>> 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<T> hoodieRecord = record.getLeft();
|
||||
final Option<Pair<HoodieRecordLocation, HoodieKey>> 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<T> deleteRecord = new HoodieRecord(recordLocationHoodieKeyPair.get()._2,
|
||||
HoodieRecord<T> 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<T> insertRecord = HoodieIndexUtils.getTaggedRecord(hoodieRecord, Option.empty());
|
||||
@@ -127,8 +127,8 @@ public class SparkHoodieGlobalBloomIndex<T extends HoodieRecordPayload> 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<T>) HoodieIndexUtils.getTaggedRecord(new HoodieRecord<>(recordLocationHoodieKeyPair.get()._2, hoodieRecord.getData()),
|
||||
Option.ofNullable(recordLocationHoodieKeyPair.get()._1))).iterator();
|
||||
(HoodieRecord<T>) HoodieIndexUtils.getTaggedRecord(new HoodieRecord<>(recordLocationHoodieKeyPair.get().getRight(), hoodieRecord.getData()),
|
||||
Option.ofNullable(recordLocationHoodieKeyPair.get().getLeft()))).iterator();
|
||||
}
|
||||
} else {
|
||||
return Collections.singletonList((HoodieRecord<T>) HoodieIndexUtils.getTaggedRecord(hoodieRecord, Option.empty())).iterator();
|
||||
@@ -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<HoodieKey, HoodieRecordLocation> findMatchingFilesForRecordKeys(
|
||||
HoodieWriteConfig config, HoodieEngineContext context, HoodieTable hoodieTable,
|
||||
HoodiePairData<String, String> partitionRecordKeyPairs,
|
||||
HoodieData<ImmutablePair<String, HoodieKey>> fileComparisonPairs,
|
||||
Map<String, List<BloomIndexFileInfo>> partitionToFileInfo, Map<String, Long> recordsPerPartition) {
|
||||
List<Pair<String, HoodieKey>> fileComparisonPairList =
|
||||
HoodieList.getList(fileComparisonPairs).stream()
|
||||
.sorted(Comparator.comparing(ImmutablePair::getLeft)).collect(toList());
|
||||
|
||||
List<HoodieKeyLookupHandle.KeyLookupResult> keyLookupResults = new ArrayList<>();
|
||||
|
||||
Iterator<List<HoodieKeyLookupHandle.KeyLookupResult>> 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()));
|
||||
});
|
||||
}
|
||||
}
|
||||
@@ -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.
|
||||
* <p>
|
||||
* ONLY USE FOR LOCAL TESTING
|
||||
*/
|
||||
@SuppressWarnings("checkstyle:LineLength")
|
||||
public class JavaInMemoryHashIndex<T extends HoodieRecordPayload> extends JavaHoodieIndex<T> {
|
||||
public class HoodieInMemoryHashIndex<T extends HoodieRecordPayload<T>>
|
||||
extends HoodieIndex<T, Object, Object, Object> {
|
||||
|
||||
private static ConcurrentMap<HoodieKey, HoodieRecordLocation> 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<T extends HoodieRecordPayload> extends JavaHo
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<HoodieRecord<T>> tagLocation(List<HoodieRecord<T>> records, HoodieEngineContext context,
|
||||
HoodieTable<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>> hoodieTable) {
|
||||
List<HoodieRecord<T>> taggedRecords = new ArrayList<>();
|
||||
records.stream().forEach(record -> {
|
||||
if (recordLocationMap.containsKey(record.getKey())) {
|
||||
record.unseal();
|
||||
record.setCurrentLocation(recordLocationMap.get(record.getKey()));
|
||||
record.seal();
|
||||
public HoodieData<HoodieRecord<T>> tagLocation(
|
||||
HoodieData<HoodieRecord<T>> records, HoodieEngineContext context,
|
||||
HoodieTable hoodieTable) {
|
||||
return records.mapPartitions(hoodieRecordIterator -> {
|
||||
List<HoodieRecord<T>> taggedRecords = new ArrayList<>();
|
||||
while (hoodieRecordIterator.hasNext()) {
|
||||
HoodieRecord<T> 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<WriteStatus> updateLocation(List<WriteStatus> writeStatusList,
|
||||
HoodieEngineContext context,
|
||||
HoodieTable<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>> hoodieTable) {
|
||||
return writeStatusList.stream().map(writeStatus -> {
|
||||
public HoodieData<WriteStatus> updateLocation(
|
||||
HoodieData<WriteStatus> 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<T extends HoodieRecordPayload> extends JavaHo
|
||||
}
|
||||
}
|
||||
return writeStatus;
|
||||
}).collect(Collectors.toList());
|
||||
});
|
||||
}
|
||||
|
||||
@Override
|
||||
@@ -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 <T>
|
||||
*/
|
||||
@SuppressWarnings("checkstyle:LineLength")
|
||||
public class SparkHoodieGlobalSimpleIndex<T extends HoodieRecordPayload> extends SparkHoodieSimpleIndex<T> {
|
||||
|
||||
public SparkHoodieGlobalSimpleIndex(HoodieWriteConfig config) {
|
||||
super(config);
|
||||
public class HoodieGlobalSimpleIndex<T extends HoodieRecordPayload<T>> extends HoodieSimpleIndex<T> {
|
||||
public HoodieGlobalSimpleIndex(HoodieWriteConfig config, Option<BaseKeyGenerator> keyGeneratorOpt) {
|
||||
super(config, keyGeneratorOpt);
|
||||
}
|
||||
|
||||
@Override
|
||||
public JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> recordRDD, HoodieEngineContext context,
|
||||
HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> hoodieTable) {
|
||||
return tagLocationInternal(recordRDD, context, hoodieTable);
|
||||
public HoodieData<HoodieRecord<T>> tagLocation(
|
||||
HoodieData<HoodieRecord<T>> 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<HoodieRecord<T>> tagLocationInternal(JavaRDD<HoodieRecord<T>> inputRecordRDD, HoodieEngineContext context,
|
||||
HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> hoodieTable) {
|
||||
protected HoodieData<HoodieRecord<T>> tagLocationInternal(
|
||||
HoodieData<HoodieRecord<T>> inputRecords, HoodieEngineContext context,
|
||||
HoodieTable hoodieTable) {
|
||||
|
||||
JavaPairRDD<String, HoodieRecord<T>> keyedInputRecordRDD = inputRecordRDD.mapToPair(entry -> new Tuple2<>(entry.getRecordKey(), entry));
|
||||
JavaPairRDD<HoodieKey, HoodieRecordLocation> allRecordLocationsInTable = fetchAllRecordLocations(context, hoodieTable,
|
||||
config.getGlobalSimpleIndexParallelism());
|
||||
return getTaggedRecords(keyedInputRecordRDD, allRecordLocationsInTable);
|
||||
HoodiePairData<String, HoodieRecord<T>> keyedInputRecords =
|
||||
inputRecords.mapToPair(entry -> new ImmutablePair<>(entry.getRecordKey(), entry));
|
||||
HoodiePairData<HoodieKey, HoodieRecordLocation> 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<HoodieKey, HoodieRecordLocation> fetchAllRecordLocations(HoodieEngineContext context,
|
||||
HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> hoodieTable,
|
||||
int parallelism) {
|
||||
protected HoodiePairData<HoodieKey, HoodieRecordLocation> fetchAllRecordLocations(
|
||||
HoodieEngineContext context, HoodieTable hoodieTable, int parallelism) {
|
||||
List<Pair<String, HoodieBaseFile>> latestBaseFiles = getAllBaseFilesInTable(context, hoodieTable);
|
||||
return fetchRecordLocations(context, hoodieTable, parallelism, latestBaseFiles);
|
||||
}
|
||||
|
||||
/**
|
||||
* Load all files for all partitions as <Partition, filename> pair RDD.
|
||||
* Load all files for all partitions as <Partition, filename> pair data.
|
||||
*/
|
||||
protected List<Pair<String, HoodieBaseFile>> getAllBaseFilesInTable(final HoodieEngineContext context,
|
||||
final HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> hoodieTable) {
|
||||
protected List<Pair<String, HoodieBaseFile>> getAllBaseFilesInTable(
|
||||
final HoodieEngineContext context, final HoodieTable hoodieTable) {
|
||||
HoodieTableMetaClient metaClient = hoodieTable.getMetaClient();
|
||||
List<String> allPartitionPaths = FSUtils.getAllPartitionPaths(context, config.getMetadataConfig(), metaClient.getBasePath());
|
||||
// Obtain the latest data files from all the partitions.
|
||||
@@ -111,16 +112,20 @@ public class SparkHoodieGlobalSimpleIndex<T extends HoodieRecordPayload> 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<HoodieRecord<T>> getTaggedRecords(JavaPairRDD<String, HoodieRecord<T>> incomingRecords, JavaPairRDD<HoodieKey, HoodieRecordLocation> existingRecords) {
|
||||
JavaPairRDD<String, Pair<String, HoodieRecordLocation>> existingRecordByRecordKey = existingRecords
|
||||
.mapToPair(entry -> new Tuple2<>(entry._1.getRecordKey(), Pair.of(entry._1.getPartitionPath(), entry._2)));
|
||||
private HoodieData<HoodieRecord<T>> getTaggedRecords(
|
||||
HoodiePairData<String, HoodieRecord<T>> incomingRecords,
|
||||
HoodiePairData<HoodieKey, HoodieRecordLocation> existingRecords) {
|
||||
HoodiePairData<String, Pair<String, HoodieRecordLocation>> existingRecordByRecordKey =
|
||||
existingRecords.mapToPair(
|
||||
entry -> new ImmutablePair<>(entry.getLeft().getRecordKey(),
|
||||
Pair.of(entry.getLeft().getPartitionPath(), entry.getRight())));
|
||||
|
||||
return incomingRecords.leftOuterJoin(existingRecordByRecordKey).values()
|
||||
.flatMap(entry -> {
|
||||
HoodieRecord<T> inputRecord = entry._1;
|
||||
Option<Pair<String, HoodieRecordLocation>> partitionPathLocationPair = Option.ofNullable(entry._2.orNull());
|
||||
HoodieRecord<T> inputRecord = entry.getLeft();
|
||||
Option<Pair<String, HoodieRecordLocation>> partitionPathLocationPair = Option.ofNullable(entry.getRight().orElse(null));
|
||||
List<HoodieRecord<T>> taggedRecords;
|
||||
|
||||
if (partitionPathLocationPair.isPresent()) {
|
||||
@@ -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 <T> type of {@link HoodieRecordPayload}
|
||||
*/
|
||||
public class HoodieSimpleIndex<T extends HoodieRecordPayload<T>>
|
||||
extends HoodieIndex<T, Object, Object, Object> {
|
||||
|
||||
private final Option<BaseKeyGenerator> keyGeneratorOpt;
|
||||
|
||||
public HoodieSimpleIndex(HoodieWriteConfig config, Option<BaseKeyGenerator> keyGeneratorOpt) {
|
||||
super(config);
|
||||
this.keyGeneratorOpt = keyGeneratorOpt;
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodieData<WriteStatus> updateLocation(
|
||||
HoodieData<WriteStatus> 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<HoodieRecord<T>> tagLocation(
|
||||
HoodieData<HoodieRecord<T>> 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<HoodieRecord<T>> tagLocationInternal(
|
||||
HoodieData<HoodieRecord<T>> inputRecords, HoodieEngineContext context,
|
||||
HoodieTable hoodieTable) {
|
||||
if (config.getSimpleIndexUseCaching()) {
|
||||
inputRecords.persist(new HoodieConfig(config.getProps())
|
||||
.getString(HoodieIndexConfig.SIMPLE_INDEX_INPUT_STORAGE_LEVEL_VALUE));
|
||||
}
|
||||
|
||||
HoodiePairData<HoodieKey, HoodieRecord<T>> keyedInputRecords =
|
||||
inputRecords.mapToPair(record -> new ImmutablePair<>(record.getKey(), record));
|
||||
HoodiePairData<HoodieKey, HoodieRecordLocation> existingLocationsOnTable =
|
||||
fetchRecordLocationsForAffectedPartitions(keyedInputRecords.keys(), context, hoodieTable,
|
||||
config.getSimpleIndexParallelism());
|
||||
|
||||
HoodieData<HoodieRecord<T>> taggedRecords =
|
||||
keyedInputRecords.leftOuterJoin(existingLocationsOnTable).map(entry -> {
|
||||
final HoodieRecord<T> untaggedRecord = entry.getRight().getLeft();
|
||||
final Option<HoodieRecordLocation> 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<HoodieKey, HoodieRecordLocation> fetchRecordLocationsForAffectedPartitions(
|
||||
HoodieData<HoodieKey> hoodieKeys, HoodieEngineContext context, HoodieTable hoodieTable,
|
||||
int parallelism) {
|
||||
List<String> affectedPartitionPathList =
|
||||
hoodieKeys.map(HoodieKey::getPartitionPath).distinct().collectAsList();
|
||||
List<Pair<String, HoodieBaseFile>> latestBaseFiles =
|
||||
getLatestBaseFilesForAllPartitions(affectedPartitionPathList, context, hoodieTable);
|
||||
return fetchRecordLocations(context, hoodieTable, parallelism, latestBaseFiles);
|
||||
}
|
||||
|
||||
protected HoodiePairData<HoodieKey, HoodieRecordLocation> fetchRecordLocations(
|
||||
HoodieEngineContext context, HoodieTable hoodieTable, int parallelism,
|
||||
List<Pair<String, HoodieBaseFile>> 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<HoodieKey, HoodieRecordLocation>) e);
|
||||
}
|
||||
}
|
||||
@@ -101,7 +101,7 @@ public abstract class HoodieTable<T extends HoodieRecordPayload, I, K, O> implem
|
||||
|
||||
protected final HoodieWriteConfig config;
|
||||
protected final HoodieTableMetaClient metaClient;
|
||||
protected final HoodieIndex<T, I, K, O> index;
|
||||
protected final HoodieIndex<T, ?, ?, ?> index;
|
||||
private SerializableConfiguration hadoopConfiguration;
|
||||
protected final TaskContextSupplier taskContextSupplier;
|
||||
private final HoodieTableMetadata metadata;
|
||||
@@ -125,7 +125,7 @@ public abstract class HoodieTable<T extends HoodieRecordPayload, I, K, O> implem
|
||||
this.taskContextSupplier = context.getTaskContextSupplier();
|
||||
}
|
||||
|
||||
protected abstract HoodieIndex<T, I, K, O> getIndex(HoodieWriteConfig config, HoodieEngineContext context);
|
||||
protected abstract HoodieIndex<T, ?, ?, ?> getIndex(HoodieWriteConfig config, HoodieEngineContext context);
|
||||
|
||||
private synchronized FileSystemViewManager getViewManager() {
|
||||
if (null == viewManager) {
|
||||
@@ -347,7 +347,7 @@ public abstract class HoodieTable<T extends HoodieRecordPayload, I, K, O> implem
|
||||
/**
|
||||
* Return the index.
|
||||
*/
|
||||
public HoodieIndex<T, I, K, O> getIndex() {
|
||||
public HoodieIndex<T, ?, ?, ?> getIndex() {
|
||||
return index;
|
||||
}
|
||||
|
||||
|
||||
@@ -63,11 +63,8 @@ public abstract class AbstractWriteHelper<T extends HoodieRecordPayload, I, K, O
|
||||
}
|
||||
}
|
||||
|
||||
private I tag(
|
||||
I dedupedRecords, HoodieEngineContext context, HoodieTable<T, I, K, O> 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<T, I, K, O> table);
|
||||
|
||||
public I combineOnCondition(
|
||||
boolean condition, I records, int parallelism, HoodieTable<T, I, K, O> table) {
|
||||
@@ -87,5 +84,5 @@ public abstract class AbstractWriteHelper<T extends HoodieRecordPayload, I, K, O
|
||||
}
|
||||
|
||||
public abstract I deduplicateRecords(
|
||||
I records, HoodieIndex<T, I, K, O> index, int parallelism);
|
||||
I records, HoodieIndex<T, ?, ?, ?> index, int parallelism);
|
||||
}
|
||||
|
||||
@@ -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<T extends HoodieRecordPayload> extends
|
||||
* Complete changes performed at the given instantTime marker with specified action.
|
||||
*/
|
||||
@Override
|
||||
protected HoodieIndex<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>> 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<T extends HoodieRecordPayload> extends
|
||||
// Create a Hoodie table which encapsulated the commits and files visible
|
||||
HoodieFlinkTable<T> table = getHoodieTable();
|
||||
Timer.Context indexTimer = metrics.getIndexCtx();
|
||||
List<HoodieRecord<T>> recordsWithLocation = getIndex().tagLocation(hoodieRecords, context, table);
|
||||
List<HoodieRecord<T>> 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());
|
||||
}
|
||||
|
||||
@@ -88,7 +88,7 @@ public class HoodieFlinkEngineContext extends HoodieEngineContext {
|
||||
}
|
||||
|
||||
@Override
|
||||
public <T> HoodieData<T> parallelize(List<T> data) {
|
||||
public <T> HoodieData<T> parallelize(List<T> data, int parallelism) {
|
||||
return HoodieList.of(data);
|
||||
}
|
||||
|
||||
|
||||
@@ -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<T extends HoodieRecordPayload> 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<WriteStatus> updateLocation(List<WriteStatus> writeStatuses,
|
||||
HoodieEngineContext context,
|
||||
HoodieTable<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>> 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<HoodieRecord<T>> tagLocation(List<HoodieRecord<T>> records,
|
||||
HoodieEngineContext context,
|
||||
HoodieTable<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>> hoodieTable) throws HoodieIndexException;
|
||||
|
||||
@Override
|
||||
@PublicAPIMethod(maturity = ApiMaturityLevel.EVOLVING)
|
||||
public HoodieData<HoodieRecord<T>> tagLocation(
|
||||
HoodieData<HoodieRecord<T>> records, HoodieEngineContext context,
|
||||
HoodieTable hoodieTable) throws HoodieIndexException {
|
||||
return HoodieList.of(tagLocation(HoodieList.getList(records), context, hoodieTable));
|
||||
}
|
||||
|
||||
@Override
|
||||
@PublicAPIMethod(maturity = ApiMaturityLevel.STABLE)
|
||||
public abstract List<WriteStatus> updateLocation(List<WriteStatus> writeStatuses,
|
||||
HoodieEngineContext context,
|
||||
HoodieTable<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>> hoodieTable) throws HoodieIndexException;
|
||||
|
||||
@Override
|
||||
@PublicAPIMethod(maturity = ApiMaturityLevel.STABLE)
|
||||
public abstract List<HoodieRecord<T>> tagLocation(List<HoodieRecord<T>> records,
|
||||
HoodieEngineContext context,
|
||||
HoodieTable<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>> hoodieTable) throws HoodieIndexException;
|
||||
@PublicAPIMethod(maturity = ApiMaturityLevel.EVOLVING)
|
||||
public HoodieData<WriteStatus> updateLocation(
|
||||
HoodieData<WriteStatus> writeStatuses, HoodieEngineContext context,
|
||||
HoodieTable hoodieTable) throws HoodieIndexException {
|
||||
return HoodieList.of(updateLocation(HoodieList.getList(writeStatuses), context, hoodieTable));
|
||||
}
|
||||
}
|
||||
|
||||
@@ -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());
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -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<T extends HoodieRecordPayload> extends HoodieBaseBloomIndex<T> {
|
||||
public FlinkHoodieBloomIndex(HoodieWriteConfig config) {
|
||||
super(config);
|
||||
}
|
||||
}
|
||||
@@ -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 <T> type of payload
|
||||
*/
|
||||
public class FlinkHoodieSimpleIndex<T extends HoodieRecordPayload> extends FlinkHoodieIndex<T> {
|
||||
|
||||
public FlinkHoodieSimpleIndex(HoodieWriteConfig config) {
|
||||
super(config);
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<WriteStatus> updateLocation(List<WriteStatus> writeStatuses, HoodieEngineContext context,
|
||||
HoodieTable<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>> 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<HoodieRecord<T>> tagLocation(List<HoodieRecord<T>> hoodieRecords, HoodieEngineContext context,
|
||||
HoodieTable<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>> hoodieTable) throws HoodieIndexException {
|
||||
return tagLocationInternal(hoodieRecords, context, hoodieTable);
|
||||
}
|
||||
|
||||
/**
|
||||
* Tags records location for incoming records.
|
||||
*/
|
||||
private List<HoodieRecord<T>> tagLocationInternal(List<HoodieRecord<T>> hoodieRecords, HoodieEngineContext context,
|
||||
HoodieTable<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>> hoodieTable) {
|
||||
Map<HoodieKey, HoodieRecord<T>> keyedInputRecords = context.mapToPair(hoodieRecords, record -> Pair.of(record.getKey(), record), 0);
|
||||
Map<HoodieKey, HoodieRecordLocation> existingLocationsOnTable = fetchRecordLocationsForAffectedPartitions(keyedInputRecords.keySet(), context, hoodieTable, config.getSimpleIndexParallelism());
|
||||
List<HoodieRecord<T>> taggedRecords = new LinkedList<>();
|
||||
|
||||
for (Map.Entry<HoodieKey, HoodieRecord<T>> hoodieKeyHoodieRecordEntry : keyedInputRecords.entrySet()) {
|
||||
HoodieKey key = hoodieKeyHoodieRecordEntry.getKey();
|
||||
HoodieRecord<T> 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<HoodieKey, HoodieRecordLocation> fetchRecordLocationsForAffectedPartitions(Set<HoodieKey> keySet,
|
||||
HoodieEngineContext context,
|
||||
HoodieTable<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>> hoodieTable,
|
||||
int parallelism) {
|
||||
List<String> affectedPartitionPathList = keySet.stream().map(HoodieKey::getPartitionPath).distinct().collect(Collectors.toList());
|
||||
List<Pair<String, HoodieBaseFile>> latestBaseFiles = getLatestBaseFilesForAllPartitions(affectedPartitionPathList, context, hoodieTable);
|
||||
return fetchRecordLocations(context, hoodieTable, parallelism, latestBaseFiles);
|
||||
}
|
||||
|
||||
private Map<HoodieKey, HoodieRecordLocation> fetchRecordLocations(HoodieEngineContext context,
|
||||
HoodieTable<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>> hoodieTable,
|
||||
int parallelism,
|
||||
List<Pair<String, HoodieBaseFile>> latestBaseFiles) {
|
||||
|
||||
List<HoodieKeyLocationFetchHandle<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>>> hoodieKeyLocationFetchHandles =
|
||||
context.map(latestBaseFiles, partitionPathBaseFile -> new HoodieKeyLocationFetchHandle<>(config, hoodieTable, partitionPathBaseFile, Option.empty()), parallelism);
|
||||
Map<HoodieKey, HoodieRecordLocation> recordLocations = new HashMap<>();
|
||||
hoodieKeyLocationFetchHandles.stream()
|
||||
.flatMap(handle -> handle.locations())
|
||||
.forEach(x -> recordLocations.put(x.getKey(), x.getRight()));
|
||||
return recordLocations;
|
||||
}
|
||||
}
|
||||
@@ -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 <T> type of payload
|
||||
*/
|
||||
public class FlinkInMemoryStateIndex<T extends HoodieRecordPayload> extends FlinkHoodieIndex<T> {
|
||||
public class FlinkInMemoryStateIndex<T extends HoodieRecordPayload<T>>
|
||||
extends HoodieIndex<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>> {
|
||||
|
||||
private static final Logger LOG = LogManager.getLogger(FlinkInMemoryStateIndex.class);
|
||||
|
||||
@@ -48,16 +50,16 @@ public class FlinkInMemoryStateIndex<T extends HoodieRecordPayload> extends Flin
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<HoodieRecord<T>> tagLocation(List<HoodieRecord<T>> records,
|
||||
HoodieEngineContext context,
|
||||
HoodieTable<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>> hoodieTable) throws HoodieIndexException {
|
||||
public HoodieData<HoodieRecord<T>> tagLocation(
|
||||
HoodieData<HoodieRecord<T>> records, HoodieEngineContext context,
|
||||
HoodieTable hoodieTable) throws HoodieIndexException {
|
||||
throw new UnsupportedOperationException("No need to tag location for FlinkInMemoryStateIndex");
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<WriteStatus> updateLocation(List<WriteStatus> writeStatuses,
|
||||
HoodieEngineContext context,
|
||||
HoodieTable<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>> hoodieTable) throws HoodieIndexException {
|
||||
public HoodieData<WriteStatus> updateLocation(
|
||||
HoodieData<WriteStatus> writeStatuses, HoodieEngineContext context,
|
||||
HoodieTable hoodieTable) throws HoodieIndexException {
|
||||
throw new UnsupportedOperationException("No need to update location for FlinkInMemoryStateIndex");
|
||||
}
|
||||
|
||||
|
||||
@@ -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<T extends HoodieRecordPayload>
|
||||
}
|
||||
|
||||
@Override
|
||||
protected HoodieIndex<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>> getIndex(HoodieWriteConfig config, HoodieEngineContext context) {
|
||||
return FlinkHoodieIndex.createIndex((HoodieFlinkEngineContext) context, config);
|
||||
protected HoodieIndex getIndex(HoodieWriteConfig config, HoodieEngineContext context) {
|
||||
return FlinkHoodieIndexFactory.createIndex((HoodieFlinkEngineContext) context, config);
|
||||
}
|
||||
|
||||
/**
|
||||
|
||||
@@ -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<R> 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<HoodieRecord<EmptyHoodieRecordPayload>> taggedRecords =
|
||||
table.getIndex().tagLocation(dedupedRecords, context, table);
|
||||
List<HoodieRecord<EmptyHoodieRecordPayload>> taggedRecords = HoodieList.getList(
|
||||
table.getIndex().tagLocation(HoodieList.of(dedupedRecords), context, table));
|
||||
Duration tagLocationDuration = Duration.between(beginTag, Instant.now());
|
||||
|
||||
// filter out non existent keys/records
|
||||
|
||||
@@ -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<T extends HoodieRecordPayload, R> extends Abstract
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<HoodieRecord<T>> deduplicateRecords(List<HoodieRecord<T>> records,
|
||||
HoodieIndex<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>> index,
|
||||
int parallelism) {
|
||||
protected List<HoodieRecord<T>> tag(List<HoodieRecord<T>> dedupedRecords, HoodieEngineContext context, HoodieTable<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>> table) {
|
||||
return HoodieList.getList(
|
||||
table.getIndex().tagLocation(HoodieList.of(dedupedRecords), context, table));
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<HoodieRecord<T>> deduplicateRecords(
|
||||
List<HoodieRecord<T>> records, HoodieIndex<T, ?, ?, ?> index, int parallelism) {
|
||||
Map<Object, List<Pair<Object, HoodieRecord<T>>>> 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();
|
||||
|
||||
@@ -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<String, List<BloomIndexFileInfo>> partitionToFileIndexInfo = new HashMap<>();
|
||||
partitionToFileIndexInfo.put("2017/10/22",
|
||||
@@ -176,14 +178,14 @@ public class TestFlinkHoodieBloomIndex extends HoodieFlinkClientTestHarness {
|
||||
Map<String, List<String>> 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<String> recordKeyList = partitionRecordKeyMap.getOrDefault(t.getLeft(), new ArrayList<>());
|
||||
recordKeyList.add(t.getRight());
|
||||
partitionRecordKeyMap.put(t.getLeft(), recordKeyList);
|
||||
});
|
||||
.forEach(t -> {
|
||||
List<String> recordKeyList = partitionRecordKeyMap.getOrDefault(t.getLeft(), new ArrayList<>());
|
||||
recordKeyList.add(t.getRight());
|
||||
partitionRecordKeyMap.put(t.getLeft(), recordKeyList);
|
||||
});
|
||||
|
||||
List<Pair<String, HoodieKey>> comparisonKeyList =
|
||||
index.explodeRecordsWithFileComparisons(partitionToFileIndexInfo, partitionRecordKeyMap);
|
||||
List<Pair<String, HoodieKey>> comparisonKeyList = HoodieList.getList(
|
||||
index.explodeRecordsWithFileComparisons(partitionToFileIndexInfo, HoodieMapPair.of(partitionRecordKeyMap)));
|
||||
|
||||
assertEquals(10, comparisonKeyList.size());
|
||||
java.util.Map<String, List<String>> 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<HoodieRecord> taggedRecords = bloomIndex.tagLocation(records, context, hoodieTable);
|
||||
HoodieBloomIndex bloomIndex = new HoodieBloomIndex(config, ListBasedHoodieBloomIndexHelper.getInstance());
|
||||
List<HoodieRecord> 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<HoodieRecord> toTagRecords = new ArrayList<>();
|
||||
toTagRecords.add(new HoodieRecord(record4.getKey(), null));
|
||||
List<HoodieRecord> taggedRecords = bloomIndex.tagLocation(toTagRecords, context, hoodieTable);
|
||||
List<HoodieRecord> taggedRecords = tagLocation(bloomIndex, toTagRecords, hoodieTable);
|
||||
Map<HoodieKey, Option<Pair<String, String>>> 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<HoodieRecord> taggedRecords = bloomIndex.tagLocation(records, context, table);
|
||||
HoodieBloomIndex bloomIndex = new HoodieBloomIndex(config, ListBasedHoodieBloomIndexHelper.getInstance());
|
||||
List<HoodieRecord> taggedRecords = tagLocation(bloomIndex, records, table);
|
||||
|
||||
// Check results
|
||||
for (HoodieRecord record : taggedRecords) {
|
||||
|
||||
@@ -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<HoodieRecord> tagLocation(
|
||||
HoodieIndex index, List<HoodieRecord> records, HoodieTable table) {
|
||||
return HoodieList.getList(index.tagLocation(HoodieList.of(records), context, table));
|
||||
}
|
||||
|
||||
/**
|
||||
* Cleanups file system.
|
||||
|
||||
@@ -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<T extends HoodieRecordPayload> extends
|
||||
// Create a Hoodie table which encapsulated the commits and files visible
|
||||
HoodieJavaTable<T> table = HoodieJavaTable.create(config, (HoodieJavaEngineContext) context);
|
||||
Timer.Context indexTimer = metrics.getIndexCtx();
|
||||
List<HoodieRecord<T>> recordsWithLocation = getIndex().tagLocation(hoodieRecords, context, table);
|
||||
List<HoodieRecord<T>> 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<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>> createIndex(HoodieWriteConfig writeConfig) {
|
||||
return JavaHoodieIndex.createIndex(config);
|
||||
protected HoodieIndex createIndex(HoodieWriteConfig writeConfig) {
|
||||
return JavaHoodieIndexFactory.createIndex(config);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
||||
@@ -78,7 +78,7 @@ public class HoodieJavaEngineContext extends HoodieEngineContext {
|
||||
}
|
||||
|
||||
@Override
|
||||
public <T> HoodieData<T> parallelize(List<T> data) {
|
||||
public <T> HoodieData<T> parallelize(List<T> data, int parallelism) {
|
||||
return HoodieList.of(data);
|
||||
}
|
||||
|
||||
|
||||
@@ -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<T extends HoodieRecordPayload> extends HoodieBaseBloomIndex<T> {
|
||||
public JavaHoodieBloomIndex(HoodieWriteConfig config) {
|
||||
super(config);
|
||||
}
|
||||
}
|
||||
@@ -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<T extends HoodieRecordPayload> 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<WriteStatus> updateLocation(List<WriteStatus> writeStatuses,
|
||||
HoodieEngineContext context,
|
||||
HoodieTable<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>> hoodieTable) throws HoodieIndexException;
|
||||
|
||||
@Override
|
||||
@PublicAPIMethod(maturity = ApiMaturityLevel.STABLE)
|
||||
@Deprecated
|
||||
@PublicAPIMethod(maturity = ApiMaturityLevel.DEPRECATED)
|
||||
public abstract List<HoodieRecord<T>> tagLocation(List<HoodieRecord<T>> records,
|
||||
HoodieEngineContext context,
|
||||
HoodieTable<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>> hoodieTable) throws HoodieIndexException;
|
||||
|
||||
@Override
|
||||
@PublicAPIMethod(maturity = ApiMaturityLevel.EVOLVING)
|
||||
public HoodieData<HoodieRecord<T>> tagLocation(
|
||||
HoodieData<HoodieRecord<T>> records, HoodieEngineContext context,
|
||||
HoodieTable hoodieTable) throws HoodieIndexException {
|
||||
return HoodieList.of(tagLocation(HoodieList.getList(records), context, hoodieTable));
|
||||
}
|
||||
|
||||
@Override
|
||||
@PublicAPIMethod(maturity = ApiMaturityLevel.EVOLVING)
|
||||
public HoodieData<WriteStatus> updateLocation(
|
||||
HoodieData<WriteStatus> writeStatuses, HoodieEngineContext context,
|
||||
HoodieTable hoodieTable) throws HoodieIndexException {
|
||||
return HoodieList.of(updateLocation(HoodieList.getList(writeStatuses), context, hoodieTable));
|
||||
}
|
||||
}
|
||||
|
||||
@@ -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());
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -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<T extends HoodieRecordPayload>
|
||||
}
|
||||
|
||||
@Override
|
||||
protected HoodieIndex<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>> getIndex(HoodieWriteConfig config, HoodieEngineContext context) {
|
||||
return JavaHoodieIndex.createIndex(config);
|
||||
protected HoodieIndex getIndex(HoodieWriteConfig config, HoodieEngineContext context) {
|
||||
return JavaHoodieIndexFactory.createIndex(config);
|
||||
}
|
||||
}
|
||||
|
||||
@@ -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<T extends HoodieRecordPayload
|
||||
protected void updateIndex(List<WriteStatus> writeStatuses, HoodieWriteMetadata<List<WriteStatus>> result) {
|
||||
Instant indexStartTime = Instant.now();
|
||||
// Update the index back
|
||||
List<WriteStatus> statuses = table.getIndex().updateLocation(writeStatuses, context, table);
|
||||
List<WriteStatus> 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<T extends HoodieRecordPayload
|
||||
public void updateIndexAndCommitIfNeeded(List<WriteStatus> writeStatuses, HoodieWriteMetadata result) {
|
||||
Instant indexStartTime = Instant.now();
|
||||
// Update the index back
|
||||
List<WriteStatus> statuses = table.getIndex().updateLocation(writeStatuses, context, table);
|
||||
List<WriteStatus> 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));
|
||||
|
||||
@@ -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<R> 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<HoodieRecord<EmptyHoodieRecordPayload>> taggedRecords =
|
||||
table.getIndex().tagLocation(dedupedRecords, context, table);
|
||||
List<HoodieRecord<EmptyHoodieRecordPayload>> taggedRecords = HoodieList.getList(
|
||||
table.getIndex().tagLocation(HoodieList.of(dedupedRecords), context, table));
|
||||
Duration tagLocationDuration = Duration.between(beginTag, Instant.now());
|
||||
|
||||
// filter out non existent keys/records
|
||||
|
||||
@@ -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<T extends HoodieRecordPayload,R> extends AbstractWr
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<HoodieRecord<T>> deduplicateRecords(List<HoodieRecord<T>> records,
|
||||
HoodieIndex<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>> index,
|
||||
int parallelism) {
|
||||
protected List<HoodieRecord<T>> tag(List<HoodieRecord<T>> dedupedRecords, HoodieEngineContext context, HoodieTable<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>> table) {
|
||||
return HoodieList.getList(
|
||||
table.getIndex().tagLocation(HoodieList.of(dedupedRecords), context, table));
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<HoodieRecord<T>> deduplicateRecords(
|
||||
List<HoodieRecord<T>> records, HoodieIndex<T, ?, ?, ?> index, int parallelism) {
|
||||
boolean isIndexingGlobal = index.isGlobal();
|
||||
Map<Object, List<Pair<Object, HoodieRecord<T>>>> keyedRecords = records.stream().map(record -> {
|
||||
HoodieKey hoodieKey = record.getKey();
|
||||
|
||||
@@ -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<T extends HoodieRecordPayload> implements Serializable {
|
||||
public class HoodieReadClient<T extends HoodieRecordPayload<T>> implements Serializable {
|
||||
|
||||
private static final long serialVersionUID = 1L;
|
||||
|
||||
@@ -65,7 +66,7 @@ public class HoodieReadClient<T extends HoodieRecordPayload> 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<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> index;
|
||||
private final transient HoodieIndex<T, ?, ?, ?> index;
|
||||
private HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> hoodieTable;
|
||||
private transient Option<SQLContext> sqlContextOpt;
|
||||
private final transient HoodieSparkEngineContext context;
|
||||
@@ -100,7 +101,7 @@ public class HoodieReadClient<T extends HoodieRecordPayload> 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<T extends HoodieRecordPayload> implements Serializ
|
||||
* component (without scheme) of the URI underlying file
|
||||
*/
|
||||
public JavaPairRDD<HoodieKey, Option<Pair<String, String>>> checkExists(JavaRDD<HoodieKey> 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<T extends HoodieRecordPayload> implements Serializ
|
||||
* @return Tagged RDD of Hoodie records
|
||||
*/
|
||||
public JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> hoodieRecords) throws HoodieIndexException {
|
||||
return index.tagLocation(hoodieRecords, context, hoodieTable);
|
||||
return HoodieJavaRDD.getJavaRDD(
|
||||
index.tagLocation(HoodieJavaRDD.of(hoodieRecords), context, hoodieTable));
|
||||
}
|
||||
|
||||
/**
|
||||
|
||||
@@ -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<T extends HoodieRecordPayload> extends
|
||||
}
|
||||
|
||||
@Override
|
||||
protected HoodieIndex<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> createIndex(HoodieWriteConfig writeConfig) {
|
||||
return SparkHoodieIndex.createIndex(config);
|
||||
protected HoodieIndex createIndex(HoodieWriteConfig writeConfig) {
|
||||
return SparkHoodieIndexFactory.createIndex(config);
|
||||
}
|
||||
|
||||
/**
|
||||
@@ -141,7 +141,8 @@ public class SparkRDDWriteClient<T extends HoodieRecordPayload> extends
|
||||
// Create a Hoodie table which encapsulated the commits and files visible
|
||||
HoodieSparkTable<T> table = HoodieSparkTable.create(config, context);
|
||||
Timer.Context indexTimer = metrics.getIndexCtx();
|
||||
JavaRDD<HoodieRecord<T>> recordsWithLocation = getIndex().tagLocation(hoodieRecords, context, table);
|
||||
JavaRDD<HoodieRecord<T>> 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());
|
||||
}
|
||||
|
||||
@@ -91,8 +91,8 @@ public class HoodieSparkEngineContext extends HoodieEngineContext {
|
||||
}
|
||||
|
||||
@Override
|
||||
public <T> HoodieData<T> parallelize(List<T> data) {
|
||||
return HoodieJavaRDD.of(javaSparkContext.parallelize(data, data.size()));
|
||||
public <T> HoodieData<T> parallelize(List<T> data, int parallelism) {
|
||||
return HoodieJavaRDD.of(javaSparkContext.parallelize(data, parallelism));
|
||||
}
|
||||
|
||||
@Override
|
||||
|
||||
@@ -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));
|
||||
}
|
||||
}
|
||||
|
||||
@@ -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 <K> type of key.
|
||||
* @param <V> type of value.
|
||||
*/
|
||||
public class HoodieJavaPairRDD<K, V> extends HoodiePairData<K, V> {
|
||||
|
||||
private final JavaPairRDD<K, V> pairRDDData;
|
||||
|
||||
private HoodieJavaPairRDD(JavaPairRDD<K, V> pairRDDData) {
|
||||
this.pairRDDData = pairRDDData;
|
||||
}
|
||||
|
||||
/**
|
||||
* @param pairRDDData a {@link JavaPairRDD} of pairs.
|
||||
* @param <K> type of key.
|
||||
* @param <V> type of value.
|
||||
* @return a new instance containing the {@link JavaPairRDD<K, V>} reference.
|
||||
*/
|
||||
public static <K, V> HoodieJavaPairRDD<K, V> of(JavaPairRDD<K, V> pairRDDData) {
|
||||
return new HoodieJavaPairRDD<>(pairRDDData);
|
||||
}
|
||||
|
||||
/**
|
||||
* @param hoodiePairData {@link HoodieJavaPairRDD <K, V>} instance containing the {@link JavaPairRDD} of pairs.
|
||||
* @param <K> type of key.
|
||||
* @param <V> type of value.
|
||||
* @return the {@link JavaPairRDD} of pairs.
|
||||
*/
|
||||
public static <K, V> JavaPairRDD<K, V> getJavaPairRDD(HoodiePairData<K, V> hoodiePairData) {
|
||||
return ((HoodieJavaPairRDD<K, V>) hoodiePairData).get();
|
||||
}
|
||||
|
||||
@Override
|
||||
public JavaPairRDD<K, V> get() {
|
||||
return pairRDDData;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void persist(String storageLevel) {
|
||||
pairRDDData.persist(StorageLevel.fromString(storageLevel));
|
||||
}
|
||||
|
||||
@Override
|
||||
public void unpersist() {
|
||||
pairRDDData.unpersist();
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodieData<K> keys() {
|
||||
return HoodieJavaRDD.of(pairRDDData.keys());
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodieData<V> values() {
|
||||
return HoodieJavaRDD.of(pairRDDData.values());
|
||||
}
|
||||
|
||||
@Override
|
||||
public long count() {
|
||||
return pairRDDData.count();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Map<K, Long> countByKey() {
|
||||
return pairRDDData.countByKey();
|
||||
}
|
||||
|
||||
@Override
|
||||
public <O> HoodieData<O> map(SerializableFunction<Pair<K, V>, O> func) {
|
||||
return HoodieJavaRDD.of(pairRDDData.map(
|
||||
tuple -> func.apply(new ImmutablePair<>(tuple._1, tuple._2))));
|
||||
}
|
||||
|
||||
@Override
|
||||
public <L, W> HoodiePairData<L, W> mapToPair(SerializablePairFunction<Pair<K, V>, L, W> mapToPairFunc) {
|
||||
return HoodieJavaPairRDD.of(pairRDDData.mapToPair(pair -> {
|
||||
Pair<L, W> newPair = mapToPairFunc.call(new ImmutablePair<>(pair._1, pair._2));
|
||||
return new Tuple2<>(newPair.getLeft(), newPair.getRight());
|
||||
}));
|
||||
}
|
||||
|
||||
@Override
|
||||
public <W> HoodiePairData<K, Pair<V, Option<W>>> leftOuterJoin(HoodiePairData<K, W> 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)))))));
|
||||
}
|
||||
}
|
||||
@@ -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<T> extends HoodieData<T> {
|
||||
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 <O> HoodieData<O> map(SerializableFunction<T, O> func) {
|
||||
return HoodieJavaRDD.of(rddData.map(func::apply));
|
||||
}
|
||||
|
||||
@Override
|
||||
public <O> HoodieData<O> mapPartitions(SerializableFunction<Iterator<T>, Iterator<O>> func, boolean preservesPartitioning) {
|
||||
return HoodieJavaRDD.of(rddData.mapPartitions(func::apply, preservesPartitioning));
|
||||
}
|
||||
|
||||
@Override
|
||||
public <O> HoodieData<O> flatMap(SerializableFunction<T, Iterator<O>> func) {
|
||||
return HoodieJavaRDD.of(rddData.flatMap(func::apply));
|
||||
return HoodieJavaRDD.of(rddData.flatMap(e -> func.apply(e)));
|
||||
}
|
||||
|
||||
@Override
|
||||
public <K, V> HoodiePairData<K, V> mapToPair(SerializablePairFunction<T, K, V> mapToPairFunc) {
|
||||
return HoodieJavaPairRDD.of(rddData.mapToPair(input -> {
|
||||
Pair<K, V> pair = mapToPairFunc.call(input);
|
||||
return new Tuple2<>(pair.getLeft(), pair.getRight());
|
||||
}));
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodieData<T> distinct() {
|
||||
return HoodieJavaRDD.of(rddData.distinct());
|
||||
}
|
||||
|
||||
@Override
|
||||
|
||||
@@ -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<T extends HoodieRecordPayload> extends HoodieIndex<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> {
|
||||
public abstract class SparkHoodieIndex<T extends HoodieRecordPayload<T>>
|
||||
extends HoodieIndex<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> {
|
||||
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<WriteStatus> updateLocation(JavaRDD<WriteStatus> writeStatusRDD,
|
||||
HoodieEngineContext context,
|
||||
HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> hoodieTable) throws HoodieIndexException;
|
||||
|
||||
@Override
|
||||
@PublicAPIMethod(maturity = ApiMaturityLevel.STABLE)
|
||||
@Deprecated
|
||||
@PublicAPIMethod(maturity = ApiMaturityLevel.DEPRECATED)
|
||||
public abstract JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> records,
|
||||
HoodieEngineContext context,
|
||||
HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> hoodieTable) throws HoodieIndexException;
|
||||
|
||||
@Override
|
||||
@PublicAPIMethod(maturity = ApiMaturityLevel.EVOLVING)
|
||||
public HoodieData<HoodieRecord<T>> tagLocation(
|
||||
HoodieData<HoodieRecord<T>> records, HoodieEngineContext context,
|
||||
HoodieTable hoodieTable) throws HoodieIndexException {
|
||||
return HoodieJavaRDD.of(tagLocation(HoodieJavaRDD.getJavaRDD(records), context, hoodieTable));
|
||||
}
|
||||
|
||||
@Override
|
||||
@PublicAPIMethod(maturity = ApiMaturityLevel.EVOLVING)
|
||||
public HoodieData<WriteStatus> updateLocation(
|
||||
HoodieData<WriteStatus> writeStatuses, HoodieEngineContext context,
|
||||
HoodieTable hoodieTable) throws HoodieIndexException {
|
||||
return HoodieJavaRDD.of(updateLocation(HoodieJavaRDD.getJavaRDD(writeStatuses), context, hoodieTable));
|
||||
}
|
||||
}
|
||||
|
||||
@@ -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<BaseKeyGenerator> 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);
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -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.
|
||||
* <p>
|
||||
* ONLY USE FOR LOCAL TESTING
|
||||
*/
|
||||
@SuppressWarnings("checkstyle:LineLength")
|
||||
public class SparkInMemoryHashIndex<T extends HoodieRecordPayload> extends SparkHoodieIndex<T> {
|
||||
|
||||
private static ConcurrentMap<HoodieKey, HoodieRecordLocation> recordLocationMap;
|
||||
|
||||
public SparkInMemoryHashIndex(HoodieWriteConfig config) {
|
||||
super(config);
|
||||
synchronized (SparkInMemoryHashIndex.class) {
|
||||
if (recordLocationMap == null) {
|
||||
recordLocationMap = new ConcurrentHashMap<>();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> recordRDD, HoodieEngineContext context,
|
||||
HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> hoodieTable) {
|
||||
return recordRDD.mapPartitionsWithIndex(this.new LocationTagFunction(), true);
|
||||
}
|
||||
|
||||
@Override
|
||||
public JavaRDD<WriteStatus> updateLocation(JavaRDD<WriteStatus> writeStatusRDD,
|
||||
HoodieEngineContext context,
|
||||
HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> hoodieTable) {
|
||||
return writeStatusRDD.map(writeStatus -> {
|
||||
for (HoodieRecord record : writeStatus.getWrittenRecords()) {
|
||||
if (!writeStatus.isErrored(record.getKey())) {
|
||||
HoodieKey key = record.getKey();
|
||||
Option<HoodieRecordLocation> 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<Integer, Iterator<HoodieRecord<T>>, Iterator<HoodieRecord<T>>> {
|
||||
|
||||
@Override
|
||||
public Iterator<HoodieRecord<T>> call(Integer partitionNum, Iterator<HoodieRecord<T>> hoodieRecordIterator) {
|
||||
List<HoodieRecord<T>> taggedRecords = new ArrayList<>();
|
||||
while (hoodieRecordIterator.hasNext()) {
|
||||
HoodieRecord<T> rec = hoodieRecordIterator.next();
|
||||
if (recordLocationMap.containsKey(rec.getKey())) {
|
||||
rec.unseal();
|
||||
rec.setCurrentLocation(recordLocationMap.get(rec.getKey()));
|
||||
rec.seal();
|
||||
}
|
||||
taggedRecords.add(rec);
|
||||
}
|
||||
return taggedRecords.iterator();
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -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<T extends HoodieRecordPayload> extends SparkHoodieIndex<T> {
|
||||
|
||||
private static final Logger LOG = LogManager.getLogger(SparkHoodieBloomIndex.class);
|
||||
|
||||
public SparkHoodieBloomIndex(HoodieWriteConfig config) {
|
||||
super(config);
|
||||
}
|
||||
|
||||
@Override
|
||||
public JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> recordRDD, HoodieEngineContext context,
|
||||
HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> 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<String, String> partitionRecordKeyPairRDD =
|
||||
recordRDD.mapToPair(record -> new Tuple2<>(record.getPartitionPath(), record.getRecordKey()));
|
||||
|
||||
// Lookup indexes for all the partition/recordkey pair
|
||||
JavaPairRDD<HoodieKey, HoodieRecordLocation> 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<HoodieRecord<T>> 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<record_key,location> for all record keys already
|
||||
* present and drop the record keys if not present.
|
||||
*/
|
||||
private JavaPairRDD<HoodieKey, HoodieRecordLocation> lookupIndex(
|
||||
JavaPairRDD<String, String> partitionRecordKeyPairRDD, final HoodieEngineContext context,
|
||||
final HoodieTable hoodieTable) {
|
||||
// Obtain records per partition, in the incoming records
|
||||
Map<String, Long> recordsPerPartition = partitionRecordKeyPairRDD.countByKey();
|
||||
List<String> affectedPartitionPathList = new ArrayList<>(recordsPerPartition.keySet());
|
||||
|
||||
// Step 2: Load all involved files as <Partition, filename> pairs
|
||||
List<Tuple2<String, BloomIndexFileInfo>> fileInfoList =
|
||||
loadInvolvedFiles(affectedPartitionPathList, context, hoodieTable);
|
||||
final Map<String, List<BloomIndexFileInfo>> 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<Tuple2<String, HoodieKey>> fileComparisonsRDD =
|
||||
explodeRecordRDDWithFileComparisons(partitionToFileInfo, partitionRecordKeyPairRDD);
|
||||
Map<String, Long> 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<String, Long> computeComparisonsPerFileGroup(final Map<String, Long> recordsPerPartition,
|
||||
final Map<String, List<BloomIndexFileInfo>> partitionToFileInfo,
|
||||
final JavaRDD<Tuple2<String, HoodieKey>> fileComparisonsRDD,
|
||||
final HoodieEngineContext context) {
|
||||
Map<String, Long> 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 <Partition, filename> pair RDD.
|
||||
*/
|
||||
List<Tuple2<String, BloomIndexFileInfo>> loadInvolvedFiles(List<String> partitions, final HoodieEngineContext context,
|
||||
final HoodieTable hoodieTable) {
|
||||
|
||||
// Obtain the latest data files from all the partitions.
|
||||
List<Pair<String, String>> 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.
|
||||
* <p>
|
||||
* 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<Tuple2<String, HoodieKey>> explodeRecordRDDWithFileComparisons(
|
||||
final Map<String, List<BloomIndexFileInfo>> partitionToFileIndexInfo,
|
||||
JavaPairRDD<String, String> 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 <RowKey, filename> pair. All workload grouped by file-level.
|
||||
* <p>
|
||||
* 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
|
||||
* <p>
|
||||
* Make sure the parallelism is atleast the groupby parallelism for tagging location
|
||||
*/
|
||||
JavaPairRDD<HoodieKey, HoodieRecordLocation> findMatchingFilesForRecordKeys(
|
||||
JavaRDD<Tuple2<String, HoodieKey>> fileComparisonsRDD,
|
||||
int shuffleParallelism,
|
||||
HoodieTable hoodieTable,
|
||||
Map<String, Long> 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 <rowKey, filename> back to the original HoodieRecord RDD.
|
||||
*/
|
||||
protected JavaRDD<HoodieRecord<T>> tagLocationBacktoRecords(
|
||||
JavaPairRDD<HoodieKey, HoodieRecordLocation> keyFilenamePairRDD, JavaRDD<HoodieRecord<T>> recordRDD) {
|
||||
JavaPairRDD<HoodieKey, HoodieRecord<T>> 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<WriteStatus> updateLocation(JavaRDD<WriteStatus> writeStatusRDD, HoodieEngineContext context,
|
||||
HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> hoodieTable) {
|
||||
return writeStatusRDD;
|
||||
}
|
||||
}
|
||||
@@ -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<HoodieKey, HoodieRecordLocation> findMatchingFilesForRecordKeys(
|
||||
HoodieWriteConfig config, HoodieEngineContext context, HoodieTable hoodieTable,
|
||||
HoodiePairData<String, String> partitionRecordKeyPairs,
|
||||
HoodieData<ImmutablePair<String, HoodieKey>> fileComparisonPairs,
|
||||
Map<String, List<BloomIndexFileInfo>> partitionToFileInfo,
|
||||
Map<String, Long> recordsPerPartition) {
|
||||
JavaRDD<Tuple2<String, HoodieKey>> fileComparisonsRDD =
|
||||
HoodieJavaRDD.getJavaRDD(fileComparisonPairs)
|
||||
.map(pair -> new Tuple2<>(pair.getLeft(), pair.getRight()));
|
||||
Map<String, Long> 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<String, Long> computeComparisonsPerFileGroup(
|
||||
final HoodieWriteConfig config,
|
||||
final Map<String, Long> recordsPerPartition,
|
||||
final Map<String, List<BloomIndexFileInfo>> partitionToFileInfo,
|
||||
final JavaRDD<Tuple2<String, HoodieKey>> fileComparisonsRDD,
|
||||
final HoodieEngineContext context) {
|
||||
Map<String, Long> 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;
|
||||
}
|
||||
}
|
||||
@@ -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<T extends HoodieRecordPayload> extends SparkHoodieIndex<T> {
|
||||
public class SparkHoodieHBaseIndex<T extends HoodieRecordPayload<T>>
|
||||
extends HoodieIndex<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> {
|
||||
|
||||
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<T extends HoodieRecordPayload> extends SparkH
|
||||
}
|
||||
|
||||
@Override
|
||||
public JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> recordRDD,
|
||||
HoodieEngineContext context,
|
||||
HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> hoodieTable) {
|
||||
return recordRDD.mapPartitionsWithIndex(locationTagFunction(hoodieTable.getMetaClient()), true);
|
||||
public HoodieData<HoodieRecord<T>> tagLocation(
|
||||
HoodieData<HoodieRecord<T>> records, HoodieEngineContext context,
|
||||
HoodieTable hoodieTable) {
|
||||
return HoodieJavaRDD.of(HoodieJavaRDD.getJavaRDD(records)
|
||||
.mapPartitionsWithIndex(locationTagFunction(hoodieTable.getMetaClient()), true));
|
||||
}
|
||||
|
||||
private Function2<Integer, Iterator<WriteStatus>, Iterator<WriteStatus>> updateLocationFunction() {
|
||||
@@ -395,16 +399,17 @@ public class SparkHoodieHBaseIndex<T extends HoodieRecordPayload> extends SparkH
|
||||
}
|
||||
|
||||
@Override
|
||||
public JavaRDD<WriteStatus> updateLocation(JavaRDD<WriteStatus> writeStatusRDD, HoodieEngineContext context,
|
||||
HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>,
|
||||
JavaRDD<WriteStatus>> hoodieTable) {
|
||||
final Option<Float> desiredQPSFraction = calculateQPSFraction(writeStatusRDD);
|
||||
public HoodieData<WriteStatus> updateLocation(
|
||||
HoodieData<WriteStatus> writeStatus, HoodieEngineContext context,
|
||||
HoodieTable hoodieTable) {
|
||||
JavaRDD<WriteStatus> writeStatusRDD = HoodieJavaRDD.getJavaRDD(writeStatus);
|
||||
final Option<Float> desiredQPSFraction = calculateQPSFraction(writeStatusRDD);
|
||||
final Map<String, Integer> fileIdPartitionMap = mapFileWithInsertsToUniquePartition(writeStatusRDD);
|
||||
JavaRDD<WriteStatus> 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<WriteStatus> writeStatusJavaRDD = partitionedRDD.mapPartitionsWithIndex(updateLocationFunction(),
|
||||
@@ -414,7 +419,7 @@ public class SparkHoodieHBaseIndex<T extends HoodieRecordPayload> extends SparkH
|
||||
// force trigger update location(hbase puts)
|
||||
writeStatusJavaRDD.count();
|
||||
this.hBaseIndexQPSResourceAllocator.releaseQPSResources();
|
||||
return writeStatusJavaRDD;
|
||||
return HoodieJavaRDD.of(writeStatusJavaRDD);
|
||||
}
|
||||
|
||||
private Option<Float> calculateQPSFraction(JavaRDD<WriteStatus> writeStatusRDD) {
|
||||
|
||||
@@ -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 <T>
|
||||
*/
|
||||
@SuppressWarnings("checkstyle:LineLength")
|
||||
public class SparkHoodieSimpleIndex<T extends HoodieRecordPayload> extends SparkHoodieIndex<T> {
|
||||
|
||||
public SparkHoodieSimpleIndex(HoodieWriteConfig config) {
|
||||
super(config);
|
||||
}
|
||||
|
||||
@Override
|
||||
public JavaRDD<WriteStatus> updateLocation(JavaRDD<WriteStatus> writeStatusRDD, HoodieEngineContext context,
|
||||
HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> 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<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> recordRDD,
|
||||
HoodieEngineContext context,
|
||||
HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> 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<HoodieRecord<T>> tagLocationInternal(JavaRDD<HoodieRecord<T>> inputRecordRDD, HoodieEngineContext context,
|
||||
HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> hoodieTable) {
|
||||
if (config.getSimpleIndexUseCaching()) {
|
||||
inputRecordRDD.persist(SparkMemoryUtils.getSimpleIndexInputStorageLevel(config.getProps()));
|
||||
}
|
||||
|
||||
JavaPairRDD<HoodieKey, HoodieRecord<T>> keyedInputRecordRDD = inputRecordRDD.mapToPair(record -> new Tuple2<>(record.getKey(), record));
|
||||
JavaPairRDD<HoodieKey, HoodieRecordLocation> existingLocationsOnTable = fetchRecordLocationsForAffectedPartitions(keyedInputRecordRDD.keys(), context, hoodieTable,
|
||||
config.getSimpleIndexParallelism());
|
||||
|
||||
JavaRDD<HoodieRecord<T>> taggedRecordRDD = keyedInputRecordRDD.leftOuterJoin(existingLocationsOnTable)
|
||||
.map(entry -> {
|
||||
final HoodieRecord<T> untaggedRecord = entry._2._1;
|
||||
final Option<HoodieRecordLocation> 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<HoodieKey, HoodieRecordLocation> fetchRecordLocationsForAffectedPartitions(JavaRDD<HoodieKey> hoodieKeys,
|
||||
HoodieEngineContext context,
|
||||
HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> hoodieTable,
|
||||
int parallelism) {
|
||||
List<String> affectedPartitionPathList = hoodieKeys.map(HoodieKey::getPartitionPath).distinct().collect();
|
||||
List<Pair<String, HoodieBaseFile>> latestBaseFiles = getLatestBaseFilesForAllPartitions(affectedPartitionPathList, context, hoodieTable);
|
||||
return fetchRecordLocations(context, hoodieTable, parallelism, latestBaseFiles);
|
||||
}
|
||||
|
||||
protected JavaPairRDD<HoodieKey, HoodieRecordLocation> fetchRecordLocations(HoodieEngineContext context,
|
||||
HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> hoodieTable,
|
||||
int parallelism,
|
||||
List<Pair<String, HoodieBaseFile>> baseFiles) {
|
||||
JavaSparkContext jsc = HoodieSparkEngineContext.getSparkContext(context);
|
||||
int fetchParallelism = Math.max(1, Math.max(baseFiles.size(), parallelism));
|
||||
|
||||
try {
|
||||
Option<BaseKeyGenerator> 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);
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -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<T extends HoodieRecordPayload>
|
||||
}
|
||||
|
||||
@Override
|
||||
protected HoodieIndex<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> getIndex(HoodieWriteConfig config, HoodieEngineContext context) {
|
||||
return SparkHoodieIndex.createIndex(config);
|
||||
protected HoodieIndex getIndex(HoodieWriteConfig config, HoodieEngineContext context) {
|
||||
return SparkHoodieIndexFactory.createIndex(config);
|
||||
}
|
||||
|
||||
/**
|
||||
|
||||
@@ -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<T extends HoodieRecordPayload<T>
|
||||
writeStatusRDD = writeStatusRDD.persist(SparkMemoryUtils.getWriteStatusStorageLevel(config.getProps()));
|
||||
Instant indexStartTime = Instant.now();
|
||||
// Update the index back
|
||||
JavaRDD<WriteStatus> statuses = table.getIndex().updateLocation(writeStatusRDD, context,
|
||||
table);
|
||||
JavaRDD<WriteStatus> statuses = HoodieJavaRDD.getJavaRDD(
|
||||
table.getIndex().updateLocation(HoodieJavaRDD.of(writeStatusRDD), context, table));
|
||||
result.setIndexUpdateDuration(Duration.between(indexStartTime, Instant.now()));
|
||||
result.setWriteStatuses(statuses);
|
||||
commitOnAutoCommit(result);
|
||||
|
||||
@@ -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<T extends HoodieRecordPayloa
|
||||
writeStatusRDD = writeStatusRDD.persist(SparkMemoryUtils.getWriteStatusStorageLevel(config.getProps()));
|
||||
Instant indexStartTime = Instant.now();
|
||||
// Update the index back
|
||||
JavaRDD<WriteStatus> statuses = table.getIndex().updateLocation(writeStatusRDD, context, table);
|
||||
JavaRDD<WriteStatus> statuses = HoodieJavaRDD.getJavaRDD(
|
||||
table.getIndex().updateLocation(HoodieJavaRDD.of(writeStatusRDD), context, table));
|
||||
result.setIndexUpdateDuration(Duration.between(indexStartTime, Instant.now()));
|
||||
result.setWriteStatuses(statuses);
|
||||
return statuses;
|
||||
|
||||
@@ -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<T extends HoodieRecordPayload,R> extends
|
||||
dedupedKeys.map(key -> new HoodieRecord(key, new EmptyHoodieRecordPayload()));
|
||||
Instant beginTag = Instant.now();
|
||||
// perform index loop up to get existing location of records
|
||||
JavaRDD<HoodieRecord<T>> taggedRecords =
|
||||
table.getIndex().tagLocation(dedupedRecords, context, table);
|
||||
JavaRDD<HoodieRecord<T>> taggedRecords = HoodieJavaRDD.getJavaRDD(
|
||||
table.getIndex().tagLocation(HoodieJavaRDD.of(dedupedRecords), context, table));
|
||||
Duration tagLocationDuration = Duration.between(beginTag, Instant.now());
|
||||
|
||||
// filter out non existent keys/records
|
||||
|
||||
@@ -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<T extends HoodieRecordPayload,R> extends AbstractW
|
||||
}
|
||||
|
||||
@Override
|
||||
public JavaRDD<HoodieRecord<T>> deduplicateRecords(JavaRDD<HoodieRecord<T>> records,
|
||||
HoodieIndex<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> index,
|
||||
int parallelism) {
|
||||
protected JavaRDD<HoodieRecord<T>> tag(JavaRDD<HoodieRecord<T>> dedupedRecords, HoodieEngineContext context,
|
||||
HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> table) {
|
||||
return HoodieJavaRDD.getJavaRDD(
|
||||
table.getIndex().tagLocation(HoodieJavaRDD.of(dedupedRecords), context, table));
|
||||
}
|
||||
|
||||
@Override
|
||||
public JavaRDD<HoodieRecord<T>> deduplicateRecords(
|
||||
JavaRDD<HoodieRecord<T>> records, HoodieIndex<T, ?, ?, ?> index, int parallelism) {
|
||||
boolean isIndexingGlobal = index.isGlobal();
|
||||
return records.mapToPair(record -> {
|
||||
HoodieKey hoodieKey = record.getKey();
|
||||
|
||||
@@ -156,7 +156,7 @@ public class TestHBaseIndex extends SparkClientFunctionalTestHarness {
|
||||
HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient);
|
||||
|
||||
// Test tagLocation without any entries in index
|
||||
JavaRDD<HoodieRecord> records1 = index.tagLocation(writeRecords, context(), hoodieTable);
|
||||
JavaRDD<HoodieRecord> 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<HoodieRecord> records2 = index.tagLocation(writeRecords, context(), hoodieTable);
|
||||
JavaRDD<HoodieRecord> 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<HoodieRecord> records3 = index.tagLocation(writeRecords, context(), hoodieTable).collect();
|
||||
List<HoodieRecord> 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<HoodieRecord> oldHoodieRecord = index.tagLocation(oldWriteRecords, context, hoodieTable);
|
||||
JavaRDD<HoodieRecord> oldHoodieRecord = tagLocation(index, oldWriteRecords, hoodieTable);
|
||||
assertEquals(0, oldHoodieRecord.filter(record -> record.isCurrentLocationKnown()).count());
|
||||
writeClient.startCommitWithTime(newCommitTime);
|
||||
JavaRDD<WriteStatus> 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<HoodieRecord> taggedRecords = index.tagLocation(newWriteRecords, context, hoodieTable).collect();
|
||||
List<HoodieRecord> 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<HoodieRecord> notAllowPathChangeRecords = index.tagLocation(newWriteRecords, context, hoodieTable).collect();
|
||||
List<HoodieRecord> 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<WriteStatus> 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<HoodieRecord> taggedRecords = index.tagLocation(writeRecords, context(), hoodieTable).collect();
|
||||
List<HoodieRecord> 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<HoodieRecord> beforeFirstTaggedRecords = index.tagLocation(oldWriteRecords, context, hoodieTable).collect();
|
||||
List<HoodieRecord> beforeFirstTaggedRecords = tagLocation(index, oldWriteRecords, hoodieTable).collect();
|
||||
JavaRDD<WriteStatus> oldWriteStatues = writeClient.upsert(oldWriteRecords, firstCommitTime);
|
||||
index.updateLocation(oldWriteStatues, context, hoodieTable);
|
||||
updateLocation(index, oldWriteStatues, hoodieTable);
|
||||
writeClient.commit(firstCommitTime, oldWriteStatues);
|
||||
List<HoodieRecord> afterFirstTaggedRecords = index.tagLocation(oldWriteRecords, context, hoodieTable).collect();
|
||||
List<HoodieRecord> afterFirstTaggedRecords = tagLocation(index, oldWriteRecords, hoodieTable).collect();
|
||||
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
hoodieTable = HoodieSparkTable.create(config, context, metaClient);
|
||||
final String secondCommitTime = writeClient.startCommit();
|
||||
List<HoodieRecord> beforeSecondTaggedRecords = index.tagLocation(newWriteRecords, context, hoodieTable).collect();
|
||||
List<HoodieRecord> beforeSecondTaggedRecords = tagLocation(index, newWriteRecords, hoodieTable).collect();
|
||||
JavaRDD<WriteStatus> newWriteStatues = writeClient.upsert(newWriteRecords, secondCommitTime);
|
||||
index.updateLocation(newWriteStatues, context, hoodieTable);
|
||||
updateLocation(index, newWriteStatues, hoodieTable);
|
||||
writeClient.commit(secondCommitTime, newWriteStatues);
|
||||
List<HoodieRecord> afterSecondTaggedRecords = index.tagLocation(newWriteRecords, context, hoodieTable).collect();
|
||||
List<HoodieRecord> afterSecondTaggedRecords = tagLocation(index, newWriteRecords, hoodieTable).collect();
|
||||
writeClient.rollback(secondCommitTime);
|
||||
List<HoodieRecord> afterRollback = index.tagLocation(newWriteRecords, context, hoodieTable).collect();
|
||||
List<HoodieRecord> 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<HoodieRecord> records2 = index.tagLocation(writeRecords, context(), hoodieTable).collect();
|
||||
List<HoodieRecord> 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<HoodieRecord> records3 = index.tagLocation(writeRecords, context(), hoodieTable).collect();
|
||||
List<HoodieRecord> 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<HoodieRecord> javaRDD0 = index.tagLocation(invalidWriteRecords, context(), hoodieTable);
|
||||
JavaRDD<HoodieRecord> 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<HoodieRecord> javaRDD1 = index.tagLocation(writeRecords, context(), hoodieTable);
|
||||
JavaRDD<HoodieRecord> 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<HoodieRecord> javaRDD2 = index.tagLocation(invalidWriteRecords, context(), hoodieTable);
|
||||
JavaRDD<HoodieRecord> 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<HoodieRecord> javaRDD1 = index.tagLocation(writeRecords1, context(), hoodieTable);
|
||||
JavaRDD<HoodieRecord> 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<HoodieRecord> javaRDD1 = index.tagLocation(writeRecords1, context(), hoodieTable);
|
||||
JavaRDD<HoodieRecord> 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<Get>) 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<Put>) 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<HoodieRecord> records1 = index.tagLocation(writeRecords, context(), hoodieTable);
|
||||
JavaRDD<HoodieRecord> 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<HoodieRecord> records2 = index.tagLocation(writeRecords, context(), hoodieTable);
|
||||
JavaRDD<HoodieRecord> 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<HoodieRecord> records3 = index.tagLocation(writeRecords, context(), hoodieTable).collect();
|
||||
List<HoodieRecord> 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<HoodieRecord> records1 = index.tagLocation(writeRecords, context(), hoodieTable);
|
||||
JavaRDD<HoodieRecord> 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<HoodieRecord> records2 = index.tagLocation(writeRecords, context(), hoodieTable).collect();
|
||||
List<HoodieRecord> 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<WriteStatus> deleteStatus = index.updateLocation(deleteWriteStatues, context(), hoodieTable);
|
||||
JavaRDD<WriteStatus> deleteStatus = updateLocation(index, deleteWriteStatues, hoodieTable);
|
||||
assertEquals(deleteStatus.count(), deleteWriteStatues.count());
|
||||
assertNoWriteErrors(deleteStatus.collect());
|
||||
|
||||
// Ensure no records can be tagged
|
||||
List<HoodieRecord> records3 = index.tagLocation(writeRecords, context(), hoodieTable).collect();
|
||||
List<HoodieRecord> 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
|
||||
|
||||
@@ -135,7 +135,7 @@ public class TestHoodieIndex extends HoodieClientTestHarness {
|
||||
HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient);
|
||||
|
||||
// Test tagLocation without any entries in index
|
||||
JavaRDD<HoodieRecord> javaRDD = (JavaRDD<HoodieRecord>) index.tagLocation(writeRecords, context, hoodieTable);
|
||||
JavaRDD<HoodieRecord> 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<HoodieRecord>) 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<HoodieRecord>) index.tagLocation(writeRecords, context, hoodieTable);
|
||||
javaRDD = tagLocation(index, writeRecords, hoodieTable);
|
||||
Map<String, String> recordKeyToPartitionPathMap = new HashMap();
|
||||
List<HoodieRecord> 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<WriteStatus> writeStatues = writeClient.upsert(writeRecords, newCommitTime);
|
||||
JavaRDD<HoodieRecord> javaRDD1 = (JavaRDD<HoodieRecord>) index.tagLocation(writeRecords, context, hoodieTable);
|
||||
JavaRDD<HoodieRecord> 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<HoodieRecord> javaRDD = (JavaRDD<HoodieRecord>) index.tagLocation(writeRecords, context, hoodieTable);
|
||||
JavaRDD<HoodieRecord> javaRDD = tagLocation(index, writeRecords, hoodieTable);
|
||||
|
||||
Map<String, String> recordKeyToPartitionPathMap = new HashMap();
|
||||
List<HoodieRecord> 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<HoodieRecord> javaRDD = (JavaRDD<HoodieRecord>) index.tagLocation(writeRecords, context, hoodieTable);
|
||||
JavaRDD<HoodieRecord> 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<HoodieRecord>) 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<HoodieRecord> taggedRecordRDD = (JavaRDD<HoodieRecord>) index.tagLocation(recordRDD, context, hoodieTable);
|
||||
JavaRDD<HoodieRecord> 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<HoodieRecord>) 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<HoodieRecord> recordRDD = jsc.parallelize(Collections.singletonList(incomingRecord));
|
||||
JavaRDD<HoodieRecord> taggedRecordRDD = (JavaRDD<HoodieRecord>) index.tagLocation(recordRDD, context, hoodieTable);
|
||||
JavaRDD<HoodieRecord> 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<HoodieRecord> recordRDDSamePartition = jsc
|
||||
.parallelize(Collections.singletonList(incomingRecordSamePartition));
|
||||
JavaRDD<HoodieRecord> taggedRecordRDDSamePartition = (JavaRDD<HoodieRecord>) index.tagLocation(recordRDDSamePartition, context, hoodieTable);
|
||||
JavaRDD<HoodieRecord> taggedRecordRDDSamePartition = tagLocation(index, recordRDDSamePartition, hoodieTable);
|
||||
|
||||
assertEquals(1, taggedRecordRDDSamePartition.count());
|
||||
HoodieRecord record = taggedRecordRDDSamePartition.first();
|
||||
@@ -463,8 +463,8 @@ public class TestHoodieIndex extends HoodieClientTestHarness {
|
||||
}
|
||||
|
||||
private JavaPairRDD<HoodieKey, Option<Pair<String, String>>> getRecordLocations(JavaRDD<HoodieKey> keyRDD, HoodieTable hoodieTable) {
|
||||
JavaRDD<HoodieRecord> recordRDD = (JavaRDD<HoodieRecord>) index.tagLocation(
|
||||
keyRDD.map(k -> new HoodieRecord(k, new EmptyHoodieRecordPayload())), context, hoodieTable);
|
||||
JavaRDD<HoodieRecord> 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())
|
||||
|
||||
@@ -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"));
|
||||
}
|
||||
|
||||
@@ -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<String> partitions = Arrays.asList("2016/01/21", "2016/04/01", "2015/03/12");
|
||||
List<Tuple2<String, BloomIndexFileInfo>> filesList = index.loadInvolvedFiles(partitions, context, hoodieTable);
|
||||
List<ImmutablePair<String, BloomIndexFileInfo>> 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<Tuple2<String, BloomIndexFileInfo>> 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<ImmutablePair<String, BloomIndexFileInfo>> 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<String, List<BloomIndexFileInfo>> 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<Tuple2<String, HoodieKey>> comparisonKeyList =
|
||||
index.explodeRecordRDDWithFileComparisons(partitionToFileIndexInfo, partitionRecordKeyPairRDD).collect();
|
||||
List<Pair<String, HoodieKey>> comparisonKeyList = HoodieJavaRDD.getJavaRDD(
|
||||
index.explodeRecordsWithFileComparisons(partitionToFileIndexInfo, HoodieJavaPairRDD.of(partitionRecordKeyPairRDD))).collect();
|
||||
|
||||
assertEquals(10, comparisonKeyList.size());
|
||||
Map<String, List<String>> 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<HoodieRecord> taggedRecordRDD = bloomIndex.tagLocation(recordRDD, context, hoodieTable);
|
||||
HoodieBloomIndex bloomIndex = new HoodieBloomIndex(config, SparkHoodieBloomIndexHelper.getInstance());
|
||||
JavaRDD<HoodieRecord> 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<HoodieRecord> taggedRecords = bloomIndex.tagLocation(keysRDD.map(k -> new HoodieRecord(k, null)), context, hoodieTable);
|
||||
HoodieBloomIndex bloomIndex = new HoodieBloomIndex(config, SparkHoodieBloomIndexHelper.getInstance());
|
||||
JavaRDD<HoodieRecord> taggedRecords = tagLocation(
|
||||
bloomIndex, keysRDD.map(k -> new HoodieRecord(k, null)), hoodieTable);
|
||||
JavaPairRDD<HoodieKey, Option<Pair<String, String>>> 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<HoodieRecord> taggedRecordRDD = bloomIndex.tagLocation(recordRDD, context, table);
|
||||
HoodieBloomIndex bloomIndex = new HoodieBloomIndex(config, SparkHoodieBloomIndexHelper.getInstance());
|
||||
JavaRDD<HoodieRecord> taggedRecordRDD = tagLocation(bloomIndex, recordRDD, table);
|
||||
|
||||
// Check results
|
||||
for (HoodieRecord record : taggedRecordRDD.collect()) {
|
||||
|
||||
@@ -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<String> partitions = Arrays.asList("2016/01/21", "2016/04/01");
|
||||
// partitions will NOT be respected by this loadInvolvedFiles(...) call
|
||||
List<Tuple2<String, BloomIndexFileInfo>> filesList = index.loadInvolvedFiles(partitions, context, hoodieTable);
|
||||
List<Pair<String, BloomIndexFileInfo>> 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<String, List<BloomIndexFileInfo>> 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<Tuple2<String, HoodieKey>> comparisonKeyList =
|
||||
index.explodeRecordRDDWithFileComparisons(partitionToFileIndexInfo, partitionRecordKeyPairRDD).collect();
|
||||
List<ImmutablePair<String, HoodieKey>> 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<String, List<String>> 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<HoodieRecord> taggedRecordRDD = index.tagLocation(recordRDD, context, hoodieTable);
|
||||
JavaRDD<HoodieRecord> 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<HoodieRecord> recordRDD = jsc.parallelize(Collections.singletonList(incomingRecord));
|
||||
JavaRDD<HoodieRecord> taggedRecordRDD = index.tagLocation(recordRDD, context, hoodieTable);
|
||||
JavaRDD<HoodieRecord> 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<HoodieRecord> recordRDDSamePartition = jsc
|
||||
.parallelize(Collections.singletonList(incomingRecordSamePartition));
|
||||
JavaRDD<HoodieRecord> taggedRecordRDDSamePartition = index.tagLocation(recordRDDSamePartition, context, hoodieTable);
|
||||
JavaRDD<HoodieRecord> 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<String, BloomIndexFileInfo> toFileMap(List<Tuple2<String, BloomIndexFileInfo>> filesList) {
|
||||
private static Map<String, BloomIndexFileInfo> toFileMap(List<Pair<String, BloomIndexFileInfo>> filesList) {
|
||||
Map<String, BloomIndexFileInfo> filesMap = new HashMap<>();
|
||||
for (Tuple2<String, BloomIndexFileInfo> t : filesList) {
|
||||
filesMap.put(t._1() + "/" + t._2().getFileId(), t._2());
|
||||
for (Pair<String, BloomIndexFileInfo> t : filesList) {
|
||||
filesMap.put(t.getKey() + "/" + t.getValue().getFileId(), t.getValue());
|
||||
}
|
||||
return filesMap;
|
||||
}
|
||||
|
||||
@@ -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<HoodieRecord> taggedRecords = ((JavaRDD<HoodieRecord>) index.tagLocation(jsc.parallelize(records, 1), context, table)).collect();
|
||||
HoodieIndex index = SparkHoodieIndexFactory.createIndex(cfg);
|
||||
List<HoodieRecord> taggedRecords = tagLocation(index, jsc.parallelize(records, 1), table).collect();
|
||||
checkTaggedRecords(taggedRecords, newCommitTime);
|
||||
}
|
||||
return Pair.of(newCommitTime, statuses);
|
||||
|
||||
@@ -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<EmptyHoodieRecordPayload,JavaRDD<HoodieRecord>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> table;
|
||||
@Mock private BaseSparkCommitActionExecutor<EmptyHoodieRecordPayload> executor;
|
||||
@Mock private HoodieWriteMetadata metadata;
|
||||
@Mock private JavaPairRDD keyPairs;
|
||||
@Mock private JavaSparkContext jsc;
|
||||
@Mock private HoodieSparkEngineContext context;
|
||||
@Mock
|
||||
private HoodieBloomIndex index;
|
||||
@Mock
|
||||
private HoodieTable<EmptyHoodieRecordPayload, JavaRDD<HoodieRecord>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> table;
|
||||
@Mock
|
||||
private BaseSparkCommitActionExecutor<EmptyHoodieRecordPayload> executor;
|
||||
@Mock
|
||||
private HoodieWriteMetadata metadata;
|
||||
@Mock
|
||||
private JavaPairRDD keyPairs;
|
||||
@Mock
|
||||
private JavaSparkContext jsc;
|
||||
@Mock
|
||||
private HoodieSparkEngineContext context;
|
||||
|
||||
private JavaRDD<HoodieKey> 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());
|
||||
|
||||
@@ -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<HoodieRecord> updatedRecords = dataGen.generateUpdates(newCommitTime, records);
|
||||
JavaRDD<HoodieRecord> updatedRecordsRDD = jsc.parallelize(updatedRecords, 1);
|
||||
HoodieIndex index = new SparkHoodieBloomIndex<>(config);
|
||||
updatedRecords = ((JavaRDD<HoodieRecord>)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)
|
||||
|
||||
@@ -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<List<HoodieRecord>, String, Integer> wrapRecordsGenFunctionForPreppedCalls(
|
||||
final HoodieWriteConfig writeConfig, final Function2<List<HoodieRecord>, String, Integer> recordGenFunction) {
|
||||
return (commit, numRecords) -> {
|
||||
final SparkHoodieIndex index = SparkHoodieIndex.createIndex(writeConfig);
|
||||
final HoodieIndex index = SparkHoodieIndexFactory.createIndex(writeConfig);
|
||||
List<HoodieRecord> 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<HoodieRecord> taggedRecords = index.tagLocation(jsc.parallelize(records, 1), context, table);
|
||||
JavaRDD<HoodieRecord> taggedRecords = tagLocation(index, jsc.parallelize(records, 1), table);
|
||||
return taggedRecords.collect();
|
||||
};
|
||||
}
|
||||
@@ -255,13 +256,13 @@ public class HoodieClientTestBase extends HoodieClientTestHarness {
|
||||
private Function<Integer, List<HoodieKey>> wrapDeleteKeysGenFunctionForPreppedCalls(
|
||||
final HoodieWriteConfig writeConfig, final Function<Integer, List<HoodieKey>> keyGenFunction) {
|
||||
return (numRecords) -> {
|
||||
final SparkHoodieIndex index = SparkHoodieIndex.createIndex(writeConfig);
|
||||
final HoodieIndex index = SparkHoodieIndexFactory.createIndex(writeConfig);
|
||||
List<HoodieKey> records = keyGenFunction.apply(numRecords);
|
||||
final HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder().setConf(hadoopConf).setBasePath(basePath).setLoadActiveTimelineOnLoad(true).build();
|
||||
HoodieSparkTable table = HoodieSparkTable.create(writeConfig, context, metaClient);
|
||||
JavaRDD<HoodieRecord> recordsToDelete = jsc.parallelize(records, 1)
|
||||
.map(key -> new HoodieRecord(key, new EmptyHoodieRecordPayload()));
|
||||
JavaRDD<HoodieRecord> taggedRecords = index.tagLocation(recordsToDelete, context, table);
|
||||
JavaRDD<HoodieRecord> taggedRecords = tagLocation(index, recordsToDelete, table);
|
||||
return taggedRecords.map(record -> record.getKey()).collect();
|
||||
};
|
||||
}
|
||||
|
||||
@@ -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<HoodieRecord> tagLocation(
|
||||
HoodieIndex index, JavaRDD<HoodieRecord> records, HoodieTable table) {
|
||||
return HoodieJavaRDD.getJavaRDD(
|
||||
index.tagLocation(HoodieJavaRDD.of(records), context, table));
|
||||
}
|
||||
|
||||
public static Pair<HashMap<String, WorkloadStat>, WorkloadStat> buildProfile(JavaRDD<HoodieRecord> inputRecordsRDD) {
|
||||
HashMap<String, WorkloadStat> partitionPathStatMap = new HashMap<>();
|
||||
WorkloadStat globalStat = new WorkloadStat();
|
||||
|
||||
@@ -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<HoodieRecord> tagLocation(
|
||||
HoodieIndex index, JavaRDD<HoodieRecord> records, HoodieTable table) {
|
||||
return HoodieJavaRDD.getJavaRDD(
|
||||
index.tagLocation(HoodieJavaRDD.of(records), context, table));
|
||||
}
|
||||
|
||||
protected JavaRDD<WriteStatus> updateLocation(
|
||||
HoodieIndex index, JavaRDD<WriteStatus> writeStatus, HoodieTable table) {
|
||||
return HoodieJavaRDD.getJavaRDD(
|
||||
index.updateLocation(HoodieJavaRDD.of(writeStatus), context, table));
|
||||
}
|
||||
|
||||
protected void insertRecords(HoodieTableMetaClient metaClient, List<HoodieRecord> records, SparkRDDWriteClient client, HoodieWriteConfig cfg, String commitTime) throws IOException {
|
||||
HoodieTableMetaClient reloadedMetaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
|
||||
|
||||
@@ -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<T> 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 <O> output object type.
|
||||
@@ -49,6 +67,16 @@ public abstract class HoodieData<T> implements Serializable {
|
||||
*/
|
||||
public abstract <O> HoodieData<O> map(SerializableFunction<T, O> 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 <O> output object type.
|
||||
* @return {@link HoodieData<O>} containing the result. Actual execution may be deferred.
|
||||
*/
|
||||
public abstract <O> HoodieData<O> mapPartitions(
|
||||
SerializableFunction<Iterator<T>, Iterator<O>> func, boolean preservesPartitioning);
|
||||
|
||||
/**
|
||||
* @param func serializable flatmap function.
|
||||
* @param <O> output object type.
|
||||
@@ -56,6 +84,19 @@ public abstract class HoodieData<T> implements Serializable {
|
||||
*/
|
||||
public abstract <O> HoodieData<O> flatMap(SerializableFunction<T, Iterator<O>> func);
|
||||
|
||||
/**
|
||||
* @param mapToPairFunc serializable map function to generate a pair.
|
||||
* @param <K> key type of the pair.
|
||||
* @param <V> value type of the pair.
|
||||
* @return {@link HoodiePairData<K, V>} containing the result. Actual execution may be deferred.
|
||||
*/
|
||||
public abstract <K, V> HoodiePairData<K, V> mapToPair(SerializablePairFunction<T, K, V> mapToPairFunc);
|
||||
|
||||
/**
|
||||
* @return distinct objects in {@link HoodieData}.
|
||||
*/
|
||||
public abstract HoodieData<T> distinct();
|
||||
|
||||
/**
|
||||
* @return collected results in {@link List<T>}.
|
||||
*/
|
||||
|
||||
@@ -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<T> extends HoodieData<T> {
|
||||
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 <O> HoodieData<O> map(SerializableFunction<T, O> func) {
|
||||
return HoodieList.of(listData.stream().parallel()
|
||||
.map(throwingMapWrapper(func)).collect(Collectors.toList()));
|
||||
}
|
||||
|
||||
@Override
|
||||
public <O> HoodieData<O> mapPartitions(SerializableFunction<Iterator<T>, Iterator<O>> func, boolean preservesPartitioning) {
|
||||
List<O> result = new ArrayList<>();
|
||||
throwingMapWrapper(func).apply(listData.iterator()).forEachRemaining(result::add);
|
||||
return HoodieList.of(result);
|
||||
}
|
||||
|
||||
@Override
|
||||
public <O> HoodieData<O> flatMap(SerializableFunction<T, Iterator<O>> func) {
|
||||
Function<T, Iterator<O>> throwableFunc = throwingMapWrapper(func);
|
||||
@@ -87,6 +115,23 @@ public class HoodieList<T> extends HoodieData<T> {
|
||||
}).collect(Collectors.toList()));
|
||||
}
|
||||
|
||||
@Override
|
||||
public <K, V> HoodiePairData<K, V> mapToPair(SerializablePairFunction<T, K, V> mapToPairFunc) {
|
||||
Map<K, List<V>> mapOfPairs = new HashMap<>();
|
||||
Function<T, Pair<K, V>> throwableMapToPairFunc = throwingMapToPairWrapper(mapToPairFunc);
|
||||
listData.forEach(data -> {
|
||||
Pair<K, V> pair = throwableMapToPairFunc.apply(data);
|
||||
List<V> list = mapOfPairs.computeIfAbsent(pair.getKey(), k -> new ArrayList<>());
|
||||
list.add(pair.getValue());
|
||||
});
|
||||
return HoodieMapPair.of(mapOfPairs);
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodieData<T> distinct() {
|
||||
return HoodieList.of(new ArrayList<>(new HashSet<>(listData)));
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<T> collectAsList() {
|
||||
return listData;
|
||||
|
||||
@@ -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 <K> type of key.
|
||||
* @param <V> type of value.
|
||||
*/
|
||||
public class HoodieMapPair<K, V> extends HoodiePairData<K, V> {
|
||||
|
||||
private final Map<K, List<V>> mapPairData;
|
||||
|
||||
private HoodieMapPair(Map<K, List<V>> mapPairData) {
|
||||
this.mapPairData = mapPairData;
|
||||
}
|
||||
|
||||
/**
|
||||
* @param mapPairData a {@link Map} of pairs.
|
||||
* @param <K> type of key.
|
||||
* @param <V> type of value.
|
||||
* @return a new instance containing the {@link Map<K, List<V>>} reference.
|
||||
*/
|
||||
public static <K, V> HoodieMapPair<K, V> of(Map<K, List<V>> mapPairData) {
|
||||
return new HoodieMapPair<>(mapPairData);
|
||||
}
|
||||
|
||||
/**
|
||||
* @param hoodiePairData {@link HoodieMapPair <K, V>} instance containing the {@link Map} of pairs.
|
||||
* @param <K> type of key.
|
||||
* @param <V> type of value.
|
||||
* @return the {@link Map} of pairs.
|
||||
*/
|
||||
public static <K, V> Map<K, List<V>> getMapPair(HoodiePairData<K, V> hoodiePairData) {
|
||||
return ((HoodieMapPair<K, V>) hoodiePairData).get();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Map<K, List<V>> get() {
|
||||
return mapPairData;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void persist(String cacheConfig) {
|
||||
// No OP
|
||||
}
|
||||
|
||||
@Override
|
||||
public void unpersist() {
|
||||
// No OP
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodieData<K> keys() {
|
||||
return HoodieList.of(new ArrayList<>(mapPairData.keySet()));
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodieData<V> 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<K, Long> countByKey() {
|
||||
return mapPairData.entrySet().stream().collect(
|
||||
Collectors.toMap(Map.Entry::getKey, entry -> (long) entry.getValue().size()));
|
||||
}
|
||||
|
||||
@Override
|
||||
public <O> HoodieData<O> map(SerializableFunction<Pair<K, V>, O> func) {
|
||||
Function<Pair<K, V>, O> throwableFunc = throwingMapWrapper(func);
|
||||
return HoodieList.of(
|
||||
streamAllPairs().map(throwableFunc).collect(Collectors.toList()));
|
||||
}
|
||||
|
||||
@Override
|
||||
public <L, W> HoodiePairData<L, W> mapToPair(SerializablePairFunction<Pair<K, V>, L, W> mapToPairFunc) {
|
||||
Map<L, List<W>> newMap = new HashMap<>();
|
||||
Function<Pair<K, V>, Pair<L, W>> throwableMapToPairFunc =
|
||||
FunctionWrapper.throwingMapToPairWrapper(mapToPairFunc);
|
||||
streamAllPairs().map(pair -> throwableMapToPairFunc.apply(pair)).forEach(newPair -> {
|
||||
List<W> list = newMap.computeIfAbsent(newPair.getKey(), k -> new ArrayList<>());
|
||||
list.add(newPair.getValue());
|
||||
});
|
||||
return HoodieMapPair.of(newMap);
|
||||
}
|
||||
|
||||
@Override
|
||||
public <W> HoodiePairData<K, Pair<V, Option<W>>> leftOuterJoin(HoodiePairData<K, W> other) {
|
||||
Map<K, List<W>> otherMapPairData = HoodieMapPair.getMapPair(other);
|
||||
Stream<ImmutablePair<K, ImmutablePair<V, Option<List<W>>>>> pairs = streamAllPairs()
|
||||
.map(pair -> new ImmutablePair<>(pair.getKey(), new ImmutablePair<>(
|
||||
pair.getValue(), Option.ofNullable(otherMapPairData.get(pair.getKey())))));
|
||||
Map<K, List<Pair<V, Option<W>>>> resultMap = new HashMap<>();
|
||||
pairs.forEach(pair -> {
|
||||
K key = pair.getKey();
|
||||
ImmutablePair<V, Option<List<W>>> valuePair = pair.getValue();
|
||||
List<Pair<V, Option<W>>> 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<ImmutablePair<K, V>> streamAllPairs() {
|
||||
return mapPairData.entrySet().stream().flatMap(
|
||||
entry -> entry.getValue().stream().map(e -> new ImmutablePair<>(entry.getKey(), e)));
|
||||
}
|
||||
}
|
||||
@@ -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 <K> type of key.
|
||||
* @param <V> type of value.
|
||||
*/
|
||||
public abstract class HoodiePairData<K, V> 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<K> keys();
|
||||
|
||||
/**
|
||||
* @return all values in {@link HoodieData}.
|
||||
*/
|
||||
public abstract HoodieData<V> values();
|
||||
|
||||
/**
|
||||
* @return the number of pairs.
|
||||
*/
|
||||
public abstract long count();
|
||||
|
||||
/**
|
||||
* @return the number of pairs per key in a {@link Map}.
|
||||
*/
|
||||
public abstract Map<K, Long> countByKey();
|
||||
|
||||
/**
|
||||
* @param func serializable map function.
|
||||
* @param <O> output object type.
|
||||
* @return {@link HoodieData<O>} containing the result. Actual execution may be deferred.
|
||||
*/
|
||||
public abstract <O> HoodieData<O> map(SerializableFunction<Pair<K, V>, O> func);
|
||||
|
||||
/**
|
||||
* @param mapToPairFunc serializable map function to generate another pair.
|
||||
* @param <L> new key type.
|
||||
* @param <W> new value type.
|
||||
* @return {@link HoodiePairData<L, W>} containing the result. Actual execution may be deferred.
|
||||
*/
|
||||
public abstract <L, W> HoodiePairData<L, W> mapToPair(
|
||||
SerializablePairFunction<Pair<K, V>, 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 <W> value type of the other {@link HoodiePairData}
|
||||
* @return {@link HoodiePairData<K, Pair<V, Option<W>>>} containing the left outer join result.
|
||||
* Actual execution may be deferred.
|
||||
*/
|
||||
public abstract <W> HoodiePairData<K, Pair<V, Option<W>>> leftOuterJoin(HoodiePairData<K, W> other);
|
||||
}
|
||||
@@ -65,7 +65,11 @@ public abstract class HoodieEngineContext {
|
||||
|
||||
public abstract <T> HoodieData<T> emptyHoodieData();
|
||||
|
||||
public abstract <T> HoodieData<T> parallelize(List<T> data);
|
||||
public <T> HoodieData<T> parallelize(List<T> data) {
|
||||
return parallelize(data, data.size());
|
||||
}
|
||||
|
||||
public abstract <T> HoodieData<T> parallelize(List<T> data, int parallelism);
|
||||
|
||||
public abstract <I, O> List<O> map(List<I> data, SerializableFunction<I, O> func, int parallelism);
|
||||
|
||||
|
||||
@@ -75,7 +75,7 @@ public final class HoodieLocalEngineContext extends HoodieEngineContext {
|
||||
}
|
||||
|
||||
@Override
|
||||
public <T> HoodieData<T> parallelize(List<T> data) {
|
||||
public <T> HoodieData<T> parallelize(List<T> data, int parallelism) {
|
||||
return HoodieList.of(data);
|
||||
}
|
||||
|
||||
|
||||
@@ -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<Pair<String, String>> TEST_PAIRS;
|
||||
private static HoodiePairData<String, String> 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<String, Long> 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<String, List<Integer>> 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<String, String> 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<String, Integer> pairData2 = constructTestMapPairData(Arrays.asList(
|
||||
ImmutablePair.of(KEY1, INTEGER_VALUE1),
|
||||
ImmutablePair.of(KEY2, INTEGER_VALUE2),
|
||||
ImmutablePair.of(KEY5, INTEGER_VALUE3)
|
||||
));
|
||||
|
||||
Map<String, List<Pair<String, Option<Integer>>>> 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<String, Integer> 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<String, List<Pair<String, Option<Integer>>>> 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<Pair<String, String>> 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 <V> HoodiePairData<String, V> constructTestMapPairData(
|
||||
final List<Pair<String, V>> pairs) {
|
||||
Map<String, List<V>> map = new HashMap<>();
|
||||
addPairsToMap(map, pairs);
|
||||
return HoodieMapPair.of(map);
|
||||
}
|
||||
|
||||
private static <V> void addPairsToMap(
|
||||
Map<String, List<V>> map, final List<Pair<String, V>> pairs) {
|
||||
for (Pair<String, V> pair : pairs) {
|
||||
String key = pair.getKey();
|
||||
V value = pair.getValue();
|
||||
List<V> list = map.computeIfAbsent(key, k -> new ArrayList<>());
|
||||
list.add(value);
|
||||
}
|
||||
}
|
||||
|
||||
private <T> void assertHoodieDataEquals(
|
||||
List<T> expectedList, HoodieData<T> hoodieData) {
|
||||
assertHoodieDataEquals(expectedList, hoodieData, Comparator.naturalOrder());
|
||||
}
|
||||
|
||||
private <T> void assertHoodieDataEquals(
|
||||
List<T> expectedList, HoodieData<T> hoodieData, Comparator comparator) {
|
||||
assertEquals(expectedList,
|
||||
hoodieData.collectAsList().stream().sorted(comparator).collect(Collectors.toList())
|
||||
);
|
||||
}
|
||||
}
|
||||
@@ -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
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user