[HUDI-3993] Replacing UDF in Bulk Insert w/ RDD transformation (#5470)
This commit is contained in:
@@ -21,6 +21,7 @@ package org.apache.hudi.client.model;
|
||||
import org.apache.spark.sql.catalyst.InternalRow;
|
||||
import org.apache.spark.sql.catalyst.expressions.GenericInternalRow;
|
||||
import org.apache.spark.sql.types.DataTypes;
|
||||
import org.apache.spark.unsafe.types.UTF8String;
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import java.util.ArrayList;
|
||||
@@ -64,7 +65,13 @@ public class TestHoodieInternalRow {
|
||||
Object[] values = getRandomValue(true);
|
||||
|
||||
InternalRow row = new GenericInternalRow(values);
|
||||
HoodieInternalRow hoodieInternalRow = new HoodieInternalRow("commitTime", "commitSeqNo", "recordKey", "partitionPath", "fileName", row);
|
||||
HoodieInternalRow hoodieInternalRow = new HoodieInternalRow(UTF8String.fromString("commitTime"),
|
||||
UTF8String.fromString("commitSeqNo"),
|
||||
UTF8String.fromString("recordKey"),
|
||||
UTF8String.fromString("partitionPath"),
|
||||
UTF8String.fromString("fileName"),
|
||||
row,
|
||||
true);
|
||||
|
||||
assertValues(hoodieInternalRow, "commitTime", "commitSeqNo", "recordKey", "partitionPath",
|
||||
"fileName", values, nullIndices);
|
||||
@@ -74,7 +81,13 @@ public class TestHoodieInternalRow {
|
||||
public void testUpdate() {
|
||||
Object[] values = getRandomValue(true);
|
||||
InternalRow row = new GenericInternalRow(values);
|
||||
HoodieInternalRow hoodieInternalRow = new HoodieInternalRow("commitTime", "commitSeqNo", "recordKey", "partitionPath", "fileName", row);
|
||||
HoodieInternalRow hoodieInternalRow = new HoodieInternalRow(UTF8String.fromString("commitTime"),
|
||||
UTF8String.fromString("commitSeqNo"),
|
||||
UTF8String.fromString("recordKey"),
|
||||
UTF8String.fromString("partitionPath"),
|
||||
UTF8String.fromString("fileName"),
|
||||
row,
|
||||
true);
|
||||
|
||||
hoodieInternalRow.update(0, "commitTime_updated");
|
||||
hoodieInternalRow.update(1, "commitSeqNo_updated");
|
||||
@@ -106,7 +119,13 @@ public class TestHoodieInternalRow {
|
||||
Object[] values = getRandomValue(true);
|
||||
|
||||
InternalRow row = new GenericInternalRow(values);
|
||||
HoodieInternalRow hoodieInternalRow = new HoodieInternalRow("commitTime", "commitSeqNo", "recordKey", "partitionPath", "fileName", row);
|
||||
HoodieInternalRow hoodieInternalRow = new HoodieInternalRow(UTF8String.fromString("commitTime"),
|
||||
UTF8String.fromString("commitSeqNo"),
|
||||
UTF8String.fromString("recordKey"),
|
||||
UTF8String.fromString("partitionPath"),
|
||||
UTF8String.fromString("fileName"),
|
||||
row,
|
||||
true);
|
||||
|
||||
hoodieInternalRow.setNullAt(i);
|
||||
nullIndices.clear();
|
||||
@@ -129,7 +148,13 @@ public class TestHoodieInternalRow {
|
||||
|
||||
Object[] values = getRandomValue(true);
|
||||
InternalRow row = new GenericInternalRow(values);
|
||||
HoodieInternalRow hoodieInternalRow = new HoodieInternalRow("commitTime", "commitSeqNo", "recordKey", "partitionPath", "fileName", row);
|
||||
HoodieInternalRow hoodieInternalRow = new HoodieInternalRow(UTF8String.fromString("commitTime"),
|
||||
UTF8String.fromString("commitSeqNo"),
|
||||
UTF8String.fromString("recordKey"),
|
||||
UTF8String.fromString("partitionPath"),
|
||||
UTF8String.fromString("fileName"),
|
||||
row,
|
||||
true);
|
||||
|
||||
nullIndices.clear();
|
||||
|
||||
@@ -173,7 +198,7 @@ public class TestHoodieInternalRow {
|
||||
}
|
||||
|
||||
private void assertValues(HoodieInternalRow hoodieInternalRow, String commitTime, String commitSeqNo, String recordKey, String partitionPath, String filename, Object[] values,
|
||||
List<Integer> nullIndexes) {
|
||||
List<Integer> nullIndexes) {
|
||||
for (Integer index : nullIndexes) {
|
||||
assertTrue(hoodieInternalRow.isNullAt(index));
|
||||
}
|
||||
|
||||
@@ -23,6 +23,7 @@ import org.apache.hudi.common.config.HoodieMetadataConfig;
|
||||
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.common.util.StringUtils;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieInsertException;
|
||||
import org.apache.hudi.exception.TableNotFoundException;
|
||||
@@ -75,8 +76,9 @@ public class TestHoodieRowCreateHandle extends HoodieClientTestHarness {
|
||||
cleanupResources();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRowCreateHandle() throws Exception {
|
||||
@ParameterizedTest
|
||||
@ValueSource(booleans = { true, false })
|
||||
public void testRowCreateHandle(boolean populateMetaFields) throws Exception {
|
||||
// init config and table
|
||||
HoodieWriteConfig cfg =
|
||||
SparkDatasetTestUtils.getConfigBuilder(basePath, timelineServicePort).build();
|
||||
@@ -93,7 +95,8 @@ public class TestHoodieRowCreateHandle extends HoodieClientTestHarness {
|
||||
String fileId = UUID.randomUUID().toString();
|
||||
String instantTime = "000";
|
||||
|
||||
HoodieRowCreateHandle handle = new HoodieRowCreateHandle(table, cfg, partitionPath, fileId, instantTime, RANDOM.nextInt(100000), RANDOM.nextLong(), RANDOM.nextLong(), SparkDatasetTestUtils.STRUCT_TYPE);
|
||||
HoodieRowCreateHandle handle = new HoodieRowCreateHandle(table, cfg, partitionPath, fileId, instantTime,
|
||||
RANDOM.nextInt(100000), RANDOM.nextLong(), RANDOM.nextLong(), SparkDatasetTestUtils.STRUCT_TYPE, populateMetaFields);
|
||||
int size = 10 + RANDOM.nextInt(1000);
|
||||
// Generate inputs
|
||||
Dataset<Row> inputRows = SparkDatasetTestUtils.getRandomRows(sqlContext, size, partitionPath, false);
|
||||
@@ -109,7 +112,7 @@ public class TestHoodieRowCreateHandle extends HoodieClientTestHarness {
|
||||
fileAbsPaths.add(basePath + "/" + writeStatus.getStat().getPath());
|
||||
fileNames.add(handle.getFileName());
|
||||
// verify output
|
||||
assertOutput(writeStatus, size, fileId, partitionPath, instantTime, totalInputRows, fileNames, fileAbsPaths);
|
||||
assertOutput(writeStatus, size, fileId, partitionPath, instantTime, totalInputRows, fileNames, fileAbsPaths, populateMetaFields);
|
||||
}
|
||||
}
|
||||
|
||||
@@ -130,7 +133,7 @@ public class TestHoodieRowCreateHandle extends HoodieClientTestHarness {
|
||||
String instantTime = "000";
|
||||
|
||||
HoodieRowCreateHandle handle =
|
||||
new HoodieRowCreateHandle(table, cfg, partitionPath, fileId, instantTime, RANDOM.nextInt(100000), RANDOM.nextLong(), RANDOM.nextLong(), SparkDatasetTestUtils.STRUCT_TYPE);
|
||||
new HoodieRowCreateHandle(table, cfg, partitionPath, fileId, instantTime, RANDOM.nextInt(100000), RANDOM.nextLong(), RANDOM.nextLong(), SparkDatasetTestUtils.STRUCT_TYPE, true);
|
||||
int size = 10 + RANDOM.nextInt(1000);
|
||||
int totalFailures = 5;
|
||||
// Generate first batch of valid rows
|
||||
@@ -169,7 +172,7 @@ public class TestHoodieRowCreateHandle extends HoodieClientTestHarness {
|
||||
// verify rows
|
||||
Dataset<Row> result = sqlContext.read().parquet(basePath + "/" + partitionPath);
|
||||
// passing only first batch of inputRows since after first batch global error would have been thrown
|
||||
assertRows(inputRows, result, instantTime, fileNames);
|
||||
assertRows(inputRows, result, instantTime, fileNames, true);
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@@ -183,7 +186,7 @@ public class TestHoodieRowCreateHandle extends HoodieClientTestHarness {
|
||||
|
||||
try {
|
||||
HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient);
|
||||
new HoodieRowCreateHandle(table, cfg, " def", UUID.randomUUID().toString(), "001", RANDOM.nextInt(100000), RANDOM.nextLong(), RANDOM.nextLong(), SparkDatasetTestUtils.STRUCT_TYPE);
|
||||
new HoodieRowCreateHandle(table, cfg, " def", UUID.randomUUID().toString(), "001", RANDOM.nextInt(100000), RANDOM.nextLong(), RANDOM.nextLong(), SparkDatasetTestUtils.STRUCT_TYPE, true);
|
||||
fail("Should have thrown exception");
|
||||
} catch (HoodieInsertException ioe) {
|
||||
// expected without metadata table
|
||||
@@ -209,8 +212,8 @@ public class TestHoodieRowCreateHandle extends HoodieClientTestHarness {
|
||||
return handle.close();
|
||||
}
|
||||
|
||||
private void assertOutput(HoodieInternalWriteStatus writeStatus, int size, String fileId, String partitionPath, String instantTime, Dataset<Row> inputRows, List<String> filenames,
|
||||
List<String> fileAbsPaths) {
|
||||
private void assertOutput(HoodieInternalWriteStatus writeStatus, int size, String fileId, String partitionPath,
|
||||
String instantTime, Dataset<Row> inputRows, List<String> filenames, List<String> fileAbsPaths, boolean populateMetaFields) {
|
||||
assertEquals(writeStatus.getPartitionPath(), partitionPath);
|
||||
assertEquals(writeStatus.getTotalRecords(), size);
|
||||
assertEquals(writeStatus.getFailedRowsSize(), 0);
|
||||
@@ -229,15 +232,25 @@ public class TestHoodieRowCreateHandle extends HoodieClientTestHarness {
|
||||
|
||||
// verify rows
|
||||
Dataset<Row> result = sqlContext.read().parquet(fileAbsPaths.toArray(new String[0]));
|
||||
assertRows(inputRows, result, instantTime, filenames);
|
||||
assertRows(inputRows, result, instantTime, filenames, populateMetaFields);
|
||||
}
|
||||
|
||||
private void assertRows(Dataset<Row> expectedRows, Dataset<Row> actualRows, String instantTime, List<String> filenames) {
|
||||
private void assertRows(Dataset<Row> expectedRows, Dataset<Row> actualRows, String instantTime, List<String> filenames, boolean populateMetaFields) {
|
||||
// verify 3 meta fields that are filled in within create handle
|
||||
actualRows.collectAsList().forEach(entry -> {
|
||||
assertEquals(entry.get(HoodieRecord.HOODIE_META_COLUMNS_NAME_TO_POS.get(HoodieRecord.COMMIT_TIME_METADATA_FIELD)).toString(), instantTime);
|
||||
assertTrue(filenames.contains(entry.get(HoodieRecord.HOODIE_META_COLUMNS_NAME_TO_POS.get(HoodieRecord.FILENAME_METADATA_FIELD)).toString()));
|
||||
assertFalse(entry.isNullAt(HoodieRecord.HOODIE_META_COLUMNS_NAME_TO_POS.get(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD)));
|
||||
String commitTime = entry.getString(HoodieRecord.HOODIE_META_COLUMNS_NAME_TO_POS.get(HoodieRecord.COMMIT_TIME_METADATA_FIELD));
|
||||
String fileName = entry.getString(HoodieRecord.HOODIE_META_COLUMNS_NAME_TO_POS.get(HoodieRecord.FILENAME_METADATA_FIELD));
|
||||
String seqId = entry.getString(HoodieRecord.HOODIE_META_COLUMNS_NAME_TO_POS.get(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD));
|
||||
|
||||
if (populateMetaFields) {
|
||||
assertEquals(instantTime, commitTime);
|
||||
assertFalse(StringUtils.isNullOrEmpty(seqId));
|
||||
assertTrue(filenames.contains(fileName));
|
||||
} else {
|
||||
assertEquals("", commitTime);
|
||||
assertEquals("", seqId);
|
||||
assertEquals("", fileName);
|
||||
}
|
||||
});
|
||||
|
||||
// after trimming 2 of the meta fields, rest of the fields should match
|
||||
|
||||
@@ -0,0 +1,166 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.spark.sql
|
||||
|
||||
import org.apache.spark.sql.HoodieUnsafeRowUtils.{composeNestedFieldPath, getNestedInternalRowValue, getNestedRowValue}
|
||||
import org.apache.spark.sql.catalyst.InternalRow
|
||||
import org.apache.spark.sql.types._
|
||||
import org.junit.jupiter.api.Assertions.{assertEquals, fail}
|
||||
import org.junit.jupiter.api.Test
|
||||
|
||||
class TestHoodieUnsafeRowUtils {
|
||||
|
||||
@Test
|
||||
def testComposeNestedFieldPath(): Unit = {
|
||||
val schema = StructType(Seq(
|
||||
StructField("foo", StringType),
|
||||
StructField(
|
||||
name = "bar",
|
||||
dataType = StructType(Seq(
|
||||
StructField("baz", DateType),
|
||||
StructField("bor", LongType)
|
||||
))
|
||||
)
|
||||
))
|
||||
|
||||
assertEquals(
|
||||
Seq((1, schema(1)), (0, schema(1).dataType.asInstanceOf[StructType](0))),
|
||||
composeNestedFieldPath(schema, "bar.baz").toSeq)
|
||||
|
||||
assertThrows(classOf[IllegalArgumentException]) { () =>
|
||||
composeNestedFieldPath(schema, "foo.baz")
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
def testGetNestedInternalRowValue(): Unit = {
|
||||
val schema = StructType(Seq(
|
||||
StructField("foo", StringType, nullable = false),
|
||||
StructField(
|
||||
name = "bar",
|
||||
dataType = StructType(Seq(
|
||||
StructField("baz", DateType),
|
||||
StructField("bor", LongType)
|
||||
))
|
||||
)
|
||||
))
|
||||
|
||||
val row = InternalRow("str", InternalRow(123, 456L))
|
||||
|
||||
assertEquals(
|
||||
123,
|
||||
getNestedInternalRowValue(row, composeNestedFieldPath(schema, "bar.baz"))
|
||||
)
|
||||
assertEquals(
|
||||
456L,
|
||||
getNestedInternalRowValue(row, composeNestedFieldPath(schema, "bar.bor"))
|
||||
)
|
||||
assertEquals(
|
||||
"str",
|
||||
getNestedInternalRowValue(row, composeNestedFieldPath(schema, "foo"))
|
||||
)
|
||||
assertEquals(
|
||||
row.getStruct(1, 2),
|
||||
getNestedInternalRowValue(row, composeNestedFieldPath(schema, "bar"))
|
||||
)
|
||||
|
||||
val rowProperNullable = InternalRow("str", null)
|
||||
|
||||
assertEquals(
|
||||
null,
|
||||
getNestedInternalRowValue(rowProperNullable, composeNestedFieldPath(schema, "bar.baz"))
|
||||
)
|
||||
assertEquals(
|
||||
null,
|
||||
getNestedInternalRowValue(rowProperNullable, composeNestedFieldPath(schema, "bar"))
|
||||
)
|
||||
|
||||
val rowInvalidNullable = InternalRow(null, InternalRow(123, 456L))
|
||||
|
||||
assertThrows(classOf[IllegalArgumentException]) { () =>
|
||||
getNestedInternalRowValue(rowInvalidNullable, composeNestedFieldPath(schema, "foo"))
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
def testGetNestedRowValue(): Unit = {
|
||||
val schema = StructType(Seq(
|
||||
StructField("foo", StringType, nullable = false),
|
||||
StructField(
|
||||
name = "bar",
|
||||
dataType = StructType(Seq(
|
||||
StructField("baz", DateType),
|
||||
StructField("bor", LongType)
|
||||
))
|
||||
)
|
||||
))
|
||||
|
||||
val row = Row("str", Row(123, 456L))
|
||||
|
||||
assertEquals(
|
||||
123,
|
||||
getNestedRowValue(row, composeNestedFieldPath(schema, "bar.baz"))
|
||||
)
|
||||
assertEquals(
|
||||
456L,
|
||||
getNestedRowValue(row, composeNestedFieldPath(schema, "bar.bor"))
|
||||
)
|
||||
assertEquals(
|
||||
"str",
|
||||
getNestedRowValue(row, composeNestedFieldPath(schema, "foo"))
|
||||
)
|
||||
assertEquals(
|
||||
row.getStruct(1),
|
||||
getNestedRowValue(row, composeNestedFieldPath(schema, "bar"))
|
||||
)
|
||||
|
||||
val rowProperNullable = Row("str", null)
|
||||
|
||||
assertEquals(
|
||||
null,
|
||||
getNestedRowValue(rowProperNullable, composeNestedFieldPath(schema, "bar.baz"))
|
||||
)
|
||||
assertEquals(
|
||||
null,
|
||||
getNestedRowValue(rowProperNullable, composeNestedFieldPath(schema, "bar"))
|
||||
)
|
||||
|
||||
val rowInvalidNullable = Row(null, Row(123, 456L))
|
||||
|
||||
assertThrows(classOf[IllegalArgumentException]) { () =>
|
||||
getNestedRowValue(rowInvalidNullable, composeNestedFieldPath(schema, "foo"))
|
||||
}
|
||||
}
|
||||
|
||||
private def assertThrows[T <: Throwable](expectedExceptionClass: Class[T])(f: () => Unit): T = {
|
||||
try {
|
||||
f.apply()
|
||||
} catch {
|
||||
case t: Throwable if expectedExceptionClass.isAssignableFrom(t.getClass) =>
|
||||
// scalastyle:off return
|
||||
return t.asInstanceOf[T]
|
||||
// scalastyle:on return
|
||||
case ot @ _ =>
|
||||
fail(s"Expected exception of class $expectedExceptionClass, but ${ot.getClass} has been thrown")
|
||||
}
|
||||
|
||||
fail(s"Expected exception of class $expectedExceptionClass, but nothing has been thrown")
|
||||
}
|
||||
|
||||
}
|
||||
Reference in New Issue
Block a user