[HUDI-242] Support for RFC-12/Bootstrapping of external datasets to hudi (#1876)
- [HUDI-418] Bootstrap Index Implementation using HFile with unit-test - [HUDI-421] FileSystem View Changes to support Bootstrap with unit-tests - [HUDI-424] Implement Query Side Integration for querying tables containing bootstrap file slices - [HUDI-423] Implement upsert functionality for handling updates to these bootstrap file slices - [HUDI-421] Bootstrap Write Client with tests - [HUDI-425] Added HoodieDeltaStreamer support - [HUDI-899] Add a knob to change partition-path style while performing metadata bootstrap - [HUDI-900] Metadata Bootstrap Key Generator needs to handle complex keys correctly - [HUDI-424] Simplify Record reader implementation - [HUDI-423] Implement upsert functionality for handling updates to these bootstrap file slices - [HUDI-420] Hoodie Demo working with hive and sparkSQL. Also, Hoodie CLI working with bootstrap tables Co-authored-by: Mehrotra <uditme@amazon.com> Co-authored-by: Vinoth Chandar <vinoth@apache.org> Co-authored-by: Balaji Varadarajan <varadarb@uber.com>
This commit is contained in:
@@ -23,6 +23,7 @@ import org.apache.hadoop.hive.conf.HiveConf;
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.utilities.deltastreamer.DeltaSync;
|
||||
import org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer;
|
||||
import org.apache.hudi.utilities.schema.SchemaProvider;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
@@ -39,14 +40,9 @@ public class HoodieDeltaStreamerWrapper extends HoodieDeltaStreamer {
|
||||
super(cfg, jssc);
|
||||
}
|
||||
|
||||
public HoodieDeltaStreamerWrapper(Config cfg, JavaSparkContext jssc, FileSystem fs, HiveConf conf) throws Exception {
|
||||
super(cfg, jssc, fs, conf);
|
||||
}
|
||||
|
||||
public JavaRDD<WriteStatus> upsert(Operation operation) throws
|
||||
Exception {
|
||||
public JavaRDD<WriteStatus> upsert(Operation operation) throws Exception {
|
||||
cfg.operation = operation;
|
||||
return deltaSyncService.getDeltaSync().syncOnce().getRight();
|
||||
return deltaSyncService.get().getDeltaSync().syncOnce().getRight();
|
||||
}
|
||||
|
||||
public JavaRDD<WriteStatus> insert() throws Exception {
|
||||
@@ -71,7 +67,8 @@ public class HoodieDeltaStreamerWrapper extends HoodieDeltaStreamer {
|
||||
}
|
||||
|
||||
public Pair<SchemaProvider, Pair<String, JavaRDD<HoodieRecord>>> fetchSource() throws Exception {
|
||||
return deltaSyncService.getDeltaSync().readFromSource(deltaSyncService.getDeltaSync().getCommitTimelineOpt());
|
||||
DeltaSync service = deltaSyncService.get().getDeltaSync();
|
||||
return service.readFromSource(service.getCommitTimelineOpt());
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
@@ -41,7 +41,7 @@ import org.apache.hudi.integ.testsuite.dag.scheduler.DagScheduler;
|
||||
import org.apache.hudi.integ.testsuite.generator.DeltaGenerator;
|
||||
import org.apache.hudi.integ.testsuite.reader.DeltaInputType;
|
||||
import org.apache.hudi.integ.testsuite.writer.DeltaOutputMode;
|
||||
import org.apache.hudi.keygen.KeyGenerator;
|
||||
import org.apache.hudi.keygen.BuiltinKeyGenerator;
|
||||
import org.apache.hudi.utilities.UtilHelpers;
|
||||
import org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer;
|
||||
import org.apache.hudi.utilities.schema.SchemaProvider;
|
||||
@@ -85,7 +85,7 @@ public class HoodieTestSuiteJob {
|
||||
*/
|
||||
private transient HiveConf hiveConf;
|
||||
|
||||
private KeyGenerator keyGenerator;
|
||||
private BuiltinKeyGenerator keyGenerator;
|
||||
|
||||
public HoodieTestSuiteJob(HoodieTestSuiteConfig cfg, JavaSparkContext jsc) throws IOException {
|
||||
this.cfg = cfg;
|
||||
@@ -96,7 +96,8 @@ public class HoodieTestSuiteJob {
|
||||
log.info("Creating workload generator with configs : {}", props.toString());
|
||||
this.schemaProvider = UtilHelpers.createSchemaProvider(cfg.schemaProviderClassName, props, jsc);
|
||||
this.hiveConf = getDefaultHiveConf(jsc.hadoopConfiguration());
|
||||
this.keyGenerator = DataSourceUtils.createKeyGenerator(props);
|
||||
this.keyGenerator = (BuiltinKeyGenerator) DataSourceUtils.createKeyGenerator(props);
|
||||
|
||||
if (!fs.exists(new Path(cfg.targetBasePath))) {
|
||||
HoodieTableMetaClient.initTableType(jsc.hadoopConfiguration(), cfg.targetBasePath,
|
||||
HoodieTableType.valueOf(cfg.tableType), cfg.targetTableName, "archived");
|
||||
|
||||
@@ -40,6 +40,7 @@ import org.apache.hudi.integ.testsuite.writer.DeltaOutputMode;
|
||||
import org.apache.hudi.integ.testsuite.writer.DeltaWriteStats;
|
||||
import org.apache.hudi.integ.testsuite.writer.DeltaWriterAdapter;
|
||||
import org.apache.hudi.integ.testsuite.writer.DeltaWriterFactory;
|
||||
import org.apache.hudi.keygen.BuiltinKeyGenerator;
|
||||
import org.apache.hudi.keygen.ComplexKeyGenerator;
|
||||
import org.apache.hudi.keygen.KeyGenerator;
|
||||
import org.apache.hudi.keygen.SimpleKeyGenerator;
|
||||
@@ -70,16 +71,13 @@ public class DeltaGenerator implements Serializable {
|
||||
private int batchId;
|
||||
|
||||
public DeltaGenerator(DeltaConfig deltaOutputConfig, JavaSparkContext jsc, SparkSession sparkSession,
|
||||
String schemaStr,
|
||||
KeyGenerator keyGenerator) {
|
||||
String schemaStr, BuiltinKeyGenerator keyGenerator) {
|
||||
this.deltaOutputConfig = deltaOutputConfig;
|
||||
this.jsc = jsc;
|
||||
this.sparkSession = sparkSession;
|
||||
this.schemaStr = schemaStr;
|
||||
this.recordRowKeyFieldNames = keyGenerator instanceof ComplexKeyGenerator ? ((ComplexKeyGenerator) keyGenerator)
|
||||
.getRecordKeyFields() : Arrays.asList(((SimpleKeyGenerator) keyGenerator).getRecordKeyField());
|
||||
this.partitionPathFieldNames = keyGenerator instanceof ComplexKeyGenerator ? ((ComplexKeyGenerator) keyGenerator)
|
||||
.getPartitionPathFields() : Arrays.asList(((SimpleKeyGenerator) keyGenerator).getPartitionPathField());
|
||||
this.recordRowKeyFieldNames = keyGenerator.getRecordKeyFields();
|
||||
this.partitionPathFieldNames = keyGenerator.getPartitionPathFields();
|
||||
}
|
||||
|
||||
public JavaRDD<DeltaWriteStats> writeRecords(JavaRDD<GenericRecord> records) {
|
||||
|
||||
Reference in New Issue
Block a user