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

@@ -31,7 +31,7 @@ import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion;
import org.apache.hudi.common.table.view.HoodieTableFileSystemView; import org.apache.hudi.common.table.view.HoodieTableFileSystemView;
import org.apache.hudi.common.testutils.HoodieTestDataGenerator; import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
import org.apache.hudi.common.testutils.SchemaTestUtil; import org.apache.hudi.common.testutils.SchemaTestUtil;
import org.apache.hudi.testutils.HoodieWriteableTestTable; import org.apache.hudi.testutils.HoodieSparkWriteableTestTable;
import org.apache.avro.Schema; import org.apache.avro.Schema;
import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileStatus;
@@ -80,7 +80,7 @@ public class ITTestRepairsCommand extends AbstractShellIntegrationTest {
// generate 200 records // generate 200 records
Schema schema = HoodieAvroUtils.addMetadataFields(SchemaTestUtil.getSimpleSchema()); Schema schema = HoodieAvroUtils.addMetadataFields(SchemaTestUtil.getSimpleSchema());
HoodieWriteableTestTable testTable = HoodieWriteableTestTable.of(HoodieCLI.getTableMetaClient(), schema); HoodieSparkWriteableTestTable testTable = HoodieSparkWriteableTestTable.of(HoodieCLI.getTableMetaClient(), schema);
HoodieRecord[] hoodieRecords1 = SchemaTestUtil.generateHoodieTestRecords(0, 100, schema).toArray(new HoodieRecord[100]); HoodieRecord[] hoodieRecords1 = SchemaTestUtil.generateHoodieTestRecords(0, 100, schema).toArray(new HoodieRecord[100]);
HoodieRecord[] hoodieRecords2 = SchemaTestUtil.generateHoodieTestRecords(100, 100, schema).toArray(new HoodieRecord[100]); HoodieRecord[] hoodieRecords2 = SchemaTestUtil.generateHoodieTestRecords(100, 100, schema).toArray(new HoodieRecord[100]);

View File

@@ -18,22 +18,24 @@
package org.apache.hudi.config; package org.apache.hudi.config;
import org.apache.hadoop.hbase.io.compress.Compression;
import org.apache.hudi.client.WriteStatus; import org.apache.hudi.client.WriteStatus;
import org.apache.hudi.client.bootstrap.BootstrapMode; import org.apache.hudi.client.bootstrap.BootstrapMode;
import org.apache.hudi.common.config.DefaultHoodieConfig; import org.apache.hudi.common.config.DefaultHoodieConfig;
import org.apache.hudi.common.fs.ConsistencyGuardConfig; import org.apache.hudi.common.fs.ConsistencyGuardConfig;
import org.apache.hudi.client.common.EngineType; import org.apache.hudi.client.common.EngineType;
import org.apache.hudi.common.model.HoodieCleaningPolicy; 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.timeline.versioning.TimelineLayoutVersion;
import org.apache.hudi.common.table.view.FileSystemViewStorageConfig; import org.apache.hudi.common.table.view.FileSystemViewStorageConfig;
import org.apache.hudi.common.util.ReflectionUtils; import org.apache.hudi.common.util.ReflectionUtils;
import org.apache.hudi.execution.bulkinsert.BulkInsertSortMode; import org.apache.hudi.execution.bulkinsert.BulkInsertSortMode;
import org.apache.hudi.index.HoodieIndex; import org.apache.hudi.index.HoodieIndex;
import org.apache.hudi.keygen.SimpleAvroKeyGenerator;
import org.apache.hudi.metrics.MetricsReporterType; import org.apache.hudi.metrics.MetricsReporterType;
import org.apache.hudi.metrics.datadog.DatadogHttpClient.ApiSite; import org.apache.hudi.metrics.datadog.DatadogHttpClient.ApiSite;
import org.apache.hudi.table.action.compact.strategy.CompactionStrategy; 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 org.apache.parquet.hadoop.metadata.CompressionCodecName;
import javax.annotation.concurrent.Immutable; import javax.annotation.concurrent.Immutable;
@@ -60,6 +62,11 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
private static final long serialVersionUID = 0L; private static final long serialVersionUID = 0L;
public static final String TABLE_NAME = "hoodie.table.name"; 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 DEFAULT_ROLLBACK_USING_MARKERS = "false";
public static final String ROLLBACK_USING_MARKERS = "hoodie.rollback.using.markers"; public static final String ROLLBACK_USING_MARKERS = "hoodie.rollback.using.markers";
public static final String TIMELINE_LAYOUT_VERSION = "hoodie.timeline.layout.version"; 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); 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() { public Boolean shouldAutoCommit() {
return Boolean.parseBoolean(props.getProperty(HOODIE_AUTO_COMMIT_PROP)); return Boolean.parseBoolean(props.getProperty(HOODIE_AUTO_COMMIT_PROP));
} }
@@ -917,6 +936,21 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
return this; 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) { public Builder withTimelineLayoutVersion(int version) {
props.setProperty(TIMELINE_LAYOUT_VERSION, String.valueOf(version)); props.setProperty(TIMELINE_LAYOUT_VERSION, String.valueOf(version));
return this; 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(DELETE_PARALLELISM), DELETE_PARALLELISM, DEFAULT_PARALLELISM);
setDefaultOnCondition(props, !props.containsKey(ROLLBACK_PARALLELISM), ROLLBACK_PARALLELISM, setDefaultOnCondition(props, !props.containsKey(ROLLBACK_PARALLELISM), ROLLBACK_PARALLELISM,
DEFAULT_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, setDefaultOnCondition(props, !props.containsKey(ROLLBACK_USING_MARKERS), ROLLBACK_USING_MARKERS,
DEFAULT_ROLLBACK_USING_MARKERS); DEFAULT_ROLLBACK_USING_MARKERS);
setDefaultOnCondition(props, !props.containsKey(COMBINE_BEFORE_INSERT_PROP), COMBINE_BEFORE_INSERT_PROP, setDefaultOnCondition(props, !props.containsKey(COMBINE_BEFORE_INSERT_PROP), COMBINE_BEFORE_INSERT_PROP,

View File

@@ -21,10 +21,8 @@ package org.apache.hudi.testutils;
import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.avro.HoodieAvroUtils;
import org.apache.hudi.avro.HoodieAvroWriteSupport; import org.apache.hudi.avro.HoodieAvroWriteSupport;
import org.apache.hudi.client.SparkTaskContextSupplier; import org.apache.hudi.client.common.TaskContextSupplier;
import org.apache.hudi.common.bloom.BloomFilter; 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.HoodieLogFile;
import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieRecordLocation; import org.apache.hudi.common.model.HoodieRecordLocation;
@@ -37,7 +35,6 @@ import org.apache.hudi.common.testutils.HoodieTestTable;
import org.apache.hudi.config.HoodieStorageConfig; import org.apache.hudi.config.HoodieStorageConfig;
import org.apache.hudi.io.storage.HoodieAvroParquetConfig; import org.apache.hudi.io.storage.HoodieAvroParquetConfig;
import org.apache.hudi.io.storage.HoodieParquetWriter; import org.apache.hudi.io.storage.HoodieParquetWriter;
import org.apache.hudi.table.HoodieTable;
import org.apache.avro.Schema; import org.apache.avro.Schema;
import org.apache.avro.generic.GenericRecord; import org.apache.avro.generic.GenericRecord;
@@ -57,7 +54,6 @@ import java.util.Arrays;
import java.util.HashMap; import java.util.HashMap;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.UUID;
import java.util.stream.Collectors; import java.util.stream.Collectors;
import static org.apache.hudi.common.testutils.FileCreateUtils.baseFileName; import static org.apache.hudi.common.testutils.FileCreateUtils.baseFileName;
@@ -65,35 +61,15 @@ import static org.apache.hudi.common.testutils.FileCreateUtils.baseFileName;
public class HoodieWriteableTestTable extends HoodieTestTable { public class HoodieWriteableTestTable extends HoodieTestTable {
private static final Logger LOG = LogManager.getLogger(HoodieWriteableTestTable.class); private static final Logger LOG = LogManager.getLogger(HoodieWriteableTestTable.class);
private final Schema schema; protected final Schema schema;
private final BloomFilter filter; protected final BloomFilter filter;
private HoodieWriteableTestTable(String basePath, FileSystem fs, HoodieTableMetaClient metaClient, Schema schema, BloomFilter filter) { protected HoodieWriteableTestTable(String basePath, FileSystem fs, HoodieTableMetaClient metaClient, Schema schema, BloomFilter filter) {
super(basePath, fs, metaClient); super(basePath, fs, metaClient);
this.schema = schema; this.schema = schema;
this.filter = filter; 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 @Override
public HoodieWriteableTestTable addCommit(String instantTime) throws Exception { public HoodieWriteableTestTable addCommit(String instantTime) throws Exception {
return (HoodieWriteableTestTable) super.addCommit(instantTime); return (HoodieWriteableTestTable) super.addCommit(instantTime);
@@ -104,29 +80,7 @@ public class HoodieWriteableTestTable extends HoodieTestTable {
return (HoodieWriteableTestTable) super.forCommit(instantTime); return (HoodieWriteableTestTable) super.forCommit(instantTime);
} }
public String getFileIdWithInserts(String partition) throws Exception { public HoodieWriteableTestTable withInserts(String partition, String fileId, List<HoodieRecord> records, TaskContextSupplier contextSupplier) 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); FileCreateUtils.createPartitionMetaFile(basePath, partition);
String fileName = baseFileName(currentInstantTime, fileId); String fileName = baseFileName(currentInstantTime, fileId);
@@ -138,7 +92,7 @@ public class HoodieWriteableTestTable extends HoodieTestTable {
try (HoodieParquetWriter writer = new HoodieParquetWriter( try (HoodieParquetWriter writer = new HoodieParquetWriter(
currentInstantTime, currentInstantTime,
new Path(Paths.get(basePath, partition, fileName).toString()), new Path(Paths.get(basePath, partition, fileName).toString()),
config, schema, new SparkTaskContextSupplier())) { config, schema, contextSupplier)) {
int seqId = 1; int seqId = 1;
for (HoodieRecord record : records) { for (HoodieRecord record : records) {
GenericRecord avroRecord = (GenericRecord) record.getData().getInsertValue(schema).get(); GenericRecord avroRecord = (GenericRecord) record.getData().getInsertValue(schema).get();

View File

@@ -115,6 +115,28 @@
<scope>test</scope> <scope>test</scope>
</dependency> </dependency>
<!-- Flink - Tests -->
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-test-utils_${scala.binary.version}</artifactId>
<version>${flink.version}</version>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-runtime_${scala.binary.version}</artifactId>
<version>${flink.version}</version>
<scope>test</scope>
<classifier>tests</classifier>
</dependency>
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-streaming-java_${scala.binary.version}</artifactId>
<version>${flink.version}</version>
<scope>test</scope>
<classifier>tests</classifier>
</dependency>
<!-- Test --> <!-- Test -->
<dependency> <dependency>
<groupId>org.junit.jupiter</groupId> <groupId>org.junit.jupiter</groupId>

View File

@@ -0,0 +1,136 @@
/*
* 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.common.fs.FSUtils;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieTableType;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.testutils.HoodieCommonTestHarness;
import org.apache.hudi.common.testutils.HoodieTestUtils;
import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration;
import org.apache.flink.streaming.api.functions.sink.SinkFunction;
import org.apache.flink.test.util.MiniClusterWithClientResource;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.LocalFileSystem;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.TestInfo;
import java.io.IOException;
import java.io.Serializable;
import java.util.ArrayList;
import java.util.List;
public class HoodieFlinkClientTestHarness extends HoodieCommonTestHarness implements Serializable {
protected static final Logger LOG = LogManager.getLogger(HoodieFlinkClientTestHarness.class);
private String testMethodName;
protected transient Configuration hadoopConf = null;
protected transient FileSystem fs;
protected transient MiniClusterWithClientResource flinkCluster = null;
@BeforeEach
public void setTestMethodName(TestInfo testInfo) {
if (testInfo.getTestMethod().isPresent()) {
testMethodName = testInfo.getTestMethod().get().getName();
} else {
testMethodName = "Unknown";
}
}
protected void initFlinkMiniCluster() {
flinkCluster = new MiniClusterWithClientResource(
new MiniClusterResourceConfiguration.Builder()
.setNumberSlotsPerTaskManager(2)
.setNumberTaskManagers(1)
.build());
}
protected void initFileSystem() {
hadoopConf = new Configuration();
initFileSystemWithConfiguration(hadoopConf);
}
private void initFileSystemWithConfiguration(Configuration configuration) {
if (basePath == null) {
throw new IllegalStateException("The base path has not been initialized.");
}
fs = FSUtils.getFs(basePath, configuration);
if (fs instanceof LocalFileSystem) {
LocalFileSystem lfs = (LocalFileSystem) fs;
// With LocalFileSystem, with checksum disabled, fs.open() returns an inputStream which is FSInputStream
// This causes ClassCastExceptions in LogRecordScanner (and potentially other places) calling fs.open
// So, for the tests, we enforce checksum verification to circumvent the problem
lfs.setVerifyChecksum(true);
}
}
/**
* Initializes an instance of {@link HoodieTableMetaClient} with a special table type specified by
* {@code getTableType()}.
*
* @throws IOException
*/
protected void initMetaClient() throws IOException {
initMetaClient(getTableType());
}
protected void initMetaClient(HoodieTableType tableType) throws IOException {
if (basePath == null) {
throw new IllegalStateException("The base path has not been initialized.");
}
metaClient = HoodieTestUtils.init(hadoopConf, basePath, tableType);
}
/**
* Cleanups file system.
*
* @throws IOException
*/
protected void cleanupFileSystem() throws IOException {
if (fs != null) {
LOG.warn("Closing file-system instance used in previous test-run");
fs.close();
fs = null;
}
}
protected void cleanupFlinkMiniCluster() {
if (flinkCluster != null) {
flinkCluster.after();
flinkCluster = null;
}
}
public static class SimpleTestSinkFunction implements SinkFunction<HoodieRecord> {
// must be static
public static List<HoodieRecord> valuesList = new ArrayList<>();
@Override
public synchronized void invoke(HoodieRecord value, Context context) throws Exception {
valuesList.add(value);
}
}
}

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.table.action.commit.SparkWriteHelper;
import org.apache.hudi.testutils.HoodieClientTestBase; import org.apache.hudi.testutils.HoodieClientTestBase;
import org.apache.hudi.testutils.HoodieClientTestUtils; 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.LogManager;
import org.apache.log4j.Logger; import org.apache.log4j.Logger;
import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaRDD;
@@ -132,7 +132,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase {
@BeforeEach @BeforeEach
public void setUpTestTable() { 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.table.HoodieTable;
import org.apache.hudi.testutils.Assertions; import org.apache.hudi.testutils.Assertions;
import org.apache.hudi.testutils.HoodieClientTestHarness; 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.hudi.testutils.MetadataMergeWriteStatus;
import org.apache.avro.Schema; 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) // 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 fileId1 = testTable.addCommit("001").getFileIdWithInserts(p1, record1);
String fileId2 = testTable.addCommit("002").getFileIdWithInserts(p1, record2); String fileId2 = testTable.addCommit("002").getFileIdWithInserts(p1, record2);
String fileId3 = testTable.addCommit("003").getFileIdWithInserts(p2, record4); String fileId3 = testTable.addCommit("003").getFileIdWithInserts(p2, record4);
@@ -337,7 +337,7 @@ public class TestHoodieIndex extends HoodieClientTestHarness {
writeClient = getHoodieWriteClient(config); writeClient = getHoodieWriteClient(config);
index = writeClient.getIndex(); index = writeClient.getIndex();
HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient); 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 p1 = "2016/01/31";
final String p2 = "2016/02/28"; 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.HoodieSparkTable;
import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.HoodieTable;
import org.apache.hudi.testutils.HoodieClientTestHarness; 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.avro.Schema;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
@@ -105,7 +105,7 @@ public class TestHoodieBloomIndex extends HoodieClientTestHarness {
HoodieWriteConfig config = makeConfig(rangePruning, treeFiltering, bucketizedChecking); HoodieWriteConfig config = makeConfig(rangePruning, treeFiltering, bucketizedChecking);
SparkHoodieBloomIndex index = new SparkHoodieBloomIndex(config); SparkHoodieBloomIndex index = new SparkHoodieBloomIndex(config);
HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient); 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 // Create some partitions, and put some files
// "2016/01/21": 0 file // "2016/01/21": 0 file
@@ -222,7 +222,7 @@ public class TestHoodieBloomIndex extends HoodieClientTestHarness {
// record2, record3). // record2, record3).
BloomFilter filter = BloomFilterFactory.createBloomFilter(10000, 0.0000001, -1, BloomFilterTypeCode.SIMPLE.name()); BloomFilter filter = BloomFilterFactory.createBloomFilter(10000, 0.0000001, -1, BloomFilterTypeCode.SIMPLE.name());
filter.add(record3.getRecordKey()); 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 fileId = testTable.addCommit("000").getFileIdWithInserts(partition, record1, record2);
String filename = testTable.getBaseFileNameById(fileId); String filename = testTable.getBaseFileNameById(fileId);
@@ -298,7 +298,7 @@ public class TestHoodieBloomIndex extends HoodieClientTestHarness {
// Also create the metadata and config // Also create the metadata and config
HoodieWriteConfig config = makeConfig(rangePruning, treeFiltering, bucketizedChecking); HoodieWriteConfig config = makeConfig(rangePruning, treeFiltering, bucketizedChecking);
HoodieSparkTable hoodieTable = HoodieSparkTable.create(config, context, metaClient); HoodieSparkTable hoodieTable = HoodieSparkTable.create(config, context, metaClient);
HoodieWriteableTestTable testTable = HoodieWriteableTestTable.of(hoodieTable, SCHEMA); HoodieSparkWriteableTestTable testTable = HoodieSparkWriteableTestTable.of(hoodieTable, SCHEMA);
// Let's tag // Let's tag
SparkHoodieBloomIndex bloomIndex = new SparkHoodieBloomIndex(config); SparkHoodieBloomIndex bloomIndex = new SparkHoodieBloomIndex(config);
@@ -363,7 +363,7 @@ public class TestHoodieBloomIndex extends HoodieClientTestHarness {
// Also create the metadata and config // Also create the metadata and config
HoodieWriteConfig config = makeConfig(rangePruning, treeFiltering, bucketizedChecking); HoodieWriteConfig config = makeConfig(rangePruning, treeFiltering, bucketizedChecking);
HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient); HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient);
HoodieWriteableTestTable testTable = HoodieWriteableTestTable.of(hoodieTable, SCHEMA); HoodieSparkWriteableTestTable testTable = HoodieSparkWriteableTestTable.of(hoodieTable, SCHEMA);
// Let's tag // Let's tag
SparkHoodieBloomIndex bloomIndex = new SparkHoodieBloomIndex(config); SparkHoodieBloomIndex bloomIndex = new SparkHoodieBloomIndex(config);
@@ -432,7 +432,7 @@ public class TestHoodieBloomIndex extends HoodieClientTestHarness {
BloomFilter filter = BloomFilterFactory.createBloomFilter(10000, 0.0000001, -1, BloomFilter filter = BloomFilterFactory.createBloomFilter(10000, 0.0000001, -1,
BloomFilterTypeCode.SIMPLE.name()); BloomFilterTypeCode.SIMPLE.name());
filter.add(record2.getRecordKey()); 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); String fileId = testTable.addCommit("000").getFileIdWithInserts("2016/01/31", record1);
assertTrue(filter.mightContain(record1.getRecordKey())); assertTrue(filter.mightContain(record1.getRecordKey()));
assertTrue(filter.mightContain(record2.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.HoodieSparkTable;
import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.HoodieTable;
import org.apache.hudi.testutils.HoodieClientTestHarness; 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.avro.Schema;
import org.apache.spark.api.java.JavaPairRDD; import org.apache.spark.api.java.JavaPairRDD;
@@ -76,7 +76,7 @@ public class TestHoodieGlobalBloomIndex extends HoodieClientTestHarness {
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build(); HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build();
SparkHoodieGlobalBloomIndex index = new SparkHoodieGlobalBloomIndex(config); SparkHoodieGlobalBloomIndex index = new SparkHoodieGlobalBloomIndex(config);
HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient); 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 // Create some partitions, and put some files, along with the meta file
// "2016/01/21": 0 file // "2016/01/21": 0 file
@@ -180,7 +180,7 @@ public class TestHoodieGlobalBloomIndex extends HoodieClientTestHarness {
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build(); HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build();
SparkHoodieGlobalBloomIndex index = new SparkHoodieGlobalBloomIndex(config); SparkHoodieGlobalBloomIndex index = new SparkHoodieGlobalBloomIndex(config);
HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient); 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 // Create some partitions, and put some files, along with the meta file
// "2016/01/21": 0 file // "2016/01/21": 0 file
@@ -261,7 +261,7 @@ public class TestHoodieGlobalBloomIndex extends HoodieClientTestHarness {
.build(); .build();
SparkHoodieGlobalBloomIndex index = new SparkHoodieGlobalBloomIndex(config); SparkHoodieGlobalBloomIndex index = new SparkHoodieGlobalBloomIndex(config);
HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient); 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 p1 = "2016/01/31";
final String p2 = "2016/02/28"; 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.HoodieSparkTable;
import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.HoodieTable;
import org.apache.hudi.testutils.HoodieClientTestHarness; 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.hudi.testutils.MetadataMergeWriteStatus;
import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.AfterEach;
@@ -87,7 +87,7 @@ public class TestHoodieKeyLocationFetchHandle extends HoodieClientTestHarness {
List<HoodieRecord> records = dataGen.generateInserts(makeNewCommitTime(), 100); List<HoodieRecord> records = dataGen.generateInserts(makeNewCommitTime(), 100);
Map<String, List<HoodieRecord>> partitionRecordsMap = recordsToPartitionRecordsMap(records); Map<String, List<HoodieRecord>> partitionRecordsMap = recordsToPartitionRecordsMap(records);
HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient); 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 = Map<Tuple2<String, String>, List<Tuple2<HoodieKey, HoodieRecordLocation>>> expectedList =
writeToParquetAndGetExpectedRecordLocations(partitionRecordsMap, testTable); writeToParquetAndGetExpectedRecordLocations(partitionRecordsMap, testTable);
@@ -103,7 +103,7 @@ public class TestHoodieKeyLocationFetchHandle extends HoodieClientTestHarness {
} }
private Map<Tuple2<String, String>, List<Tuple2<HoodieKey, HoodieRecordLocation>>> writeToParquetAndGetExpectedRecordLocations( 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<>(); Map<Tuple2<String, String>, List<Tuple2<HoodieKey, HoodieRecordLocation>>> expectedList = new HashMap<>();
for (Map.Entry<String, List<HoodieRecord>> entry : partitionRecordsMap.entrySet()) { for (Map.Entry<String, List<HoodieRecord>> entry : partitionRecordsMap.entrySet()) {
int totalRecordsPerPartition = entry.getValue().size(); 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.HoodieClientTestHarness;
import org.apache.hudi.testutils.HoodieClientTestUtils; import org.apache.hudi.testutils.HoodieClientTestUtils;
import org.apache.hudi.testutils.HoodieMergeOnReadTestUtils; 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.hudi.testutils.MetadataMergeWriteStatus;
import org.apache.avro.generic.GenericRecord; import org.apache.avro.generic.GenericRecord;
@@ -1001,7 +1001,7 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness {
// Write them to corresponding avro logfiles // Write them to corresponding avro logfiles
metaClient = HoodieTableMetaClient.reload(metaClient); metaClient = HoodieTableMetaClient.reload(metaClient);
HoodieTable table = HoodieSparkTable.create(config, context, 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); .withLogAppends(updatedRecords);
// In writeRecordsToLogFiles, no commit files are getting added, so resetting file-system view state // In writeRecordsToLogFiles, no commit files are getting added, so resetting file-system view state
((SyncableFileSystemView) (table.getSliceView())).reset(); ((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.HoodieSparkTable;
import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.HoodieTable;
import org.apache.hudi.testutils.HoodieClientTestHarness; 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.hadoop.conf.Configuration;
import org.apache.spark.api.java.JavaRDD; 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(); updatedRecords = ((JavaRDD<HoodieRecord>)index.tagLocation(updatedRecordsRDD, context, table)).collect();
// Write them to corresponding avro logfiles. Also, set the state transition properly. // 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); .withLogAppends(updatedRecords);
metaClient.getActiveTimeline().transitionRequestedToInflight(new HoodieInstant(State.REQUESTED, metaClient.getActiveTimeline().transitionRequestedToInflight(new HoodieInstant(State.REQUESTED,
HoodieTimeline.DELTA_COMMIT_ACTION, newCommitTime), Option.empty()); 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.timeline.HoodieTimeline;
import org.apache.hudi.common.table.view.HoodieTableFileSystemView; import org.apache.hudi.common.table.view.HoodieTableFileSystemView;
import org.apache.hudi.common.testutils.HoodieCommonTestHarness; 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.HoodieTestUtils;
import org.apache.hudi.common.testutils.minicluster.HdfsTestService; import org.apache.hudi.common.testutils.minicluster.HdfsTestService;
import org.apache.hudi.common.util.Option; 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 Configuration hadoopConf = null;
protected transient SQLContext sqlContext; protected transient SQLContext sqlContext;
protected transient FileSystem fs; protected transient FileSystem fs;
protected transient HoodieTestDataGenerator dataGen = null;
protected transient ExecutorService executorService; protected transient ExecutorService executorService;
protected transient HoodieTableMetaClient metaClient; protected transient HoodieTableMetaClient metaClient;
protected transient SparkRDDWriteClient writeClient; 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. * 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

@@ -35,6 +35,7 @@ import java.io.IOException;
public class HoodieCommonTestHarness { public class HoodieCommonTestHarness {
protected String basePath = null; protected String basePath = null;
protected transient HoodieTestDataGenerator dataGen = null;
protected transient HoodieTableMetaClient metaClient; protected transient HoodieTableMetaClient metaClient;
@TempDir @TempDir
public java.nio.file.Path tempDir; public java.nio.file.Path tempDir;
@@ -52,6 +53,24 @@ public class HoodieCommonTestHarness {
} }
} }
/**
* 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 an instance of {@link HoodieTableMetaClient} with a special table type specified by * Initializes an instance of {@link HoodieTableMetaClient} with a special table type specified by
* {@code getTableType()}. * {@code getTableType()}.

View File

@@ -173,5 +173,102 @@
<artifactId>bijection-avro_${scala.binary.version}</artifactId> <artifactId>bijection-avro_${scala.binary.version}</artifactId>
<version>0.9.7</version> <version>0.9.7</version>
</dependency> </dependency>
<!-- Junit Test Suite -->
<dependency>
<groupId>org.junit.jupiter</groupId>
<artifactId>junit-jupiter-api</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.junit.jupiter</groupId>
<artifactId>junit-jupiter-engine</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.junit.vintage</groupId>
<artifactId>junit-vintage-engine</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.junit.jupiter</groupId>
<artifactId>junit-jupiter-params</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.mockito</groupId>
<artifactId>mockito-junit-jupiter</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.junit.platform</groupId>
<artifactId>junit-platform-runner</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.junit.platform</groupId>
<artifactId>junit-platform-suite-api</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.junit.platform</groupId>
<artifactId>junit-platform-commons</artifactId>
<scope>test</scope>
</dependency>
<!-- Hoodie - Test -->
<dependency>
<groupId>org.apache.hudi</groupId>
<artifactId>hudi-common</artifactId>
<version>${project.version}</version>
<classifier>tests</classifier>
<type>test-jar</type>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.apache.hudi</groupId>
<artifactId>hudi-client-common</artifactId>
<version>${project.version}</version>
<classifier>tests</classifier>
<type>test-jar</type>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.apache.hudi</groupId>
<artifactId>hudi-flink-client</artifactId>
<version>${project.version}</version>
<classifier>tests</classifier>
<type>test-jar</type>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.apache.hudi</groupId>
<artifactId>hudi-hadoop-mr</artifactId>
<version>${project.version}</version>
<scope>test</scope>
</dependency>
<!-- Flink - Tests -->
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-test-utils_${scala.binary.version}</artifactId>
<version>${flink.version}</version>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-runtime_${scala.binary.version}</artifactId>
<version>${flink.version}</version>
<scope>test</scope>
<classifier>tests</classifier>
</dependency>
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-streaming-java_${scala.binary.version}</artifactId>
<version>${flink.version}</version>
<scope>test</scope>
<classifier>tests</classifier>
</dependency>
</dependencies> </dependencies>
</project> </project>

View File

@@ -19,9 +19,11 @@
package org.apache.hudi; package org.apache.hudi;
import org.apache.hudi.client.WriteStatus; import org.apache.hudi.client.WriteStatus;
import org.apache.hudi.common.config.TypedProperties;
import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.OverwriteWithLatestAvroPayload; import org.apache.hudi.common.model.OverwriteWithLatestAvroPayload;
import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.common.model.WriteOperationType;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.operator.InstantGenerateOperator; import org.apache.hudi.operator.InstantGenerateOperator;
import org.apache.hudi.operator.KeyedWriteProcessFunction; import org.apache.hudi.operator.KeyedWriteProcessFunction;
import org.apache.hudi.operator.KeyedWriteProcessOperator; import org.apache.hudi.operator.KeyedWriteProcessOperator;
@@ -42,11 +44,11 @@ import org.apache.flink.runtime.state.filesystem.FsStateBackend;
import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.datastream.DataStream;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer; import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer;
import org.apache.kafka.clients.consumer.ConsumerConfig;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.List; import java.util.List;
import java.util.Objects; import java.util.Objects;
import java.util.Properties;
/** /**
* An Utility which can incrementally consume data from Kafka and apply it to the target table. * An Utility which can incrementally consume data from Kafka and apply it to the target table.
@@ -73,13 +75,21 @@ public class HoodieFlinkStreamer {
env.setStateBackend(new FsStateBackend(cfg.flinkCheckPointPath)); env.setStateBackend(new FsStateBackend(cfg.flinkCheckPointPath));
} }
Properties kafkaProps = StreamerUtil.getKafkaProps(cfg); TypedProperties props = StreamerUtil.getProps(cfg);
// add kafka config
props.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, cfg.kafkaBootstrapServers);
props.put(ConsumerConfig.GROUP_ID_CONFIG, cfg.kafkaGroupId);
// add data source config
props.put(HoodieWriteConfig.WRITE_PAYLOAD_CLASS, cfg.payloadClassName);
props.put(HoodieWriteConfig.PRECOMBINE_FIELD_PROP, cfg.sourceOrderingField);
// Read from kafka source // Read from kafka source
DataStream<HoodieRecord> inputRecords = DataStream<HoodieRecord> inputRecords =
env.addSource(new FlinkKafkaConsumer<>(cfg.kafkaTopic, new SimpleStringSchema(), kafkaProps)) env.addSource(new FlinkKafkaConsumer<>(cfg.kafkaTopic, new SimpleStringSchema(), props))
.filter(Objects::nonNull) .filter(Objects::nonNull)
.map(new JsonStringToHoodieRecordMapFunction(cfg)) .map(new JsonStringToHoodieRecordMapFunction(props))
.name("kafka_to_hudi_record") .name("kafka_to_hudi_record")
.uid("kafka_to_hudi_record_uid"); .uid("kafka_to_hudi_record_uid");

View File

@@ -18,18 +18,21 @@
package org.apache.hudi.source; package org.apache.hudi.source;
import org.apache.hudi.HoodieFlinkStreamer;
import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.avro.HoodieAvroUtils;
import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.config.TypedProperties;
import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.HoodieRecordPayload;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.exception.HoodieFlinkStreamerException; import org.apache.hudi.exception.HoodieFlinkStreamerException;
import org.apache.hudi.common.model.OverwriteWithLatestAvroPayload;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.keygen.KeyGenerator; import org.apache.hudi.keygen.KeyGenerator;
import org.apache.hudi.keygen.SimpleAvroKeyGenerator; import org.apache.hudi.keygen.SimpleAvroKeyGenerator;
import org.apache.hudi.schema.FilebasedSchemaProvider; import org.apache.hudi.schema.FilebasedSchemaProvider;
import org.apache.hudi.util.AvroConvertor; import org.apache.hudi.util.AvroConvertor;
import org.apache.hudi.util.StreamerUtil; import org.apache.hudi.util.StreamerUtil;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericRecord; import org.apache.avro.generic.GenericRecord;
import org.apache.flink.api.common.functions.MapFunction; import org.apache.flink.api.common.functions.MapFunction;
@@ -40,32 +43,46 @@ import java.io.IOException;
*/ */
public class JsonStringToHoodieRecordMapFunction implements MapFunction<String, HoodieRecord> { public class JsonStringToHoodieRecordMapFunction implements MapFunction<String, HoodieRecord> {
private final HoodieFlinkStreamer.Config cfg; private TypedProperties props;
private KeyGenerator keyGenerator; private KeyGenerator keyGenerator;
private AvroConvertor avroConvertor; private AvroConvertor avroConvertor;
private Option<String> schemaStr;
private String payloadClassName;
private String orderingField;
public JsonStringToHoodieRecordMapFunction(HoodieFlinkStreamer.Config cfg) { public JsonStringToHoodieRecordMapFunction(TypedProperties props) {
this.cfg = cfg; this(props, Option.empty());
}
public JsonStringToHoodieRecordMapFunction(TypedProperties props, Option<String> schemaStr) {
this.props = props;
this.schemaStr = schemaStr;
init(); init();
} }
@Override @Override
public HoodieRecord map(String value) throws Exception { public HoodieRecord map(String value) throws Exception {
GenericRecord gr = avroConvertor.fromJson(value); GenericRecord gr = this.avroConvertor.fromJson(value);
HoodieRecordPayload payload = StreamerUtil.createPayload(cfg.payloadClassName, gr, HoodieRecordPayload payload = StreamerUtil.createPayload(this.payloadClassName, gr,
(Comparable) HoodieAvroUtils.getNestedFieldVal(gr, cfg.sourceOrderingField, false)); (Comparable) HoodieAvroUtils.getNestedFieldVal(gr, this.orderingField, false));
return new HoodieRecord<>(keyGenerator.getKey(gr), payload); return new HoodieRecord<>(this.keyGenerator.getKey(gr), payload);
} }
private void init() { private void init() {
TypedProperties props = StreamerUtil.getProps(cfg); if (schemaStr.isPresent()) {
avroConvertor = new AvroConvertor(new FilebasedSchemaProvider(props).getSourceSchema()); this.avroConvertor = new AvroConvertor(new Schema.Parser().parse(schemaStr.get()));
} else {
this.avroConvertor = new AvroConvertor(new FilebasedSchemaProvider(props).getSourceSchema());
}
this.payloadClassName = props.getString(HoodieWriteConfig.WRITE_PAYLOAD_CLASS,
OverwriteWithLatestAvroPayload.class.getName());
this.orderingField = props.getString(HoodieWriteConfig.PRECOMBINE_FIELD_PROP, "ts");
try { try {
keyGenerator = StreamerUtil.createKeyGenerator(props); this.keyGenerator = StreamerUtil.createKeyGenerator(props);
} catch (IOException e) { } catch (IOException e) {
throw new HoodieFlinkStreamerException(String.format("KeyGenerator %s initialization failed", throw new HoodieFlinkStreamerException(String.format("KeyGenerator %s initialization failed",
props.getString("hoodie.datasource.write.keygenerator.class", SimpleAvroKeyGenerator.class.getName())), e); props.getString(HoodieWriteConfig.KEYGENERATOR_CLASS_PROP, SimpleAvroKeyGenerator.class.getName())), e);
} }
} }
} }

View File

@@ -38,7 +38,6 @@ import org.apache.avro.generic.GenericRecord;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.kafka.clients.consumer.ConsumerConfig;
import org.slf4j.Logger; import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
@@ -46,19 +45,11 @@ import java.io.BufferedReader;
import java.io.IOException; import java.io.IOException;
import java.io.StringReader; import java.io.StringReader;
import java.util.List; import java.util.List;
import java.util.Properties;
public class StreamerUtil { public class StreamerUtil {
private static Logger LOG = LoggerFactory.getLogger(StreamerUtil.class); private static Logger LOG = LoggerFactory.getLogger(StreamerUtil.class);
public static Properties getKafkaProps(HoodieFlinkStreamer.Config cfg) {
Properties result = new Properties();
result.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, cfg.kafkaBootstrapServers);
result.put(ConsumerConfig.GROUP_ID_CONFIG, cfg.kafkaGroupId);
return result;
}
public static TypedProperties getProps(HoodieFlinkStreamer.Config cfg) { public static TypedProperties getProps(HoodieFlinkStreamer.Config cfg) {
return readConfig( return readConfig(
FSUtils.getFs(cfg.propsFilePath, getHadoopConf()), FSUtils.getFs(cfg.propsFilePath, getHadoopConf()),

View File

@@ -0,0 +1,89 @@
/*
* 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.source;
import org.apache.hudi.common.config.TypedProperties;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.OverwriteWithLatestAvroPayload;
import org.apache.hudi.common.testutils.RawTripTestPayload;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.keygen.constant.KeyGeneratorOptions;
import org.apache.hudi.testutils.HoodieFlinkClientTestHarness;
import org.apache.avro.Schema;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.Assertions;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import java.util.List;
import java.util.Set;
import java.util.stream.Collectors;
import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.AVRO_SCHEMA;
public class TestJsonStringToHoodieRecordMapFunction extends HoodieFlinkClientTestHarness {
@BeforeEach
public void init() {
initPath();
initTestDataGenerator();
initFileSystem();
initFlinkMiniCluster();
}
@AfterEach
public void clean() throws Exception {
cleanupTestDataGenerator();
cleanupFileSystem();
cleanupFlinkMiniCluster();
}
@Test
public void testMapFunction() throws Exception {
final String newCommitTime = "001";
final int numRecords = 10;
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, numRecords);
List<String> recordStr = RawTripTestPayload.recordsToStrings(records);
Schema schema = AVRO_SCHEMA;
TypedProperties props = new TypedProperties();
props.put(HoodieWriteConfig.WRITE_PAYLOAD_CLASS, OverwriteWithLatestAvroPayload.class.getName());
props.put(HoodieWriteConfig.PRECOMBINE_FIELD_PROP, "timestamp");
props.put(KeyGeneratorOptions.RECORDKEY_FIELD_OPT_KEY, "_row_key");
props.put(KeyGeneratorOptions.PARTITIONPATH_FIELD_OPT_KEY, "partitionPath");
StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
env.setParallelism(2);
SimpleTestSinkFunction.valuesList.clear();
env.fromCollection(recordStr)
.map(new JsonStringToHoodieRecordMapFunction(props, Option.of(schema.toString())))
.addSink(new SimpleTestSinkFunction());
env.execute();
// input records all present in the sink
Assertions.assertEquals(10, SimpleTestSinkFunction.valuesList.size());
// input keys all present in the sink
Set<String> inputKeySet = records.stream().map(r -> r.getKey().getRecordKey()).collect(Collectors.toSet());
Assertions.assertEquals(10, SimpleTestSinkFunction.valuesList.stream()
.map(r -> inputKeySet.contains(r.getRecordKey())).filter(b -> b).count());
}
}

View File

@@ -0,0 +1,30 @@
###
# 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.
###
log4j.rootLogger=WARN, CONSOLE
log4j.logger.org.apache.hudi=DEBUG
log4j.logger.org.apache.hadoop.hbase=ERROR
# CONSOLE is set to be a ConsoleAppender.
log4j.appender.CONSOLE=org.apache.log4j.ConsoleAppender
# CONSOLE uses PatternLayout.
log4j.appender.CONSOLE.layout=org.apache.log4j.PatternLayout
log4j.appender.CONSOLE.layout.ConversionPattern=[%-5p] %d %c %x - %m%n
log4j.appender.CONSOLE.filter.a=org.apache.log4j.varia.LevelRangeFilter
log4j.appender.CONSOLE.filter.a.AcceptOnMatch=true
log4j.appender.CONSOLE.filter.a.LevelMin=WARN
log4j.appender.CONSOLE.filter.a.LevelMax=FATAL

View File

@@ -0,0 +1,31 @@
###
# 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.
###
log4j.rootLogger=WARN, CONSOLE
log4j.logger.org.apache=INFO
log4j.logger.org.apache.hudi=DEBUG
log4j.logger.org.apache.hadoop.hbase=ERROR
# A1 is set to be a ConsoleAppender.
log4j.appender.CONSOLE=org.apache.log4j.ConsoleAppender
# A1 uses PatternLayout.
log4j.appender.CONSOLE.layout=org.apache.log4j.PatternLayout
log4j.appender.CONSOLE.layout.ConversionPattern=%-4r [%t] %-5p %c %x - %m%n
log4j.appender.CONSOLE.filter.a=org.apache.log4j.varia.LevelRangeFilter
log4j.appender.CONSOLE.filter.a.AcceptOnMatch=true
log4j.appender.CONSOLE.filter.a.LevelMin=WARN
log4j.appender.CONSOLE.filter.a.LevelMax=FATAL

View File

@@ -20,6 +20,7 @@ package org.apache.hudi
import org.apache.hudi.common.model.HoodieTableType import org.apache.hudi.common.model.HoodieTableType
import org.apache.hudi.common.model.OverwriteWithLatestAvroPayload import org.apache.hudi.common.model.OverwriteWithLatestAvroPayload
import org.apache.hudi.common.model.WriteOperationType import org.apache.hudi.common.model.WriteOperationType
import org.apache.hudi.config.HoodieWriteConfig
import org.apache.hudi.hive.HiveSyncTool import org.apache.hudi.hive.HiveSyncTool
import org.apache.hudi.hive.SlashEncodedDayPartitionValueExtractor import org.apache.hudi.hive.SlashEncodedDayPartitionValueExtractor
import org.apache.hudi.keygen.SimpleKeyGenerator import org.apache.hudi.keygen.SimpleKeyGenerator
@@ -202,14 +203,14 @@ object DataSourceWriteOptions {
* key value, we will pick the one with the largest value for the precombine field, * key value, we will pick the one with the largest value for the precombine field,
* determined by Object.compareTo(..) * determined by Object.compareTo(..)
*/ */
val PRECOMBINE_FIELD_OPT_KEY = "hoodie.datasource.write.precombine.field" val PRECOMBINE_FIELD_OPT_KEY = HoodieWriteConfig.PRECOMBINE_FIELD_PROP
val DEFAULT_PRECOMBINE_FIELD_OPT_VAL = "ts" val DEFAULT_PRECOMBINE_FIELD_OPT_VAL = "ts"
/** /**
* Payload class used. Override this, if you like to roll your own merge logic, when upserting/inserting. * Payload class used. Override this, if you like to roll your own merge logic, when upserting/inserting.
* This will render any value set for `PRECOMBINE_FIELD_OPT_VAL` in-effective * This will render any value set for `PRECOMBINE_FIELD_OPT_VAL` in-effective
*/ */
val PAYLOAD_CLASS_OPT_KEY = "hoodie.datasource.write.payload.class" val PAYLOAD_CLASS_OPT_KEY = HoodieWriteConfig.WRITE_PAYLOAD_CLASS
val DEFAULT_PAYLOAD_OPT_VAL = classOf[OverwriteWithLatestAvroPayload].getName val DEFAULT_PAYLOAD_OPT_VAL = classOf[OverwriteWithLatestAvroPayload].getName
/** /**
@@ -241,7 +242,7 @@ object DataSourceWriteOptions {
* Key generator class, that implements will extract the key out of incoming record * Key generator class, that implements will extract the key out of incoming record
* *
*/ */
val KEYGENERATOR_CLASS_OPT_KEY = "hoodie.datasource.write.keygenerator.class" val KEYGENERATOR_CLASS_OPT_KEY = HoodieWriteConfig.KEYGENERATOR_CLASS_PROP
val DEFAULT_KEYGENERATOR_CLASS_OPT_VAL = classOf[SimpleKeyGenerator].getName val DEFAULT_KEYGENERATOR_CLASS_OPT_VAL = classOf[SimpleKeyGenerator].getName
/** /**