1
0

[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:
wenningd
2020-12-09 15:52:23 -08:00
committed by GitHub
parent 3a91d26d62
commit fce1453fa6
79 changed files with 1040 additions and 172 deletions

View File

@@ -72,7 +72,7 @@ public class TestHoodieRowCreateHandle extends HoodieClientTestHarness {
}
@Test
public void testRowCreateHandle() throws IOException {
public void testRowCreateHandle() throws Exception {
// init config and table
HoodieWriteConfig cfg = SparkDatasetTestUtils.getConfigBuilder(basePath).build();
HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient);
@@ -113,7 +113,7 @@ public class TestHoodieRowCreateHandle extends HoodieClientTestHarness {
* should be thrown.
*/
@Test
public void testGlobalFailure() throws IOException {
public void testGlobalFailure() throws Exception {
// init config and table
HoodieWriteConfig cfg = SparkDatasetTestUtils.getConfigBuilder(basePath).build();
HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient);
@@ -179,7 +179,8 @@ public class TestHoodieRowCreateHandle extends HoodieClientTestHarness {
}
}
private HoodieInternalWriteStatus writeAndGetWriteStatus(Dataset<Row> inputRows, HoodieRowCreateHandle handle) throws IOException {
private HoodieInternalWriteStatus writeAndGetWriteStatus(Dataset<Row> inputRows, HoodieRowCreateHandle handle)
throws Exception {
List<InternalRow> internalRows = SparkDatasetTestUtils.toInternalRows(inputRows, SparkDatasetTestUtils.ENCODER);
// issue writes
for (InternalRow internalRow : internalRows) {

View File

@@ -35,7 +35,6 @@ import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import java.io.IOException;
import java.util.List;
import java.util.Random;
import java.util.UUID;
@@ -64,7 +63,7 @@ public class TestHoodieInternalRowParquetWriter extends HoodieClientTestHarness
}
@Test
public void endToEndTest() throws IOException {
public void endToEndTest() throws Exception {
HoodieWriteConfig cfg = SparkDatasetTestUtils.getConfigBuilder(basePath).build();
for (int i = 0; i < 5; i++) {
// init write support and parquet config

View File

@@ -43,7 +43,6 @@ import org.apache.hadoop.mapred.RecordReader;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import java.util.stream.Collectors;
@@ -84,36 +83,32 @@ public class HoodieMergeOnReadTestUtils {
.map(f -> new Schema.Field(f.name(), f.schema(), f.doc(), f.defaultVal()))
.collect(Collectors.toList()));
return inputPaths.stream().map(path -> {
setInputPath(jobConf, path);
List<GenericRecord> records = new ArrayList<>();
try {
List<InputSplit> splits = Arrays.asList(inputFormat.getSplits(jobConf, 1));
for (InputSplit split : splits) {
RecordReader recordReader = inputFormat.getRecordReader(split, jobConf, null);
Object key = recordReader.createKey();
ArrayWritable writable = (ArrayWritable) recordReader.createValue();
while (recordReader.next(key, writable)) {
GenericRecordBuilder newRecord = new GenericRecordBuilder(projectedSchema);
// writable returns an array with [field1, field2, _hoodie_commit_time,
// _hoodie_commit_seqno]
Writable[] values = writable.get();
schema.getFields().stream()
.filter(f -> !projectCols || projectedColumns.contains(f.name()))
.map(f -> Pair.of(projectedSchema.getFields().stream()
.filter(p -> f.name().equals(p.name())).findFirst().get(), f))
.forEach(fieldsPair -> newRecord.set(fieldsPair.getKey(), values[fieldsPair.getValue().pos()]));
records.add(newRecord.build());
}
List<GenericRecord> records = new ArrayList<>();
try {
FileInputFormat.setInputPaths(jobConf, String.join(",", inputPaths));
InputSplit[] splits = inputFormat.getSplits(jobConf, inputPaths.size());
for (InputSplit split : splits) {
RecordReader recordReader = inputFormat.getRecordReader(split, jobConf, null);
Object key = recordReader.createKey();
ArrayWritable writable = (ArrayWritable) recordReader.createValue();
while (recordReader.next(key, writable)) {
GenericRecordBuilder newRecord = new GenericRecordBuilder(projectedSchema);
// writable returns an array with [field1, field2, _hoodie_commit_time,
// _hoodie_commit_seqno]
Writable[] values = writable.get();
schema.getFields().stream()
.filter(f -> !projectCols || projectedColumns.contains(f.name()))
.map(f -> Pair.of(projectedSchema.getFields().stream()
.filter(p -> f.name().equals(p.name())).findFirst().get(), f))
.forEach(fieldsPair -> newRecord.set(fieldsPair.getKey(), values[fieldsPair.getValue().pos()]));
records.add(newRecord.build());
}
} catch (IOException ie) {
ie.printStackTrace();
}
return records;
}).reduce((a, b) -> {
a.addAll(b);
return a;
}).orElse(new ArrayList<>());
} catch (IOException ie) {
ie.printStackTrace();
}
return records;
}
private static void setPropsForInputFormat(FileInputFormat inputFormat, JobConf jobConf, Schema schema, String hiveColumnTypes, boolean projectCols, List<String> projectedCols) {
@@ -156,10 +151,4 @@ public class HoodieMergeOnReadTestUtils {
configurable.setConf(conf);
jobConf.addResource(conf);
}
private static void setInputPath(JobConf jobConf, String inputPath) {
jobConf.set("mapreduce.input.fileinputformat.inputdir", inputPath);
jobConf.set("mapreduce.input.fileinputformat.inputdir", inputPath);
jobConf.set("map.input.dir", inputPath);
}
}

View File

@@ -26,6 +26,7 @@ import org.apache.hudi.config.HoodieStorageConfig;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.index.HoodieIndex;
import org.apache.spark.package$;
import org.apache.spark.sql.Dataset;
import org.apache.spark.sql.Row;
import org.apache.spark.sql.SQLContext;
@@ -41,6 +42,8 @@ import org.apache.spark.sql.types.Metadata;
import org.apache.spark.sql.types.StructField;
import org.apache.spark.sql.types.StructType;
import java.lang.reflect.InvocationTargetException;
import java.lang.reflect.Method;
import java.util.ArrayList;
import java.util.List;
import java.util.UUID;
@@ -139,11 +142,11 @@ public class SparkDatasetTestUtils {
* @param rows Dataset<Row>s to be converted
* @return the List of {@link InternalRow}s thus converted.
*/
public static List<InternalRow> toInternalRows(Dataset<Row> rows, ExpressionEncoder encoder) {
public static List<InternalRow> toInternalRows(Dataset<Row> rows, ExpressionEncoder encoder) throws Exception {
List<InternalRow> toReturn = new ArrayList<>();
List<Row> rowList = rows.collectAsList();
for (Row row : rowList) {
toReturn.add(encoder.toRow(row).copy());
toReturn.add(serializeRow(encoder, row).copy());
}
return toReturn;
}
@@ -173,4 +176,17 @@ public class SparkDatasetTestUtils {
.withBulkInsertParallelism(2);
}
private static InternalRow serializeRow(ExpressionEncoder encoder, Row row)
throws InvocationTargetException, IllegalAccessException, NoSuchMethodException, ClassNotFoundException {
// TODO remove reflection if Spark 2.x support is dropped
if (package$.MODULE$.SPARK_VERSION().startsWith("2.")) {
Method spark2method = encoder.getClass().getMethod("toRow", Object.class);
return (InternalRow) spark2method.invoke(encoder, row);
} else {
Class<?> serializerClass = Class.forName("org.apache.spark.sql.catalyst.encoders.ExpressionEncoder$Serializer");
Object serializer = encoder.getClass().getMethod("createSerializer").invoke(encoder);
Method aboveSpark2method = serializerClass.getMethod("apply", Object.class);
return (InternalRow) aboveSpark2method.invoke(serializer, row);
}
}
}