Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -17,13 +17,15 @@

package org.apache.hudi

import org.apache.spark.SPARK_VERSION
import org.apache.avro.generic.{GenericRecord, GenericRecordBuilder, IndexedRecord}
import org.apache.hudi.common.model.HoodieKey
import org.apache.avro.Schema
import org.apache.hudi.avro.HoodieAvroUtils
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.avro.SchemaConverters
import org.apache.spark.sql.catalyst.encoders.RowEncoder
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.encoders.{ExpressionEncoder, RowEncoder}
import org.apache.spark.sql.types._
import org.apache.spark.sql.{DataFrame, Dataset, Row, SparkSession}

Expand All @@ -41,7 +43,7 @@ object AvroConversionUtils {
// Use the Avro schema to derive the StructType which has the correct nullability information
val dataType = SchemaConverters.toSqlType(avroSchema).dataType.asInstanceOf[StructType]
val encoder = RowEncoder.apply(dataType).resolveAndBind()
df.queryExecution.toRdd.map(encoder.fromRow)
df.queryExecution.toRdd.map[Row](internalRow => deserializeRow(encoder, internalRow))
.mapPartitions { records =>
if (records.isEmpty) Iterator.empty
else {
Expand Down Expand Up @@ -96,4 +98,15 @@ object AvroConversionUtils {
val name = HoodieAvroUtils.sanitizeName(tableName)
(s"${name}_record", s"hoodie.${name}")
}

private def deserializeRow(encoder: ExpressionEncoder[Row], internalRow: InternalRow): Row = {
if (SPARK_VERSION.startsWith("2.")) {
val spark2method = encoder.getClass.getMethod("fromRow", classOf[InternalRow])
spark2method.invoke(encoder, internalRow).asInstanceOf[Row]
} else {
val deserializer = encoder.getClass.getMethod("createDeserializer").invoke(encoder)
val aboveSpark2method = deserializer.getClass.getMethod("apply", classOf[InternalRow])
aboveSpark2method.invoke(deserializer, internalRow).asInstanceOf[Row]
}
}
}
6 changes: 2 additions & 4 deletions hudi-spark/src/test/java/HoodieJavaStreamingApp.java
Original file line number Diff line number Diff line change
Expand Up @@ -43,13 +43,12 @@
import org.apache.spark.sql.SparkSession;
import org.apache.spark.sql.streaming.DataStreamWriter;
import org.apache.spark.sql.streaming.OutputMode;
import org.apache.spark.sql.streaming.ProcessingTime;
import org.apache.spark.sql.streaming.Trigger;

import java.util.List;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.Future;
import org.apache.spark.sql.streaming.StreamingQuery;

import static org.apache.hudi.common.testutils.RawTripTestPayload.recordsToStrings;

Expand Down Expand Up @@ -363,8 +362,7 @@ public void stream(Dataset<Row> streamingInput, String operationType, String che
.outputMode(OutputMode.Append());

updateHiveSyncConfig(writer);
StreamingQuery query = writer.trigger(new ProcessingTime(500)).start(tablePath);
query.awaitTermination(streamingDurationInMs);
writer.trigger(Trigger.ProcessingTime(500)).start(tablePath).awaitTermination(streamingDurationInMs);
}

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -47,7 +47,6 @@
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;
Expand All @@ -60,6 +59,7 @@
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;
Expand Down Expand Up @@ -254,7 +254,7 @@ public static HoodieWriteClient createHoodieClient(JavaSparkContext jsc, String
}

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);
Expand Down