[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:
@@ -19,7 +19,7 @@
|
||||
package org.apache.hudi;
|
||||
|
||||
import org.apache.hudi.avro.HoodieAvroUtils;
|
||||
import org.apache.hudi.client.HoodieWriteClient;
|
||||
import org.apache.hudi.client.SparkRDDWriteClient;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.common.model.WriteOperationType;
|
||||
@@ -63,7 +63,7 @@ import static org.mockito.Mockito.when;
|
||||
public class TestDataSourceUtils {
|
||||
|
||||
@Mock
|
||||
private HoodieWriteClient hoodieWriteClient;
|
||||
private SparkRDDWriteClient hoodieWriteClient;
|
||||
|
||||
@Mock
|
||||
private JavaRDD<HoodieRecord> hoodieRecords;
|
||||
@@ -172,7 +172,7 @@ public class TestDataSourceUtils {
|
||||
}
|
||||
|
||||
public static class NoOpBulkInsertPartitioner<T extends HoodieRecordPayload>
|
||||
implements BulkInsertPartitioner<T> {
|
||||
implements BulkInsertPartitioner<JavaRDD<HoodieRecord<T>>> {
|
||||
|
||||
@Override
|
||||
public JavaRDD<HoodieRecord<T>> repartitionRecords(JavaRDD<HoodieRecord<T>> records, int outputSparkPartitions) {
|
||||
|
||||
@@ -25,6 +25,7 @@ import org.apache.hudi.client.bootstrap.FullRecordBootstrapDataProvider;
|
||||
import org.apache.hudi.client.bootstrap.selector.BootstrapModeSelector;
|
||||
import org.apache.hudi.client.bootstrap.selector.FullRecordBootstrapModeSelector;
|
||||
import org.apache.hudi.client.bootstrap.selector.MetadataOnlyBootstrapModeSelector;
|
||||
import org.apache.hudi.client.common.HoodieSparkEngineContext;
|
||||
import org.apache.hudi.common.bootstrap.FileStatusUtils;
|
||||
import org.apache.hudi.common.bootstrap.index.BootstrapIndex;
|
||||
import org.apache.hudi.common.config.TypedProperties;
|
||||
@@ -166,7 +167,7 @@ public class TestBootstrap extends HoodieClientTestBase {
|
||||
df.write().format("parquet").mode(SaveMode.Overwrite).save(srcPath);
|
||||
}
|
||||
String filePath = FileStatusUtils.toPath(BootstrapUtils.getAllLeafFoldersWithFiles(metaClient, metaClient.getFs(),
|
||||
srcPath, jsc).stream().findAny().map(p -> p.getValue().stream().findAny())
|
||||
srcPath, context).stream().findAny().map(p -> p.getValue().stream().findAny())
|
||||
.orElse(null).get().getPath()).toString();
|
||||
ParquetFileReader reader = ParquetFileReader.open(metaClient.getHadoopConf(), new Path(filePath));
|
||||
MessageType schema = reader.getFooter().getFileMetaData().getSchema();
|
||||
@@ -249,7 +250,7 @@ public class TestBootstrap extends HoodieClientTestBase {
|
||||
.withBootstrapParallelism(3)
|
||||
.withBootstrapModeSelector(bootstrapModeSelectorClass).build())
|
||||
.build();
|
||||
HoodieWriteClient client = new HoodieWriteClient(jsc, config);
|
||||
SparkRDDWriteClient client = new SparkRDDWriteClient(context, config);
|
||||
client.bootstrap(Option.empty());
|
||||
checkBootstrapResults(totalRecords, schema, bootstrapCommitInstantTs, checkNumRawFiles, numInstantsAfterBootstrap,
|
||||
numInstantsAfterBootstrap, timestamp, timestamp, deltaCommit, bootstrapInstants);
|
||||
@@ -260,14 +261,14 @@ public class TestBootstrap extends HoodieClientTestBase {
|
||||
client.rollBackInflightBootstrap();
|
||||
metaClient.reloadActiveTimeline();
|
||||
assertEquals(0, metaClient.getCommitsTimeline().countInstants());
|
||||
assertEquals(0L, BootstrapUtils.getAllLeafFoldersWithFiles(metaClient, metaClient.getFs(), basePath, jsc)
|
||||
assertEquals(0L, BootstrapUtils.getAllLeafFoldersWithFiles(metaClient, metaClient.getFs(), basePath, context)
|
||||
.stream().flatMap(f -> f.getValue().stream()).count());
|
||||
|
||||
BootstrapIndex index = BootstrapIndex.getBootstrapIndex(metaClient);
|
||||
assertFalse(index.useIndex());
|
||||
|
||||
// Run bootstrap again
|
||||
client = new HoodieWriteClient(jsc, config);
|
||||
client = new SparkRDDWriteClient(context, config);
|
||||
client.bootstrap(Option.empty());
|
||||
|
||||
metaClient.reloadActiveTimeline();
|
||||
@@ -286,7 +287,7 @@ public class TestBootstrap extends HoodieClientTestBase {
|
||||
String updateSPath = tmpFolder.toAbsolutePath().toString() + "/data2";
|
||||
generateNewDataSetAndReturnSchema(updateTimestamp, totalRecords, partitions, updateSPath);
|
||||
JavaRDD<HoodieRecord> updateBatch =
|
||||
generateInputBatch(jsc, BootstrapUtils.getAllLeafFoldersWithFiles(metaClient, metaClient.getFs(), updateSPath, jsc),
|
||||
generateInputBatch(jsc, BootstrapUtils.getAllLeafFoldersWithFiles(metaClient, metaClient.getFs(), updateSPath, context),
|
||||
schema);
|
||||
String newInstantTs = client.startCommit();
|
||||
client.upsert(updateBatch, newInstantTs);
|
||||
@@ -348,7 +349,7 @@ public class TestBootstrap extends HoodieClientTestBase {
|
||||
original.registerTempTable("original");
|
||||
if (checkNumRawFiles) {
|
||||
List<HoodieFileStatus> files = BootstrapUtils.getAllLeafFoldersWithFiles(metaClient, metaClient.getFs(),
|
||||
bootstrapBasePath, jsc).stream().flatMap(x -> x.getValue().stream()).collect(Collectors.toList());
|
||||
bootstrapBasePath, context).stream().flatMap(x -> x.getValue().stream()).collect(Collectors.toList());
|
||||
assertEquals(files.size() * numVersions,
|
||||
sqlContext.sql("select distinct _hoodie_file_name from bootstrapped").count());
|
||||
}
|
||||
@@ -466,18 +467,19 @@ public class TestBootstrap extends HoodieClientTestBase {
|
||||
assertEquals(totalRecords, seenKeys.size());
|
||||
}
|
||||
|
||||
public static class TestFullBootstrapDataProvider extends FullRecordBootstrapDataProvider {
|
||||
public static class TestFullBootstrapDataProvider extends FullRecordBootstrapDataProvider<JavaRDD<HoodieRecord>> {
|
||||
|
||||
public TestFullBootstrapDataProvider(TypedProperties props, JavaSparkContext jsc) {
|
||||
super(props, jsc);
|
||||
public TestFullBootstrapDataProvider(TypedProperties props, HoodieSparkEngineContext context) {
|
||||
super(props, context);
|
||||
}
|
||||
|
||||
@Override
|
||||
public JavaRDD<HoodieRecord> generateInputRecordRDD(String tableName, String sourceBasePath,
|
||||
public JavaRDD<HoodieRecord> generateInputRecords(String tableName, String sourceBasePath,
|
||||
List<Pair<String, List<HoodieFileStatus>>> partitionPaths) {
|
||||
String filePath = FileStatusUtils.toPath(partitionPaths.stream().flatMap(p -> p.getValue().stream())
|
||||
.findAny().get().getPath()).toString();
|
||||
ParquetFileReader reader = null;
|
||||
JavaSparkContext jsc = HoodieSparkEngineContext.getSparkContext(context);
|
||||
try {
|
||||
reader = ParquetFileReader.open(jsc.hadoopConfiguration(), new Path(filePath));
|
||||
} catch (IOException e) {
|
||||
|
||||
@@ -23,6 +23,7 @@ import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieWriteStat;
|
||||
import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.table.HoodieSparkTable;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.hudi.testutils.HoodieClientTestHarness;
|
||||
|
||||
@@ -76,7 +77,7 @@ public class TestHoodieBulkInsertDataInternalWriter extends HoodieClientTestHarn
|
||||
public void testDataInternalWriter() throws IOException {
|
||||
// init config and table
|
||||
HoodieWriteConfig cfg = getConfigBuilder(basePath).build();
|
||||
HoodieTable table = HoodieTable.create(metaClient, cfg, hadoopConf);
|
||||
HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient);
|
||||
// execute N rounds
|
||||
for (int i = 0; i < 5; i++) {
|
||||
String instantTime = "00" + i;
|
||||
@@ -121,7 +122,7 @@ public class TestHoodieBulkInsertDataInternalWriter extends HoodieClientTestHarn
|
||||
public void testGlobalFailure() throws IOException {
|
||||
// init config and table
|
||||
HoodieWriteConfig cfg = getConfigBuilder(basePath).build();
|
||||
HoodieTable table = HoodieTable.create(metaClient, cfg, hadoopConf);
|
||||
HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient);
|
||||
String partitionPath = HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS[0];
|
||||
|
||||
String instantTime = "001";
|
||||
|
||||
Reference in New Issue
Block a user