1
0

[HUDI-1418] Set up flink client unit test infra (#2281)

This commit is contained in:
Gary Li
2020-12-31 08:57:22 +08:00
committed by GitHub
parent 605b617cfa
commit c5e8a024f6
23 changed files with 642 additions and 125 deletions

View File

@@ -70,7 +70,7 @@ import org.apache.hudi.table.action.HoodieWriteMetadata;
import org.apache.hudi.table.action.commit.SparkWriteHelper;
import org.apache.hudi.testutils.HoodieClientTestBase;
import org.apache.hudi.testutils.HoodieClientTestUtils;
import org.apache.hudi.testutils.HoodieWriteableTestTable;
import org.apache.hudi.testutils.HoodieSparkWriteableTestTable;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import org.apache.spark.api.java.JavaRDD;
@@ -132,7 +132,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase {
@BeforeEach
public void setUpTestTable() {
testTable = HoodieWriteableTestTable.of(metaClient);
testTable = HoodieSparkWriteableTestTable.of(metaClient);
}
/**

View File

@@ -39,7 +39,7 @@ import org.apache.hudi.table.HoodieSparkTable;
import org.apache.hudi.table.HoodieTable;
import org.apache.hudi.testutils.Assertions;
import org.apache.hudi.testutils.HoodieClientTestHarness;
import org.apache.hudi.testutils.HoodieWriteableTestTable;
import org.apache.hudi.testutils.HoodieSparkWriteableTestTable;
import org.apache.hudi.testutils.MetadataMergeWriteStatus;
import org.apache.avro.Schema;
@@ -280,7 +280,7 @@ public class TestHoodieIndex extends HoodieClientTestHarness {
}
// We create three parquet file, each having one record. (two different partitions)
HoodieWriteableTestTable testTable = HoodieWriteableTestTable.of(hoodieTable, SCHEMA);
HoodieSparkWriteableTestTable testTable = HoodieSparkWriteableTestTable.of(hoodieTable, SCHEMA);
String fileId1 = testTable.addCommit("001").getFileIdWithInserts(p1, record1);
String fileId2 = testTable.addCommit("002").getFileIdWithInserts(p1, record2);
String fileId3 = testTable.addCommit("003").getFileIdWithInserts(p2, record4);
@@ -337,7 +337,7 @@ public class TestHoodieIndex extends HoodieClientTestHarness {
writeClient = getHoodieWriteClient(config);
index = writeClient.getIndex();
HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient);
HoodieWriteableTestTable testTable = HoodieWriteableTestTable.of(hoodieTable, SCHEMA);
HoodieSparkWriteableTestTable testTable = HoodieSparkWriteableTestTable.of(hoodieTable, SCHEMA);
final String p1 = "2016/01/31";
final String p2 = "2016/02/28";

View File

@@ -33,7 +33,7 @@ import org.apache.hudi.io.HoodieKeyLookupHandle;
import org.apache.hudi.table.HoodieSparkTable;
import org.apache.hudi.table.HoodieTable;
import org.apache.hudi.testutils.HoodieClientTestHarness;
import org.apache.hudi.testutils.HoodieWriteableTestTable;
import org.apache.hudi.testutils.HoodieSparkWriteableTestTable;
import org.apache.avro.Schema;
import org.apache.hadoop.fs.Path;
@@ -105,7 +105,7 @@ public class TestHoodieBloomIndex extends HoodieClientTestHarness {
HoodieWriteConfig config = makeConfig(rangePruning, treeFiltering, bucketizedChecking);
SparkHoodieBloomIndex index = new SparkHoodieBloomIndex(config);
HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient);
HoodieWriteableTestTable testTable = HoodieWriteableTestTable.of(hoodieTable, SCHEMA);
HoodieSparkWriteableTestTable testTable = HoodieSparkWriteableTestTable.of(hoodieTable, SCHEMA);
// Create some partitions, and put some files
// "2016/01/21": 0 file
@@ -222,7 +222,7 @@ public class TestHoodieBloomIndex extends HoodieClientTestHarness {
// record2, record3).
BloomFilter filter = BloomFilterFactory.createBloomFilter(10000, 0.0000001, -1, BloomFilterTypeCode.SIMPLE.name());
filter.add(record3.getRecordKey());
HoodieWriteableTestTable testTable = HoodieWriteableTestTable.of(metaClient, SCHEMA, filter);
HoodieSparkWriteableTestTable testTable = HoodieSparkWriteableTestTable.of(metaClient, SCHEMA, filter);
String fileId = testTable.addCommit("000").getFileIdWithInserts(partition, record1, record2);
String filename = testTable.getBaseFileNameById(fileId);
@@ -298,7 +298,7 @@ public class TestHoodieBloomIndex extends HoodieClientTestHarness {
// Also create the metadata and config
HoodieWriteConfig config = makeConfig(rangePruning, treeFiltering, bucketizedChecking);
HoodieSparkTable hoodieTable = HoodieSparkTable.create(config, context, metaClient);
HoodieWriteableTestTable testTable = HoodieWriteableTestTable.of(hoodieTable, SCHEMA);
HoodieSparkWriteableTestTable testTable = HoodieSparkWriteableTestTable.of(hoodieTable, SCHEMA);
// Let's tag
SparkHoodieBloomIndex bloomIndex = new SparkHoodieBloomIndex(config);
@@ -363,7 +363,7 @@ public class TestHoodieBloomIndex extends HoodieClientTestHarness {
// Also create the metadata and config
HoodieWriteConfig config = makeConfig(rangePruning, treeFiltering, bucketizedChecking);
HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient);
HoodieWriteableTestTable testTable = HoodieWriteableTestTable.of(hoodieTable, SCHEMA);
HoodieSparkWriteableTestTable testTable = HoodieSparkWriteableTestTable.of(hoodieTable, SCHEMA);
// Let's tag
SparkHoodieBloomIndex bloomIndex = new SparkHoodieBloomIndex(config);
@@ -432,7 +432,7 @@ public class TestHoodieBloomIndex extends HoodieClientTestHarness {
BloomFilter filter = BloomFilterFactory.createBloomFilter(10000, 0.0000001, -1,
BloomFilterTypeCode.SIMPLE.name());
filter.add(record2.getRecordKey());
HoodieWriteableTestTable testTable = HoodieWriteableTestTable.of(metaClient, SCHEMA, filter);
HoodieSparkWriteableTestTable testTable = HoodieSparkWriteableTestTable.of(metaClient, SCHEMA, filter);
String fileId = testTable.addCommit("000").getFileIdWithInserts("2016/01/31", record1);
assertTrue(filter.mightContain(record1.getRecordKey()));
assertTrue(filter.mightContain(record2.getRecordKey()));

View File

@@ -27,7 +27,7 @@ import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.table.HoodieSparkTable;
import org.apache.hudi.table.HoodieTable;
import org.apache.hudi.testutils.HoodieClientTestHarness;
import org.apache.hudi.testutils.HoodieWriteableTestTable;
import org.apache.hudi.testutils.HoodieSparkWriteableTestTable;
import org.apache.avro.Schema;
import org.apache.spark.api.java.JavaPairRDD;
@@ -76,7 +76,7 @@ public class TestHoodieGlobalBloomIndex extends HoodieClientTestHarness {
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build();
SparkHoodieGlobalBloomIndex index = new SparkHoodieGlobalBloomIndex(config);
HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient);
HoodieWriteableTestTable testTable = HoodieWriteableTestTable.of(hoodieTable, SCHEMA);
HoodieSparkWriteableTestTable testTable = HoodieSparkWriteableTestTable.of(hoodieTable, SCHEMA);
// Create some partitions, and put some files, along with the meta file
// "2016/01/21": 0 file
@@ -180,7 +180,7 @@ public class TestHoodieGlobalBloomIndex extends HoodieClientTestHarness {
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build();
SparkHoodieGlobalBloomIndex index = new SparkHoodieGlobalBloomIndex(config);
HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient);
HoodieWriteableTestTable testTable = HoodieWriteableTestTable.of(hoodieTable, SCHEMA);
HoodieSparkWriteableTestTable testTable = HoodieSparkWriteableTestTable.of(hoodieTable, SCHEMA);
// Create some partitions, and put some files, along with the meta file
// "2016/01/21": 0 file
@@ -261,7 +261,7 @@ public class TestHoodieGlobalBloomIndex extends HoodieClientTestHarness {
.build();
SparkHoodieGlobalBloomIndex index = new SparkHoodieGlobalBloomIndex(config);
HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient);
HoodieWriteableTestTable testTable = HoodieWriteableTestTable.of(hoodieTable, SCHEMA);
HoodieSparkWriteableTestTable testTable = HoodieSparkWriteableTestTable.of(hoodieTable, SCHEMA);
final String p1 = "2016/01/31";
final String p2 = "2016/02/28";

View File

@@ -35,7 +35,7 @@ import org.apache.hudi.index.HoodieIndexUtils;
import org.apache.hudi.table.HoodieSparkTable;
import org.apache.hudi.table.HoodieTable;
import org.apache.hudi.testutils.HoodieClientTestHarness;
import org.apache.hudi.testutils.HoodieWriteableTestTable;
import org.apache.hudi.testutils.HoodieSparkWriteableTestTable;
import org.apache.hudi.testutils.MetadataMergeWriteStatus;
import org.junit.jupiter.api.AfterEach;
@@ -87,7 +87,7 @@ public class TestHoodieKeyLocationFetchHandle extends HoodieClientTestHarness {
List<HoodieRecord> records = dataGen.generateInserts(makeNewCommitTime(), 100);
Map<String, List<HoodieRecord>> partitionRecordsMap = recordsToPartitionRecordsMap(records);
HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient);
HoodieWriteableTestTable testTable = HoodieWriteableTestTable.of(hoodieTable, AVRO_SCHEMA_WITH_METADATA_FIELDS);
HoodieSparkWriteableTestTable testTable = HoodieSparkWriteableTestTable.of(hoodieTable, AVRO_SCHEMA_WITH_METADATA_FIELDS);
Map<Tuple2<String, String>, List<Tuple2<HoodieKey, HoodieRecordLocation>>> expectedList =
writeToParquetAndGetExpectedRecordLocations(partitionRecordsMap, testTable);
@@ -103,7 +103,7 @@ public class TestHoodieKeyLocationFetchHandle extends HoodieClientTestHarness {
}
private Map<Tuple2<String, String>, List<Tuple2<HoodieKey, HoodieRecordLocation>>> writeToParquetAndGetExpectedRecordLocations(
Map<String, List<HoodieRecord>> partitionRecordsMap, HoodieWriteableTestTable testTable) throws Exception {
Map<String, List<HoodieRecord>> partitionRecordsMap, HoodieSparkWriteableTestTable testTable) throws Exception {
Map<Tuple2<String, String>, List<Tuple2<HoodieKey, HoodieRecordLocation>>> expectedList = new HashMap<>();
for (Map.Entry<String, List<HoodieRecord>> entry : partitionRecordsMap.entrySet()) {
int totalRecordsPerPartition = entry.getValue().size();

View File

@@ -68,7 +68,7 @@ import org.apache.hudi.table.action.deltacommit.SparkDeleteDeltaCommitActionExec
import org.apache.hudi.testutils.HoodieClientTestHarness;
import org.apache.hudi.testutils.HoodieClientTestUtils;
import org.apache.hudi.testutils.HoodieMergeOnReadTestUtils;
import org.apache.hudi.testutils.HoodieWriteableTestTable;
import org.apache.hudi.testutils.HoodieSparkWriteableTestTable;
import org.apache.hudi.testutils.MetadataMergeWriteStatus;
import org.apache.avro.generic.GenericRecord;
@@ -1001,7 +1001,7 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness {
// Write them to corresponding avro logfiles
metaClient = HoodieTableMetaClient.reload(metaClient);
HoodieTable table = HoodieSparkTable.create(config, context, metaClient);
HoodieWriteableTestTable.of(table, HoodieTestDataGenerator.AVRO_SCHEMA_WITH_METADATA_FIELDS)
HoodieSparkWriteableTestTable.of(table, HoodieTestDataGenerator.AVRO_SCHEMA_WITH_METADATA_FIELDS)
.withLogAppends(updatedRecords);
// In writeRecordsToLogFiles, no commit files are getting added, so resetting file-system view state
((SyncableFileSystemView) (table.getSliceView())).reset();

View File

@@ -44,7 +44,7 @@ import org.apache.hudi.index.bloom.SparkHoodieBloomIndex;
import org.apache.hudi.table.HoodieSparkTable;
import org.apache.hudi.table.HoodieTable;
import org.apache.hudi.testutils.HoodieClientTestHarness;
import org.apache.hudi.testutils.HoodieWriteableTestTable;
import org.apache.hudi.testutils.HoodieSparkWriteableTestTable;
import org.apache.hadoop.conf.Configuration;
import org.apache.spark.api.java.JavaRDD;
@@ -155,7 +155,7 @@ public class TestHoodieCompactor extends HoodieClientTestHarness {
updatedRecords = ((JavaRDD<HoodieRecord>)index.tagLocation(updatedRecordsRDD, context, table)).collect();
// Write them to corresponding avro logfiles. Also, set the state transition properly.
HoodieWriteableTestTable.of(table, HoodieTestDataGenerator.AVRO_SCHEMA_WITH_METADATA_FIELDS)
HoodieSparkWriteableTestTable.of(table, HoodieTestDataGenerator.AVRO_SCHEMA_WITH_METADATA_FIELDS)
.withLogAppends(updatedRecords);
metaClient.getActiveTimeline().transitionRequestedToInflight(new HoodieInstant(State.REQUESTED,
HoodieTimeline.DELTA_COMMIT_ACTION, newCommitTime), Option.empty());

View File

@@ -28,7 +28,6 @@ import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.timeline.HoodieTimeline;
import org.apache.hudi.common.table.view.HoodieTableFileSystemView;
import org.apache.hudi.common.testutils.HoodieCommonTestHarness;
import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
import org.apache.hudi.common.testutils.HoodieTestUtils;
import org.apache.hudi.common.testutils.minicluster.HdfsTestService;
import org.apache.hudi.common.util.Option;
@@ -72,7 +71,6 @@ public abstract class HoodieClientTestHarness extends HoodieCommonTestHarness im
protected transient Configuration hadoopConf = null;
protected transient SQLContext sqlContext;
protected transient FileSystem fs;
protected transient HoodieTestDataGenerator dataGen = null;
protected transient ExecutorService executorService;
protected transient HoodieTableMetaClient metaClient;
protected transient SparkRDDWriteClient writeClient;
@@ -237,24 +235,6 @@ public abstract class HoodieClientTestHarness extends HoodieCommonTestHarness im
}
}
/**
* Initializes a test data generator which used to generate test datas.
*
*/
protected void initTestDataGenerator() {
dataGen = new HoodieTestDataGenerator();
}
/**
* Cleanups test data generator.
*
*/
protected void cleanupTestDataGenerator() {
if (dataGen != null) {
dataGen = null;
}
}
/**
* Initializes a distributed file system and base directory.
*

View File

@@ -0,0 +1,102 @@
/*
* 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.testutils;
import org.apache.hudi.client.SparkTaskContextSupplier;
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.model.HoodieRecord;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.table.HoodieTable;
import org.apache.avro.Schema;
import org.apache.hadoop.fs.FileSystem;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import java.util.Arrays;
import java.util.List;
import java.util.UUID;
public class HoodieSparkWriteableTestTable extends HoodieWriteableTestTable {
private static final Logger LOG = LogManager.getLogger(HoodieSparkWriteableTestTable.class);
private HoodieSparkWriteableTestTable(String basePath, FileSystem fs, HoodieTableMetaClient metaClient, Schema schema, BloomFilter filter) {
super(basePath, fs, metaClient, schema, filter);
}
public static HoodieSparkWriteableTestTable of(HoodieTableMetaClient metaClient, Schema schema, BloomFilter filter) {
return new HoodieSparkWriteableTestTable(metaClient.getBasePath(), metaClient.getRawFs(), metaClient, schema, filter);
}
public static HoodieSparkWriteableTestTable of(HoodieTableMetaClient metaClient, Schema schema) {
BloomFilter filter = BloomFilterFactory
.createBloomFilter(10000, 0.0000001, -1, BloomFilterTypeCode.SIMPLE.name());
return of(metaClient, schema, filter);
}
public static HoodieSparkWriteableTestTable of(HoodieTable hoodieTable, Schema schema) {
HoodieTableMetaClient metaClient = hoodieTable.getMetaClient();
return of(metaClient, schema);
}
public static HoodieSparkWriteableTestTable of(HoodieTable hoodieTable, Schema schema, BloomFilter filter) {
HoodieTableMetaClient metaClient = hoodieTable.getMetaClient();
return of(metaClient, schema, filter);
}
@Override
public HoodieSparkWriteableTestTable addCommit(String instantTime) throws Exception {
return (HoodieSparkWriteableTestTable) super.addCommit(instantTime);
}
@Override
public HoodieSparkWriteableTestTable forCommit(String instantTime) {
return (HoodieSparkWriteableTestTable) super.forCommit(instantTime);
}
public String getFileIdWithInserts(String partition) throws Exception {
return getFileIdWithInserts(partition, new HoodieRecord[0]);
}
public String getFileIdWithInserts(String partition, HoodieRecord... records) throws Exception {
return getFileIdWithInserts(partition, Arrays.asList(records));
}
public String getFileIdWithInserts(String partition, List<HoodieRecord> records) throws Exception {
String fileId = UUID.randomUUID().toString();
withInserts(partition, fileId, records);
return fileId;
}
public HoodieSparkWriteableTestTable withInserts(String partition, String fileId) throws Exception {
return withInserts(partition, fileId, new HoodieRecord[0]);
}
public HoodieSparkWriteableTestTable withInserts(String partition, String fileId, HoodieRecord... records) throws Exception {
return withInserts(partition, fileId, Arrays.asList(records));
}
public HoodieSparkWriteableTestTable withInserts(String partition, String fileId, List<HoodieRecord> records) throws Exception {
super.withInserts(partition, fileId, records, new SparkTaskContextSupplier());
return this;
}
}

View File

@@ -1,188 +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.testutils;
import org.apache.hudi.avro.HoodieAvroUtils;
import org.apache.hudi.avro.HoodieAvroWriteSupport;
import org.apache.hudi.client.SparkTaskContextSupplier;
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.model.HoodieLogFile;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieRecordLocation;
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;
import org.apache.hudi.common.testutils.FileCreateUtils;
import org.apache.hudi.common.testutils.HoodieTestTable;
import org.apache.hudi.config.HoodieStorageConfig;
import org.apache.hudi.io.storage.HoodieAvroParquetConfig;
import org.apache.hudi.io.storage.HoodieParquetWriter;
import org.apache.hudi.table.HoodieTable;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericRecord;
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.log4j.LogManager;
import org.apache.log4j.Logger;
import org.apache.parquet.avro.AvroSchemaConverter;
import org.apache.parquet.hadoop.ParquetWriter;
import org.apache.parquet.hadoop.metadata.CompressionCodecName;
import java.io.IOException;
import java.nio.file.Paths;
import java.util.Arrays;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.UUID;
import java.util.stream.Collectors;
import static org.apache.hudi.common.testutils.FileCreateUtils.baseFileName;
public class HoodieWriteableTestTable extends HoodieTestTable {
private static final Logger LOG = LogManager.getLogger(HoodieWriteableTestTable.class);
private final Schema schema;
private final BloomFilter filter;
private HoodieWriteableTestTable(String basePath, FileSystem fs, HoodieTableMetaClient metaClient, Schema schema, BloomFilter filter) {
super(basePath, fs, metaClient);
this.schema = schema;
this.filter = filter;
}
public static HoodieWriteableTestTable of(HoodieTableMetaClient metaClient, Schema schema, BloomFilter filter) {
return new HoodieWriteableTestTable(metaClient.getBasePath(), metaClient.getRawFs(), metaClient, schema, filter);
}
public static HoodieWriteableTestTable of(HoodieTableMetaClient metaClient, Schema schema) {
BloomFilter filter = BloomFilterFactory
.createBloomFilter(10000, 0.0000001, -1, BloomFilterTypeCode.SIMPLE.name());
return of(metaClient, schema, filter);
}
public static HoodieWriteableTestTable of(HoodieTable hoodieTable, Schema schema) {
HoodieTableMetaClient metaClient = hoodieTable.getMetaClient();
return of(metaClient, schema);
}
public static HoodieWriteableTestTable of(HoodieTable hoodieTable, Schema schema, BloomFilter filter) {
HoodieTableMetaClient metaClient = hoodieTable.getMetaClient();
return of(metaClient, schema, filter);
}
@Override
public HoodieWriteableTestTable addCommit(String instantTime) throws Exception {
return (HoodieWriteableTestTable) super.addCommit(instantTime);
}
@Override
public HoodieWriteableTestTable forCommit(String instantTime) {
return (HoodieWriteableTestTable) super.forCommit(instantTime);
}
public String getFileIdWithInserts(String partition) throws Exception {
return getFileIdWithInserts(partition, new HoodieRecord[0]);
}
public String getFileIdWithInserts(String partition, HoodieRecord... records) throws Exception {
return getFileIdWithInserts(partition, Arrays.asList(records));
}
public String getFileIdWithInserts(String partition, List<HoodieRecord> records) throws Exception {
String fileId = UUID.randomUUID().toString();
withInserts(partition, fileId, records);
return fileId;
}
public HoodieWriteableTestTable withInserts(String partition, String fileId) throws Exception {
return withInserts(partition, fileId, new HoodieRecord[0]);
}
public HoodieWriteableTestTable withInserts(String partition, String fileId, HoodieRecord... records) throws Exception {
return withInserts(partition, fileId, Arrays.asList(records));
}
public HoodieWriteableTestTable withInserts(String partition, String fileId, List<HoodieRecord> records) throws Exception {
FileCreateUtils.createPartitionMetaFile(basePath, partition);
String fileName = baseFileName(currentInstantTime, fileId);
HoodieAvroWriteSupport writeSupport = new HoodieAvroWriteSupport(
new AvroSchemaConverter().convert(schema), schema, filter);
HoodieAvroParquetConfig config = new HoodieAvroParquetConfig(writeSupport, CompressionCodecName.GZIP,
ParquetWriter.DEFAULT_BLOCK_SIZE, ParquetWriter.DEFAULT_PAGE_SIZE, 120 * 1024 * 1024,
new Configuration(), Double.parseDouble(HoodieStorageConfig.DEFAULT_STREAM_COMPRESSION_RATIO));
try (HoodieParquetWriter writer = new HoodieParquetWriter(
currentInstantTime,
new Path(Paths.get(basePath, partition, fileName).toString()),
config, schema, new SparkTaskContextSupplier())) {
int seqId = 1;
for (HoodieRecord record : records) {
GenericRecord avroRecord = (GenericRecord) record.getData().getInsertValue(schema).get();
HoodieAvroUtils.addCommitMetadataToRecord(avroRecord, currentInstantTime, String.valueOf(seqId++));
HoodieAvroUtils.addHoodieKeyToRecord(avroRecord, record.getRecordKey(), record.getPartitionPath(), fileName);
writer.writeAvro(record.getRecordKey(), avroRecord);
filter.add(record.getRecordKey());
}
}
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()
.collect(Collectors.groupingBy(HoodieRecord::getCurrentLocation)).values()) {
appendRecordsToLogFile(groupedRecords);
}
return this;
}
private void 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))
.withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId(location.getFileId())
.overBaseCommit(location.getInstantTime()).withFs(fs).build()) {
Map<HoodieLogBlock.HeaderMetadataType, String> header = new HashMap<>();
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, location.getInstantTime());
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString());
logWriter.appendBlock(new HoodieAvroDataBlock(groupedRecords.stream().map(r -> {
try {
GenericRecord val = (GenericRecord) r.getData().getInsertValue(schema).get();
HoodieAvroUtils.addHoodieKeyToRecord(val, r.getRecordKey(), r.getPartitionPath(), "");
return (IndexedRecord) val;
} catch (IOException e) {
LOG.warn("Failed to convert record " + r.toString(), e);
return null;
}
}).collect(Collectors.toList()), header));
}
}
}