1
0

[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:
Y Ethan Guo
2021-10-28 01:16:00 -07:00
committed by GitHub
parent e5b6b8602c
commit 0223c442ec
70 changed files with 2196 additions and 1567 deletions

View File

@@ -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());
}

View File

@@ -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);
}

View File

@@ -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));
}
}

View File

@@ -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());
}
}
}

View File

@@ -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);
}
}

View File

@@ -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;
}
}

View File

@@ -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");
}

View File

@@ -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);
}
/**

View File

@@ -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

View File

@@ -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();

View File

@@ -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) {

View File

@@ -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.