1
0

[HUDI-3896] Porting Nested Schema Pruning optimization for Hudi's custom Relations (#5428)

Currently, all Hudi Relations bear performance gap relative to Spark's HadoopFsRelation 
and the reason to that is SchemaPruning optimization rule (pruning nested schemas) 
that is unfortunately predicated on usage of HadoopFsRelation, meaning that it's 
not applied in cases when any other relation is used.

This change is porting this rule to Hudi relations (MOR, Incremental, etc) 
by the virtue of leveraging HoodieSparkSessionExtensions mechanism 
injecting modified version of the original SchemaPruning rule 
that is adopted to work w/ Hudi's custom relations.

- Added customOptimizerRules to HoodieAnalysis
- Added NestedSchemaPrunning Spark's Optimizer rule
- Handle Spark's Optimizer pruned data schema (to effectively prune nested schemas)
- Enable HoodieClientTestHarness to inject HoodieSparkSessionExtensions
- Injecting Spark Session extensions for TestMORDataSource, TestCOWDataSource
- Disabled fallback to HadoopFsRelation
This commit is contained in:
Alexey Kudinkin
2022-07-21 02:36:06 -07:00
committed by GitHub
parent 2394c62973
commit de37774e12
42 changed files with 1220 additions and 500 deletions

View File

