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

@@ -33,8 +33,9 @@ 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;
import org.apache.hudi.common.testutils.FileCreateUtils;
import org.apache.hudi.common.testutils.HoodieTestTable;
import org.apache.hudi.common.testutils.HoodieMetadataTestTable;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.collection.Pair;
import org.apache.hudi.config.HoodieStorageConfig;
import org.apache.hudi.io.storage.HoodieAvroParquetConfig;
import org.apache.hudi.io.storage.HoodieOrcConfig;
@@ -47,6 +48,7 @@ import org.apache.avro.generic.IndexedRecord;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hudi.metadata.HoodieTableMetadataWriter;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import org.apache.orc.CompressionKind;
@@ -56,7 +58,7 @@ import org.apache.parquet.hadoop.metadata.CompressionCodecName;
import java.io.IOException;
import java.nio.file.Paths;
import java.util.Arrays;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
@@ -64,15 +66,21 @@ import java.util.stream.Collectors;
import static org.apache.hudi.common.testutils.FileCreateUtils.baseFileName;
public class HoodieWriteableTestTable extends HoodieTestTable {
public class HoodieWriteableTestTable extends HoodieMetadataTestTable {
private static final Logger LOG = LogManager.getLogger(HoodieWriteableTestTable.class);
protected final Schema schema;
protected final BloomFilter filter;
protected final boolean populateMetaFields;
protected HoodieWriteableTestTable(String basePath, FileSystem fs, HoodieTableMetaClient metaClient, Schema schema, BloomFilter filter) {
super(basePath, fs, metaClient);
protected HoodieWriteableTestTable(String basePath, FileSystem fs, HoodieTableMetaClient metaClient,
Schema schema, BloomFilter filter) {
this(basePath, fs, metaClient, schema, filter, null);
}
protected HoodieWriteableTestTable(String basePath, FileSystem fs, HoodieTableMetaClient metaClient, Schema schema,
BloomFilter filter, HoodieTableMetadataWriter metadataWriter) {
super(basePath, fs, metaClient, metadataWriter);
this.schema = schema;
this.filter = filter;
this.populateMetaFields = metaClient.getTableConfig().populateMetaFields();
@@ -139,19 +147,18 @@ public class HoodieWriteableTestTable extends HoodieTestTable {
return this;
}
public HoodieWriteableTestTable withLogAppends(HoodieRecord... records) throws Exception {
return withLogAppends(Arrays.asList(records));
}
public HoodieWriteableTestTable withLogAppends(List<HoodieRecord> records) throws Exception {
for (List<HoodieRecord> groupedRecords: records.stream()
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()) {
appendRecordsToLogFile(groupedRecords);
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))
@@ -170,6 +177,7 @@ public class HoodieWriteableTestTable extends HoodieTestTable {
return null;
}
}).collect(Collectors.toList()), header));
return Pair.of(partitionPath, logWriter.getLogFile());
}
}
}