[HUDI-1418] Set up flink client unit test infra (#2281)
This commit is contained in:
@@ -18,22 +18,24 @@
|
||||
|
||||
package org.apache.hudi.config;
|
||||
|
||||
import org.apache.hadoop.hbase.io.compress.Compression;
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
import org.apache.hudi.client.bootstrap.BootstrapMode;
|
||||
import org.apache.hudi.common.config.DefaultHoodieConfig;
|
||||
import org.apache.hudi.common.fs.ConsistencyGuardConfig;
|
||||
import org.apache.hudi.client.common.EngineType;
|
||||
import org.apache.hudi.common.model.HoodieCleaningPolicy;
|
||||
import org.apache.hudi.common.model.OverwriteWithLatestAvroPayload;
|
||||
import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion;
|
||||
import org.apache.hudi.common.table.view.FileSystemViewStorageConfig;
|
||||
import org.apache.hudi.common.util.ReflectionUtils;
|
||||
import org.apache.hudi.execution.bulkinsert.BulkInsertSortMode;
|
||||
import org.apache.hudi.index.HoodieIndex;
|
||||
import org.apache.hudi.keygen.SimpleAvroKeyGenerator;
|
||||
import org.apache.hudi.metrics.MetricsReporterType;
|
||||
import org.apache.hudi.metrics.datadog.DatadogHttpClient.ApiSite;
|
||||
import org.apache.hudi.table.action.compact.strategy.CompactionStrategy;
|
||||
|
||||
import org.apache.hadoop.hbase.io.compress.Compression;
|
||||
import org.apache.parquet.hadoop.metadata.CompressionCodecName;
|
||||
|
||||
import javax.annotation.concurrent.Immutable;
|
||||
@@ -60,6 +62,11 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
|
||||
private static final long serialVersionUID = 0L;
|
||||
|
||||
public static final String TABLE_NAME = "hoodie.table.name";
|
||||
public static final String PRECOMBINE_FIELD_PROP = "hoodie.datasource.write.precombine.field";
|
||||
public static final String WRITE_PAYLOAD_CLASS = "hoodie.datasource.write.payload.class";
|
||||
public static final String DEFAULT_WRITE_PAYLOAD_CLASS = OverwriteWithLatestAvroPayload.class.getName();
|
||||
public static final String KEYGENERATOR_CLASS_PROP = "hoodie.datasource.write.keygenerator.class";
|
||||
public static final String DEFAULT_KEYGENERATOR_CLASS = SimpleAvroKeyGenerator.class.getName();
|
||||
public static final String DEFAULT_ROLLBACK_USING_MARKERS = "false";
|
||||
public static final String ROLLBACK_USING_MARKERS = "hoodie.rollback.using.markers";
|
||||
public static final String TIMELINE_LAYOUT_VERSION = "hoodie.timeline.layout.version";
|
||||
@@ -196,6 +203,18 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
|
||||
return props.getProperty(TABLE_NAME);
|
||||
}
|
||||
|
||||
public String getPreCombineField() {
|
||||
return props.getProperty(PRECOMBINE_FIELD_PROP);
|
||||
}
|
||||
|
||||
public String getWritePayloadClass() {
|
||||
return props.getProperty(WRITE_PAYLOAD_CLASS);
|
||||
}
|
||||
|
||||
public String getKeyGeneratorClass() {
|
||||
return props.getProperty(KEYGENERATOR_CLASS_PROP);
|
||||
}
|
||||
|
||||
public Boolean shouldAutoCommit() {
|
||||
return Boolean.parseBoolean(props.getProperty(HOODIE_AUTO_COMMIT_PROP));
|
||||
}
|
||||
@@ -917,6 +936,21 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withPreCombineField(String preCombineField) {
|
||||
props.setProperty(PRECOMBINE_FIELD_PROP, preCombineField);
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withWritePayLoad(String payload) {
|
||||
props.setProperty(WRITE_PAYLOAD_CLASS, payload);
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withKeyGenerator(String keyGeneratorClass) {
|
||||
props.setProperty(KEYGENERATOR_CLASS_PROP, keyGeneratorClass);
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withTimelineLayoutVersion(int version) {
|
||||
props.setProperty(TIMELINE_LAYOUT_VERSION, String.valueOf(version));
|
||||
return this;
|
||||
@@ -1109,6 +1143,10 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
|
||||
setDefaultOnCondition(props, !props.containsKey(DELETE_PARALLELISM), DELETE_PARALLELISM, DEFAULT_PARALLELISM);
|
||||
setDefaultOnCondition(props, !props.containsKey(ROLLBACK_PARALLELISM), ROLLBACK_PARALLELISM,
|
||||
DEFAULT_ROLLBACK_PARALLELISM);
|
||||
setDefaultOnCondition(props, !props.containsKey(KEYGENERATOR_CLASS_PROP),
|
||||
KEYGENERATOR_CLASS_PROP, DEFAULT_KEYGENERATOR_CLASS);
|
||||
setDefaultOnCondition(props, !props.containsKey(WRITE_PAYLOAD_CLASS),
|
||||
WRITE_PAYLOAD_CLASS, DEFAULT_WRITE_PAYLOAD_CLASS);
|
||||
setDefaultOnCondition(props, !props.containsKey(ROLLBACK_USING_MARKERS), ROLLBACK_USING_MARKERS,
|
||||
DEFAULT_ROLLBACK_USING_MARKERS);
|
||||
setDefaultOnCondition(props, !props.containsKey(COMBINE_BEFORE_INSERT_PROP), COMBINE_BEFORE_INSERT_PROP,
|
||||
|
||||
@@ -0,0 +1,142 @@
|
||||
/*
|
||||
* 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.common.TaskContextSupplier;
|
||||
import org.apache.hudi.common.bloom.BloomFilter;
|
||||
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.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.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);
|
||||
|
||||
protected final Schema schema;
|
||||
protected final BloomFilter filter;
|
||||
|
||||
protected HoodieWriteableTestTable(String basePath, FileSystem fs, HoodieTableMetaClient metaClient, Schema schema, BloomFilter filter) {
|
||||
super(basePath, fs, metaClient);
|
||||
this.schema = schema;
|
||||
this.filter = 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 HoodieWriteableTestTable withInserts(String partition, String fileId, List<HoodieRecord> records, TaskContextSupplier contextSupplier) 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, contextSupplier)) {
|
||||
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));
|
||||
}
|
||||
}
|
||||
}
|
||||
Reference in New Issue
Block a user