Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[HUDI-4249] Fixing in-memory HoodieData implementation to operate lazily #5855

Merged
merged 19 commits into from
Jul 16, 2022
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -40,7 +40,6 @@
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;

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,6 @@
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;
Expand Down Expand Up @@ -60,7 +59,7 @@ public HoodiePairData<HoodieKey, HoodieRecordLocation> findMatchingFilesForRecor
HoodieData<Pair<String, HoodieKey>> fileComparisonPairs,
Map<String, List<BloomIndexFileInfo>> partitionToFileInfo, Map<String, Long> recordsPerPartition) {
List<Pair<String, HoodieKey>> fileComparisonPairList =
HoodieList.getList(fileComparisonPairs).stream()
fileComparisonPairs.collectAsList().stream()
.sorted(Comparator.comparing(Pair::getLeft)).collect(toList());

List<HoodieKeyLookupResult> keyLookupResults = new ArrayList<>();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -83,7 +83,8 @@ protected HoodieRecord<R> computeNext() {
Option<HoodieRecordLocation> loc = mapper.getRecordLocation(record.getKey(), record.getPartitionPath());
return HoodieIndexUtils.getTaggedRecord(record, loc);
}
}
},
false
);
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,7 @@

import org.apache.hudi.async.AsyncCleanerService;
import org.apache.hudi.client.common.HoodieFlinkEngineContext;
import org.apache.hudi.common.data.HoodieList;
import org.apache.hudi.common.data.HoodieListData;
import org.apache.hudi.common.engine.HoodieEngineContext;
import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.model.FileSlice;
Expand Down Expand Up @@ -127,8 +127,7 @@ public List<HoodieRecord<T>> filterExists(List<HoodieRecord<T>> hoodieRecords) {
// Create a Hoodie table which encapsulated the commits and files visible
HoodieFlinkTable<T> table = getHoodieTable();
Timer.Context indexTimer = metrics.getIndexCtx();
List<HoodieRecord<T>> recordsWithLocation = HoodieList.getList(
getIndex().tagLocation(HoodieList.of(hoodieRecords), context, table));
List<HoodieRecord<T>> recordsWithLocation = getIndex().tagLocation(HoodieListData.eager(hoodieRecords), context, table).collectAsList();
metrics.updateIndexMetrics(LOOKUP_STR, metrics.getDurationInMs(indexTimer == null ? 0L : indexTimer.stop()));
return recordsWithLocation.stream().filter(v1 -> !v1.isCurrentLocationKnown()).collect(Collectors.toList());
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,7 +23,7 @@
import org.apache.hudi.common.data.HoodieAccumulator;
import org.apache.hudi.common.data.HoodieAtomicLongAccumulator;
import org.apache.hudi.common.data.HoodieData;
import org.apache.hudi.common.data.HoodieList;
import org.apache.hudi.common.data.HoodieListData;
import org.apache.hudi.common.engine.EngineProperty;
import org.apache.hudi.common.engine.HoodieEngineContext;
import org.apache.hudi.common.engine.TaskContextSupplier;
Expand Down Expand Up @@ -84,12 +84,12 @@ public HoodieAccumulator newAccumulator() {

@Override
public <T> HoodieData<T> emptyHoodieData() {
return HoodieList.of(Collections.emptyList());
return HoodieListData.eager(Collections.emptyList());
}

@Override
public <T> HoodieData<T> parallelize(List<T> data, int parallelism) {
return HoodieList.of(data);
return HoodieListData.eager(data);
}

public RuntimeContext getRuntimeContext() {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,7 +23,7 @@
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.data.HoodieListData;
import org.apache.hudi.common.engine.HoodieEngineContext;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieRecordPayload;
Expand Down Expand Up @@ -61,15 +61,15 @@ public abstract List<HoodieRecord<T>> tagLocation(List<HoodieRecord<T>> records,
public <R> HoodieData<HoodieRecord<R>> tagLocation(
HoodieData<HoodieRecord<R>> records, HoodieEngineContext context,
HoodieTable hoodieTable) throws HoodieIndexException {
List<HoodieRecord<T>> hoodieRecords = tagLocation(HoodieList.getList(records.map(record -> (HoodieRecord<T>) record)), context, hoodieTable);
return HoodieList.of(hoodieRecords.stream().map(r -> (HoodieRecord<R>) r).collect(Collectors.toList()));
List<HoodieRecord<T>> hoodieRecords = tagLocation(records.map(record -> (HoodieRecord<T>) record).collectAsList(), context, hoodieTable);
return HoodieListData.eager(hoodieRecords.stream().map(r -> (HoodieRecord<R>) r).collect(Collectors.toList()));
}

@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));
return HoodieListData.eager(updateLocation(writeStatuses.collectAsList(), context, hoodieTable));
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,6 @@
import org.apache.hudi.client.HoodieFlinkWriteClient;
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.metrics.Registry;
import org.apache.hudi.common.model.HoodieRecord;
Expand Down Expand Up @@ -106,7 +105,7 @@ protected void commit(String instantTime, Map<MetadataPartitionType, HoodieData<
ValidationUtils.checkState(enabled, "Metadata table cannot be committed to as it is not enabled");
ValidationUtils.checkState(metadataMetaClient != null, "Metadata table is not fully initialized yet.");
HoodieData<HoodieRecord> preppedRecords = prepRecords(partitionRecordsMap);
List<HoodieRecord> preppedRecordList = HoodieList.getList(preppedRecords);
List<HoodieRecord> preppedRecordList = preppedRecords.collectAsList();

try (HoodieFlinkWriteClient writeClient = new HoodieFlinkWriteClient(engineContext, metadataWriteConfig)) {
if (canTriggerTableService) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -40,8 +40,6 @@

import java.util.List;

import static org.apache.hudi.common.data.HoodieList.getList;

public abstract class HoodieFlinkTable<T extends HoodieRecordPayload>
extends HoodieTable<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>>
implements ExplicitWriteHandleTable<T> {
Expand Down Expand Up @@ -78,7 +76,7 @@ public static <T extends HoodieRecordPayload> HoodieFlinkTable<T> create(HoodieW

public static HoodieWriteMetadata<List<WriteStatus>> convertMetadata(
HoodieWriteMetadata<HoodieData<WriteStatus>> metadata) {
return metadata.clone(getList(metadata.getWriteStatuses()));
return metadata.clone(metadata.getWriteStatuses().collectAsList());
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +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.data.HoodieListData;
import org.apache.hudi.common.engine.HoodieEngineContext;
import org.apache.hudi.common.model.EmptyHoodieRecordPayload;
import org.apache.hudi.common.model.HoodieAvroRecord;
Expand Down Expand Up @@ -97,8 +97,7 @@ public HoodieWriteMetadata<List<WriteStatus>> execute(String instantTime,
dedupedKeys.stream().map(key -> new HoodieAvroRecord<>(key, new EmptyHoodieRecordPayload())).collect(Collectors.toList());
Instant beginTag = Instant.now();
// perform index look up to get existing location of records
List<HoodieRecord<EmptyHoodieRecordPayload>> taggedRecords = HoodieList.getList(
table.getIndex().tagLocation(HoodieList.of(dedupedRecords), context, table));
List<HoodieRecord<EmptyHoodieRecordPayload>> taggedRecords = table.getIndex().tagLocation(HoodieListData.eager(dedupedRecords), context, table).collectAsList();
Duration tagLocationDuration = Duration.between(beginTag, Instant.now());

// filter out non existent keys/records
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +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.data.HoodieListData;
import org.apache.hudi.common.engine.HoodieEngineContext;
import org.apache.hudi.common.model.HoodieAvroRecord;
import org.apache.hudi.common.model.HoodieKey;
Expand Down Expand Up @@ -83,8 +83,7 @@ public HoodieWriteMetadata<List<WriteStatus>> write(String instantTime, List<Hoo

@Override
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));
return table.getIndex().tagLocation(HoodieListData.eager(dedupedRecords), context, table).collectAsList();
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,11 +18,12 @@

package org.apache.hudi.index.bloom;

import org.apache.avro.Schema;
import org.apache.hadoop.fs.Path;
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.data.HoodieListPairData;
import org.apache.hudi.common.model.HoodieAvroRecord;
import org.apache.hudi.common.model.HoodieKey;
import org.apache.hudi.common.model.HoodieRecord;
Expand All @@ -37,9 +38,6 @@
import org.apache.hudi.table.HoodieTable;
import org.apache.hudi.testutils.HoodieFlinkClientTestHarness;
import org.apache.hudi.testutils.HoodieFlinkWriteableTestTable;

import org.apache.avro.Schema;
import org.apache.hadoop.fs.Path;
import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
Expand Down Expand Up @@ -185,8 +183,7 @@ public void testRangePruning(boolean rangePruning, boolean treeFiltering, boolea
partitionRecordKeyMap.put(t.getLeft(), recordKeyList);
});

List<Pair<String, HoodieKey>> comparisonKeyList = HoodieList.getList(
index.explodeRecordsWithFileComparisons(partitionToFileIndexInfo, HoodieMapPair.of(partitionRecordKeyMap)));
List<Pair<String, HoodieKey>> comparisonKeyList = index.explodeRecordsWithFileComparisons(partitionToFileIndexInfo, HoodieListPairData.lazy(partitionRecordKeyMap)).collectAsList();

assertEquals(10, comparisonKeyList.size());
java.util.Map<String, List<String>> recordKeyToFileComps = comparisonKeyList.stream()
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,8 @@
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.data.HoodieData;
import org.apache.hudi.common.data.HoodieListData;
import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieTableType;
Expand Down Expand Up @@ -133,7 +134,7 @@ protected void initMetaClient(HoodieTableType tableType) throws IOException {

protected List<HoodieRecord> tagLocation(
HoodieIndex index, List<HoodieRecord> records, HoodieTable table) {
return HoodieList.getList(index.tagLocation(HoodieList.of(records), context, table));
return ((HoodieData<HoodieRecord>) index.tagLocation(HoodieListData.eager(records), context, table)).collectAsList();
}

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,7 @@

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.data.HoodieListData;
import org.apache.hudi.common.engine.HoodieEngineContext;
import org.apache.hudi.common.model.HoodieCommitMetadata;
import org.apache.hudi.common.model.HoodieKey;
Expand Down Expand Up @@ -67,8 +67,7 @@ public List<HoodieRecord<T>> filterExists(List<HoodieRecord<T>> hoodieRecords) {
// 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 = HoodieList.getList(
getIndex().tagLocation(HoodieList.of(hoodieRecords), context, table));
List<HoodieRecord<T>> recordsWithLocation = getIndex().tagLocation(HoodieListData.eager(hoodieRecords), context, table).collectAsList();
metrics.updateIndexMetrics(LOOKUP_STR, metrics.getDurationInMs(indexTimer == null ? 0L : indexTimer.stop()));
return recordsWithLocation.stream().filter(v1 -> !v1.isCurrentLocationKnown()).collect(Collectors.toList());
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,7 @@
import org.apache.hudi.client.WriteStatus;
import org.apache.hudi.client.common.JavaTaskContextSupplier;
import org.apache.hudi.common.data.HoodieData;
import org.apache.hudi.common.data.HoodieList;
import org.apache.hudi.common.data.HoodieListData;
import org.apache.hudi.common.engine.HoodieEngineContext;
import org.apache.hudi.common.model.ClusteringOperation;
import org.apache.hudi.common.model.HoodieAvroRecord;
Expand Down Expand Up @@ -94,7 +94,7 @@ public HoodieWriteMetadata<HoodieData<WriteStatus>> performClustering(
Option.ofNullable(clusteringPlan.getPreserveHoodieMetadata()).orElse(false),
instantTime)));
HoodieWriteMetadata<HoodieData<WriteStatus>> writeMetadata = new HoodieWriteMetadata<>();
writeMetadata.setWriteStatuses(HoodieList.of(writeStatusList));
writeMetadata.setWriteStatuses(HoodieListData.eager(writeStatusList));
return writeMetadata;
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,7 +24,7 @@
import org.apache.hudi.common.data.HoodieAccumulator;
import org.apache.hudi.common.data.HoodieAtomicLongAccumulator;
import org.apache.hudi.common.data.HoodieData;
import org.apache.hudi.common.data.HoodieList;
import org.apache.hudi.common.data.HoodieListData;
import org.apache.hudi.common.engine.EngineProperty;
import org.apache.hudi.common.engine.HoodieEngineContext;
import org.apache.hudi.common.engine.TaskContextSupplier;
Expand Down Expand Up @@ -74,12 +74,12 @@ public HoodieAccumulator newAccumulator() {

@Override
public <T> HoodieData<T> emptyHoodieData() {
return HoodieList.of(Collections.emptyList());
return HoodieListData.eager(Collections.emptyList());
}

@Override
public <T> HoodieData<T> parallelize(List<T> data, int parallelism) {
return HoodieList.of(data);
return HoodieListData.eager(data);
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,7 +23,7 @@
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.data.HoodieListData;
import org.apache.hudi.common.engine.HoodieEngineContext;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieRecordPayload;
Expand Down Expand Up @@ -58,15 +58,15 @@ public abstract List<HoodieRecord<T>> tagLocation(List<HoodieRecord<T>> records,
public <R> HoodieData<HoodieRecord<R>> tagLocation(
HoodieData<HoodieRecord<R>> records, HoodieEngineContext context,
HoodieTable hoodieTable) throws HoodieIndexException {
List<HoodieRecord<T>> hoodieRecords = tagLocation(HoodieList.getList(records.map(record -> (HoodieRecord<T>) record)), context, hoodieTable);
return HoodieList.of(hoodieRecords.stream().map(r -> (HoodieRecord<R>) r).collect(Collectors.toList()));
List<HoodieRecord<T>> hoodieRecords = tagLocation(records.map(record -> (HoodieRecord<T>) record).collectAsList(), context, hoodieTable);
return HoodieListData.eager(hoodieRecords.stream().map(r -> (HoodieRecord<R>) r).collect(Collectors.toList()));
}

@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));
return HoodieListData.eager(updateLocation(writeStatuses.collectAsList(), context, hoodieTable));
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -36,8 +36,6 @@

import java.util.List;

import static org.apache.hudi.common.data.HoodieList.getList;

public abstract class HoodieJavaTable<T extends HoodieRecordPayload>
extends HoodieTable<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>> {
protected HoodieJavaTable(HoodieWriteConfig config, HoodieEngineContext context, HoodieTableMetaClient metaClient) {
Expand Down Expand Up @@ -67,7 +65,7 @@ public static <T extends HoodieRecordPayload> HoodieJavaTable<T> create(HoodieWr

public static HoodieWriteMetadata<List<WriteStatus>> convertMetadata(
HoodieWriteMetadata<HoodieData<WriteStatus>> metadata) {
return metadata.clone(getList(metadata.getWriteStatuses()));
return metadata.clone(metadata.getWriteStatuses().collectAsList());
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +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.data.HoodieListData;
import org.apache.hudi.common.engine.HoodieEngineContext;
import org.apache.hudi.common.model.HoodieBaseFile;
import org.apache.hudi.common.model.HoodieCommitMetadata;
Expand Down Expand Up @@ -130,8 +130,7 @@ public HoodieWriteMetadata<List<WriteStatus>> execute(List<HoodieRecord<T>> inpu
protected List<WriteStatus> updateIndex(List<WriteStatus> writeStatuses, HoodieWriteMetadata<List<WriteStatus>> result) {
Instant indexStartTime = Instant.now();
// Update the index back
List<WriteStatus> statuses = HoodieList.getList(
table.getIndex().updateLocation(HoodieList.of(writeStatuses), context, table));
List<WriteStatus> statuses = table.getIndex().updateLocation(HoodieListData.eager(writeStatuses), context, table).collectAsList();
result.setIndexUpdateDuration(Duration.between(indexStartTime, Instant.now()));
result.setWriteStatuses(statuses);
return statuses;
Expand Down Expand Up @@ -339,8 +338,7 @@ public Partitioner getInsertPartitioner(WorkloadProfile profile) {
public void updateIndexAndCommitIfNeeded(List<WriteStatus> writeStatuses, HoodieWriteMetadata result) {
Instant indexStartTime = Instant.now();
// Update the index back
List<WriteStatus> statuses = HoodieList.getList(
table.getIndex().updateLocation(HoodieList.of(writeStatuses), context, table));
List<WriteStatus> statuses = table.getIndex().updateLocation(HoodieListData.eager(writeStatuses), context, table).collectAsList();
result.setIndexUpdateDuration(Duration.between(indexStartTime, Instant.now()));
result.setWriteStatuses(statuses);
result.setPartitionToReplaceFileIds(getPartitionToReplacedFileIds(result));
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +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.data.HoodieListData;
import org.apache.hudi.common.engine.HoodieEngineContext;
import org.apache.hudi.common.model.EmptyHoodieRecordPayload;
import org.apache.hudi.common.model.HoodieAvroRecord;
Expand Down Expand Up @@ -99,8 +99,7 @@ public HoodieWriteMetadata<List<WriteStatus>> execute(String instantTime,
dedupedKeys.stream().map(key -> new HoodieAvroRecord<>(key, new EmptyHoodieRecordPayload())).collect(Collectors.toList());
Instant beginTag = Instant.now();
// perform index look up to get existing location of records
List<HoodieRecord<EmptyHoodieRecordPayload>> taggedRecords = HoodieList.getList(
table.getIndex().tagLocation(HoodieList.of(dedupedRecords), context, table));
List<HoodieRecord<EmptyHoodieRecordPayload>> taggedRecords = table.getIndex().tagLocation(HoodieListData.eager(dedupedRecords), context, table).collectAsList();
Duration tagLocationDuration = Duration.between(beginTag, Instant.now());

// filter out non existent keys/records
Expand Down
Loading