1
0

[HUDI-2472] Enabling metadata table for TestHoodieMergeOnReadTable and TestHoodieCompactor (#4023)

This commit is contained in:
Manoj Govindassamy
2021-11-19 17:02:21 -08:00
committed by GitHub
parent 459b34240b
commit c8617d9390
6 changed files with 84 additions and 56 deletions

View File

@@ -30,6 +30,7 @@ import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.log.HoodieLogFormat;
import org.apache.hudi.common.table.log.block.HoodieAvroDataBlock;
import org.apache.hudi.common.table.log.block.HoodieLogBlock.HeaderMetadataType;
import org.apache.hudi.common.util.collection.Pair;
import org.apache.hudi.table.HoodieTable;
import org.apache.avro.Schema;
@@ -38,7 +39,9 @@ import org.apache.hadoop.fs.Path;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.stream.Collectors;
@@ -105,14 +108,18 @@ public class HoodieFlinkWriteableTestTable extends HoodieWriteableTestTable {
return (HoodieFlinkWriteableTestTable) withInserts(partition, fileId, records, new org.apache.hudi.client.FlinkTaskContextSupplier(null));
}
public HoodieFlinkWriteableTestTable withLogAppends(List<HoodieRecord> records) throws Exception {
for (List<HoodieRecord> groupedRecords: records.stream().collect(Collectors.groupingBy(HoodieRecord::getCurrentLocation)).values()) {
appendRecordsToLogFile(groupedRecords);
public Map<String, List<HoodieLogFile>> withLogAppends(List<HoodieRecord> records) throws Exception {
Map<String, List<HoodieLogFile>> partitionToLogfilesMap = new HashMap<>();
for (List<HoodieRecord> groupedRecords : records.stream().collect(
Collectors.groupingBy(HoodieRecord::getCurrentLocation)).values()) {
final Pair<String, HoodieLogFile> appendedLogFile = appendRecordsToLogFile(groupedRecords);
partitionToLogfilesMap.computeIfAbsent(
appendedLogFile.getKey(), k -> new ArrayList<>()).add(appendedLogFile.getValue());
}
return this;
return partitionToLogfilesMap;
}
private void appendRecordsToLogFile(List<HoodieRecord> groupedRecords) throws Exception {
private Pair<String, HoodieLogFile> appendRecordsToLogFile(List<HoodieRecord> groupedRecords) throws Exception {
String partitionPath = groupedRecords.get(0).getPartitionPath();
HoodieRecordLocation location = groupedRecords.get(0).getCurrentLocation();
try (HoodieLogFormat.Writer logWriter = HoodieLogFormat.newWriterBuilder().onParentPath(new Path(basePath, partitionPath))
@@ -131,6 +138,7 @@ public class HoodieFlinkWriteableTestTable extends HoodieWriteableTestTable {
return null;
}
}).collect(Collectors.toList()), header));
return Pair.of(partitionPath, logWriter.getLogFile());
}
}
}