1
0

[HUDI-759] Integrate checkpoint provider with delta streamer (#1486)

This commit is contained in:
Gary Li
2020-04-14 14:51:04 -07:00
committed by GitHub
parent 644c1cc8bd
commit 14d4fea833
7 changed files with 147 additions and 64 deletions

View File

@@ -256,21 +256,21 @@ public class TestHoodieDeltaStreamer extends UtilitiesTestBase {
}
static HoodieDeltaStreamer.Config makeConfig(String basePath, Operation op, List<String> transformerClassNames,
String propsFilename, boolean enableHiveSync) {
String propsFilename, boolean enableHiveSync) {
return makeConfig(basePath, op, transformerClassNames, propsFilename, enableHiveSync, true,
false, null, null);
}
static HoodieDeltaStreamer.Config makeConfig(String basePath, Operation op, List<String> transformerClassNames,
String propsFilename, boolean enableHiveSync, boolean useSchemaProviderClass, boolean updatePayloadClass,
String payloadClassName, String tableType) {
String propsFilename, boolean enableHiveSync, boolean useSchemaProviderClass, boolean updatePayloadClass,
String payloadClassName, String tableType) {
return makeConfig(basePath, op, TestDataSource.class.getName(), transformerClassNames, propsFilename, enableHiveSync,
useSchemaProviderClass, 1000, updatePayloadClass, payloadClassName, tableType, "timestamp");
}
static HoodieDeltaStreamer.Config makeConfig(String basePath, Operation op, String sourceClassName,
List<String> transformerClassNames, String propsFilename, boolean enableHiveSync, boolean useSchemaProviderClass,
int sourceLimit, boolean updatePayloadClass, String payloadClassName, String tableType, String sourceOrderingField) {
List<String> transformerClassNames, String propsFilename, boolean enableHiveSync, boolean useSchemaProviderClass,
int sourceLimit, boolean updatePayloadClass, String payloadClassName, String tableType, String sourceOrderingField) {
HoodieDeltaStreamer.Config cfg = new HoodieDeltaStreamer.Config();
cfg.targetBasePath = basePath;
cfg.targetTableName = "hoodie_trips";
@@ -394,6 +394,28 @@ public class TestHoodieDeltaStreamer extends UtilitiesTestBase {
props.getString("hoodie.datasource.write.keygenerator.class"));
}
@Test
public void testKafkaConnectCheckpointProvider() throws IOException {
String tableBasePath = dfsBasePath + "/test_table";
String bootstrapPath = dfsBasePath + "/kafka_topic1";
String partitionPath = bootstrapPath + "/year=2016/month=05/day=01";
String filePath = partitionPath + "/kafka_topic1+0+100+200.parquet";
String checkpointProviderClass = "org.apache.hudi.utilities.checkpointing.KafkaConnectHdfsProvider";
HoodieDeltaStreamer.Config cfg = TestHelpers.makeDropAllConfig(tableBasePath, Operation.UPSERT);
TypedProperties props =
new DFSPropertiesConfiguration(dfs, new Path(dfsBasePath + "/" + PROPS_FILENAME_TEST_SOURCE)).getConfig();
props.put("hoodie.deltastreamer.checkpoint.provider.path", bootstrapPath);
cfg.initialCheckpointProvider = checkpointProviderClass;
// create regular kafka connect hdfs dirs
dfs.mkdirs(new Path(bootstrapPath));
dfs.mkdirs(new Path(partitionPath));
// generate parquet files using kafka connect naming convention
HoodieTestDataGenerator dataGenerator = new HoodieTestDataGenerator();
Helpers.saveParquetToDFS(Helpers.toGenericRecords(dataGenerator.generateInserts("000", 100)), new Path(filePath));
HoodieDeltaStreamer deltaStreamer = new HoodieDeltaStreamer(cfg, jsc, dfs, hdfsTestService.getHadoopConf(), props);
assertEquals(deltaStreamer.getConfig().checkpoint, "kafka_topic1,0:200");
}
@Test
public void testPropsWithInvalidKeyGenerator() throws Exception {
try {
@@ -595,7 +617,7 @@ public class TestHoodieDeltaStreamer extends UtilitiesTestBase {
assertTrue(e.getMessage().contains("Please provide a valid schema provider class!"));
}
}
@Test
public void testPayloadClassUpdate() throws Exception {
String dataSetBasePath = dfsBasePath + "/test_dataset_mor";

View File

@@ -19,13 +19,11 @@
package org.apache.hudi.utilities.checkpointing;
import org.apache.hudi.common.HoodieCommonTestHarness;
import org.apache.hudi.common.config.TypedProperties;
import org.apache.hudi.common.model.HoodieTestUtils;
import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.exception.HoodieException;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.junit.Before;
import org.junit.Test;
@@ -34,15 +32,14 @@ import java.io.File;
import static org.junit.Assert.assertEquals;
public class TestKafkaConnectHdfsProvider extends HoodieCommonTestHarness {
private FileSystem fs = null;
private String topicPath = null;
private Configuration hadoopConf = null;
@Before
public void init() {
// Prepare directories
initPath();
final Configuration hadoopConf = HoodieTestUtils.getDefaultHadoopConf();
fs = FSUtils.getFs(basePath, hadoopConf);
hadoopConf = HoodieTestUtils.getDefaultHadoopConf();
}
@Test
@@ -70,7 +67,10 @@ public class TestKafkaConnectHdfsProvider extends HoodieCommonTestHarness {
+ "random_snappy_1.parquet").createNewFile();
new File(topicPath + "/year=2016/month=05/day=02/"
+ "random_snappy_2.parquet").createNewFile();
InitialCheckPointProvider provider = new KafkaConnectHdfsProvider(new Path(topicPath), fs);
final TypedProperties props = new TypedProperties();
props.put("hoodie.deltastreamer.checkpoint.provider.path", topicPath);
final InitialCheckPointProvider provider = new KafkaConnectHdfsProvider(props);
provider.init(hadoopConf);
assertEquals(provider.getCheckpoint(), "topic1,0:300,1:200");
}
@@ -88,7 +88,10 @@ public class TestKafkaConnectHdfsProvider extends HoodieCommonTestHarness {
+ "topic1+2+100+200.parquet").createNewFile();
new File(topicPath + "/year=2016/month=05/day=02/"
+ "topic1+0+201+300.parquet").createNewFile();
InitialCheckPointProvider provider = new KafkaConnectHdfsProvider(new Path(topicPath), fs);
final TypedProperties props = new TypedProperties();
props.put("hoodie.deltastreamer.checkpoint.provider.path", topicPath);
final InitialCheckPointProvider provider = new KafkaConnectHdfsProvider(props);
provider.init(hadoopConf);
provider.getCheckpoint();
}
}