1
0

[HUDI-3338] Custom relation instead of HadoopFsRelation (#4709)

Currently, HadoopFsRelation will use the value of the real partition path as the value of the partition field. However, different from the normal table, Hudi will persist the partition value in the parquet file. And in some cases, it's different between the value of the real partition path and the value of the partition field.
So here we implement BaseFileOnlyViewRelation which lets Hudi manage its own relation.
This commit is contained in:
Yann Byron
2022-02-12 02:48:44 +08:00
committed by GitHub
parent 10474e0962
commit b431246710
20 changed files with 685 additions and 235 deletions

View File

@@ -18,18 +18,24 @@
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.testutils.DataSourceTestUtils
import org.apache.spark.sql.types.StructType
import org.apache.spark.sql.types.{StructType, TimestampType}
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 java.io.File
import java.nio.file.Paths
import scala.collection.JavaConverters
class TestHoodieSparkUtils {
@@ -232,6 +238,29 @@ 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

@@ -1,50 +0,0 @@
/*
* 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.spark.sql.types.TimestampType
import org.junit.jupiter.api.Assertions.assertEquals
import org.junit.jupiter.api.Test
class TestMergeOnReadSnapshotRelation {
@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) =
MergeOnReadSnapshotRelation.getRequiredSchema(tableAvroSchema, Array("ts"))
assertEquals("timestamp-millis",
requiredAvroSchema.getField("ts").schema().getTypes.get(1).getLogicalType.getName)
assertEquals(TimestampType, requiredStructSchema.fields(0).dataType)
}
}

View File

@@ -833,4 +833,48 @@ class TestCOWDataSource extends HoodieClientTestBase {
assertEquals(inputDF2.sort("_row_key").select("shortDecimal").collect().map(_.getDecimal(0).toPlainString).mkString(","),
readResult.sort("_row_key").select("shortDecimal").collect().map(_.getDecimal(0).toPlainString).mkString(","))
}
@Test
def testHoodieBaseFileOnlyViewRelation(): Unit = {
val _spark = spark
import _spark.implicits._
val df = Seq((1, "z3", 30, "v1", "2018-09-23"), (2, "z3", 35, "v1", "2018-09-24"))
.toDF("id", "name", "age", "ts", "data_date")
df.write.format("hudi")
.options(commonOpts)
.option(DataSourceWriteOptions.RECORDKEY_FIELD.key, "id")
.option(DataSourceWriteOptions.RECORDKEY_FIELD.key, "id")
.option(DataSourceWriteOptions.RECORDKEY_FIELD.key, "id")
.option("hoodie.insert.shuffle.parallelism", "4")
.option("hoodie.upsert.shuffle.parallelism", "4")
.option("hoodie.bulkinsert.shuffle.parallelism", "2")
.option(DataSourceWriteOptions.RECORDKEY_FIELD.key, "id")
.option(DataSourceWriteOptions.PARTITIONPATH_FIELD.key, "data_date")
.option(DataSourceWriteOptions.PRECOMBINE_FIELD.key, "ts")
.option(DataSourceWriteOptions.KEYGENERATOR_CLASS_NAME.key, "org.apache.hudi.keygen.TimestampBasedKeyGenerator")
.option(TimestampBasedAvroKeyGenerator.Config.TIMESTAMP_TYPE_FIELD_PROP, "DATE_STRING")
.option(TimestampBasedAvroKeyGenerator.Config.TIMESTAMP_OUTPUT_DATE_FORMAT_PROP, "yyyy/MM/dd")
.option(TimestampBasedAvroKeyGenerator.Config.TIMESTAMP_TIMEZONE_FORMAT_PROP, "GMT+8:00")
.option(TimestampBasedAvroKeyGenerator.Config.TIMESTAMP_INPUT_DATE_FORMAT_PROP, "yyyy-MM-dd")
.mode(org.apache.spark.sql.SaveMode.Append)
.save(basePath)
val res = spark.read.format("hudi").load(basePath)
assert(res.count() == 2)
// data_date is the partition field. Persist to the parquet file using the origin values, and read it.
assertTrue(
res.select("data_date").map(_.get(0).toString).collect().sorted.sameElements(
Array("2018-09-23", "2018-09-24")
)
)
assertTrue(
res.select("_hoodie_partition_path").map(_.get(0).toString).collect().sorted.sameElements(
Array("2018/09/23", "2018/09/24")
)
)
}
}