[HUDI-1040] Make Hudi support Spark 3 (#2208)
* Fix flaky MOR unit test * Update Spark APIs to make it be compatible with both spark2 & spark3 * Refactor bulk insert v2 part to make Hudi be able to compile with Spark3 * Add spark3 profile to handle fasterxml & spark version * Create hudi-spark-common module & refactor hudi-spark related modules Co-authored-by: Wenning Ding <wenningd@amazon.com>
This commit is contained in:
@@ -55,7 +55,6 @@ import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.Accumulator;
|
||||
import org.apache.spark.SparkConf;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
@@ -68,6 +67,7 @@ import org.apache.spark.sql.execution.datasources.jdbc.JdbcUtils;
|
||||
import org.apache.spark.sql.jdbc.JdbcDialect;
|
||||
import org.apache.spark.sql.jdbc.JdbcDialects;
|
||||
import org.apache.spark.sql.types.StructType;
|
||||
import org.apache.spark.util.LongAccumulator;
|
||||
|
||||
import java.io.BufferedReader;
|
||||
import java.io.IOException;
|
||||
@@ -292,7 +292,7 @@ public class UtilHelpers {
|
||||
}
|
||||
|
||||
public static int handleErrors(JavaSparkContext jsc, String instantTime, JavaRDD<WriteStatus> writeResponse) {
|
||||
Accumulator<Integer> errors = jsc.accumulator(0);
|
||||
LongAccumulator errors = jsc.sc().longAccumulator();
|
||||
writeResponse.foreach(writeStatus -> {
|
||||
if (writeStatus.hasErrors()) {
|
||||
errors.add(1);
|
||||
|
||||
@@ -18,8 +18,8 @@
|
||||
|
||||
package org.apache.hudi.utilities.deltastreamer;
|
||||
|
||||
import org.apache.hudi.AvroConversionUtils;
|
||||
import org.apache.hudi.DataSourceUtils;
|
||||
import org.apache.hudi.HoodieSparkUtils;
|
||||
import org.apache.hudi.avro.HoodieAvroUtils;
|
||||
import org.apache.hudi.client.SparkRDDWriteClient;
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
@@ -342,7 +342,7 @@ public class DeltaSync implements Serializable {
|
||||
// pass in the schema for the Row-to-Avro conversion
|
||||
// to avoid nullability mismatch between Avro schema and Row schema
|
||||
avroRDDOptional = transformed
|
||||
.map(t -> AvroConversionUtils.createRdd(
|
||||
.map(t -> HoodieSparkUtils.createRdd(
|
||||
t, this.userProvidedSchemaProvider.getTargetSchema(),
|
||||
HOODIE_RECORD_STRUCT_NAME, HOODIE_RECORD_NAMESPACE).toJavaRDD());
|
||||
schemaProvider = this.userProvidedSchemaProvider;
|
||||
@@ -356,7 +356,7 @@ public class DeltaSync implements Serializable {
|
||||
UtilHelpers.createRowBasedSchemaProvider(r.schema(), props, jssc)))
|
||||
.orElse(dataAndCheckpoint.getSchemaProvider());
|
||||
avroRDDOptional = transformed
|
||||
.map(t -> AvroConversionUtils.createRdd(
|
||||
.map(t -> HoodieSparkUtils.createRdd(
|
||||
t, HOODIE_RECORD_STRUCT_NAME, HOODIE_RECORD_NAMESPACE).toJavaRDD());
|
||||
}
|
||||
} else {
|
||||
|
||||
@@ -19,6 +19,7 @@
|
||||
package org.apache.hudi.utilities.deltastreamer;
|
||||
|
||||
import org.apache.hudi.AvroConversionUtils;
|
||||
import org.apache.hudi.HoodieSparkUtils;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.utilities.UtilHelpers;
|
||||
import org.apache.hudi.utilities.schema.FilebasedSchemaProvider;
|
||||
@@ -73,8 +74,8 @@ public final class SourceFormatAdapter {
|
||||
// If the source schema is specified through Avro schema,
|
||||
// pass in the schema for the Row-to-Avro conversion
|
||||
// to avoid nullability mismatch between Avro schema and Row schema
|
||||
? AvroConversionUtils.createRdd(rdd, r.getSchemaProvider().getSourceSchema(),
|
||||
HOODIE_RECORD_STRUCT_NAME, HOODIE_RECORD_NAMESPACE).toJavaRDD() : AvroConversionUtils.createRdd(rdd,
|
||||
? HoodieSparkUtils.createRdd(rdd, r.getSchemaProvider().getSourceSchema(),
|
||||
HOODIE_RECORD_STRUCT_NAME, HOODIE_RECORD_NAMESPACE).toJavaRDD() : HoodieSparkUtils.createRdd(rdd,
|
||||
HOODIE_RECORD_STRUCT_NAME, HOODIE_RECORD_NAMESPACE).toJavaRDD();
|
||||
})
|
||||
.orElse(null)), r.getCheckpointForNextBatch(), r.getSchemaProvider());
|
||||
|
||||
Reference in New Issue
Block a user