[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:
@@ -18,8 +18,9 @@
|
||||
|
||||
package org.apache.hudi.examples.spark;
|
||||
|
||||
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.fs.FSUtils;
|
||||
import org.apache.hudi.common.model.HoodieAvroPayload;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
@@ -48,7 +49,7 @@ import java.util.stream.Collectors;
|
||||
|
||||
|
||||
/**
|
||||
* Simple examples of #{@link HoodieWriteClient}.
|
||||
* Simple examples of #{@link SparkRDDWriteClient}.
|
||||
*
|
||||
* To run this example, you should
|
||||
* <pre>
|
||||
@@ -94,7 +95,7 @@ public class HoodieWriteClientExample {
|
||||
.withDeleteParallelism(2).forTable(tableName)
|
||||
.withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build())
|
||||
.withCompactionConfig(HoodieCompactionConfig.newBuilder().archiveCommitsWith(20, 30).build()).build();
|
||||
HoodieWriteClient<HoodieAvroPayload> client = new HoodieWriteClient<>(jsc, cfg);
|
||||
SparkRDDWriteClient<HoodieAvroPayload> client = new SparkRDDWriteClient<>(new HoodieSparkEngineContext(jsc), cfg);
|
||||
|
||||
// inserts
|
||||
String newCommitTime = client.startCommit();
|
||||
|
||||
Reference in New Issue
Block a user