@@ -57,7 +57,7 @@ case class IndexRow(fileName: String,
class TestDataSkippingUtils extends HoodieClientTestBase with SparkAdapterSupport {
val exprUtils: HoodieCatalystExpressionUtils = sparkAdapter.createCatalystExpressionUtils()
val exprUtils: HoodieCatalystExpressionUtils = sparkAdapter.getCatalystExpressionUtils()
var spark: SparkSession = _

View File

@@ -0,0 +1,56 @@
/*
* 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.hudi
import org.apache.avro.Schema
import org.apache.hudi.AvroConversionUtils.convertAvroSchemaToStructType
import org.junit.jupiter.api.Assertions.assertEquals
import org.junit.jupiter.api.Test
import scala.collection.JavaConverters.asScalaBufferConverter
class TestHoodieRelations {
@Test
def testPruningSchema(): Unit = {
val avroSchemaString = "{\"type\":\"record\",\"name\":\"record\"," +
"\"fields\":[{\"name\":\"_hoodie_commit_time\",\"type\":[\"null\",\"string\"],\"doc\":\"\",\"default\":null}," +
"{\"name\":\"_hoodie_commit_seqno\",\"type\":[\"null\",\"string\"],\"doc\":\"\",\"default\":null}," +
"{\"name\":\"_hoodie_record_key\",\"type\":[\"null\",\"string\"],\"doc\":\"\",\"default\":null}," +
"{\"name\":\"_hoodie_partition_path\",\"type\":[\"null\",\"string\"],\"doc\":\"\",\"default\":null}," +
"{\"name\":\"_hoodie_file_name\",\"type\":[\"null\",\"string\"],\"doc\":\"\",\"default\":null}," +
"{\"name\":\"uuid\",\"type\":\"string\"},{\"name\":\"name\",\"type\":[\"null\",\"string\"],\"default\":null}," +
"{\"name\":\"age\",\"type\":[\"null\",\"int\"],\"default\":null}," +
"{\"name\":\"ts\",\"type\":[\"null\",{\"type\":\"long\",\"logicalType\":\"timestamp-millis\"}],\"default\":null}," +
"{\"name\":\"partition\",\"type\":[\"null\",\"string\"],\"default\":null}]}"
val tableAvroSchema = new Schema.Parser().parse(avroSchemaString)
val tableStructSchema = convertAvroSchemaToStructType(tableAvroSchema)
val (requiredAvroSchema, requiredStructSchema, _) =
HoodieBaseRelation.projectSchema(Left(tableAvroSchema), Array("ts"))
assertEquals(Seq(tableAvroSchema.getField("ts")), requiredAvroSchema.getFields.asScala)
assertEquals(
Seq(tableStructSchema.fields.apply(tableStructSchema.fieldIndex("ts"))),
requiredStructSchema.fields.toSeq
)
}
}

View File

@@ -18,17 +18,17 @@
package org.apache.hudi
import org.apache.avro.Schema
import org.apache.avro.generic.GenericRecord
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.Path
import org.apache.hudi.exception.SchemaCompatibilityException
import org.apache.hudi.testutils.DataSourceTestUtils
import org.apache.spark.sql.types.{StructType, TimestampType}
import org.apache.spark.sql.types.StructType
import org.apache.spark.sql.{Row, SparkSession}
import org.junit.jupiter.api.Assertions._
import org.junit.jupiter.api.Test
import org.junit.jupiter.api.io.TempDir
import org.junit.jupiter.params.ParameterizedTest
import org.junit.jupiter.params.provider.ValueSource
import java.io.File
import java.nio.file.Paths
@@ -36,6 +36,60 @@ import scala.collection.JavaConverters
class TestHoodieSparkUtils {
@ParameterizedTest
@ValueSource(strings = Array("2.4.4", "3.1.0", "3.2.0", "3.3.0"))
def testSparkVersionCheckers(sparkVersion: String): Unit = {
val vsMock = new SparkVersionsSupport {
override def getSparkVersion: String = sparkVersion
}
sparkVersion match {
case "2.4.4" =>
assertTrue(vsMock.isSpark2)
assertFalse(vsMock.isSpark3)
assertFalse(vsMock.isSpark3_1)
assertFalse(vsMock.isSpark3_0)
assertFalse(vsMock.isSpark3_2)
assertFalse(vsMock.gteqSpark3_1)
assertFalse(vsMock.gteqSpark3_1_3)
assertFalse(vsMock.gteqSpark3_2)
case "3.1.0" =>
assertTrue(vsMock.isSpark3)
assertTrue(vsMock.isSpark3_1)
assertTrue(vsMock.gteqSpark3_1)
assertFalse(vsMock.isSpark2)
assertFalse(vsMock.isSpark3_0)
assertFalse(vsMock.isSpark3_2)
assertFalse(vsMock.gteqSpark3_1_3)
assertFalse(vsMock.gteqSpark3_2)
case "3.2.0" =>
assertTrue(vsMock.isSpark3)
assertTrue(vsMock.isSpark3_2)
assertTrue(vsMock.gteqSpark3_1)
assertTrue(vsMock.gteqSpark3_1_3)
assertTrue(vsMock.gteqSpark3_2)
assertFalse(vsMock.isSpark2)
assertFalse(vsMock.isSpark3_0)
assertFalse(vsMock.isSpark3_1)
case "3.3.0" =>
assertTrue(vsMock.isSpark3)
assertTrue(vsMock.gteqSpark3_1)
assertTrue(vsMock.gteqSpark3_1_3)
assertTrue(vsMock.gteqSpark3_2)
assertFalse(vsMock.isSpark3_2)
assertFalse(vsMock.isSpark2)
assertFalse(vsMock.isSpark3_0)
assertFalse(vsMock.isSpark3_1)
}
}
@Test
def testGlobPaths(@TempDir tempDir: File): Unit = {
val folders: Seq[Path] = Seq(
@@ -204,29 +258,6 @@ class TestHoodieSparkUtils {
spark.stop()
}
@Test
def testGetRequiredSchema(): Unit = {
val avroSchemaString = "{\"type\":\"record\",\"name\":\"record\"," +
"\"fields\":[{\"name\":\"_hoodie_commit_time\",\"type\":[\"null\",\"string\"],\"doc\":\"\",\"default\":null}," +
"{\"name\":\"_hoodie_commit_seqno\",\"type\":[\"null\",\"string\"],\"doc\":\"\",\"default\":null}," +
"{\"name\":\"_hoodie_record_key\",\"type\":[\"null\",\"string\"],\"doc\":\"\",\"default\":null}," +
"{\"name\":\"_hoodie_partition_path\",\"type\":[\"null\",\"string\"],\"doc\":\"\",\"default\":null}," +
"{\"name\":\"_hoodie_file_name\",\"type\":[\"null\",\"string\"],\"doc\":\"\",\"default\":null}," +
"{\"name\":\"uuid\",\"type\":\"string\"},{\"name\":\"name\",\"type\":[\"null\",\"string\"],\"default\":null}," +
"{\"name\":\"age\",\"type\":[\"null\",\"int\"],\"default\":null}," +
"{\"name\":\"ts\",\"type\":[\"null\",{\"type\":\"long\",\"logicalType\":\"timestamp-millis\"}],\"default\":null}," +
"{\"name\":\"partition\",\"type\":[\"null\",\"string\"],\"default\":null}]}"
val tableAvroSchema = new Schema.Parser().parse(avroSchemaString)
val (requiredAvroSchema, requiredStructSchema, _) =
HoodieSparkUtils.getRequiredSchema(tableAvroSchema, Array("ts"))
assertEquals("timestamp-millis",
requiredAvroSchema.getField("ts").schema().getTypes.get(1).getLogicalType.getName)
assertEquals(TimestampType, requiredStructSchema.fields(0).dataType)
}
def convertRowListToSeq(inputList: java.util.List[Row]): Seq[Row] =
JavaConverters.asScalaIteratorConverter(inputList.iterator).asScala.toSeq
}

View File

@@ -19,27 +19,31 @@ package org.apache.hudi.functional
import org.apache.hadoop.fs.Path
import org.apache.hudi.DataSourceWriteOptions._
import org.apache.hudi.HoodieConversionUtils.toJavaOption
import org.apache.hudi.common.config.HoodieMetadataConfig
import org.apache.hudi.common.model.{DefaultHoodieRecordPayload, HoodieTableType}
import org.apache.hudi.common.table.HoodieTableMetaClient
import org.apache.hudi.common.testutils.HoodieTestDataGenerator
import org.apache.hudi.common.testutils.RawTripTestPayload.recordsToStrings
import org.apache.hudi.common.util
import org.apache.hudi.config.{HoodieIndexConfig, HoodieWriteConfig}
import org.apache.hudi.index.HoodieIndex.IndexType
import org.apache.hudi.keygen.NonpartitionedKeyGenerator
import org.apache.hudi.keygen.constant.KeyGeneratorOptions.Config
import org.apache.hudi.testutils.{DataSourceTestUtils, HoodieClientTestBase}
import org.apache.hudi.util.JFunction
import org.apache.hudi.{DataSourceReadOptions, DataSourceWriteOptions, HoodieDataSourceHelpers, SparkDatasetMixin}
import org.apache.log4j.LogManager
import org.apache.spark.sql._
import org.apache.spark.sql.functions._
import org.apache.spark.sql.hudi.HoodieSparkSessionExtension
import org.apache.spark.sql.types.BooleanType
import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue}
import org.junit.jupiter.api.{AfterEach, BeforeEach, Test}
import org.junit.jupiter.params.ParameterizedTest
import org.junit.jupiter.params.provider.CsvSource
import scala.collection.JavaConversions._
import java.util.function.Consumer
import scala.collection.JavaConverters._
/**
@@ -76,11 +80,17 @@ class TestMORDataSource extends HoodieClientTestBase with SparkDatasetMixin {
cleanupFileSystem()
}
override def getSparkSessionExtensionsInjector: util.Option[Consumer[SparkSessionExtensions]] =
toJavaOption(
Some(
JFunction.toJava((receiver: SparkSessionExtensions) => new HoodieSparkSessionExtension().apply(receiver)))
)
@Test def testCount() {
// First Operation:
// Producing parquet files to three default partitions.
// SNAPSHOT view on MOR table with parquet files only.
val records1 = recordsToStrings(dataGen.generateInserts("001", 100)).toList
val records1 = recordsToStrings(dataGen.generateInserts("001", 100)).asScala
val inputDF1 = spark.read.json(spark.sparkContext.parallelize(records1, 2))
inputDF1.write.format("org.apache.hudi")
.options(commonOpts)
@@ -98,7 +108,7 @@ class TestMORDataSource extends HoodieClientTestBase with SparkDatasetMixin {
// Second Operation:
// Upsert the update to the default partitions with duplicate records. Produced a log file for each parquet.
// SNAPSHOT view should read the log files only with the latest commit time.
val records2 = recordsToStrings(dataGen.generateUniqueUpdates("002", 100)).toList
val records2 = recordsToStrings(dataGen.generateUniqueUpdates("002", 100)).asScala
val inputDF2: Dataset[Row] = spark.read.json(spark.sparkContext.parallelize(records2, 2))
inputDF2.write.format("org.apache.hudi")
.options(commonOpts)
@@ -173,7 +183,7 @@ class TestMORDataSource extends HoodieClientTestBase with SparkDatasetMixin {
// Third Operation:
// Upsert another update to the default partitions with 50 duplicate records. Produced the second log file for each parquet.
// SNAPSHOT view should read the latest log files.
val records3 = recordsToStrings(dataGen.generateUniqueUpdates("003", 50)).toList
val records3 = recordsToStrings(dataGen.generateUniqueUpdates("003", 50)).asScala
val inputDF3: Dataset[Row] = spark.read.json(spark.sparkContext.parallelize(records3, 2))
inputDF3.write.format("org.apache.hudi")
.options(commonOpts)
@@ -213,7 +223,7 @@ class TestMORDataSource extends HoodieClientTestBase with SparkDatasetMixin {
val partitionPaths = new Array[String](1)
partitionPaths.update(0, "2020/01/10")
val newDataGen = new HoodieTestDataGenerator(partitionPaths)
val records4 = recordsToStrings(newDataGen.generateInserts("004", 100)).toList
val records4 = recordsToStrings(newDataGen.generateInserts("004", 100)).asScala
val inputDF4: Dataset[Row] = spark.read.json(spark.sparkContext.parallelize(records4, 2))
inputDF4.write.format("org.apache.hudi")
.options(commonOpts)
@@ -238,7 +248,7 @@ class TestMORDataSource extends HoodieClientTestBase with SparkDatasetMixin {
// Upsert records to the new partition. Produced a newer version of parquet file.
// SNAPSHOT view should read the latest log files from the default partition
// and the latest parquet from the new partition.
val records5 = recordsToStrings(newDataGen.generateUniqueUpdates("005", 50)).toList
val records5 = recordsToStrings(newDataGen.generateUniqueUpdates("005", 50)).asScala
val inputDF5: Dataset[Row] = spark.read.json(spark.sparkContext.parallelize(records5, 2))
inputDF5.write.format("org.apache.hudi")
.options(commonOpts)
@@ -252,7 +262,7 @@ class TestMORDataSource extends HoodieClientTestBase with SparkDatasetMixin {
// Sixth Operation:
// Insert 2 records and trigger compaction.
val records6 = recordsToStrings(newDataGen.generateInserts("006", 2)).toList
val records6 = recordsToStrings(newDataGen.generateInserts("006", 2)).asScala
val inputDF6: Dataset[Row] = spark.read.json(spark.sparkContext.parallelize(records6, 2))
inputDF6.write.format("org.apache.hudi")
.options(commonOpts)
@@ -279,7 +289,7 @@ class TestMORDataSource extends HoodieClientTestBase with SparkDatasetMixin {
// First Operation:
// Producing parquet files to three default partitions.
// SNAPSHOT view on MOR table with parquet files only.
val records1 = recordsToStrings(dataGen.generateInserts("001", 100)).toList
val records1 = recordsToStrings(dataGen.generateInserts("001", 100)).asScala
val inputDF1 = spark.read.json(spark.sparkContext.parallelize(records1, 2))
inputDF1.write.format("org.apache.hudi")
.options(commonOpts)
@@ -297,7 +307,7 @@ class TestMORDataSource extends HoodieClientTestBase with SparkDatasetMixin {
// Second Operation:
// Upsert 50 delete records
// Snopshot view should only read 50 records
val records2 = recordsToStrings(dataGen.generateUniqueDeleteRecords("002", 50)).toList
val records2 = recordsToStrings(dataGen.generateUniqueDeleteRecords("002", 50)).asScala
val inputDF2: Dataset[Row] = spark.read.json(spark.sparkContext.parallelize(records2, 2))
inputDF2.write.format("org.apache.hudi")
.options(commonOpts)
@@ -330,7 +340,7 @@ class TestMORDataSource extends HoodieClientTestBase with SparkDatasetMixin {
// Third Operation:
// Upsert 50 delete records to delete the reset
// Snopshot view should read 0 record
val records3 = recordsToStrings(dataGen.generateUniqueDeleteRecords("003", 50)).toList
val records3 = recordsToStrings(dataGen.generateUniqueDeleteRecords("003", 50)).asScala
val inputDF3: Dataset[Row] = spark.read.json(spark.sparkContext.parallelize(records3, 2))
inputDF3.write.format("org.apache.hudi")
.options(commonOpts)
@@ -447,7 +457,7 @@ class TestMORDataSource extends HoodieClientTestBase with SparkDatasetMixin {
// Vectorized Reader will only be triggered with AtomicType schema,
// which is not null, UDTs, arrays, structs, and maps.
val schema = HoodieTestDataGenerator.SHORT_TRIP_SCHEMA
val records1 = recordsToStrings(dataGen.generateInsertsAsPerSchema("001", 100, schema)).toList
val records1 = recordsToStrings(dataGen.generateInsertsAsPerSchema("001", 100, schema)).asScala
val inputDF1 = spark.read.json(spark.sparkContext.parallelize(records1, 2))
inputDF1.write.format("org.apache.hudi")
.options(commonOpts)
@@ -462,7 +472,7 @@ class TestMORDataSource extends HoodieClientTestBase with SparkDatasetMixin {
assertEquals(100, hudiSnapshotDF1.count())
val records2 = recordsToStrings(dataGen.generateUniqueUpdatesAsPerSchema("002", 50, schema))
.toList
.asScala
val inputDF2: Dataset[Row] = spark.read.json(spark.sparkContext.parallelize(records2, 2))
inputDF2.write.format("org.apache.hudi")
.options(commonOpts)
@@ -488,7 +498,7 @@ class TestMORDataSource extends HoodieClientTestBase with SparkDatasetMixin {
@Test def testNoPrecombine() {
// Insert Operation
val records = recordsToStrings(dataGen.generateInserts("000", 100)).toList
val records = recordsToStrings(dataGen.generateInserts("000", 100)).asScala
val inputDF = spark.read.json(spark.sparkContext.parallelize(records, 2))
val commonOptsNoPreCombine = Map(
@@ -594,7 +604,7 @@ class TestMORDataSource extends HoodieClientTestBase with SparkDatasetMixin {
val N = 20
// Test query with partition prune if URL_ENCODE_PARTITIONING has enable
val records1 = dataGen.generateInsertsContainsAllPartitions("000", N)
val inputDF1 = spark.read.json(spark.sparkContext.parallelize(recordsToStrings(records1), 2))
val inputDF1 = spark.read.json(spark.sparkContext.parallelize(recordsToStrings(records1).asScala, 2))
inputDF1.write.format("hudi")
.options(commonOpts)
.option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
@@ -624,7 +634,7 @@ class TestMORDataSource extends HoodieClientTestBase with SparkDatasetMixin {
// Second write with Append mode
val records2 = dataGen.generateInsertsContainsAllPartitions("000", N + 1)
val inputDF2 = spark.read.json(spark.sparkContext.parallelize(recordsToStrings(records2), 2))
val inputDF2 = spark.read.json(spark.sparkContext.parallelize(recordsToStrings(records2).asScala, 2))
inputDF2.write.format("hudi")
.options(commonOpts)
.option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
@@ -646,8 +656,8 @@ class TestMORDataSource extends HoodieClientTestBase with SparkDatasetMixin {
def testMORPartitionPrune(partitionEncode: Boolean, hiveStylePartition: Boolean): Unit = {
val partitions = Array("2021/03/01", "2021/03/02", "2021/03/03", "2021/03/04", "2021/03/05")
val newDataGen = new HoodieTestDataGenerator(partitions)
val records1 = newDataGen.generateInsertsContainsAllPartitions("000", 100)
val inputDF1 = spark.read.json(spark.sparkContext.parallelize(recordsToStrings(records1), 2))
val records1 = newDataGen.generateInsertsContainsAllPartitions("000", 100).asScala
val inputDF1 = spark.read.json(spark.sparkContext.parallelize(recordsToStrings(records1.asJava).asScala, 2))
val partitionCounts = partitions.map(p => p -> records1.count(r => r.getPartitionPath == p)).toMap
@@ -676,7 +686,7 @@ class TestMORDataSource extends HoodieClientTestBase with SparkDatasetMixin {
.load(basePath)
.filter("partition != '2021/03/01'")
.count()
assertEquals(records1.size() - partitionCounts("2021/03/01"), count3)
assertEquals(records1.size - partitionCounts("2021/03/01"), count3)
val count4 = spark.read.format("hudi")
.load(basePath)
@@ -688,7 +698,7 @@ class TestMORDataSource extends HoodieClientTestBase with SparkDatasetMixin {
.load(basePath)
.filter("partition like '%2021/03/%'")
.count()
assertEquals(records1.size(), count5)
assertEquals(records1.size, count5)
val count6 = spark.read.format("hudi")
.load(basePath)
@@ -708,7 +718,7 @@ class TestMORDataSource extends HoodieClientTestBase with SparkDatasetMixin {
def testReadPathsForMergeOnReadTable(): Unit = {
// Paths only baseFiles
val records1 = dataGen.generateInserts("001", 100)
val inputDF1 = spark.read.json(spark.sparkContext.parallelize(recordsToStrings(records1), 2))
val inputDF1 = spark.read.json(spark.sparkContext.parallelize(recordsToStrings(records1).asScala, 2))
inputDF1.write.format("org.apache.hudi")
.options(commonOpts)
.option("hoodie.compact.inline", "false") // else fails due to compaction & deltacommit instant times being same
@@ -722,7 +732,7 @@ class TestMORDataSource extends HoodieClientTestBase with SparkDatasetMixin {
.map(_.getPath.toString)
.mkString(",")
val records2 = dataGen.generateUniqueDeleteRecords("002", 100)
val inputDF2: Dataset[Row] = spark.read.json(spark.sparkContext.parallelize(recordsToStrings(records2), 2))
val inputDF2: Dataset[Row] = spark.read.json(spark.sparkContext.parallelize(recordsToStrings(records2).asScala, 2))
inputDF2.write.format("org.apache.hudi")
.options(commonOpts)
.mode(SaveMode.Append)
@@ -754,7 +764,7 @@ class TestMORDataSource extends HoodieClientTestBase with SparkDatasetMixin {
def testReadPathsForOnlyLogFiles(): Unit = {
initMetaClient(HoodieTableType.MERGE_ON_READ)
val records1 = dataGen.generateInsertsContainsAllPartitions("000", 20)
val inputDF1 = spark.read.json(spark.sparkContext.parallelize(recordsToStrings(records1), 2))
val inputDF1 = spark.read.json(spark.sparkContext.parallelize(recordsToStrings(records1).asScala, 2))
inputDF1.write.format("hudi")
.options(commonOpts)
.option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
@@ -772,7 +782,7 @@ class TestMORDataSource extends HoodieClientTestBase with SparkDatasetMixin {
.mkString(",")
val records2 = dataGen.generateInsertsContainsAllPartitions("000", 20)
val inputDF2 = spark.read.json(spark.sparkContext.parallelize(recordsToStrings(records2), 2))
val inputDF2 = spark.read.json(spark.sparkContext.parallelize(recordsToStrings(records2).asScala, 2))
inputDF2.write.format("hudi")
.options(commonOpts)
.option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
@@ -798,7 +808,7 @@ class TestMORDataSource extends HoodieClientTestBase with SparkDatasetMixin {
def testReadLogOnlyMergeOnReadTable(): Unit = {
initMetaClient(HoodieTableType.MERGE_ON_READ)
val records1 = dataGen.generateInsertsContainsAllPartitions("000", 20)
val inputDF = spark.read.json(spark.sparkContext.parallelize(recordsToStrings(records1), 2))
val inputDF = spark.read.json(spark.sparkContext.parallelize(recordsToStrings(records1).asScala, 2))
inputDF.write.format("hudi")
.options(commonOpts)
.option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
@@ -817,7 +827,7 @@ class TestMORDataSource extends HoodieClientTestBase with SparkDatasetMixin {
@Test
def testTempFilesCleanForClustering(): Unit = {
val records1 = recordsToStrings(dataGen.generateInserts("001", 1000)).toList
val records1 = recordsToStrings(dataGen.generateInserts("001", 1000)).asScala
val inputDF1: Dataset[Row] = spark.read.json(spark.sparkContext.parallelize(records1, 2))
inputDF1.write.format("org.apache.hudi")
.options(commonOpts)
@@ -835,7 +845,7 @@ class TestMORDataSource extends HoodieClientTestBase with SparkDatasetMixin {
@Test
def testClusteringOnNullableColumn(): Unit = {
val records1 = recordsToStrings(dataGen.generateInserts("001", 1000)).toList
val records1 = recordsToStrings(dataGen.generateInserts("001", 1000)).asScala
val inputDF1: Dataset[Row] = spark.read.json(spark.sparkContext.parallelize(records1, 2))
.withColumn("cluster_id", when(expr("end_lon < 0.2 "), lit(null).cast("string"))
.otherwise(col("_row_key")))
@@ -860,7 +870,7 @@ class TestMORDataSource extends HoodieClientTestBase with SparkDatasetMixin {
val numRecords = 100
val numRecordsToDelete = 2
val schema = HoodieTestDataGenerator.SHORT_TRIP_SCHEMA
val records0 = recordsToStrings(dataGen.generateInsertsAsPerSchema("000", numRecords, schema)).toList
val records0 = recordsToStrings(dataGen.generateInsertsAsPerSchema("000", numRecords, schema)).asScala
val inputDF0 = spark.read.json(spark.sparkContext.parallelize(records0, 2))
inputDF0.write.format("org.apache.hudi")
.options(commonOpts)
@@ -911,7 +921,7 @@ class TestMORDataSource extends HoodieClientTestBase with SparkDatasetMixin {
)
val dataGen1 = new HoodieTestDataGenerator(Array("2022-01-01"))
val records1 = recordsToStrings(dataGen1.generateInserts("001", 50)).toList
val records1 = recordsToStrings(dataGen1.generateInserts("001", 50)).asScala
val inputDF1 = spark.read.json(spark.sparkContext.parallelize(records1, 2))
inputDF1.write.format("org.apache.hudi")
.options(options)
@@ -924,7 +934,7 @@ class TestMORDataSource extends HoodieClientTestBase with SparkDatasetMixin {
val commit1Time = metaClient.getActiveTimeline.lastInstant().get().getTimestamp
val dataGen2 = new HoodieTestDataGenerator(Array("2022-01-02"))
val records2 = recordsToStrings(dataGen2.generateInserts("002", 60)).toList
val records2 = recordsToStrings(dataGen2.generateInserts("002", 60)).asScala
val inputDF2 = spark.read.json(spark.sparkContext.parallelize(records2, 2))
inputDF2.write.format("org.apache.hudi")
.options(options)
@@ -932,7 +942,7 @@ class TestMORDataSource extends HoodieClientTestBase with SparkDatasetMixin {
.save(basePath)
val commit2Time = metaClient.reloadActiveTimeline.lastInstant().get().getTimestamp
val records3 = recordsToStrings(dataGen2.generateUniqueUpdates("003", 20)).toList
val records3 = recordsToStrings(dataGen2.generateUniqueUpdates("003", 20)).asScala
val inputDF3 = spark.read.json(spark.sparkContext.parallelize(records3, 2))
inputDF3.write.format("org.apache.hudi")
.options(options)

View File

@@ -18,6 +18,7 @@
package org.apache.hudi.functional
import org.apache.avro.Schema
import org.apache.hudi.HoodieBaseRelation.projectSchema
import org.apache.hudi.common.config.HoodieMetadataConfig
import org.apache.hudi.common.model.{HoodieRecord, OverwriteNonDefaultsWithLatestAvroPayload}
import org.apache.hudi.common.table.HoodieTableConfig
@@ -333,7 +334,7 @@ class TestParquetColumnProjection extends SparkClientFunctionalTestHarness with
}
val readColumns = targetColumns ++ relation.mandatoryFields
val (_, projectedStructType, _) = HoodieSparkUtils.getRequiredSchema(tableState.schema, readColumns)
val (_, projectedStructType, _) = projectSchema(Left(tableState.schema), readColumns)
val row: InternalRow = rows.take(1).head

View File

@@ -0,0 +1,61 @@
/*
* 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.avro
import org.apache.avro.generic.GenericData
import org.apache.hudi.SparkAdapterSupport
import org.apache.hudi.avro.model.{HoodieMetadataColumnStats, IntWrapper}
import org.apache.spark.sql.avro.SchemaConverters.SchemaType
import org.junit.jupiter.api.Assertions.assertEquals
import org.junit.jupiter.api.Test
class TestAvroSerDe extends SparkAdapterSupport {
@Test
def testAvroUnionSerDe(): Unit = {
val originalAvroRecord = {
val minValue = new GenericData.Record(IntWrapper.SCHEMA$)
minValue.put("value", 9)
val maxValue = new GenericData.Record(IntWrapper.SCHEMA$)
maxValue.put("value", 10)
val record = new GenericData.Record(HoodieMetadataColumnStats.SCHEMA$)
record.put("fileName", "9388c460-4ace-4274-9a0b-d44606af60af-0_2-25-35_20220520154514641.parquet")
record.put("columnName", "c8")
record.put("minValue", minValue)
record.put("maxValue", maxValue)
record.put("valueCount", 10L)
record.put("nullCount", 0L)
record.put("totalSize", 94L)
record.put("totalUncompressedSize", 54L)
record.put("isDeleted", false)
record
}
val avroSchema = HoodieMetadataColumnStats.SCHEMA$
val SchemaType(catalystSchema, _) = SchemaConverters.toSqlType(avroSchema)
val deserializer = sparkAdapter.createAvroDeserializer(avroSchema, catalystSchema)
val serializer = sparkAdapter.createAvroSerializer(catalystSchema, avroSchema, nullable = false)
val row = deserializer.deserialize(originalAvroRecord).get
val deserializedAvroRecord = serializer.serialize(row)
assertEquals(originalAvroRecord, deserializedAvroRecord)
}
}

View File

@@ -0,0 +1,40 @@
/*
* 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.avro
import org.apache.hudi.avro.model.HoodieMetadataColumnStats
import org.apache.spark.sql.avro.SchemaConverters.SchemaType
import org.junit.Test
import org.junit.jupiter.api.Assertions.assertEquals
class TestSchemaConverters {
@Test
def testAvroUnionConversion(): Unit = {
val originalAvroSchema = HoodieMetadataColumnStats.SCHEMA$
val SchemaType(convertedStructType, _) = SchemaConverters.toSqlType(originalAvroSchema)
val convertedAvroSchema = SchemaConverters.toAvroType(convertedStructType)
// NOTE: Here we're validating that converting Avro -> Catalyst and Catalyst -> Avro are inverse
// transformations, but since it's not an easy endeavor to match Avro scehams, we match
// derived Catalyst schemas instead
assertEquals(convertedStructType, SchemaConverters.toSqlType(convertedAvroSchema).dataType)
}
}

View File

@@ -0,0 +1,121 @@
/*
* 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.hudi
import org.apache.hudi.{HoodieSparkUtils, SparkAdapterSupport}
import org.apache.spark.sql.DataFrame
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
import org.apache.spark.sql.execution.{FileSourceScanExec, ProjectExec, RowDataSourceScanExec, SparkPlan}
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.types.{IntegerType, StringType, StructField, StructType}
import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue}
class TestNestedSchemaPruningOptimization extends HoodieSparkSqlTestBase with SparkAdapterSupport {
private def explain(plan: LogicalPlan): String = {
val explainCommand = sparkAdapter.getCatalystPlanUtils.createExplainCommand(plan, extended = true)
executePlan(explainCommand)
.executeCollect()
.mkString("\n")
}
private def executePlan(plan: LogicalPlan): SparkPlan =
spark.sessionState.executePlan(plan).executedPlan
test("Test NestedSchemaPruning optimization (COW/MOR)") {
withTempDir { tmp =>
// NOTE: This tests are only relevant for Spark >= 3.1
// TODO extract tests into a separate spark-version-specific module
if (HoodieSparkUtils.gteqSpark3_1) {
Seq("cow", "mor").foreach { tableType =>
val tableName = generateTableName
val tablePath = s"${tmp.getCanonicalPath}/$tableName"
spark.sql(
s"""
|CREATE TABLE $tableName (
| id int,
| item STRUCT<name: string, price: double>,
| ts long
|) USING HUDI TBLPROPERTIES (
| type = '$tableType',
| primaryKey = 'id',
| preCombineField = 'ts',
| hoodie.populate.meta.fields = 'false'
|)
|LOCATION '$tablePath'
""".stripMargin)
spark.sql(
s"""
|INSERT INTO $tableName
|SELECT 1 AS id, named_struct('name', 'a1', 'price', 10) AS item, 123456 AS ts
""".stripMargin)
val selectDF = spark.sql(s"SELECT id, item.name FROM $tableName")
val expectedSchema = StructType(Seq(
StructField("id", IntegerType),
StructField("item" , StructType(Seq(StructField("name", StringType))))
))
spark.sessionState.conf.setConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED, false)
val expectedReadSchemaClause = "ReadSchema: struct<id:int,item:struct<name:string>>"
val hint =
s"""
|Following is expected to be present in the plan (where ReadSchema has properly pruned nested structs, which
|is an optimization performed by NestedSchemaPruning rule):
|
|== Physical Plan ==
|*(1) Project [id#45, item#46.name AS name#55]
|+- FileScan parquet default.h0[id#45,item#46] Batched: false, DataFilters: [], Format: Parquet, Location: HoodieFileIndex(1 paths)[file:/private/var/folders/kb/cnff55vj041g2nnlzs5ylqk00000gn/T/spark-7137..., PartitionFilters: [], PushedFilters: [], $expectedReadSchemaClause
|]
|""".stripMargin
val executedPlan = executePlan(selectDF.logicalPlan)
// NOTE: Unfortunately, we can't use pattern-matching to extract required fields, due to a need to maintain
// compatibility w/ Spark 2.4
executedPlan match {
// COW
case ProjectExec(_, fileScan: FileSourceScanExec) =>
val tableIdentifier = fileScan.tableIdentifier
val requiredSchema = fileScan.requiredSchema
assertEquals(tableName, tableIdentifier.get.table)
assertEquals(expectedSchema, requiredSchema, hint)
// MOR
case ProjectExec(_, dataScan: RowDataSourceScanExec) =>
// NOTE: This is temporary solution to assert for Spark 2.4, until it's deprecated
val explainedPlan = explain(selectDF.queryExecution.logical)
assertTrue(explainedPlan.contains(expectedReadSchemaClause))
// TODO replace w/ after Spark 2.4 deprecation
//val tableIdentifier = dataScan.tableIdentifier
//val requiredSchema = dataScan.requiredSchema
//
//assertEquals(tableName, tableIdentifier.get.table)
//assertEquals(expectedSchema, requiredSchema, hint)
}
}
}
}
}
}