1
0

[HUDI-1089] Refactor hudi-client to support multi-engine (#1827)

- This change breaks `hudi-client` into `hudi-client-common` and `hudi-spark-client` modules 
- Simple usages of Spark using jsc.parallelize() has been redone using EngineContext#map, EngineContext#flatMap etc
- Code changes in the PR, break classes into `BaseXYZ` parent classes with no spark dependencies living in `hudi-client-common`
- Classes on `hudi-spark-client` are named `SparkXYZ` extending the parent classes with all the Spark dependencies
- To simplify/cleanup, HoodieIndex#fetchRecordLocation has been removed and its usages in tests replaced with alternatives

Co-authored-by: Vinoth Chandar <vinoth@apache.org>
This commit is contained in:
Mathieu
2020-10-02 05:25:29 +08:00
committed by GitHub
parent 5aaaf8bff1
commit 1f7add9291
380 changed files with 6071 additions and 4128 deletions

View File

@@ -27,8 +27,9 @@ import java.util.Set;
import org.apache.hadoop.conf.Configuration;
import org.apache.hudi.avro.model.HoodieCompactionPlan;
import org.apache.hudi.client.HoodieReadClient;
import org.apache.hudi.client.HoodieWriteClient;
import org.apache.hudi.client.SparkRDDWriteClient;
import org.apache.hudi.client.WriteStatus;
import org.apache.hudi.client.common.HoodieSparkEngineContext;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
import org.apache.hudi.common.util.Option;
@@ -50,12 +51,12 @@ import org.apache.spark.api.java.JavaSparkContext;
/**
* A writer abstraction for the Hudi test suite. This class wraps different implementations of writers used to perform
* write operations into the target hudi dataset. Current supported writers are {@link HoodieDeltaStreamerWrapper}
* and {@link HoodieWriteClient}.
* and {@link SparkRDDWriteClient}.
*/
public class HoodieTestSuiteWriter {
private HoodieDeltaStreamerWrapper deltaStreamerWrapper;
private HoodieWriteClient writeClient;
private SparkRDDWriteClient writeClient;
protected HoodieTestSuiteConfig cfg;
private Option<String> lastCheckpoint;
private HoodieReadClient hoodieReadClient;
@@ -76,10 +77,11 @@ public class HoodieTestSuiteWriter {
// We ensure that only 1 instance of HoodieWriteClient is instantiated for a HoodieTestSuiteWriter
// This does not instantiate a HoodieWriteClient until a
// {@link HoodieDeltaStreamer#commit(HoodieWriteClient, JavaRDD, Option)} is invoked.
HoodieSparkEngineContext context = new HoodieSparkEngineContext(jsc);
this.deltaStreamerWrapper = new HoodieDeltaStreamerWrapper(cfg, jsc);
this.hoodieReadClient = new HoodieReadClient(jsc, cfg.targetBasePath);
this.hoodieReadClient = new HoodieReadClient(context, cfg.targetBasePath);
if (!cfg.useDeltaStreamer) {
this.writeClient = new HoodieWriteClient(jsc, getHoodieClientConfig(cfg, props, schema), rollbackInflight);
this.writeClient = new SparkRDDWriteClient(context, getHoodieClientConfig(cfg, props, schema), rollbackInflight);
}
this.cfg = cfg;
this.configuration = jsc.hadoopConfiguration();
@@ -162,7 +164,7 @@ public class HoodieTestSuiteWriter {
}
}
if (instantTime.isPresent()) {
return writeClient.compact(instantTime.get());
return (JavaRDD<WriteStatus>) writeClient.compact(instantTime.get());
} else {
return null;
}
@@ -183,19 +185,19 @@ public class HoodieTestSuiteWriter {
if (!cfg.useDeltaStreamer) {
Map<String, String> extraMetadata = new HashMap<>();
/** Store the checkpoint in the commit metadata just like
* {@link HoodieDeltaStreamer#commit(HoodieWriteClient, JavaRDD, Option)} **/
* {@link HoodieDeltaStreamer#commit(SparkRDDWriteClient, JavaRDD, Option)} **/
extraMetadata.put(HoodieDeltaStreamerWrapper.CHECKPOINT_KEY, lastCheckpoint.get());
writeClient.commit(instantTime.get(), records, Option.of(extraMetadata));
}
}
public HoodieWriteClient getWriteClient(DagNode dagNode) throws IllegalAccessException {
public SparkRDDWriteClient getWriteClient(DagNode dagNode) throws IllegalAccessException {
if (cfg.useDeltaStreamer & !allowWriteClientAccess(dagNode)) {
throw new IllegalAccessException("cannot access write client when testing in deltastreamer mode");
}
synchronized (this) {
if (writeClient == null) {
this.writeClient = new HoodieWriteClient(this.sparkContext, getHoodieClientConfig(cfg, props, schema), false);
this.writeClient = new SparkRDDWriteClient(new HoodieSparkEngineContext(this.sparkContext), getHoodieClientConfig(cfg, props, schema), false);
}
}
return writeClient;

View File

@@ -26,8 +26,9 @@ import java.util.List;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericRecord;
import org.apache.hudi.avro.HoodieAvroUtils;
import org.apache.hudi.client.HoodieWriteClient;
import org.apache.hudi.client.SparkRDDWriteClient;
import org.apache.hudi.client.WriteStatus;
import org.apache.hudi.client.common.HoodieSparkEngineContext;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
import org.apache.hudi.common.testutils.HoodieTestUtils;
@@ -70,7 +71,7 @@ public class TestDFSHoodieDatasetInputReader extends UtilitiesTestBase {
public void testSimpleHoodieDatasetReader() throws Exception {
HoodieWriteConfig config = makeHoodieClientConfig();
HoodieWriteClient client = new HoodieWriteClient(jsc, config);
SparkRDDWriteClient client = new SparkRDDWriteClient(new HoodieSparkEngineContext(jsc), config);
String commitTime = client.startCommit();
HoodieTestDataGenerator generator = new HoodieTestDataGenerator();
// Insert 100 records across 3 partitions