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

@@ -18,8 +18,10 @@
package org.apache.hudi.utilities;
import org.apache.hudi.client.HoodieWriteClient;
import org.apache.hudi.client.SparkRDDWriteClient;
import org.apache.hudi.client.WriteStatus;
import org.apache.hudi.client.common.HoodieEngineContext;
import org.apache.hudi.client.common.HoodieSparkEngineContext;
import org.apache.hudi.common.HoodieJsonPayload;
import org.apache.hudi.common.config.TypedProperties;
import org.apache.hudi.common.fs.FSUtils;
@@ -142,7 +144,7 @@ public class HDFSParquetImporter implements Serializable {
// Get schema.
String schemaStr = UtilHelpers.parseSchema(fs, cfg.schemaFile);
HoodieWriteClient client =
SparkRDDWriteClient client =
UtilHelpers.createHoodieClient(jsc, cfg.targetPath, schemaStr, cfg.parallelism, Option.empty(), props);
JavaRDD<HoodieRecord<HoodieRecordPayload>> hoodieRecords = buildHoodieRecordsForImport(jsc, schemaStr);
@@ -166,7 +168,8 @@ public class HDFSParquetImporter implements Serializable {
AvroReadSupport.setAvroReadSchema(jsc.hadoopConfiguration(), (new Schema.Parser().parse(schemaStr)));
ParquetInputFormat.setReadSupportClass(job, (AvroReadSupport.class));
jsc.setJobGroup(this.getClass().getSimpleName(), "Build records for import");
HoodieEngineContext context = new HoodieSparkEngineContext(jsc);
context.setJobStatus(this.getClass().getSimpleName(), "Build records for import");
return jsc.newAPIHadoopFile(cfg.srcPath, ParquetInputFormat.class, Void.class, GenericRecord.class,
job.getConfiguration())
// To reduce large number of tasks.
@@ -203,8 +206,8 @@ public class HDFSParquetImporter implements Serializable {
* @param hoodieRecords Hoodie Records
* @param <T> Type
*/
protected <T extends HoodieRecordPayload> JavaRDD<WriteStatus> load(HoodieWriteClient client, String instantTime,
JavaRDD<HoodieRecord<T>> hoodieRecords) {
protected <T extends HoodieRecordPayload> JavaRDD<WriteStatus> load(SparkRDDWriteClient client, String instantTime,
JavaRDD<HoodieRecord<T>> hoodieRecords) {
switch (cfg.command.toLowerCase()) {
case "upsert": {
return client.upsert(hoodieRecords, instantTime);

View File

@@ -18,7 +18,8 @@
package org.apache.hudi.utilities;
import org.apache.hudi.client.HoodieWriteClient;
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.fs.FSUtils;
import org.apache.hudi.config.HoodieWriteConfig;
@@ -68,7 +69,7 @@ public class HoodieCleaner {
public void run() {
HoodieWriteConfig hoodieCfg = getHoodieClientConfig();
HoodieWriteClient client = new HoodieWriteClient<>(jssc, hoodieCfg, false);
SparkRDDWriteClient client = new SparkRDDWriteClient<>(new HoodieSparkEngineContext(jssc), hoodieCfg, false);
client.clean();
}

View File

@@ -21,6 +21,7 @@ package org.apache.hudi.utilities;
import org.apache.hudi.client.CompactionAdminClient;
import org.apache.hudi.client.CompactionAdminClient.RenameOpResult;
import org.apache.hudi.client.CompactionAdminClient.ValidationOpResult;
import org.apache.hudi.client.common.HoodieSparkEngineContext;
import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.model.HoodieFileGroupId;
import org.apache.hudi.common.table.HoodieTableMetaClient;
@@ -60,7 +61,7 @@ public class HoodieCompactionAdminTool {
*/
public void run(JavaSparkContext jsc) throws Exception {
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.basePath);
try (CompactionAdminClient admin = new CompactionAdminClient(jsc, cfg.basePath)) {
try (CompactionAdminClient admin = new CompactionAdminClient(new HoodieSparkEngineContext(jsc), cfg.basePath)) {
final FileSystem fs = FSUtils.getFs(cfg.basePath, jsc.hadoopConfiguration());
if (cfg.outputPath != null && fs.exists(new Path(cfg.outputPath))) {
throw new IllegalStateException("Output File Path already exists");

View File

@@ -18,7 +18,7 @@
package org.apache.hudi.utilities;
import org.apache.hudi.client.HoodieWriteClient;
import org.apache.hudi.client.SparkRDDWriteClient;
import org.apache.hudi.client.WriteStatus;
import org.apache.hudi.common.config.TypedProperties;
import org.apache.hudi.common.fs.FSUtils;
@@ -130,15 +130,15 @@ public class HoodieCompactor {
private int doCompact(JavaSparkContext jsc) throws Exception {
// Get schema.
String schemaStr = UtilHelpers.parseSchema(fs, cfg.schemaFile);
HoodieWriteClient client =
SparkRDDWriteClient client =
UtilHelpers.createHoodieClient(jsc, cfg.basePath, schemaStr, cfg.parallelism, Option.empty(), props);
JavaRDD<WriteStatus> writeResponse = client.compact(cfg.compactionInstantTime);
JavaRDD<WriteStatus> writeResponse = (JavaRDD<WriteStatus>) client.compact(cfg.compactionInstantTime);
return UtilHelpers.handleErrors(jsc, cfg.compactionInstantTime, writeResponse);
}
private int doSchedule(JavaSparkContext jsc) throws Exception {
// Get schema.
HoodieWriteClient client =
SparkRDDWriteClient client =
UtilHelpers.createHoodieClient(jsc, cfg.basePath, "", cfg.parallelism, Option.of(cfg.strategyClassName), props);
client.scheduleCompactionAtInstant(cfg.compactionInstantTime, Option.empty());
return 0;

View File

@@ -18,6 +18,8 @@
package org.apache.hudi.utilities;
import org.apache.hudi.client.common.HoodieEngineContext;
import org.apache.hudi.client.common.HoodieSparkEngineContext;
import org.apache.hudi.common.config.SerializableConfiguration;
import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.model.HoodieBaseFile;
@@ -97,8 +99,10 @@ public class HoodieSnapshotCopier implements Serializable {
fs.delete(new Path(outputDir), true);
}
jsc.setJobGroup(this.getClass().getSimpleName(), "Creating a snapshot");
jsc.parallelize(partitions, partitions.size()).flatMap(partition -> {
HoodieEngineContext context = new HoodieSparkEngineContext(jsc);
context.setJobStatus(this.getClass().getSimpleName(), "Creating a snapshot");
List<Tuple2<String, String>> filesToCopy = context.flatMap(partitions, partition -> {
// Only take latest version files <= latestCommit.
FileSystem fs1 = FSUtils.getFs(baseDir, serConf.newCopy());
List<Tuple2<String, String>> filePaths = new ArrayList<>();
@@ -112,8 +116,10 @@ public class HoodieSnapshotCopier implements Serializable {
filePaths.add(new Tuple2<>(partition, partitionMetaFile.toString()));
}
return filePaths.iterator();
}).foreach(tuple -> {
return filePaths.stream();
}, partitions.size());
context.foreach(filesToCopy, tuple -> {
String partition = tuple._1();
Path sourceFilePath = new Path(tuple._2());
Path toPartitionPath = new Path(outputDir, partition);
@@ -124,8 +130,8 @@ public class HoodieSnapshotCopier implements Serializable {
}
FileUtil.copy(ifs, sourceFilePath, ifs, new Path(toPartitionPath, sourceFilePath.getName()), false,
ifs.getConf());
});
}, filesToCopy.size());
// Also copy the .commit files
LOG.info(String.format("Copying .commit files which are no-late-than %s.", latestCommitTimestamp));
FileStatus[] commitFilesToCopy =

View File

@@ -18,6 +18,8 @@
package org.apache.hudi.utilities;
import org.apache.hudi.client.common.HoodieEngineContext;
import org.apache.hudi.client.common.HoodieSparkEngineContext;
import org.apache.hudi.common.config.SerializableConfiguration;
import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.model.HoodieBaseFile;
@@ -175,7 +177,8 @@ public class HoodieSnapshotExporter {
? defaultPartitioner
: ReflectionUtils.loadClass(cfg.outputPartitioner);
jsc.setJobGroup(this.getClass().getSimpleName(), "Exporting as non-HUDI dataset");
HoodieEngineContext context = new HoodieSparkEngineContext(jsc);
context.setJobStatus(this.getClass().getSimpleName(), "Exporting as non-HUDI dataset");
final BaseFileOnlyView fsView = getBaseFileOnlyView(jsc, cfg);
Iterator<String> exportingFilePaths = jsc
.parallelize(partitions, partitions.size())
@@ -193,14 +196,16 @@ public class HoodieSnapshotExporter {
private void exportAsHudi(JavaSparkContext jsc, Config cfg, List<String> partitions, String latestCommitTimestamp) throws IOException {
final BaseFileOnlyView fsView = getBaseFileOnlyView(jsc, cfg);
final SerializableConfiguration serConf = new SerializableConfiguration(jsc.hadoopConfiguration());
jsc.setJobGroup(this.getClass().getSimpleName(), "Exporting as HUDI dataset");
jsc.parallelize(partitions, partitions.size()).flatMap(partition -> {
final HoodieEngineContext context = new HoodieSparkEngineContext(jsc);
final SerializableConfiguration serConf = context.getHadoopConf();
context.setJobStatus(this.getClass().getSimpleName(), "Exporting as HUDI dataset");
List<Tuple2<String, String>> files = context.flatMap(partitions, partition -> {
// Only take latest version files <= latestCommit.
List<Tuple2<String, String>> filePaths = new ArrayList<>();
Stream<HoodieBaseFile> dataFiles = fsView.getLatestBaseFilesBeforeOrOn(partition, latestCommitTimestamp);
dataFiles.forEach(hoodieDataFile -> filePaths.add(new Tuple2<>(partition, hoodieDataFile.getPath())));
// also need to copy over partition metadata
Path partitionMetaFile =
new Path(new Path(cfg.sourceBasePath, partition), HoodiePartitionMetadata.HOODIE_PARTITION_METAFILE);
@@ -208,9 +213,10 @@ public class HoodieSnapshotExporter {
if (fs.exists(partitionMetaFile)) {
filePaths.add(new Tuple2<>(partition, partitionMetaFile.toString()));
}
return filePaths.stream();
}, partitions.size());
return filePaths.iterator();
}).foreach(tuple -> {
context.foreach(files, tuple -> {
String partition = tuple._1();
Path sourceFilePath = new Path(tuple._2());
Path toPartitionPath = new Path(cfg.targetOutputPath, partition);
@@ -221,7 +227,7 @@ public class HoodieSnapshotExporter {
}
FileUtil.copy(fs, sourceFilePath, fs, new Path(toPartitionPath, sourceFilePath.getName()), false,
fs.getConf());
});
}, files.size());
// Also copy the .commit files
LOG.info(String.format("Copying .commit files which are no-late-than %s.", latestCommitTimestamp));

View File

@@ -18,6 +18,8 @@
package org.apache.hudi.utilities;
import org.apache.hudi.client.common.HoodieEngineContext;
import org.apache.hudi.client.common.HoodieSparkEngineContext;
import org.apache.hudi.common.util.ValidationUtils;
import com.beust.jcommander.JCommander;
@@ -86,8 +88,10 @@ public class HoodieWithTimelineServer implements Serializable {
System.out.println("Driver Hostname is :" + driverHost);
List<String> messages = new ArrayList<>();
IntStream.range(0, cfg.numPartitions).forEach(i -> messages.add("Hello World"));
jsc.setJobGroup(this.getClass().getSimpleName(), "Sending requests to driver host");
List<String> gotMessages = jsc.parallelize(messages).map(msg -> sendRequest(driverHost, cfg.serverPort)).collect();
HoodieEngineContext context = new HoodieSparkEngineContext(jsc);
context.setJobStatus(this.getClass().getSimpleName(), "Sending requests to driver host");
List<String> gotMessages = context.map(messages, msg -> sendRequest(driverHost, cfg.serverPort), messages.size());
System.out.println("Got Messages :" + gotMessages);
ValidationUtils.checkArgument(gotMessages.equals(messages), "Got expected reply from Server");
}

View File

@@ -20,8 +20,9 @@ package org.apache.hudi.utilities;
import org.apache.hadoop.conf.Configuration;
import org.apache.hudi.AvroConversionUtils;
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.config.DFSPropertiesConfiguration;
import org.apache.hudi.common.config.TypedProperties;
import org.apache.hudi.common.util.Option;
@@ -230,7 +231,7 @@ public class UtilHelpers {
sparkConf.set("spark.hadoop.mapred.output.compression.type", "BLOCK");
additionalConfigs.forEach(sparkConf::set);
return HoodieWriteClient.registerClasses(sparkConf);
return SparkRDDWriteClient.registerClasses(sparkConf);
}
public static JavaSparkContext buildSparkContext(String appName, String defaultMaster, Map<String, String> configs) {
@@ -260,8 +261,8 @@ public class UtilHelpers {
* @param schemaStr Schema
* @param parallelism Parallelism
*/
public static HoodieWriteClient createHoodieClient(JavaSparkContext jsc, String basePath, String schemaStr,
int parallelism, Option<String> compactionStrategyClass, TypedProperties properties) {
public static SparkRDDWriteClient createHoodieClient(JavaSparkContext jsc, String basePath, String schemaStr,
int parallelism, Option<String> compactionStrategyClass, TypedProperties properties) {
HoodieCompactionConfig compactionConfig = compactionStrategyClass
.map(strategy -> HoodieCompactionConfig.newBuilder().withInlineCompaction(false)
.withCompactionStrategy(ReflectionUtils.loadClass(strategy)).build())
@@ -274,7 +275,7 @@ public class UtilHelpers {
.withSchema(schemaStr).combineInput(true, true).withCompactionConfig(compactionConfig)
.withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build())
.withProps(properties).build();
return new HoodieWriteClient(jsc, config);
return new SparkRDDWriteClient(new HoodieSparkEngineContext(jsc), config);
}
public static int handleErrors(JavaSparkContext jsc, String instantTime, JavaRDD<WriteStatus> writeResponse) {

View File

@@ -20,7 +20,8 @@ package org.apache.hudi.utilities.deltastreamer;
import org.apache.hudi.DataSourceUtils;
import org.apache.hudi.DataSourceWriteOptions;
import org.apache.hudi.client.HoodieWriteClient;
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.HoodieTableType;
import org.apache.hudi.common.table.HoodieTableConfig;
@@ -137,7 +138,7 @@ public class BootstrapExecutor implements Serializable {
*/
public void execute() throws IOException {
initializeTable();
HoodieWriteClient bootstrapClient = new HoodieWriteClient(jssc, bootstrapConfig, true);
SparkRDDWriteClient bootstrapClient = new SparkRDDWriteClient(new HoodieSparkEngineContext(jssc), bootstrapConfig, true);
try {
HashMap<String, String> checkpointCommitMetadata = new HashMap<>();

View File

@@ -21,8 +21,9 @@ package org.apache.hudi.utilities.deltastreamer;
import org.apache.hudi.AvroConversionUtils;
import org.apache.hudi.DataSourceUtils;
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.config.TypedProperties;
import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.model.HoodieCommitMetadata;
@@ -158,7 +159,7 @@ public class DeltaSync implements Serializable {
/**
* Callback when write client is instantiated.
*/
private transient Function<HoodieWriteClient, Boolean> onInitializingHoodieWriteClient;
private transient Function<SparkRDDWriteClient, Boolean> onInitializingHoodieWriteClient;
/**
* Timeline with completed commits.
@@ -168,13 +169,13 @@ public class DeltaSync implements Serializable {
/**
* Write Client.
*/
private transient HoodieWriteClient writeClient;
private transient SparkRDDWriteClient writeClient;
private transient HoodieDeltaStreamerMetrics metrics;
public DeltaSync(HoodieDeltaStreamer.Config cfg, SparkSession sparkSession, SchemaProvider schemaProvider,
TypedProperties props, JavaSparkContext jssc, FileSystem fs, Configuration conf,
Function<HoodieWriteClient, Boolean> onInitializingHoodieWriteClient) throws IOException {
Function<SparkRDDWriteClient, Boolean> onInitializingHoodieWriteClient) throws IOException {
this.cfg = cfg;
this.jssc = jssc;
@@ -543,7 +544,7 @@ public class DeltaSync implements Serializable {
if ((null != schemaProvider) && (null == writeClient)) {
registerAvroSchemas(schemaProvider);
HoodieWriteConfig hoodieCfg = getHoodieClientConfig(schemaProvider);
writeClient = new HoodieWriteClient<>(jssc, hoodieCfg, true);
writeClient = new SparkRDDWriteClient<>(new HoodieSparkEngineContext(jssc), hoodieCfg, true);
onInitializingHoodieWriteClient.apply(writeClient);
}
}

View File

@@ -18,10 +18,12 @@
package org.apache.hudi.utilities.deltastreamer;
import org.apache.hudi.async.AbstractAsyncService;
import org.apache.hudi.async.HoodieAsyncService;
import org.apache.hudi.async.AsyncCompactService;
import org.apache.hudi.client.HoodieWriteClient;
import org.apache.hudi.async.SparkAsyncCompactService;
import org.apache.hudi.client.SparkRDDWriteClient;
import org.apache.hudi.client.WriteStatus;
import org.apache.hudi.client.common.HoodieSparkEngineContext;
import org.apache.hudi.common.bootstrap.index.HFileBootstrapIndex;
import org.apache.hudi.common.config.TypedProperties;
import org.apache.hudi.common.fs.FSUtils;
@@ -470,7 +472,7 @@ public class HoodieDeltaStreamer implements Serializable {
/**
* Syncs data either in single-run or in continuous mode.
*/
public static class DeltaSyncService extends AbstractAsyncService {
public static class DeltaSyncService extends HoodieAsyncService {
private static final long serialVersionUID = 1L;
/**
@@ -620,9 +622,9 @@ public class HoodieDeltaStreamer implements Serializable {
* @param writeClient HoodieWriteClient
* @return
*/
protected Boolean onInitializingWriteClient(HoodieWriteClient writeClient) {
protected Boolean onInitializingWriteClient(SparkRDDWriteClient writeClient) {
if (cfg.isAsyncCompactionEnabled()) {
asyncCompactService = new AsyncCompactService(jssc, writeClient);
asyncCompactService = new SparkAsyncCompactService(new HoodieSparkEngineContext(jssc), writeClient);
// Enqueue existing pending compactions first
HoodieTableMetaClient meta =
new HoodieTableMetaClient(new Configuration(jssc.hadoopConfiguration()), cfg.targetBasePath, true);

View File

@@ -18,6 +18,8 @@
package org.apache.hudi.utilities.perf;
import org.apache.hudi.client.common.HoodieEngineContext;
import org.apache.hudi.client.common.HoodieSparkEngineContext;
import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.model.FileSlice;
import org.apache.hudi.common.table.HoodieTableMetaClient;
@@ -132,8 +134,9 @@ public class TimelineServerPerf implements Serializable {
public List<PerfStats> runLookups(JavaSparkContext jsc, List<String> partitionPaths, SyncableFileSystemView fsView,
int numIterations, int concurrency) {
jsc.setJobGroup(this.getClass().getSimpleName(), "Lookup all performance stats");
return jsc.parallelize(partitionPaths, cfg.numExecutors).flatMap(p -> {
HoodieEngineContext context = new HoodieSparkEngineContext(jsc);
context.setJobStatus(this.getClass().getSimpleName(), "Lookup all performance stats");
return context.flatMap(partitionPaths, p -> {
ScheduledThreadPoolExecutor executor = new ScheduledThreadPoolExecutor(100);
final List<PerfStats> result = new ArrayList<>();
final List<ScheduledFuture<PerfStats>> futures = new ArrayList<>();
@@ -141,7 +144,7 @@ public class TimelineServerPerf implements Serializable {
String fileId = slices.isEmpty() ? "dummyId"
: slices.get(new Random(Double.doubleToLongBits(Math.random())).nextInt(slices.size())).getFileId();
IntStream.range(0, concurrency).forEach(i -> futures.add(executor.schedule(() -> runOneRound(fsView, p, fileId,
i, numIterations), 0, TimeUnit.NANOSECONDS)));
i, numIterations), 0, TimeUnit.NANOSECONDS)));
futures.forEach(x -> {
try {
result.add(x.get());
@@ -151,8 +154,8 @@ public class TimelineServerPerf implements Serializable {
});
System.out.println("SLICES are=");
slices.forEach(s -> System.out.println("\t\tFileSlice=" + s));
return result.iterator();
}).collect();
return result.stream();
}, cfg.numExecutors);
}
private static PerfStats runOneRound(SyncableFileSystemView fsView, String partition, String fileId, int id,

View File

@@ -18,7 +18,7 @@
package org.apache.hudi.utilities.functional;
import org.apache.hudi.client.HoodieWriteClient;
import org.apache.hudi.client.SparkRDDWriteClient;
import org.apache.hudi.common.model.HoodieAvroPayload;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieTableType;
@@ -85,7 +85,7 @@ public class TestHoodieSnapshotExporter extends FunctionalTestHarness {
// Prepare data as source Hudi dataset
HoodieWriteConfig cfg = getHoodieWriteConfig(sourcePath);
HoodieWriteClient hdfsWriteClient = new HoodieWriteClient(jsc(), cfg);
SparkRDDWriteClient hdfsWriteClient = new SparkRDDWriteClient(context(), cfg);
hdfsWriteClient.startCommitWithTime(COMMIT_TIME);
HoodieTestDataGenerator dataGen = new HoodieTestDataGenerator(new String[] {PARTITION_PATH});
List<HoodieRecord> records = dataGen.generateInserts(COMMIT_TIME, NUM_RECORDS);

View File

@@ -19,6 +19,8 @@
package org.apache.hudi.utilities.testutils;
import java.io.FileInputStream;
import org.apache.hudi.client.common.HoodieSparkEngineContext;
import org.apache.hudi.common.config.TypedProperties;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieTableType;
@@ -84,6 +86,7 @@ public class UtilitiesTestBase {
protected static MiniDFSCluster dfsCluster;
protected static DistributedFileSystem dfs;
protected transient JavaSparkContext jsc = null;
protected transient HoodieSparkEngineContext context = null;
protected transient SparkSession sparkSession = null;
protected transient SQLContext sqlContext;
protected static HiveServer2 hiveServer;
@@ -129,6 +132,7 @@ public class UtilitiesTestBase {
public void setup() throws Exception {
TestDataSource.initDataGen();
jsc = UtilHelpers.buildSparkContext(this.getClass().getName() + "-hoodie", "local[2]");
context = new HoodieSparkEngineContext(jsc);
sqlContext = new SQLContext(jsc);
sparkSession = SparkSession.builder().config(jsc.getConf()).getOrCreate();
}
@@ -139,6 +143,9 @@ public class UtilitiesTestBase {
if (jsc != null) {
jsc.stop();
}
if (context != null) {
context = null;
}
}
/**