[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:
@@ -22,8 +22,9 @@ import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hudi.client.HoodieReadClient;
|
||||
import org.apache.hudi.client.HoodieWriteClient;
|
||||
import org.apache.hudi.client.HoodieWriteResult;
|
||||
import org.apache.hudi.client.SparkRDDWriteClient;
|
||||
import org.apache.hudi.client.common.HoodieSparkEngineContext;
|
||||
import org.apache.hudi.common.config.TypedProperties;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
@@ -171,9 +172,9 @@ public class DataSourceUtils {
|
||||
.withProps(parameters).build();
|
||||
}
|
||||
|
||||
public static HoodieWriteClient createHoodieClient(JavaSparkContext jssc, String schemaStr, String basePath,
|
||||
String tblName, Map<String, String> parameters) {
|
||||
return new HoodieWriteClient<>(jssc, createHoodieConfig(schemaStr, basePath, tblName, parameters), true);
|
||||
public static SparkRDDWriteClient createHoodieClient(JavaSparkContext jssc, String schemaStr, String basePath,
|
||||
String tblName, Map<String, String> parameters) {
|
||||
return new SparkRDDWriteClient<>(new HoodieSparkEngineContext(jssc), createHoodieConfig(schemaStr, basePath, tblName, parameters), true);
|
||||
}
|
||||
|
||||
public static String getCommitActionType(WriteOperationType operation, HoodieTableType tableType) {
|
||||
@@ -184,7 +185,7 @@ public class DataSourceUtils {
|
||||
}
|
||||
}
|
||||
|
||||
public static HoodieWriteResult doWriteOperation(HoodieWriteClient client, JavaRDD<HoodieRecord> hoodieRecords,
|
||||
public static HoodieWriteResult doWriteOperation(SparkRDDWriteClient client, JavaRDD<HoodieRecord> hoodieRecords,
|
||||
String instantTime, WriteOperationType operation) throws HoodieException {
|
||||
switch (operation) {
|
||||
case BULK_INSERT:
|
||||
@@ -202,7 +203,7 @@ public class DataSourceUtils {
|
||||
}
|
||||
}
|
||||
|
||||
public static HoodieWriteResult doDeleteOperation(HoodieWriteClient client, JavaRDD<HoodieKey> hoodieKeys,
|
||||
public static HoodieWriteResult doDeleteOperation(SparkRDDWriteClient client, JavaRDD<HoodieKey> hoodieKeys,
|
||||
String instantTime) {
|
||||
return new HoodieWriteResult(client.delete(hoodieKeys, instantTime));
|
||||
}
|
||||
@@ -224,7 +225,7 @@ public class DataSourceUtils {
|
||||
public static JavaRDD<HoodieRecord> dropDuplicates(JavaSparkContext jssc, JavaRDD<HoodieRecord> incomingHoodieRecords,
|
||||
HoodieWriteConfig writeConfig) {
|
||||
try {
|
||||
HoodieReadClient client = new HoodieReadClient<>(jssc, writeConfig);
|
||||
HoodieReadClient client = new HoodieReadClient<>(new HoodieSparkEngineContext(jssc), writeConfig);
|
||||
return client.tagLocation(incomingHoodieRecords)
|
||||
.filter(r -> !((HoodieRecord<HoodieRecordPayload>) r).isCurrentLocationKnown());
|
||||
} catch (TableNotFoundException e) {
|
||||
|
||||
@@ -18,8 +18,10 @@
|
||||
|
||||
package org.apache.hudi.async;
|
||||
|
||||
import org.apache.hudi.client.HoodieWriteClient;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
import org.apache.hudi.client.AbstractCompactor;
|
||||
import org.apache.hudi.client.AbstractHoodieWriteClient;
|
||||
import org.apache.hudi.client.HoodieSparkCompactor;
|
||||
import org.apache.hudi.client.common.HoodieEngineContext;
|
||||
|
||||
/**
|
||||
* Async Compaction Service used by Structured Streaming. Here, async compaction is run in daemon mode to prevent
|
||||
@@ -29,7 +31,12 @@ public class SparkStreamingAsyncCompactService extends AsyncCompactService {
|
||||
|
||||
private static final long serialVersionUID = 1L;
|
||||
|
||||
public SparkStreamingAsyncCompactService(JavaSparkContext jssc, HoodieWriteClient client) {
|
||||
super(jssc, client, true);
|
||||
public SparkStreamingAsyncCompactService(HoodieEngineContext context, AbstractHoodieWriteClient client) {
|
||||
super(context, client, true);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected AbstractCompactor createCompactor(AbstractHoodieWriteClient client) {
|
||||
return new HoodieSparkCompactor(client);
|
||||
}
|
||||
}
|
||||
|
||||
@@ -23,6 +23,7 @@ import org.apache.hudi.DataSourceUtils;
|
||||
import org.apache.hudi.avro.HoodieAvroUtils;
|
||||
import org.apache.hudi.avro.model.HoodieFileStatus;
|
||||
import org.apache.hudi.client.bootstrap.FullRecordBootstrapDataProvider;
|
||||
import org.apache.hudi.client.common.HoodieSparkEngineContext;
|
||||
import org.apache.hudi.common.bootstrap.FileStatusUtils;
|
||||
import org.apache.hudi.common.config.TypedProperties;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
@@ -32,7 +33,6 @@ import org.apache.hudi.keygen.KeyGenerator;
|
||||
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
import org.apache.spark.rdd.RDD;
|
||||
import org.apache.spark.sql.Dataset;
|
||||
import org.apache.spark.sql.SparkSession;
|
||||
@@ -43,18 +43,18 @@ import java.util.List;
|
||||
/**
|
||||
* Spark Data frame based bootstrap input provider.
|
||||
*/
|
||||
public class SparkParquetBootstrapDataProvider extends FullRecordBootstrapDataProvider {
|
||||
public class SparkParquetBootstrapDataProvider extends FullRecordBootstrapDataProvider<JavaRDD<HoodieRecord>> {
|
||||
|
||||
private final transient SparkSession sparkSession;
|
||||
|
||||
public SparkParquetBootstrapDataProvider(TypedProperties props,
|
||||
JavaSparkContext jsc) {
|
||||
super(props, jsc);
|
||||
this.sparkSession = SparkSession.builder().config(jsc.getConf()).getOrCreate();
|
||||
HoodieSparkEngineContext context) {
|
||||
super(props, context);
|
||||
this.sparkSession = SparkSession.builder().config(context.getJavaSparkContext().getConf()).getOrCreate();
|
||||
}
|
||||
|
||||
@Override
|
||||
public JavaRDD<HoodieRecord> generateInputRecordRDD(String tableName, String sourceBasePath,
|
||||
public JavaRDD<HoodieRecord> generateInputRecords(String tableName, String sourceBasePath,
|
||||
List<Pair<String, List<HoodieFileStatus>>> partitionPathsWithFiles) {
|
||||
String[] filePaths = partitionPathsWithFiles.stream().map(Pair::getValue)
|
||||
.flatMap(f -> f.stream().map(fs -> FileStatusUtils.toPath(fs.getPath()).toString()))
|
||||
|
||||
@@ -20,7 +20,8 @@ package org.apache.hudi.internal;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hudi.DataSourceUtils;
|
||||
import org.apache.hudi.client.HoodieWriteClient;
|
||||
import org.apache.hudi.client.SparkRDDWriteClient;
|
||||
import org.apache.hudi.client.common.HoodieSparkEngineContext;
|
||||
import org.apache.hudi.common.model.HoodieWriteStat;
|
||||
import org.apache.hudi.common.model.WriteOperationType;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
@@ -30,6 +31,7 @@ import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.table.HoodieSparkTable;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
@@ -59,7 +61,7 @@ public class HoodieDataSourceInternalWriter implements DataSourceWriter {
|
||||
private final HoodieTableMetaClient metaClient;
|
||||
private final HoodieWriteConfig writeConfig;
|
||||
private final StructType structType;
|
||||
private final HoodieWriteClient writeClient;
|
||||
private final SparkRDDWriteClient writeClient;
|
||||
private final HoodieTable hoodieTable;
|
||||
private final WriteOperationType operationType;
|
||||
|
||||
@@ -69,11 +71,11 @@ public class HoodieDataSourceInternalWriter implements DataSourceWriter {
|
||||
this.writeConfig = writeConfig;
|
||||
this.structType = structType;
|
||||
this.operationType = WriteOperationType.BULK_INSERT;
|
||||
this.writeClient = new HoodieWriteClient<>(new JavaSparkContext(sparkSession.sparkContext()), writeConfig, true);
|
||||
this.writeClient = new SparkRDDWriteClient<>(new HoodieSparkEngineContext(new JavaSparkContext(sparkSession.sparkContext())), writeConfig, true);
|
||||
writeClient.setOperationType(operationType);
|
||||
writeClient.startCommitWithTime(instantTime);
|
||||
this.metaClient = new HoodieTableMetaClient(configuration, writeConfig.getBasePath());
|
||||
this.hoodieTable = HoodieTable.create(metaClient, writeConfig, metaClient.getHadoopConf());
|
||||
this.hoodieTable = HoodieSparkTable.create(writeConfig, new HoodieSparkEngineContext(new JavaSparkContext(sparkSession.sparkContext())), metaClient);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
||||
@@ -36,7 +36,7 @@ import java.util.List;
|
||||
* Abstract class to extend for plugging in extraction of {@link HoodieKey} from an Avro record.
|
||||
*/
|
||||
@PublicAPIClass(maturity = ApiMaturityLevel.STABLE)
|
||||
public abstract class KeyGenerator implements Serializable, KeyGeneratorInterface {
|
||||
public abstract class KeyGenerator implements Serializable, SparkKeyGeneratorInterface {
|
||||
|
||||
private static final String STRUCT_NAME = "hoodieRowTopLevelField";
|
||||
private static final String NAMESPACE = "hoodieRow";
|
||||
|
||||
@@ -27,7 +27,8 @@ import org.apache.hadoop.fs.{FileSystem, Path}
|
||||
import org.apache.hadoop.hive.conf.HiveConf
|
||||
import org.apache.hudi.DataSourceWriteOptions._
|
||||
import org.apache.hudi.avro.HoodieAvroUtils
|
||||
import org.apache.hudi.client.{HoodieWriteClient, HoodieWriteResult}
|
||||
import org.apache.hudi.client.{SparkRDDWriteClient, HoodieWriteResult}
|
||||
import org.apache.hudi.client.{SparkRDDWriteClient, WriteStatus}
|
||||
import org.apache.hudi.common.config.TypedProperties
|
||||
import org.apache.hudi.common.model.{HoodieRecordPayload, HoodieTableType, WriteOperationType}
|
||||
import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient}
|
||||
@@ -59,11 +60,11 @@ private[hudi] object HoodieSparkSqlWriter {
|
||||
parameters: Map[String, String],
|
||||
df: DataFrame,
|
||||
hoodieTableConfigOpt: Option[HoodieTableConfig] = Option.empty,
|
||||
hoodieWriteClient: Option[HoodieWriteClient[HoodieRecordPayload[Nothing]]] = Option.empty,
|
||||
asyncCompactionTriggerFn: Option[Function1[HoodieWriteClient[HoodieRecordPayload[Nothing]], Unit]] = Option.empty
|
||||
hoodieWriteClient: Option[SparkRDDWriteClient[HoodieRecordPayload[Nothing]]] = Option.empty,
|
||||
asyncCompactionTriggerFn: Option[Function1[SparkRDDWriteClient[HoodieRecordPayload[Nothing]], Unit]] = Option.empty
|
||||
)
|
||||
: (Boolean, common.util.Option[String], common.util.Option[String],
|
||||
HoodieWriteClient[HoodieRecordPayload[Nothing]], HoodieTableConfig) = {
|
||||
SparkRDDWriteClient[HoodieRecordPayload[Nothing]], HoodieTableConfig) = {
|
||||
|
||||
val sparkContext = sqlContext.sparkContext
|
||||
val path = parameters.get("path")
|
||||
@@ -126,7 +127,7 @@ private[hudi] object HoodieSparkSqlWriter {
|
||||
}
|
||||
// scalastyle:on
|
||||
|
||||
val (writeResult, writeClient: HoodieWriteClient[HoodieRecordPayload[Nothing]]) =
|
||||
val (writeResult, writeClient: SparkRDDWriteClient[HoodieRecordPayload[Nothing]]) =
|
||||
if (operation != WriteOperationType.DELETE) {
|
||||
// register classes & schemas
|
||||
val (structName, nameSpace) = AvroConversionUtils.getAvroRecordNameAndNamespace(tblName)
|
||||
@@ -151,7 +152,7 @@ private[hudi] object HoodieSparkSqlWriter {
|
||||
// Create a HoodieWriteClient & issue the write.
|
||||
val client = hoodieWriteClient.getOrElse(DataSourceUtils.createHoodieClient(jsc, schema.toString, path.get,
|
||||
tblName, mapAsJavaMap(parameters)
|
||||
)).asInstanceOf[HoodieWriteClient[HoodieRecordPayload[Nothing]]]
|
||||
)).asInstanceOf[SparkRDDWriteClient[HoodieRecordPayload[Nothing]]]
|
||||
|
||||
if (isAsyncCompactionEnabled(client, tableConfig, parameters, jsc.hadoopConfiguration())) {
|
||||
asyncCompactionTriggerFn.get.apply(client)
|
||||
@@ -190,7 +191,7 @@ private[hudi] object HoodieSparkSqlWriter {
|
||||
// Create a HoodieWriteClient & issue the delete.
|
||||
val client = hoodieWriteClient.getOrElse(DataSourceUtils.createHoodieClient(jsc,
|
||||
Schema.create(Schema.Type.NULL).toString, path.get, tblName,
|
||||
mapAsJavaMap(parameters))).asInstanceOf[HoodieWriteClient[HoodieRecordPayload[Nothing]]]
|
||||
mapAsJavaMap(parameters))).asInstanceOf[SparkRDDWriteClient[HoodieRecordPayload[Nothing]]]
|
||||
|
||||
if (isAsyncCompactionEnabled(client, tableConfig, parameters, jsc.hadoopConfiguration())) {
|
||||
asyncCompactionTriggerFn.get.apply(client)
|
||||
@@ -389,7 +390,7 @@ private[hudi] object HoodieSparkSqlWriter {
|
||||
|
||||
private def commitAndPerformPostOperations(writeResult: HoodieWriteResult,
|
||||
parameters: Map[String, String],
|
||||
client: HoodieWriteClient[HoodieRecordPayload[Nothing]],
|
||||
client: SparkRDDWriteClient[HoodieRecordPayload[Nothing]],
|
||||
tableConfig: HoodieTableConfig,
|
||||
jsc: JavaSparkContext,
|
||||
tableInstantInfo: TableInstantInfo
|
||||
@@ -446,7 +447,7 @@ private[hudi] object HoodieSparkSqlWriter {
|
||||
}
|
||||
}
|
||||
|
||||
private def isAsyncCompactionEnabled(client: HoodieWriteClient[HoodieRecordPayload[Nothing]],
|
||||
private def isAsyncCompactionEnabled(client: SparkRDDWriteClient[HoodieRecordPayload[Nothing]],
|
||||
tableConfig: HoodieTableConfig,
|
||||
parameters: Map[String, String], configuration: Configuration) : Boolean = {
|
||||
log.info(s"Config.isInlineCompaction ? ${client.getConfig.isInlineCompaction}")
|
||||
|
||||
@@ -20,7 +20,8 @@ import java.lang
|
||||
import java.util.function.{Function, Supplier}
|
||||
|
||||
import org.apache.hudi.async.{AsyncCompactService, SparkStreamingAsyncCompactService}
|
||||
import org.apache.hudi.client.HoodieWriteClient
|
||||
import org.apache.hudi.client.SparkRDDWriteClient
|
||||
import org.apache.hudi.client.common.HoodieSparkEngineContext
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload
|
||||
import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient}
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant.State
|
||||
@@ -60,7 +61,7 @@ class HoodieStreamingSink(sqlContext: SQLContext,
|
||||
}
|
||||
|
||||
private var asyncCompactorService : AsyncCompactService = _
|
||||
private var writeClient : Option[HoodieWriteClient[HoodieRecordPayload[Nothing]]] = Option.empty
|
||||
private var writeClient : Option[SparkRDDWriteClient[HoodieRecordPayload[Nothing]]] = Option.empty
|
||||
private var hoodieTableConfig : Option[HoodieTableConfig] = Option.empty
|
||||
|
||||
override def addBatch(batchId: Long, data: DataFrame): Unit = this.synchronized {
|
||||
@@ -153,10 +154,10 @@ class HoodieStreamingSink(sqlContext: SQLContext,
|
||||
}
|
||||
}
|
||||
|
||||
protected def triggerAsyncCompactor(client: HoodieWriteClient[HoodieRecordPayload[Nothing]]): Unit = {
|
||||
protected def triggerAsyncCompactor(client: SparkRDDWriteClient[HoodieRecordPayload[Nothing]]): Unit = {
|
||||
if (null == asyncCompactorService) {
|
||||
log.info("Triggering Async compaction !!")
|
||||
asyncCompactorService = new SparkStreamingAsyncCompactService(new JavaSparkContext(sqlContext.sparkContext),
|
||||
asyncCompactorService = new SparkStreamingAsyncCompactService(new HoodieSparkEngineContext(new JavaSparkContext(sqlContext.sparkContext)),
|
||||
client)
|
||||
asyncCompactorService.start(new Function[java.lang.Boolean, java.lang.Boolean] {
|
||||
override def apply(errored: lang.Boolean): lang.Boolean = {
|
||||
|
||||
@@ -17,7 +17,6 @@
|
||||
|
||||
package org.apache.hudi
|
||||
|
||||
|
||||
import com.google.common.collect.Lists
|
||||
import org.apache.avro.Schema
|
||||
import org.apache.hadoop.fs.GlobPattern
|
||||
@@ -30,10 +29,11 @@ import org.apache.hudi.common.table.timeline.HoodieTimeline
|
||||
import org.apache.hudi.common.util.ParquetUtils
|
||||
import org.apache.hudi.config.HoodieWriteConfig
|
||||
import org.apache.hudi.exception.HoodieException
|
||||
import org.apache.hudi.table.HoodieTable
|
||||
|
||||
import org.apache.hadoop.fs.GlobPattern
|
||||
import org.apache.hudi.client.common.HoodieSparkEngineContext
|
||||
import org.apache.hudi.table.HoodieSparkTable
|
||||
import org.apache.log4j.LogManager
|
||||
import org.apache.spark.api.java.JavaSparkContext
|
||||
import org.apache.spark.rdd.RDD
|
||||
import org.apache.spark.sql.sources.{BaseRelation, TableScan}
|
||||
import org.apache.spark.sql.types.{StringType, StructField, StructType}
|
||||
@@ -64,8 +64,9 @@ class IncrementalRelation(val sqlContext: SQLContext,
|
||||
throw new HoodieException("Incremental view not implemented yet, for merge-on-read tables")
|
||||
}
|
||||
// TODO : Figure out a valid HoodieWriteConfig
|
||||
private val hoodieTable = HoodieTable.create(metaClient, HoodieWriteConfig.newBuilder().withPath(basePath).build(),
|
||||
sqlContext.sparkContext.hadoopConfiguration)
|
||||
private val hoodieTable = HoodieSparkTable.create(HoodieWriteConfig.newBuilder().withPath(basePath).build(),
|
||||
new HoodieSparkEngineContext(new JavaSparkContext(sqlContext.sparkContext)),
|
||||
metaClient)
|
||||
private val commitTimeline = hoodieTable.getMetaClient.getCommitTimeline.filterCompletedInstants()
|
||||
if (commitTimeline.empty()) {
|
||||
throw new HoodieException("No instants to incrementally pull")
|
||||
|
||||
@@ -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";
|
||||
|
||||
@@ -22,7 +22,7 @@ import java.util.{Date, UUID}
|
||||
|
||||
import org.apache.commons.io.FileUtils
|
||||
import org.apache.hudi.DataSourceWriteOptions._
|
||||
import org.apache.hudi.client.HoodieWriteClient
|
||||
import org.apache.hudi.client.SparkRDDWriteClient
|
||||
import org.apache.hudi.common.model.{HoodieRecord, HoodieRecordPayload}
|
||||
import org.apache.hudi.common.testutils.HoodieTestDataGenerator
|
||||
import org.apache.hudi.config.HoodieWriteConfig
|
||||
@@ -250,7 +250,7 @@ class HoodieSparkSqlWriterSuite extends FunSuite with Matchers {
|
||||
schema.toString,
|
||||
path.toAbsolutePath.toString,
|
||||
hoodieFooTableName,
|
||||
mapAsJavaMap(fooTableParams)).asInstanceOf[HoodieWriteClient[HoodieRecordPayload[Nothing]]])
|
||||
mapAsJavaMap(fooTableParams)).asInstanceOf[SparkRDDWriteClient[HoodieRecordPayload[Nothing]]])
|
||||
|
||||
// write to Hudi
|
||||
HoodieSparkSqlWriter.write(sqlContext, SaveMode.Append, fooTableParams, df, Option.empty,
|
||||
|
||||
Reference in New Issue
Block a user