1
0

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

This commit is contained in:
Alexey Kudinkin
2022-07-16 16:26:48 -07:00
committed by GitHub
parent 80368a049d
commit 4bda6afe0b
35 changed files with 868 additions and 555 deletions

View File

@@ -20,7 +20,7 @@ package org.apache.hudi.client;
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;
@@ -127,8 +127,7 @@ 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 = 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());
}

View File

@@ -23,7 +23,7 @@ import org.apache.hudi.common.config.SerializableConfiguration;
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;
@@ -84,12 +84,12 @@ public class HoodieFlinkEngineContext extends HoodieEngineContext {
@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() {

View File

@@ -23,7 +23,7 @@ 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.data.HoodieListData;
import org.apache.hudi.common.engine.HoodieEngineContext;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieRecordPayload;
@@ -61,8 +61,8 @@ public abstract class FlinkHoodieIndex<T extends HoodieRecordPayload> extends Ho
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
@@ -70,6 +70,6 @@ public abstract class FlinkHoodieIndex<T extends HoodieRecordPayload> extends Ho
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));
}
}

View File

@@ -21,7 +21,6 @@ package org.apache.hudi.metadata;
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;
@@ -106,7 +105,7 @@ public class FlinkHoodieBackedTableMetadataWriter extends HoodieBackedTableMetad
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) {

View File

@@ -40,8 +40,6 @@ import org.apache.avro.specific.SpecificRecordBase;
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> {
@@ -78,7 +76,7 @@ public abstract class HoodieFlinkTable<T extends HoodieRecordPayload>
public static HoodieWriteMetadata<List<WriteStatus>> convertMetadata(
HoodieWriteMetadata<HoodieData<WriteStatus>> metadata) {
return metadata.clone(getList(metadata.getWriteStatuses()));
return metadata.clone(metadata.getWriteStatuses().collectAsList());
}
@Override

View File

@@ -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;
@@ -97,8 +97,7 @@ public class FlinkDeleteHelper<R> extends
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

View File

@@ -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;
@@ -83,8 +83,7 @@ public class FlinkWriteHelper<T extends HoodieRecordPayload, R> extends BaseWrit
@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

View File

@@ -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;
@@ -37,9 +38,6 @@ import org.apache.hudi.table.HoodieFlinkTable;
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;
@@ -185,8 +183,7 @@ public class TestFlinkHoodieBloomIndex extends HoodieFlinkClientTestHarness {
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()

View File

@@ -21,7 +21,8 @@ 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.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;
@@ -133,7 +134,7 @@ public class HoodieFlinkClientTestHarness extends HoodieCommonTestHarness implem
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();
}
/**