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

@@ -32,6 +32,7 @@ import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.exception.HoodieException; import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.exception.HoodieIOException;
import org.apache.hudi.index.HoodieIndex; import org.apache.hudi.index.HoodieIndex;
import org.apache.hudi.utilities.checkpointing.InitialCheckPointProvider;
import org.apache.hudi.utilities.schema.SchemaProvider; import org.apache.hudi.utilities.schema.SchemaProvider;
import org.apache.hudi.utilities.sources.Source; import org.apache.hudi.utilities.sources.Source;
import org.apache.hudi.utilities.transform.ChainedTransformer; import org.apache.hudi.utilities.transform.ChainedTransformer;
@@ -117,7 +118,17 @@ public class UtilHelpers {
} }
} }
public static InitialCheckPointProvider createInitialCheckpointProvider(
String className, TypedProperties props) throws IOException {
try {
return (InitialCheckPointProvider) ReflectionUtils.loadClass(className, new Class<?>[] {TypedProperties.class}, props);
} catch (Throwable e) {
throw new IOException("Could not load initial checkpoint provider class " + className, e);
}
}
/** /**
*
*/ */
public static DFSPropertiesConfiguration readConfig(FileSystem fs, Path cfgPath, List<String> overriddenProps) { public static DFSPropertiesConfiguration readConfig(FileSystem fs, Path cfgPath, List<String> overriddenProps) {
DFSPropertiesConfiguration conf; DFSPropertiesConfiguration conf;
@@ -264,6 +275,7 @@ public class UtilHelpers {
/** /**
* Returns a factory for creating connections to the given JDBC URL. * Returns a factory for creating connections to the given JDBC URL.
*
* @param options - JDBC options that contains url, table and other information. * @param options - JDBC options that contains url, table and other information.
* @return * @return
* @throws SQLException if the driver could not open a JDBC connection. * @throws SQLException if the driver could not open a JDBC connection.

View File

@@ -18,14 +18,43 @@
package org.apache.hudi.utilities.checkpointing; package org.apache.hudi.utilities.checkpointing;
import org.apache.hudi.common.config.TypedProperties;
import org.apache.hudi.exception.HoodieException; import org.apache.hudi.exception.HoodieException;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
/** /**
* Provide the initial checkpoint for delta streamer. * Provide the initial checkpoint for delta streamer.
*/ */
public interface InitialCheckPointProvider { public abstract class InitialCheckPointProvider {
protected transient Path path;
protected transient FileSystem fs;
protected transient TypedProperties props;
static class Config {
private static String CHECKPOINT_PROVIDER_PATH_PROP = "hoodie.deltastreamer.checkpoint.provider.path";
}
/**
* Construct InitialCheckPointProvider.
* @param props All properties passed to Delta Streamer
*/
public InitialCheckPointProvider(TypedProperties props) {
this.props = props;
this.path = new Path(props.getString(Config.CHECKPOINT_PROVIDER_PATH_PROP));
}
/**
* Initialize the class with the current filesystem.
*
* @param config Hadoop configuration
*/
public abstract void init(Configuration config) throws HoodieException;
/** /**
* Get checkpoint string recognizable for delta streamer. * Get checkpoint string recognizable for delta streamer.
*/ */
String getCheckpoint() throws HoodieException; public abstract String getCheckpoint() throws HoodieException;
} }

View File

@@ -18,8 +18,10 @@
package org.apache.hudi.utilities.checkpointing; package org.apache.hudi.utilities.checkpointing;
import org.apache.hudi.common.config.TypedProperties;
import org.apache.hudi.exception.HoodieException; import org.apache.hudi.exception.HoodieException;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
@@ -35,15 +37,20 @@ import java.util.regex.Pattern;
* Generate checkpoint from Kafka-Connect-HDFS managed data set. * Generate checkpoint from Kafka-Connect-HDFS managed data set.
* Documentation: https://docs.confluent.io/current/connect/kafka-connect-hdfs/index.html * Documentation: https://docs.confluent.io/current/connect/kafka-connect-hdfs/index.html
*/ */
public class KafkaConnectHdfsProvider implements InitialCheckPointProvider { public class KafkaConnectHdfsProvider extends InitialCheckPointProvider {
private final Path path; private static String FILENAME_SEPARATOR = "[\\+\\.]";
private final FileSystem fs;
private static final String FILENAME_SEPARATOR = "[\\+\\.]"; public KafkaConnectHdfsProvider(TypedProperties props) {
super(props);
}
public KafkaConnectHdfsProvider(final Path basePath, final FileSystem fileSystem) { @Override
this.path = basePath; public void init(Configuration config) throws HoodieException {
this.fs = fileSystem; try {
this.fs = FileSystem.get(config);
} catch (IOException e) {
throw new HoodieException("KafkaConnectHdfsProvider initialization failed");
}
} }
/** /**
@@ -72,6 +79,7 @@ public class KafkaConnectHdfsProvider implements InitialCheckPointProvider {
/** /**
* Convert map contains max offset of each partition to string. * Convert map contains max offset of each partition to string.
*
* @param topic Topic name * @param topic Topic name
* @param checkpoint Map with partition as key and max offset as value * @param checkpoint Map with partition as key and max offset as value
* @return Checkpoint string * @return Checkpoint string
@@ -88,6 +96,7 @@ public class KafkaConnectHdfsProvider implements InitialCheckPointProvider {
/** /**
* List file status recursively. * List file status recursively.
*
* @param curPath Current Path * @param curPath Current Path
* @param filter PathFilter * @param filter PathFilter
* @return All file status match kafka connect naming convention * @return All file status match kafka connect naming convention

View File

@@ -130,7 +130,7 @@ public class DeltaSync implements Serializable {
/** /**
* Hive Config. * Hive Config.
*/ */
private transient HiveConf hiveConf; private transient Configuration conf;
/** /**
* Bag of properties with source, hoodie client, key generator etc. * Bag of properties with source, hoodie client, key generator etc.
@@ -153,7 +153,7 @@ public class DeltaSync implements Serializable {
private transient HoodieWriteClient writeClient; private transient HoodieWriteClient writeClient;
public DeltaSync(HoodieDeltaStreamer.Config cfg, SparkSession sparkSession, SchemaProvider schemaProvider, public DeltaSync(HoodieDeltaStreamer.Config cfg, SparkSession sparkSession, SchemaProvider schemaProvider,
TypedProperties props, JavaSparkContext jssc, FileSystem fs, HiveConf hiveConf, TypedProperties props, JavaSparkContext jssc, FileSystem fs, Configuration conf,
Function<HoodieWriteClient, Boolean> onInitializingHoodieWriteClient) throws IOException { Function<HoodieWriteClient, Boolean> onInitializingHoodieWriteClient) throws IOException {
this.cfg = cfg; this.cfg = cfg;
@@ -172,7 +172,7 @@ public class DeltaSync implements Serializable {
this.formatAdapter = new SourceFormatAdapter( this.formatAdapter = new SourceFormatAdapter(
UtilHelpers.createSource(cfg.sourceClassName, props, jssc, sparkSession, schemaProvider)); UtilHelpers.createSource(cfg.sourceClassName, props, jssc, sparkSession, schemaProvider));
this.hiveConf = hiveConf; this.conf = conf;
// If schemaRegistry already resolved, setup write-client // If schemaRegistry already resolved, setup write-client
setupWriteClient(); setupWriteClient();
@@ -449,8 +449,7 @@ public class DeltaSync implements Serializable {
HiveSyncConfig hiveSyncConfig = DataSourceUtils.buildHiveSyncConfig(props, cfg.targetBasePath); HiveSyncConfig hiveSyncConfig = DataSourceUtils.buildHiveSyncConfig(props, cfg.targetBasePath);
LOG.info("Syncing target hoodie table with hive table(" + hiveSyncConfig.tableName + "). Hive metastore URL :" LOG.info("Syncing target hoodie table with hive table(" + hiveSyncConfig.tableName + "). Hive metastore URL :"
+ hiveSyncConfig.jdbcUrl + ", basePath :" + cfg.targetBasePath); + hiveSyncConfig.jdbcUrl + ", basePath :" + cfg.targetBasePath);
new HiveSyncTool(hiveSyncConfig, new HiveConf(conf, HiveConf.class), fs).syncHoodieTable();
new HiveSyncTool(hiveSyncConfig, hiveConf, fs).syncHoodieTable();
} }
} }

View File

@@ -35,6 +35,7 @@ import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.exception.HoodieIOException;
import org.apache.hudi.utilities.HiveIncrementalPuller; import org.apache.hudi.utilities.HiveIncrementalPuller;
import org.apache.hudi.utilities.UtilHelpers; import org.apache.hudi.utilities.UtilHelpers;
import org.apache.hudi.utilities.checkpointing.InitialCheckPointProvider;
import org.apache.hudi.utilities.schema.SchemaProvider; import org.apache.hudi.utilities.schema.SchemaProvider;
import org.apache.hudi.utilities.sources.JsonDFSSource; import org.apache.hudi.utilities.sources.JsonDFSSource;
@@ -45,7 +46,6 @@ import com.beust.jcommander.ParameterException;
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.hadoop.hive.conf.HiveConf;
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.JavaSparkContext; import org.apache.spark.api.java.JavaSparkContext;
@@ -90,35 +90,34 @@ public class HoodieDeltaStreamer implements Serializable {
public HoodieDeltaStreamer(Config cfg, JavaSparkContext jssc) throws IOException { public HoodieDeltaStreamer(Config cfg, JavaSparkContext jssc) throws IOException {
this(cfg, jssc, FSUtils.getFs(cfg.targetBasePath, jssc.hadoopConfiguration()), this(cfg, jssc, FSUtils.getFs(cfg.targetBasePath, jssc.hadoopConfiguration()),
getDefaultHiveConf(jssc.hadoopConfiguration())); jssc.hadoopConfiguration(), null);
} }
public HoodieDeltaStreamer(Config cfg, JavaSparkContext jssc, TypedProperties props) throws IOException { public HoodieDeltaStreamer(Config cfg, JavaSparkContext jssc, TypedProperties props) throws IOException {
this(cfg, jssc, FSUtils.getFs(cfg.targetBasePath, jssc.hadoopConfiguration()), this(cfg, jssc, FSUtils.getFs(cfg.targetBasePath, jssc.hadoopConfiguration()),
getDefaultHiveConf(jssc.hadoopConfiguration()), props); jssc.hadoopConfiguration(), props);
} }
public HoodieDeltaStreamer(Config cfg, JavaSparkContext jssc, FileSystem fs, HiveConf hiveConf, public HoodieDeltaStreamer(Config cfg, JavaSparkContext jssc, FileSystem fs, Configuration conf) throws IOException {
this(cfg, jssc, fs, conf, null);
}
public HoodieDeltaStreamer(Config cfg, JavaSparkContext jssc, FileSystem fs, Configuration conf,
TypedProperties properties) throws IOException { TypedProperties properties) throws IOException {
this.cfg = cfg; if (cfg.initialCheckpointProvider != null && cfg.checkpoint == null) {
this.deltaSyncService = new DeltaSyncService(cfg, jssc, fs, hiveConf, properties); InitialCheckPointProvider checkPointProvider =
UtilHelpers.createInitialCheckpointProvider(cfg.initialCheckpointProvider, properties);
checkPointProvider.init(conf);
cfg.checkpoint = checkPointProvider.getCheckpoint();
} }
public HoodieDeltaStreamer(Config cfg, JavaSparkContext jssc, FileSystem fs, HiveConf hiveConf) throws IOException {
this.cfg = cfg; this.cfg = cfg;
this.deltaSyncService = new DeltaSyncService(cfg, jssc, fs, hiveConf); this.deltaSyncService = new DeltaSyncService(cfg, jssc, fs, conf, properties);
} }
public void shutdownGracefully() { public void shutdownGracefully() {
deltaSyncService.shutdown(false); deltaSyncService.shutdown(false);
} }
private static HiveConf getDefaultHiveConf(Configuration cfg) {
HiveConf hiveConf = new HiveConf();
hiveConf.addResource(cfg);
return hiveConf;
}
/** /**
* Main method to start syncing. * Main method to start syncing.
* *
@@ -143,6 +142,10 @@ public class HoodieDeltaStreamer implements Serializable {
} }
} }
public Config getConfig() {
return cfg;
}
private boolean onDeltaSyncShutdown(boolean error) { private boolean onDeltaSyncShutdown(boolean error) {
LOG.info("DeltaSync shutdown. Closing write client. Error?" + error); LOG.info("DeltaSync shutdown. Closing write client. Error?" + error);
deltaSyncService.close(); deltaSyncService.close();
@@ -293,6 +296,12 @@ public class HoodieDeltaStreamer implements Serializable {
@Parameter(names = {"--checkpoint"}, description = "Resume Delta Streamer from this checkpoint.") @Parameter(names = {"--checkpoint"}, description = "Resume Delta Streamer from this checkpoint.")
public String checkpoint = null; public String checkpoint = null;
@Parameter(names = {"--initial-checkpoint-provider"}, description = "subclass of "
+ "org.apache.hudi.utilities.checkpointing.InitialCheckpointProvider. Generate check point for delta streamer "
+ "for the first run. This field will override the checkpoint of last commit using the checkpoint field. "
+ "Use this field only when switching source, for example, from DFS source to Kafka Source.")
public String initialCheckpointProvider = null;
@Parameter(names = {"--help", "-h"}, help = true) @Parameter(names = {"--help", "-h"}, help = true)
public Boolean help = false; public Boolean help = false;
@@ -371,7 +380,7 @@ public class HoodieDeltaStreamer implements Serializable {
*/ */
private transient DeltaSync deltaSync; private transient DeltaSync deltaSync;
public DeltaSyncService(Config cfg, JavaSparkContext jssc, FileSystem fs, HiveConf hiveConf, public DeltaSyncService(Config cfg, JavaSparkContext jssc, FileSystem fs, Configuration conf,
TypedProperties properties) throws IOException { TypedProperties properties) throws IOException {
this.cfg = cfg; this.cfg = cfg;
this.jssc = jssc; this.jssc = jssc;
@@ -395,13 +404,13 @@ public class HoodieDeltaStreamer implements Serializable {
LOG.info("Creating delta streamer with configs : " + props.toString()); LOG.info("Creating delta streamer with configs : " + props.toString());
this.schemaProvider = UtilHelpers.createSchemaProvider(cfg.schemaProviderClassName, props, jssc); this.schemaProvider = UtilHelpers.createSchemaProvider(cfg.schemaProviderClassName, props, jssc);
deltaSync = new DeltaSync(cfg, sparkSession, schemaProvider, props, jssc, fs, hiveConf, deltaSync = new DeltaSync(cfg, sparkSession, schemaProvider, props, jssc, fs, conf,
this::onInitializingWriteClient); this::onInitializingWriteClient);
} }
public DeltaSyncService(HoodieDeltaStreamer.Config cfg, JavaSparkContext jssc, FileSystem fs, HiveConf hiveConf) public DeltaSyncService(HoodieDeltaStreamer.Config cfg, JavaSparkContext jssc, FileSystem fs, Configuration conf)
throws IOException { throws IOException {
this(cfg, jssc, fs, hiveConf, null); this(cfg, jssc, fs, conf, null);
} }
public DeltaSync getDeltaSync() { public DeltaSync getDeltaSync() {

View File

@@ -394,6 +394,28 @@ public class TestHoodieDeltaStreamer extends UtilitiesTestBase {
props.getString("hoodie.datasource.write.keygenerator.class")); 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 @Test
public void testPropsWithInvalidKeyGenerator() throws Exception { public void testPropsWithInvalidKeyGenerator() throws Exception {
try { try {

View File

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