[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:
@@ -0,0 +1,58 @@
|
||||
/*
|
||||
* 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 java.time.LocalDate
|
||||
|
||||
import org.apache.avro.Schema
|
||||
import org.apache.avro.generic.GenericData
|
||||
import org.apache.spark.sql.catalyst.expressions.GenericRow
|
||||
import org.scalatest.{FunSuite, Matchers}
|
||||
|
||||
class TestAvroConversionHelper extends FunSuite with Matchers {
|
||||
|
||||
val dateSchema = s"""
|
||||
{
|
||||
"namespace": "logical",
|
||||
"type": "record",
|
||||
"name": "test",
|
||||
"fields": [
|
||||
{"name": "date", "type": {"type": "int", "logicalType": "date"}}
|
||||
]
|
||||
}
|
||||
"""
|
||||
|
||||
val dateInputData = Seq(7, 365, 0)
|
||||
|
||||
test("Logical type: date") {
|
||||
val schema = new Schema.Parser().parse(dateSchema)
|
||||
val convertor = AvroConversionHelper.createConverterToRow(schema, AvroConversionUtils.convertAvroSchemaToStructType(schema))
|
||||
|
||||
val dateOutputData = dateInputData.map(x => {
|
||||
val record = new GenericData.Record(schema) {{ put("date", x) }}
|
||||
convertor(record).asInstanceOf[GenericRow].get(0)
|
||||
})
|
||||
|
||||
println(s"trans data from int [ ${dateInputData.mkString(", ")} ] to date [ ${dateOutputData.mkString(", ")} ]")
|
||||
|
||||
assert(dateOutputData(0).toString === LocalDate.ofEpochDay(dateInputData(0)).toString)
|
||||
assert(dateOutputData(1).toString === LocalDate.ofEpochDay(dateInputData(1)).toString)
|
||||
assert(dateOutputData(2).toString === LocalDate.ofEpochDay(dateInputData(2)).toString)
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,581 @@
|
||||
/*
|
||||
* 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.generic.GenericRecord
|
||||
import org.apache.hudi.avro.HoodieAvroUtils
|
||||
import org.apache.hudi.common.config.TypedProperties
|
||||
import org.apache.hudi.common.model.{EmptyHoodieRecordPayload, HoodieKey, OverwriteWithLatestAvroPayload}
|
||||
import org.apache.hudi.common.testutils.SchemaTestUtil
|
||||
import org.apache.hudi.common.util.Option
|
||||
import org.apache.hudi.exception.{HoodieException, HoodieKeyException}
|
||||
import org.apache.hudi.keygen._
|
||||
import org.apache.hudi.testutils.KeyGeneratorTestUtilities
|
||||
import org.apache.spark.sql.Row
|
||||
import org.junit.jupiter.api.Assertions.assertEquals
|
||||
import org.junit.jupiter.api.{BeforeEach, Test}
|
||||
import org.scalatest.Assertions.fail
|
||||
|
||||
/**
|
||||
* Tests on the default key generator, payload classes.
|
||||
*/
|
||||
class TestDataSourceDefaults {
|
||||
|
||||
val schema = SchemaTestUtil.getComplexEvolvedSchema
|
||||
val structType = AvroConversionUtils.convertAvroSchemaToStructType(schema)
|
||||
var baseRecord: GenericRecord = _
|
||||
var baseRow: Row = _
|
||||
val testStructName = "testStructName"
|
||||
val testNamespace = "testNamespace"
|
||||
|
||||
@BeforeEach def initialize(): Unit = {
|
||||
baseRecord = SchemaTestUtil
|
||||
.generateAvroRecordFromJson(schema, 1, "001", "f1")
|
||||
baseRow = KeyGeneratorTestUtilities.getRow(baseRecord, schema, structType)
|
||||
}
|
||||
|
||||
private def getKeyConfig(recordKeyFieldName: String, partitionPathField: String, hiveStylePartitioning: String): TypedProperties = {
|
||||
val props = new TypedProperties()
|
||||
props.setProperty(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY, recordKeyFieldName)
|
||||
props.setProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY, partitionPathField)
|
||||
props.setProperty(DataSourceWriteOptions.HIVE_STYLE_PARTITIONING_OPT_KEY, hiveStylePartitioning)
|
||||
props
|
||||
}
|
||||
|
||||
@Test def testSimpleKeyGenerator() = {
|
||||
|
||||
// top level, valid fields
|
||||
var keyGen = new SimpleKeyGenerator(getKeyConfig("field1", "name", "false"))
|
||||
val hk1 = keyGen.getKey(baseRecord)
|
||||
assertEquals("field1", hk1.getRecordKey)
|
||||
assertEquals("name1", hk1.getPartitionPath)
|
||||
|
||||
assertEquals("field1", keyGen.getRecordKey(baseRow))
|
||||
assertEquals("name1", keyGen.getPartitionPath(baseRow))
|
||||
|
||||
// partition path field not specified
|
||||
try {
|
||||
val props = new TypedProperties()
|
||||
props.setProperty(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY, "field1")
|
||||
new SimpleKeyGenerator(props).getKey(baseRecord)
|
||||
fail("Should have errored out")
|
||||
} catch {
|
||||
case e: IllegalArgumentException =>
|
||||
// do nothing
|
||||
}
|
||||
|
||||
// partition path field not specified using Row
|
||||
try {
|
||||
val props = new TypedProperties()
|
||||
props.setProperty(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY, "field1")
|
||||
val keyGen = new SimpleKeyGenerator(props)
|
||||
keyGen.getRecordKey(baseRow)
|
||||
fail("Should have errored out")
|
||||
} catch {
|
||||
case e: IllegalArgumentException =>
|
||||
// do nothing
|
||||
}
|
||||
|
||||
// recordkey field not specified
|
||||
try {
|
||||
val props = new TypedProperties()
|
||||
props.setProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY, "partitionField")
|
||||
new SimpleKeyGenerator(props).getKey(baseRecord)
|
||||
fail("Should have errored out")
|
||||
} catch {
|
||||
case e: IllegalArgumentException =>
|
||||
// do nothing
|
||||
}
|
||||
|
||||
// recordkey field not specified using Row
|
||||
try {
|
||||
val props = new TypedProperties()
|
||||
props.setProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY, "partitionField")
|
||||
val keyGen = new SimpleKeyGenerator(props)
|
||||
keyGen.getPartitionPath(baseRow)
|
||||
fail("Should have errored out")
|
||||
} catch {
|
||||
case e: IllegalArgumentException =>
|
||||
// do nothing
|
||||
}
|
||||
|
||||
// nested field as record key and partition path
|
||||
val hk2 = new SimpleKeyGenerator(getKeyConfig("testNestedRecord.userId", "testNestedRecord.isAdmin", "false"))
|
||||
.getKey(baseRecord)
|
||||
assertEquals("UserId1@001", hk2.getRecordKey)
|
||||
assertEquals("false", hk2.getPartitionPath)
|
||||
|
||||
// Nested record key not found
|
||||
try {
|
||||
new SimpleKeyGenerator(getKeyConfig("testNestedRecord.NotThere", "testNestedRecord.isAdmin", "false"))
|
||||
.getKey(baseRecord)
|
||||
fail("Should have errored out")
|
||||
} catch {
|
||||
case e: HoodieException =>
|
||||
// do nothing
|
||||
}
|
||||
|
||||
// if partition path can't be found, return default partition path
|
||||
val hk3 = new SimpleKeyGenerator(getKeyConfig("testNestedRecord.userId", "testNestedRecord.notThere", "false"))
|
||||
.getKey(baseRecord)
|
||||
assertEquals("default", hk3.getPartitionPath)
|
||||
|
||||
// if partition path can't be found, return default partition path using row
|
||||
keyGen = new SimpleKeyGenerator(getKeyConfig("testNestedRecord.userId", "testNestedRecord.notThere", "false"))
|
||||
val hk3_row = keyGen.getPartitionPath(baseRow)
|
||||
assertEquals("default", hk3_row)
|
||||
|
||||
// if enable hive style partitioning
|
||||
val hk4 = new SimpleKeyGenerator(getKeyConfig("field1", "name", "true")).getKey(baseRecord)
|
||||
assertEquals("name=name1", hk4.getPartitionPath)
|
||||
|
||||
// if enable hive style partitioning using row
|
||||
keyGen = new SimpleKeyGenerator(getKeyConfig("field1", "name", "true"))
|
||||
val hk4_row = keyGen.getPartitionPath(baseRow)
|
||||
assertEquals("name=name1", hk4_row)
|
||||
|
||||
// if partition is null, return default partition path
|
||||
baseRecord.put("name", "")
|
||||
val hk5 = new SimpleKeyGenerator(getKeyConfig("field1", "name", "false"))
|
||||
.getKey(baseRecord)
|
||||
assertEquals("default", hk5.getPartitionPath)
|
||||
|
||||
// if partition is null, return default partition path using Row
|
||||
keyGen = new SimpleKeyGenerator(getKeyConfig("field1", "name", "false"))
|
||||
baseRow = KeyGeneratorTestUtilities.getRow(baseRecord, schema, structType)
|
||||
val hk5_row = keyGen.getPartitionPath(baseRow)
|
||||
assertEquals("default", hk5_row)
|
||||
|
||||
// if partition is empty, return default partition path
|
||||
baseRecord.put("name", null)
|
||||
val hk6 = new SimpleKeyGenerator(getKeyConfig("field1", "name", "false"))
|
||||
.getKey(baseRecord)
|
||||
assertEquals("default", hk6.getPartitionPath)
|
||||
|
||||
// if partition is empty, return default partition path using Row
|
||||
keyGen = new SimpleKeyGenerator(getKeyConfig("field1", "name", "false"))
|
||||
baseRow = KeyGeneratorTestUtilities.getRow(baseRecord, schema, structType)
|
||||
val hk6_row = keyGen.getPartitionPath(baseRow)
|
||||
assertEquals("default", hk6_row)
|
||||
|
||||
// if record key is empty, throw error
|
||||
try {
|
||||
baseRecord.put("field1", "")
|
||||
val props = new TypedProperties()
|
||||
props.setProperty(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY, "field1")
|
||||
props.setProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY, "name")
|
||||
new SimpleKeyGenerator(props).getKey(baseRecord)
|
||||
fail("Should have errored out")
|
||||
} catch {
|
||||
case e: HoodieKeyException =>
|
||||
// do nothing
|
||||
}
|
||||
|
||||
// if record key is empty, throw error. Using Row
|
||||
try {
|
||||
val props = new TypedProperties()
|
||||
props.setProperty(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY, "field1")
|
||||
props.setProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY, "name")
|
||||
keyGen = new SimpleKeyGenerator(props)
|
||||
baseRow = KeyGeneratorTestUtilities.getRow(baseRecord, schema, structType)
|
||||
keyGen.getRecordKey(baseRow)
|
||||
fail("Should have errored out")
|
||||
} catch {
|
||||
case e: HoodieKeyException =>
|
||||
// do nothing
|
||||
}
|
||||
|
||||
// if record key is null, throw error
|
||||
try {
|
||||
baseRecord.put("field1", null)
|
||||
val props = new TypedProperties()
|
||||
props.setProperty(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY, "field1")
|
||||
props.setProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY, "name")
|
||||
new SimpleKeyGenerator(props).getKey(baseRecord)
|
||||
fail("Should have errored out")
|
||||
} catch {
|
||||
case e: HoodieKeyException =>
|
||||
// do nothing
|
||||
}
|
||||
|
||||
// if record key is null, throw error. Using Row
|
||||
try {
|
||||
val props = new TypedProperties()
|
||||
props.setProperty(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY, "field1")
|
||||
props.setProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY, "name")
|
||||
keyGen = new SimpleKeyGenerator(props)
|
||||
baseRow = KeyGeneratorTestUtilities.getRow(baseRecord, schema, structType)
|
||||
keyGen.getRecordKey(baseRow)
|
||||
fail("Should have errored out")
|
||||
} catch {
|
||||
case e: HoodieKeyException =>
|
||||
// do nothing
|
||||
}
|
||||
}
|
||||
|
||||
@Test def testUserDefinedKeyGeneratorWorksWithRows(): Unit = {
|
||||
val keyGen = new UserDefinedKeyGenerator(getKeyConfig("field1", "name", "false"))
|
||||
assertEquals("field1", keyGen.getRecordKey(baseRow))
|
||||
assertEquals("name1", keyGen.getPartitionPath(baseRow))
|
||||
}
|
||||
|
||||
class UserDefinedKeyGenerator(props: TypedProperties) extends KeyGenerator(props) with SparkKeyGeneratorInterface {
|
||||
val recordKeyProp: String = props.getString(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY)
|
||||
val partitionPathProp: String = props.getString(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY)
|
||||
val STRUCT_NAME: String = "hoodieRowTopLevelField"
|
||||
val NAMESPACE: String = "hoodieRow"
|
||||
var converterFn: Function1[Any, Any] = _
|
||||
|
||||
override def getKey(record: GenericRecord): HoodieKey = {
|
||||
new HoodieKey(HoodieAvroUtils.getNestedFieldValAsString(record, recordKeyProp, true),
|
||||
HoodieAvroUtils.getNestedFieldValAsString(record, partitionPathProp, true))
|
||||
}
|
||||
|
||||
override def getRecordKey(row: Row): String = {
|
||||
if (null == converterFn) converterFn = AvroConversionHelper.createConverterToAvro(row.schema, STRUCT_NAME, NAMESPACE)
|
||||
val genericRecord = converterFn.apply(row).asInstanceOf[GenericRecord]
|
||||
getKey(genericRecord).getRecordKey
|
||||
}
|
||||
|
||||
override def getPartitionPath(row: Row): String = {
|
||||
if (null == converterFn) converterFn = AvroConversionHelper.createConverterToAvro(row.schema, STRUCT_NAME, NAMESPACE)
|
||||
val genericRecord = converterFn.apply(row).asInstanceOf[GenericRecord]
|
||||
getKey(genericRecord).getPartitionPath
|
||||
}
|
||||
}
|
||||
|
||||
@Test def testComplexKeyGenerator() = {
|
||||
// top level, valid fields
|
||||
var keyGen = new ComplexKeyGenerator(getKeyConfig("field1,name", "field1,name", "false"))
|
||||
val hk1 = keyGen.getKey(baseRecord)
|
||||
assertEquals("field1:field1,name:name1", hk1.getRecordKey)
|
||||
assertEquals("field1/name1", hk1.getPartitionPath)
|
||||
|
||||
// top level, valid fields with Row
|
||||
assertEquals("field1:field1,name:name1", keyGen.getRecordKey(baseRow))
|
||||
assertEquals("field1/name1", keyGen.getPartitionPath(baseRow))
|
||||
|
||||
// partition path field not specified
|
||||
try {
|
||||
val props = new TypedProperties()
|
||||
props.setProperty(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY, "field1")
|
||||
new ComplexKeyGenerator(props).getKey(baseRecord)
|
||||
fail("Should have errored out")
|
||||
} catch {
|
||||
case e: IllegalArgumentException =>
|
||||
// do nothing
|
||||
}
|
||||
|
||||
// partition path field not specified using Row
|
||||
try {
|
||||
val props = new TypedProperties()
|
||||
props.setProperty(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY, "field1")
|
||||
val keyGen = new ComplexKeyGenerator(props)
|
||||
keyGen.getRecordKey(baseRow)
|
||||
fail("Should have errored out")
|
||||
} catch {
|
||||
case e: IllegalArgumentException =>
|
||||
// do nothing
|
||||
}
|
||||
|
||||
// recordkey field not specified
|
||||
try {
|
||||
val props = new TypedProperties()
|
||||
props.setProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY, "partitionField")
|
||||
new ComplexKeyGenerator(props).getKey(baseRecord)
|
||||
fail("Should have errored out")
|
||||
} catch {
|
||||
case e: IllegalArgumentException =>
|
||||
// do nothing
|
||||
}
|
||||
|
||||
// recordkey field not specified
|
||||
try {
|
||||
val props = new TypedProperties()
|
||||
props.setProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY, "partitionField")
|
||||
val keyGen = new ComplexKeyGenerator(props)
|
||||
keyGen.getPartitionPath(baseRow)
|
||||
fail("Should have errored out")
|
||||
} catch {
|
||||
case e: IllegalArgumentException =>
|
||||
// do nothing
|
||||
}
|
||||
|
||||
// nested field as record key and partition path
|
||||
keyGen = new ComplexKeyGenerator(getKeyConfig("testNestedRecord.userId,testNestedRecord.isAdmin", "testNestedRecord.userId,testNestedRecord.isAdmin", "false"))
|
||||
val hk2 = keyGen.getKey(baseRecord)
|
||||
assertEquals("testNestedRecord.userId:UserId1@001,testNestedRecord.isAdmin:false", hk2.getRecordKey)
|
||||
assertEquals("UserId1@001/false", hk2.getPartitionPath)
|
||||
|
||||
// nested field as record key and partition path
|
||||
assertEquals("testNestedRecord.userId:UserId1@001,testNestedRecord.isAdmin:false", keyGen.getRecordKey(baseRow))
|
||||
assertEquals("UserId1@001/false", keyGen.getPartitionPath(baseRow))
|
||||
|
||||
// Nested record key not found
|
||||
try {
|
||||
new ComplexKeyGenerator(getKeyConfig("testNestedRecord.NotThere", "testNestedRecord.isAdmin", "false"))
|
||||
.getKey(baseRecord)
|
||||
fail("Should have errored out")
|
||||
} catch {
|
||||
case e: HoodieException =>
|
||||
// do nothing
|
||||
}
|
||||
|
||||
// Nested record key not found
|
||||
try {
|
||||
val keyGen = new ComplexKeyGenerator(getKeyConfig("testNestedRecord.NotThere", "testNestedRecord.isAdmin", "false"))
|
||||
keyGen.getRecordKey(baseRow)
|
||||
fail("Should have errored out")
|
||||
} catch {
|
||||
case e: HoodieException =>
|
||||
// do nothing
|
||||
}
|
||||
|
||||
// if partition path can't be found, return default partition path
|
||||
keyGen = new ComplexKeyGenerator(getKeyConfig("testNestedRecord.userId", "testNestedRecord.notThere", "false"))
|
||||
val hk3 = keyGen.getKey(baseRecord)
|
||||
assertEquals("default", hk3.getPartitionPath)
|
||||
|
||||
assertEquals("default", keyGen.getPartitionPath(baseRow))
|
||||
|
||||
// if enable hive style partitioning
|
||||
keyGen = new ComplexKeyGenerator(getKeyConfig("field1,name", "field1,name", "true"))
|
||||
val hk4 = keyGen.getKey(baseRecord)
|
||||
assertEquals("field1:field1,name:name1", hk4.getRecordKey)
|
||||
assertEquals("field1=field1/name=name1", hk4.getPartitionPath)
|
||||
|
||||
assertEquals("field1:field1,name:name1", keyGen.getRecordKey(baseRow))
|
||||
assertEquals("field1=field1/name=name1", keyGen.getPartitionPath(baseRow))
|
||||
|
||||
// if one part of the record key is empty, replace with "__empty__"
|
||||
baseRecord.put("name", "")
|
||||
keyGen = new ComplexKeyGenerator(getKeyConfig("field1,name", "field1,name", "false"))
|
||||
val hk5 = keyGen.getKey(baseRecord)
|
||||
assertEquals("field1:field1,name:__empty__", hk5.getRecordKey)
|
||||
assertEquals("field1/default", hk5.getPartitionPath)
|
||||
|
||||
baseRow = KeyGeneratorTestUtilities.getRow(baseRecord, schema, structType)
|
||||
assertEquals("field1:field1,name:__empty__", keyGen.getRecordKey(baseRow))
|
||||
assertEquals("field1/default", keyGen.getPartitionPath(baseRow))
|
||||
|
||||
// if one part of the record key is null, replace with "__null__"
|
||||
baseRecord.put("name", null)
|
||||
keyGen = new ComplexKeyGenerator(getKeyConfig("field1,name", "field1,name", "false"))
|
||||
val hk6 = keyGen.getKey(baseRecord)
|
||||
assertEquals("field1:field1,name:__null__", hk6.getRecordKey)
|
||||
assertEquals("field1/default", hk6.getPartitionPath)
|
||||
|
||||
baseRow = KeyGeneratorTestUtilities.getRow(baseRecord, schema, structType)
|
||||
assertEquals("field1:field1,name:__null__", keyGen.getRecordKey(baseRow))
|
||||
assertEquals("field1/default", keyGen.getPartitionPath(baseRow))
|
||||
|
||||
// if all parts of the composite record key are null/empty, throw error
|
||||
try {
|
||||
baseRecord.put("name", "")
|
||||
baseRecord.put("field1", null)
|
||||
val props = new TypedProperties()
|
||||
props.setProperty(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY, "field1,name")
|
||||
props.setProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY, "field1,name")
|
||||
new ComplexKeyGenerator(props).getKey(baseRecord)
|
||||
fail("Should have errored out")
|
||||
} catch {
|
||||
case e: HoodieKeyException =>
|
||||
// do nothing
|
||||
}
|
||||
|
||||
// if all parts of the composite record key are null/empty, throw error
|
||||
try {
|
||||
baseRecord.put("name", "")
|
||||
baseRecord.put("field1", null)
|
||||
val props = new TypedProperties()
|
||||
props.setProperty(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY, "field1,name")
|
||||
props.setProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY, "field1,name")
|
||||
keyGen = new ComplexKeyGenerator(props)
|
||||
baseRow = KeyGeneratorTestUtilities.getRow(baseRecord, schema, structType)
|
||||
keyGen.getRecordKey(baseRow)
|
||||
fail("Should have errored out")
|
||||
} catch {
|
||||
case e: HoodieKeyException =>
|
||||
// do nothing
|
||||
}
|
||||
|
||||
// reset name and field1 values.
|
||||
baseRecord.put("name", "name1")
|
||||
baseRecord.put("field1", "field1")
|
||||
keyGen = new ComplexKeyGenerator(getKeyConfig("field1, name", "field1, name", "false"))
|
||||
val hk7 = keyGen.getKey(baseRecord)
|
||||
assertEquals("field1:field1,name:name1", hk7.getRecordKey)
|
||||
assertEquals("field1/name1", hk7.getPartitionPath)
|
||||
|
||||
baseRow = KeyGeneratorTestUtilities.getRow(baseRecord, schema, structType)
|
||||
assertEquals("field1:field1,name:name1", keyGen.getRecordKey(baseRow))
|
||||
assertEquals("field1/name1", keyGen.getPartitionPath(baseRow))
|
||||
|
||||
keyGen = new ComplexKeyGenerator(getKeyConfig("field1,", "field1,", "false"))
|
||||
val hk8 = keyGen.getKey(baseRecord)
|
||||
assertEquals("field1:field1", hk8.getRecordKey)
|
||||
assertEquals("field1", hk8.getPartitionPath)
|
||||
|
||||
assertEquals("field1:field1", keyGen.getRecordKey(baseRow))
|
||||
assertEquals("field1", keyGen.getPartitionPath(baseRow))
|
||||
}
|
||||
|
||||
@Test def testGlobalDeleteKeyGenerator() = {
|
||||
// top level, partition value included but not actually used
|
||||
var keyGen = new GlobalDeleteKeyGenerator(getKeyConfig("field1,name", "field1,name", "false"))
|
||||
val hk1 = keyGen.getKey(baseRecord)
|
||||
assertEquals("field1:field1,name:name1", hk1.getRecordKey)
|
||||
assertEquals("", hk1.getPartitionPath)
|
||||
|
||||
assertEquals("field1:field1,name:name1", keyGen.getRecordKey(baseRow))
|
||||
assertEquals("", keyGen.getPartitionPath(baseRow))
|
||||
|
||||
// top level, partition value not included
|
||||
val props = new TypedProperties()
|
||||
props.setProperty(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY, "field1,name")
|
||||
keyGen = new GlobalDeleteKeyGenerator(props)
|
||||
val hk2 = keyGen.getKey(baseRecord)
|
||||
assertEquals("field1:field1,name:name1", hk2.getRecordKey)
|
||||
assertEquals("", hk2.getPartitionPath)
|
||||
|
||||
assertEquals("field1:field1,name:name1", keyGen.getRecordKey(baseRow))
|
||||
assertEquals("", keyGen.getPartitionPath(baseRow))
|
||||
|
||||
// if one part of the record key is empty, replace with "__empty__"
|
||||
baseRecord.put("name", "")
|
||||
keyGen = new GlobalDeleteKeyGenerator(getKeyConfig("field1,name", "field1,name", "false"))
|
||||
val hk3 = keyGen.getKey(baseRecord)
|
||||
assertEquals("field1:field1,name:__empty__", hk3.getRecordKey)
|
||||
assertEquals("", hk3.getPartitionPath)
|
||||
|
||||
baseRow = KeyGeneratorTestUtilities.getRow(baseRecord, schema, structType)
|
||||
assertEquals("field1:field1,name:__empty__", keyGen.getRecordKey(baseRow))
|
||||
assertEquals("", keyGen.getPartitionPath(baseRow))
|
||||
|
||||
// if one part of the record key is null, replace with "__null__"
|
||||
baseRecord.put("name", null)
|
||||
keyGen = new GlobalDeleteKeyGenerator(getKeyConfig("field1,name", "field1,name", "false"))
|
||||
val hk4 = keyGen.getKey(baseRecord)
|
||||
assertEquals("field1:field1,name:__null__", hk4.getRecordKey)
|
||||
assertEquals("", hk4.getPartitionPath)
|
||||
|
||||
baseRow = KeyGeneratorTestUtilities.getRow(baseRecord, schema, structType)
|
||||
assertEquals("field1:field1,name:__null__", keyGen.getRecordKey(baseRow))
|
||||
assertEquals("", keyGen.getPartitionPath(baseRow))
|
||||
|
||||
// recordkey field not specified
|
||||
try {
|
||||
val props = new TypedProperties()
|
||||
props.setProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY, "partitionField")
|
||||
new GlobalDeleteKeyGenerator(props).getKey(baseRecord)
|
||||
fail("Should have errored out")
|
||||
} catch {
|
||||
case e: IllegalArgumentException =>
|
||||
// do nothing
|
||||
}
|
||||
|
||||
// recordkey field not specified
|
||||
try {
|
||||
val props = new TypedProperties()
|
||||
props.setProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY, "partitionField")
|
||||
val keyGen = new GlobalDeleteKeyGenerator(props)
|
||||
keyGen.getRecordKey(baseRow)
|
||||
fail("Should have errored out")
|
||||
} catch {
|
||||
case e: IllegalArgumentException =>
|
||||
// do nothing
|
||||
}
|
||||
|
||||
// Nested record key not found
|
||||
try {
|
||||
new GlobalDeleteKeyGenerator(getKeyConfig("testNestedRecord.NotThere", "testNestedRecord.isAdmin", "false"))
|
||||
.getKey(baseRecord)
|
||||
fail("Should have errored out")
|
||||
} catch {
|
||||
case e: HoodieException =>
|
||||
// do nothing
|
||||
}
|
||||
|
||||
// Nested record key not found
|
||||
try {
|
||||
val keyGen = new GlobalDeleteKeyGenerator(getKeyConfig("testNestedRecord.NotThere", "testNestedRecord.isAdmin", "false"))
|
||||
keyGen.getRecordKey(baseRow)
|
||||
fail("Should have errored out")
|
||||
} catch {
|
||||
case e: HoodieException =>
|
||||
// do nothing
|
||||
}
|
||||
|
||||
// if all parts of the composite record key are null/empty, throw error
|
||||
try {
|
||||
baseRecord.put("name", "")
|
||||
baseRecord.put("field1", null)
|
||||
val props = new TypedProperties()
|
||||
props.setProperty(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY, "field1,name")
|
||||
new GlobalDeleteKeyGenerator(props).getKey(baseRecord)
|
||||
fail("Should have errored out")
|
||||
} catch {
|
||||
case e: HoodieKeyException =>
|
||||
// do nothing
|
||||
}
|
||||
|
||||
// if all parts of the composite record key are null/empty, throw error
|
||||
try {
|
||||
baseRecord.put("name", "")
|
||||
baseRecord.put("field1", null)
|
||||
baseRow = KeyGeneratorTestUtilities.getRow(baseRecord, schema, structType)
|
||||
val props = new TypedProperties()
|
||||
props.setProperty(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY, "field1,name")
|
||||
val keyGen = new GlobalDeleteKeyGenerator(props)
|
||||
keyGen.getRecordKey(baseRow)
|
||||
fail("Should have errored out")
|
||||
} catch {
|
||||
case e: HoodieKeyException =>
|
||||
// do nothing
|
||||
}
|
||||
}
|
||||
|
||||
@Test def testOverwriteWithLatestAvroPayload() = {
|
||||
val overWritePayload1 = new OverwriteWithLatestAvroPayload(baseRecord, 1)
|
||||
val laterRecord = SchemaTestUtil
|
||||
.generateAvroRecordFromJson(schema, 2, "001", "f1")
|
||||
val overWritePayload2 = new OverwriteWithLatestAvroPayload(laterRecord, 2)
|
||||
|
||||
// it will provide the record with greatest combine value
|
||||
val combinedPayload12 = overWritePayload1.preCombine(overWritePayload2)
|
||||
val combinedGR12 = combinedPayload12.getInsertValue(schema).get().asInstanceOf[GenericRecord]
|
||||
assertEquals("field2", combinedGR12.get("field1").toString)
|
||||
|
||||
// and it will be deterministic, to order of processing.
|
||||
val combinedPayload21 = overWritePayload2.preCombine(overWritePayload1)
|
||||
val combinedGR21 = combinedPayload21.getInsertValue(schema).get().asInstanceOf[GenericRecord]
|
||||
assertEquals("field2", combinedGR21.get("field1").toString)
|
||||
}
|
||||
|
||||
@Test def testEmptyHoodieRecordPayload() = {
|
||||
val emptyPayload1 = new EmptyHoodieRecordPayload(baseRecord, 1)
|
||||
val laterRecord = SchemaTestUtil
|
||||
.generateAvroRecordFromJson(schema, 2, "001", "f1")
|
||||
val emptyPayload2 = new EmptyHoodieRecordPayload(laterRecord, 2)
|
||||
|
||||
// it will provide an empty record
|
||||
val combinedPayload12 = emptyPayload1.preCombine(emptyPayload2)
|
||||
val combined12 = combinedPayload12.getInsertValue(schema)
|
||||
assertEquals(Option.empty(), combined12)
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,106 @@
|
||||
/*
|
||||
* 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 java.io.File
|
||||
import java.nio.file.Paths
|
||||
|
||||
import org.apache.hadoop.conf.Configuration
|
||||
import org.apache.hadoop.fs.Path
|
||||
import org.apache.spark.sql.SparkSession
|
||||
import org.junit.jupiter.api.Assertions.assertEquals
|
||||
import org.junit.jupiter.api.Test
|
||||
import org.junit.jupiter.api.io.TempDir
|
||||
|
||||
class TestHoodieSparkUtils {
|
||||
|
||||
@Test
|
||||
def testGlobPaths(@TempDir tempDir: File): Unit = {
|
||||
val folders: Seq[Path] = Seq(
|
||||
new Path(Paths.get(tempDir.getAbsolutePath, "folder1").toUri),
|
||||
new Path(Paths.get(tempDir.getAbsolutePath, "folder2").toUri)
|
||||
)
|
||||
|
||||
val files: Seq[Path] = Seq(
|
||||
new Path(Paths.get(tempDir.getAbsolutePath, "folder1", "file1").toUri),
|
||||
new Path(Paths.get(tempDir.getAbsolutePath, "folder1", "file2").toUri),
|
||||
new Path(Paths.get(tempDir.getAbsolutePath, "folder2", "file3").toUri),
|
||||
new Path(Paths.get(tempDir.getAbsolutePath, "folder2", "file4").toUri)
|
||||
)
|
||||
|
||||
folders.foreach(folder => new File(folder.toUri).mkdir())
|
||||
files.foreach(file => new File(file.toUri).createNewFile())
|
||||
|
||||
var paths = Seq(tempDir.getAbsolutePath + "/*")
|
||||
var globbedPaths = HoodieSparkUtils.checkAndGlobPathIfNecessary(paths,
|
||||
new Path(paths.head).getFileSystem(new Configuration()))
|
||||
assertEquals(folders.sortWith(_.toString < _.toString), globbedPaths.sortWith(_.toString < _.toString))
|
||||
|
||||
paths = Seq(tempDir.getAbsolutePath + "/*/*")
|
||||
globbedPaths = HoodieSparkUtils.checkAndGlobPathIfNecessary(paths,
|
||||
new Path(paths.head).getFileSystem(new Configuration()))
|
||||
assertEquals(files.sortWith(_.toString < _.toString), globbedPaths.sortWith(_.toString < _.toString))
|
||||
|
||||
paths = Seq(tempDir.getAbsolutePath + "/folder1/*")
|
||||
globbedPaths = HoodieSparkUtils.checkAndGlobPathIfNecessary(paths,
|
||||
new Path(paths.head).getFileSystem(new Configuration()))
|
||||
assertEquals(Seq(files(0), files(1)).sortWith(_.toString < _.toString),
|
||||
globbedPaths.sortWith(_.toString < _.toString))
|
||||
|
||||
paths = Seq(tempDir.getAbsolutePath + "/folder2/*")
|
||||
globbedPaths = HoodieSparkUtils.checkAndGlobPathIfNecessary(paths,
|
||||
new Path(paths.head).getFileSystem(new Configuration()))
|
||||
assertEquals(Seq(files(2), files(3)).sortWith(_.toString < _.toString),
|
||||
globbedPaths.sortWith(_.toString < _.toString))
|
||||
|
||||
paths = Seq(tempDir.getAbsolutePath + "/folder1/*", tempDir.getAbsolutePath + "/folder2/*")
|
||||
globbedPaths = HoodieSparkUtils.checkAndGlobPathIfNecessary(paths,
|
||||
new Path(paths.head).getFileSystem(new Configuration()))
|
||||
assertEquals(files.sortWith(_.toString < _.toString), globbedPaths.sortWith(_.toString < _.toString))
|
||||
}
|
||||
|
||||
@Test
|
||||
def testCreateInMemoryIndex(@TempDir tempDir: File): Unit = {
|
||||
val spark = SparkSession.builder
|
||||
.appName("Hoodie Datasource test")
|
||||
.master("local[2]")
|
||||
.config("spark.serializer", "org.apache.spark.serializer.KryoSerializer")
|
||||
.getOrCreate
|
||||
|
||||
val folders: Seq[Path] = Seq(
|
||||
new Path(Paths.get(tempDir.getAbsolutePath, "folder1").toUri),
|
||||
new Path(Paths.get(tempDir.getAbsolutePath, "folder2").toUri)
|
||||
)
|
||||
|
||||
val files: Seq[Path] = Seq(
|
||||
new Path(Paths.get(tempDir.getAbsolutePath, "folder1", "file1").toUri),
|
||||
new Path(Paths.get(tempDir.getAbsolutePath, "folder1", "file2").toUri),
|
||||
new Path(Paths.get(tempDir.getAbsolutePath, "folder2", "file3").toUri),
|
||||
new Path(Paths.get(tempDir.getAbsolutePath, "folder2", "file4").toUri)
|
||||
)
|
||||
|
||||
folders.foreach(folder => new File(folder.toUri).mkdir())
|
||||
files.foreach(file => new File(file.toUri).createNewFile())
|
||||
|
||||
val index = HoodieSparkUtils.createInMemoryFileIndex(spark, Seq(folders(0), folders(1)))
|
||||
val indexedFilePaths = index.allFiles().map(fs => fs.getPath)
|
||||
assertEquals(files.sortWith(_.toString < _.toString), indexedFilePaths.sortWith(_.toString < _.toString))
|
||||
spark.stop()
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,417 @@
|
||||
/*
|
||||
* 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.functional
|
||||
|
||||
import java.time.Instant
|
||||
import java.util
|
||||
import java.util.{Collections, Date, UUID}
|
||||
|
||||
import org.apache.commons.io.FileUtils
|
||||
import org.apache.hudi.DataSourceWriteOptions._
|
||||
import org.apache.hudi.client.{SparkRDDWriteClient, TestBootstrap}
|
||||
import org.apache.hudi.common.model.{HoodieRecord, HoodieRecordPayload}
|
||||
import org.apache.hudi.common.testutils.HoodieTestDataGenerator
|
||||
import org.apache.hudi.config.{HoodieBootstrapConfig, HoodieWriteConfig}
|
||||
import org.apache.hudi.exception.HoodieException
|
||||
import org.apache.hudi.keygen.{NonpartitionedKeyGenerator, SimpleKeyGenerator}
|
||||
import org.apache.hudi.testutils.DataSourceTestUtils
|
||||
import org.apache.hudi.{AvroConversionUtils, DataSourceUtils, DataSourceWriteOptions, HoodieSparkSqlWriter, HoodieWriterUtils}
|
||||
import org.apache.spark.SparkContext
|
||||
import org.apache.spark.api.java.JavaSparkContext
|
||||
import org.apache.spark.sql.{Row, SQLContext, SaveMode, SparkSession}
|
||||
import org.mockito.ArgumentMatchers.any
|
||||
import org.mockito.Mockito.{spy, times, verify}
|
||||
import org.scalatest.{FunSuite, Matchers}
|
||||
|
||||
import scala.collection.JavaConversions._
|
||||
|
||||
class HoodieSparkSqlWriterSuite extends FunSuite with Matchers {
|
||||
|
||||
var spark: SparkSession = _
|
||||
var sc: SparkContext = _
|
||||
var sqlContext: SQLContext = _
|
||||
|
||||
test("Parameters With Write Defaults") {
|
||||
val originals = HoodieWriterUtils.parametersWithWriteDefaults(Map.empty)
|
||||
val rhsKey = "hoodie.right.hand.side.key"
|
||||
val rhsVal = "hoodie.right.hand.side.val"
|
||||
val modifier = Map(OPERATION_OPT_KEY -> INSERT_OPERATION_OPT_VAL, TABLE_TYPE_OPT_KEY -> MOR_TABLE_TYPE_OPT_VAL, rhsKey -> rhsVal)
|
||||
val modified = HoodieWriterUtils.parametersWithWriteDefaults(modifier)
|
||||
val matcher = (k: String, v: String) => modified(k) should be(v)
|
||||
|
||||
originals foreach {
|
||||
case (OPERATION_OPT_KEY, _) => matcher(OPERATION_OPT_KEY, INSERT_OPERATION_OPT_VAL)
|
||||
case (TABLE_TYPE_OPT_KEY, _) => matcher(TABLE_TYPE_OPT_KEY, MOR_TABLE_TYPE_OPT_VAL)
|
||||
case (`rhsKey`, _) => matcher(rhsKey, rhsVal)
|
||||
case (k, v) => matcher(k, v)
|
||||
}
|
||||
}
|
||||
|
||||
test("throw hoodie exception when invalid serializer") {
|
||||
val session = SparkSession.builder().appName("hoodie_test").master("local").getOrCreate()
|
||||
try {
|
||||
val sqlContext = session.sqlContext
|
||||
val options = Map("path" -> "hoodie/test/path", HoodieWriteConfig.TABLE_NAME -> "hoodie_test_tbl")
|
||||
val e = intercept[HoodieException](HoodieSparkSqlWriter.write(sqlContext, SaveMode.ErrorIfExists, options,
|
||||
session.emptyDataFrame))
|
||||
assert(e.getMessage.contains("spark.serializer"))
|
||||
} finally {
|
||||
session.stop()
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
test("throw hoodie exception when there already exist a table with different name with Append Save mode") {
|
||||
|
||||
initSparkContext("test_append_mode")
|
||||
val path = java.nio.file.Files.createTempDirectory("hoodie_test_path")
|
||||
try {
|
||||
|
||||
val hoodieFooTableName = "hoodie_foo_tbl"
|
||||
|
||||
//create a new table
|
||||
val fooTableModifier = Map("path" -> path.toAbsolutePath.toString,
|
||||
HoodieWriteConfig.TABLE_NAME -> hoodieFooTableName,
|
||||
"hoodie.insert.shuffle.parallelism" -> "4",
|
||||
"hoodie.upsert.shuffle.parallelism" -> "4")
|
||||
val fooTableParams = HoodieWriterUtils.parametersWithWriteDefaults(fooTableModifier)
|
||||
val dataFrame = spark.createDataFrame(Seq(Test(UUID.randomUUID().toString, new Date().getTime)))
|
||||
HoodieSparkSqlWriter.write(sqlContext, SaveMode.Append, fooTableParams, dataFrame)
|
||||
|
||||
//on same path try append with different("hoodie_bar_tbl") table name which should throw an exception
|
||||
val barTableModifier = Map("path" -> path.toAbsolutePath.toString,
|
||||
HoodieWriteConfig.TABLE_NAME -> "hoodie_bar_tbl",
|
||||
"hoodie.insert.shuffle.parallelism" -> "4",
|
||||
"hoodie.upsert.shuffle.parallelism" -> "4")
|
||||
val barTableParams = HoodieWriterUtils.parametersWithWriteDefaults(barTableModifier)
|
||||
val dataFrame2 = spark.createDataFrame(Seq(Test(UUID.randomUUID().toString, new Date().getTime)))
|
||||
val tableAlreadyExistException = intercept[HoodieException](HoodieSparkSqlWriter.write(sqlContext, SaveMode.Append, barTableParams, dataFrame2))
|
||||
assert(tableAlreadyExistException.getMessage.contains("hoodie table with name " + hoodieFooTableName + " already exist"))
|
||||
|
||||
//on same path try append with delete operation and different("hoodie_bar_tbl") table name which should throw an exception
|
||||
val deleteTableParams = barTableParams ++ Map(OPERATION_OPT_KEY -> "delete")
|
||||
val deleteCmdException = intercept[HoodieException](HoodieSparkSqlWriter.write(sqlContext, SaveMode.Append, deleteTableParams, dataFrame2))
|
||||
assert(deleteCmdException.getMessage.contains("hoodie table with name " + hoodieFooTableName + " already exist"))
|
||||
} finally {
|
||||
spark.stop()
|
||||
FileUtils.deleteDirectory(path.toFile)
|
||||
}
|
||||
}
|
||||
|
||||
test("test bulk insert dataset with datasource impl") {
|
||||
initSparkContext("test_bulk_insert_datasource")
|
||||
val path = java.nio.file.Files.createTempDirectory("hoodie_test_path")
|
||||
try {
|
||||
|
||||
val hoodieFooTableName = "hoodie_foo_tbl"
|
||||
|
||||
//create a new table
|
||||
val fooTableModifier = Map("path" -> path.toAbsolutePath.toString,
|
||||
HoodieWriteConfig.TABLE_NAME -> hoodieFooTableName,
|
||||
"hoodie.bulkinsert.shuffle.parallelism" -> "4",
|
||||
DataSourceWriteOptions.OPERATION_OPT_KEY -> DataSourceWriteOptions.BULK_INSERT_OPERATION_OPT_VAL,
|
||||
DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY -> "_row_key",
|
||||
DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY -> "partition",
|
||||
DataSourceWriteOptions.KEYGENERATOR_CLASS_OPT_KEY -> "org.apache.hudi.keygen.SimpleKeyGenerator")
|
||||
val fooTableParams = HoodieWriterUtils.parametersWithWriteDefaults(fooTableModifier)
|
||||
|
||||
// generate the inserts
|
||||
val schema = DataSourceTestUtils.getStructTypeExampleSchema
|
||||
val structType = AvroConversionUtils.convertAvroSchemaToStructType(schema)
|
||||
val records = DataSourceTestUtils.generateRandomRows(100)
|
||||
val recordsSeq = convertRowListToSeq(records)
|
||||
val df = spark.createDataFrame(sc.parallelize(recordsSeq), structType)
|
||||
// write to Hudi
|
||||
HoodieSparkSqlWriter.write(sqlContext, SaveMode.Append, fooTableParams, df)
|
||||
|
||||
// collect all parition paths to issue read of parquet files
|
||||
val partitions = Seq(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, HoodieTestDataGenerator.DEFAULT_SECOND_PARTITION_PATH,
|
||||
HoodieTestDataGenerator.DEFAULT_THIRD_PARTITION_PATH)
|
||||
// Check the entire dataset has all records still
|
||||
val fullPartitionPaths = new Array[String](3)
|
||||
for (i <- 0 until fullPartitionPaths.length) {
|
||||
fullPartitionPaths(i) = String.format("%s/%s/*", path.toAbsolutePath.toString, partitions(i))
|
||||
}
|
||||
|
||||
// fetch all records from parquet files generated from write to hudi
|
||||
val actualDf = sqlContext.read.parquet(fullPartitionPaths(0), fullPartitionPaths(1), fullPartitionPaths(2))
|
||||
|
||||
// remove metadata columns so that expected and actual DFs can be compared as is
|
||||
val trimmedDf = actualDf.drop(HoodieRecord.HOODIE_META_COLUMNS.get(0)).drop(HoodieRecord.HOODIE_META_COLUMNS.get(1))
|
||||
.drop(HoodieRecord.HOODIE_META_COLUMNS.get(2)).drop(HoodieRecord.HOODIE_META_COLUMNS.get(3))
|
||||
.drop(HoodieRecord.HOODIE_META_COLUMNS.get(4))
|
||||
|
||||
assert(df.except(trimmedDf).count() == 0)
|
||||
} finally {
|
||||
spark.stop()
|
||||
FileUtils.deleteDirectory(path.toFile)
|
||||
}
|
||||
}
|
||||
|
||||
test("test insert dataset without precombine field") {
|
||||
val session = SparkSession.builder()
|
||||
.appName("test_insert_without_precombine")
|
||||
.master("local[2]")
|
||||
.config("spark.serializer", "org.apache.spark.serializer.KryoSerializer")
|
||||
.getOrCreate()
|
||||
val path = java.nio.file.Files.createTempDirectory("hoodie_test_path")
|
||||
try {
|
||||
|
||||
val sqlContext = session.sqlContext
|
||||
val sc = session.sparkContext
|
||||
val hoodieFooTableName = "hoodie_foo_tbl"
|
||||
|
||||
//create a new table
|
||||
val fooTableModifier = Map("path" -> path.toAbsolutePath.toString,
|
||||
HoodieWriteConfig.TABLE_NAME -> hoodieFooTableName,
|
||||
"hoodie.bulkinsert.shuffle.parallelism" -> "1",
|
||||
DataSourceWriteOptions.OPERATION_OPT_KEY -> DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL,
|
||||
DataSourceWriteOptions.INSERT_DROP_DUPS_OPT_KEY -> "false",
|
||||
DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY -> "_row_key",
|
||||
DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY -> "partition",
|
||||
DataSourceWriteOptions.KEYGENERATOR_CLASS_OPT_KEY -> "org.apache.hudi.keygen.SimpleKeyGenerator")
|
||||
val fooTableParams = HoodieWriterUtils.parametersWithWriteDefaults(fooTableModifier)
|
||||
|
||||
// generate the inserts
|
||||
val schema = DataSourceTestUtils.getStructTypeExampleSchema
|
||||
val structType = AvroConversionUtils.convertAvroSchemaToStructType(schema)
|
||||
val records = DataSourceTestUtils.generateRandomRows(100)
|
||||
val recordsSeq = convertRowListToSeq(records)
|
||||
val df = session.createDataFrame(sc.parallelize(recordsSeq), structType)
|
||||
// write to Hudi
|
||||
HoodieSparkSqlWriter.write(sqlContext, SaveMode.Append, fooTableParams - DataSourceWriteOptions.PRECOMBINE_FIELD_OPT_KEY, df)
|
||||
|
||||
// collect all parition paths to issue read of parquet files
|
||||
val partitions = Seq(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, HoodieTestDataGenerator.DEFAULT_SECOND_PARTITION_PATH,
|
||||
HoodieTestDataGenerator.DEFAULT_THIRD_PARTITION_PATH)
|
||||
// Check the entire dataset has all records still
|
||||
val fullPartitionPaths = new Array[String](3)
|
||||
for (i <- 0 until fullPartitionPaths.length) {
|
||||
fullPartitionPaths(i) = String.format("%s/%s/*", path.toAbsolutePath.toString, partitions(i))
|
||||
}
|
||||
|
||||
// fetch all records from parquet files generated from write to hudi
|
||||
val actualDf = session.sqlContext.read.parquet(fullPartitionPaths(0), fullPartitionPaths(1), fullPartitionPaths(2))
|
||||
|
||||
// remove metadata columns so that expected and actual DFs can be compared as is
|
||||
val trimmedDf = actualDf.drop(HoodieRecord.HOODIE_META_COLUMNS.get(0)).drop(HoodieRecord.HOODIE_META_COLUMNS.get(1))
|
||||
.drop(HoodieRecord.HOODIE_META_COLUMNS.get(2)).drop(HoodieRecord.HOODIE_META_COLUMNS.get(3))
|
||||
.drop(HoodieRecord.HOODIE_META_COLUMNS.get(4))
|
||||
|
||||
assert(df.except(trimmedDf).count() == 0)
|
||||
} finally {
|
||||
session.stop()
|
||||
FileUtils.deleteDirectory(path.toFile)
|
||||
}
|
||||
}
|
||||
|
||||
test("test bulk insert dataset with datasource impl multiple rounds") {
|
||||
initSparkContext("test_bulk_insert_datasource")
|
||||
val path = java.nio.file.Files.createTempDirectory("hoodie_test_path")
|
||||
try {
|
||||
|
||||
val hoodieFooTableName = "hoodie_foo_tbl"
|
||||
|
||||
//create a new table
|
||||
val fooTableModifier = Map("path" -> path.toAbsolutePath.toString,
|
||||
HoodieWriteConfig.TABLE_NAME -> hoodieFooTableName,
|
||||
"hoodie.bulkinsert.shuffle.parallelism" -> "4",
|
||||
DataSourceWriteOptions.OPERATION_OPT_KEY -> DataSourceWriteOptions.BULK_INSERT_OPERATION_OPT_VAL,
|
||||
DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY -> "_row_key",
|
||||
DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY -> "partition",
|
||||
DataSourceWriteOptions.KEYGENERATOR_CLASS_OPT_KEY -> "org.apache.hudi.keygen.SimpleKeyGenerator")
|
||||
val fooTableParams = HoodieWriterUtils.parametersWithWriteDefaults(fooTableModifier)
|
||||
|
||||
val partitions = Seq(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, HoodieTestDataGenerator.DEFAULT_SECOND_PARTITION_PATH,
|
||||
HoodieTestDataGenerator.DEFAULT_THIRD_PARTITION_PATH)
|
||||
val fullPartitionPaths = new Array[String](3)
|
||||
for (i <- 0 to 2) {
|
||||
fullPartitionPaths(i) = String.format("%s/%s/*", path.toAbsolutePath.toString, partitions(i))
|
||||
}
|
||||
|
||||
val schema = DataSourceTestUtils.getStructTypeExampleSchema
|
||||
val structType = AvroConversionUtils.convertAvroSchemaToStructType(schema)
|
||||
var totalExpectedDf = spark.createDataFrame(sc.emptyRDD[Row], structType)
|
||||
|
||||
for (_ <- 0 to 2) {
|
||||
// generate the inserts
|
||||
val records = DataSourceTestUtils.generateRandomRows(200)
|
||||
val recordsSeq = convertRowListToSeq(records)
|
||||
val df = spark.createDataFrame(sc.parallelize(recordsSeq), structType)
|
||||
// write to Hudi
|
||||
HoodieSparkSqlWriter.write(sqlContext, SaveMode.Append, fooTableParams, df)
|
||||
|
||||
// Fetch records from entire dataset
|
||||
val actualDf = sqlContext.read.parquet(fullPartitionPaths(0), fullPartitionPaths(1), fullPartitionPaths(2))
|
||||
|
||||
// remove metadata columns so that expected and actual DFs can be compared as is
|
||||
val trimmedDf = actualDf.drop(HoodieRecord.HOODIE_META_COLUMNS.get(0)).drop(HoodieRecord.HOODIE_META_COLUMNS.get(1))
|
||||
.drop(HoodieRecord.HOODIE_META_COLUMNS.get(2)).drop(HoodieRecord.HOODIE_META_COLUMNS.get(3))
|
||||
.drop(HoodieRecord.HOODIE_META_COLUMNS.get(4))
|
||||
|
||||
// find total df (union from multiple rounds)
|
||||
totalExpectedDf = totalExpectedDf.union(df)
|
||||
// find mismatch between actual and expected df
|
||||
assert(totalExpectedDf.except(trimmedDf).count() == 0)
|
||||
}
|
||||
} finally {
|
||||
spark.stop()
|
||||
FileUtils.deleteDirectory(path.toFile)
|
||||
}
|
||||
}
|
||||
|
||||
List(DataSourceWriteOptions.COW_TABLE_TYPE_OPT_VAL, DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL)
|
||||
.foreach(tableType => {
|
||||
test("test basic HoodieSparkSqlWriter functionality with datasource insert for " + tableType) {
|
||||
initSparkContext("test_insert_datasource")
|
||||
val path = java.nio.file.Files.createTempDirectory("hoodie_test_path")
|
||||
try {
|
||||
|
||||
val hoodieFooTableName = "hoodie_foo_tbl"
|
||||
|
||||
//create a new table
|
||||
val fooTableModifier = Map("path" -> path.toAbsolutePath.toString,
|
||||
HoodieWriteConfig.TABLE_NAME -> hoodieFooTableName,
|
||||
DataSourceWriteOptions.TABLE_TYPE_OPT_KEY -> tableType,
|
||||
HoodieWriteConfig.INSERT_PARALLELISM -> "4",
|
||||
DataSourceWriteOptions.OPERATION_OPT_KEY -> DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL,
|
||||
DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY -> "_row_key",
|
||||
DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY -> "partition",
|
||||
DataSourceWriteOptions.KEYGENERATOR_CLASS_OPT_KEY -> classOf[SimpleKeyGenerator].getCanonicalName)
|
||||
val fooTableParams = HoodieWriterUtils.parametersWithWriteDefaults(fooTableModifier)
|
||||
|
||||
// generate the inserts
|
||||
val schema = DataSourceTestUtils.getStructTypeExampleSchema
|
||||
val structType = AvroConversionUtils.convertAvroSchemaToStructType(schema)
|
||||
val records = DataSourceTestUtils.generateRandomRows(100)
|
||||
val recordsSeq = convertRowListToSeq(records)
|
||||
val df = spark.createDataFrame(sc.parallelize(recordsSeq), structType)
|
||||
|
||||
val client = spy(DataSourceUtils.createHoodieClient(
|
||||
new JavaSparkContext(sc),
|
||||
schema.toString,
|
||||
path.toAbsolutePath.toString,
|
||||
hoodieFooTableName,
|
||||
mapAsJavaMap(fooTableParams)).asInstanceOf[SparkRDDWriteClient[HoodieRecordPayload[Nothing]]])
|
||||
|
||||
// write to Hudi
|
||||
HoodieSparkSqlWriter.write(sqlContext, SaveMode.Append, fooTableParams, df, Option.empty,
|
||||
Option(client))
|
||||
// Verify that asynchronous compaction is not scheduled
|
||||
verify(client, times(0)).scheduleCompaction(any())
|
||||
// Verify that HoodieWriteClient is closed correctly
|
||||
verify(client, times(1)).close()
|
||||
|
||||
// collect all partition paths to issue read of parquet files
|
||||
val partitions = Seq(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH,
|
||||
HoodieTestDataGenerator.DEFAULT_SECOND_PARTITION_PATH, HoodieTestDataGenerator.DEFAULT_THIRD_PARTITION_PATH)
|
||||
// Check the entire dataset has all records still
|
||||
val fullPartitionPaths = new Array[String](3)
|
||||
for (i <- fullPartitionPaths.indices) {
|
||||
fullPartitionPaths(i) = String.format("%s/%s/*", path.toAbsolutePath.toString, partitions(i))
|
||||
}
|
||||
|
||||
// fetch all records from parquet files generated from write to hudi
|
||||
val actualDf = sqlContext.read.parquet(fullPartitionPaths(0), fullPartitionPaths(1), fullPartitionPaths(2))
|
||||
|
||||
// remove metadata columns so that expected and actual DFs can be compared as is
|
||||
val trimmedDf = actualDf.drop(HoodieRecord.HOODIE_META_COLUMNS.get(0)).drop(HoodieRecord.HOODIE_META_COLUMNS.get(1))
|
||||
.drop(HoodieRecord.HOODIE_META_COLUMNS.get(2)).drop(HoodieRecord.HOODIE_META_COLUMNS.get(3))
|
||||
.drop(HoodieRecord.HOODIE_META_COLUMNS.get(4))
|
||||
|
||||
assert(df.except(trimmedDf).count() == 0)
|
||||
} finally {
|
||||
spark.stop()
|
||||
FileUtils.deleteDirectory(path.toFile)
|
||||
}
|
||||
}
|
||||
})
|
||||
|
||||
List(DataSourceWriteOptions.COW_TABLE_TYPE_OPT_VAL, DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL)
|
||||
.foreach(tableType => {
|
||||
test("test HoodieSparkSqlWriter functionality with datasource bootstrap for " + tableType) {
|
||||
initSparkContext("test_bootstrap_datasource")
|
||||
val path = java.nio.file.Files.createTempDirectory("hoodie_test_path")
|
||||
val srcPath = java.nio.file.Files.createTempDirectory("hoodie_bootstrap_source_path")
|
||||
|
||||
try {
|
||||
|
||||
val hoodieFooTableName = "hoodie_foo_tbl"
|
||||
|
||||
val sourceDF = TestBootstrap.generateTestRawTripDataset(Instant.now.toEpochMilli, 0, 100, Collections.emptyList(), sc,
|
||||
spark.sqlContext)
|
||||
|
||||
// Write source data non-partitioned
|
||||
sourceDF.write
|
||||
.format("parquet")
|
||||
.mode(SaveMode.Overwrite)
|
||||
.save(srcPath.toAbsolutePath.toString)
|
||||
|
||||
val fooTableModifier = Map("path" -> path.toAbsolutePath.toString,
|
||||
HoodieBootstrapConfig.BOOTSTRAP_BASE_PATH_PROP -> srcPath.toAbsolutePath.toString,
|
||||
HoodieWriteConfig.TABLE_NAME -> hoodieFooTableName,
|
||||
DataSourceWriteOptions.TABLE_TYPE_OPT_KEY -> tableType,
|
||||
HoodieBootstrapConfig.BOOTSTRAP_PARALLELISM -> "4",
|
||||
DataSourceWriteOptions.OPERATION_OPT_KEY -> DataSourceWriteOptions.BOOTSTRAP_OPERATION_OPT_VAL,
|
||||
DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY -> "_row_key",
|
||||
DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY -> "partition",
|
||||
HoodieBootstrapConfig.BOOTSTRAP_KEYGEN_CLASS -> classOf[NonpartitionedKeyGenerator].getCanonicalName)
|
||||
val fooTableParams = HoodieWriterUtils.parametersWithWriteDefaults(fooTableModifier)
|
||||
|
||||
val client = spy(DataSourceUtils.createHoodieClient(
|
||||
new JavaSparkContext(sc),
|
||||
null,
|
||||
path.toAbsolutePath.toString,
|
||||
hoodieFooTableName,
|
||||
mapAsJavaMap(fooTableParams)).asInstanceOf[SparkRDDWriteClient[HoodieRecordPayload[Nothing]]])
|
||||
|
||||
HoodieSparkSqlWriter.bootstrap(sqlContext, SaveMode.Append, fooTableParams, spark.emptyDataFrame, Option.empty,
|
||||
Option(client))
|
||||
|
||||
// Verify that HoodieWriteClient is closed correctly
|
||||
verify(client, times(1)).close()
|
||||
|
||||
// fetch all records from parquet files generated from write to hudi
|
||||
val actualDf = sqlContext.read.parquet(path.toAbsolutePath.toString)
|
||||
assert(actualDf.count == 100)
|
||||
} finally {
|
||||
spark.stop()
|
||||
FileUtils.deleteDirectory(path.toFile)
|
||||
FileUtils.deleteDirectory(srcPath.toFile)
|
||||
}
|
||||
}
|
||||
})
|
||||
|
||||
case class Test(uuid: String, ts: Long)
|
||||
|
||||
import scala.collection.JavaConverters
|
||||
|
||||
def convertRowListToSeq(inputList: util.List[Row]): Seq[Row] =
|
||||
JavaConverters.asScalaIteratorConverter(inputList.iterator).asScala.toSeq
|
||||
|
||||
def initSparkContext(appName: String): Unit = {
|
||||
spark = SparkSession.builder()
|
||||
.appName(appName)
|
||||
.master("local[2]")
|
||||
.config("spark.serializer", "org.apache.spark.serializer.KryoSerializer")
|
||||
.getOrCreate()
|
||||
sc = spark.sparkContext
|
||||
sc.setLogLevel("ERROR")
|
||||
sqlContext = spark.sqlContext
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,323 @@
|
||||
/*
|
||||
* 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.functional
|
||||
|
||||
import java.sql.{Date, Timestamp}
|
||||
import java.util.function.Supplier
|
||||
import java.util.stream.Stream
|
||||
|
||||
import org.apache.hadoop.fs.Path
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant
|
||||
import org.apache.hudi.common.testutils.HoodieTestDataGenerator
|
||||
import org.apache.hudi.common.testutils.RawTripTestPayload.recordsToStrings
|
||||
import org.apache.hudi.config.HoodieWriteConfig
|
||||
import org.apache.hudi.testutils.HoodieClientTestBase
|
||||
import org.apache.hudi.{DataSourceReadOptions, DataSourceWriteOptions, HoodieDataSourceHelpers}
|
||||
import org.apache.spark.sql._
|
||||
import org.apache.spark.sql.functions.col
|
||||
import org.apache.spark.sql.types.{DataTypes, DateType, IntegerType, StringType, StructField, StructType, TimestampType}
|
||||
import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue}
|
||||
import org.junit.jupiter.api.{AfterEach, BeforeEach, Test}
|
||||
|
||||
import scala.collection.JavaConversions._
|
||||
|
||||
/**
|
||||
* Basic tests on the spark datasource for COW table.
|
||||
*/
|
||||
|
||||
class TestCOWDataSource extends HoodieClientTestBase {
|
||||
var spark: SparkSession = null
|
||||
val commonOpts = Map(
|
||||
"hoodie.insert.shuffle.parallelism" -> "4",
|
||||
"hoodie.upsert.shuffle.parallelism" -> "4",
|
||||
DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY -> "_row_key",
|
||||
DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY -> "partition",
|
||||
DataSourceWriteOptions.PRECOMBINE_FIELD_OPT_KEY -> "timestamp",
|
||||
HoodieWriteConfig.TABLE_NAME -> "hoodie_test"
|
||||
)
|
||||
|
||||
@BeforeEach override def setUp() {
|
||||
initPath()
|
||||
initSparkContexts()
|
||||
spark = sqlContext.sparkSession
|
||||
initTestDataGenerator()
|
||||
initFileSystem()
|
||||
}
|
||||
|
||||
@AfterEach override def tearDown() = {
|
||||
cleanupSparkContexts()
|
||||
cleanupTestDataGenerator()
|
||||
cleanupFileSystem()
|
||||
}
|
||||
|
||||
@Test def testShortNameStorage() {
|
||||
// Insert Operation
|
||||
val records = recordsToStrings(dataGen.generateInserts("000", 100)).toList
|
||||
val inputDF = spark.read.json(spark.sparkContext.parallelize(records, 2))
|
||||
inputDF.write.format("hudi")
|
||||
.options(commonOpts)
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
|
||||
.mode(SaveMode.Overwrite)
|
||||
.save(basePath)
|
||||
|
||||
assertTrue(HoodieDataSourceHelpers.hasNewCommits(fs, basePath, "000"))
|
||||
}
|
||||
|
||||
@Test def testCopyOnWriteStorage() {
|
||||
// Insert Operation
|
||||
val records1 = recordsToStrings(dataGen.generateInserts("000", 100)).toList
|
||||
val inputDF1 = spark.read.json(spark.sparkContext.parallelize(records1, 2))
|
||||
inputDF1.write.format("org.apache.hudi")
|
||||
.options(commonOpts)
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
|
||||
.mode(SaveMode.Overwrite)
|
||||
.save(basePath)
|
||||
|
||||
assertTrue(HoodieDataSourceHelpers.hasNewCommits(fs, basePath, "000"))
|
||||
val commitInstantTime1 = HoodieDataSourceHelpers.latestCommit(fs, basePath)
|
||||
|
||||
// Snapshot query
|
||||
val snapshotDF1 = spark.read.format("org.apache.hudi").load(basePath + "/*/*/*/*")
|
||||
assertEquals(100, snapshotDF1.count())
|
||||
|
||||
val records2 = recordsToStrings(dataGen.generateUpdates("001", 100)).toList
|
||||
val inputDF2 = spark.read.json(spark.sparkContext.parallelize(records2, 2))
|
||||
val uniqueKeyCnt = inputDF2.select("_row_key").distinct().count()
|
||||
|
||||
// Upsert Operation
|
||||
inputDF2.write.format("org.apache.hudi")
|
||||
.options(commonOpts)
|
||||
.mode(SaveMode.Append)
|
||||
.save(basePath)
|
||||
|
||||
val commitInstantTime2 = HoodieDataSourceHelpers.latestCommit(fs, basePath)
|
||||
assertEquals(2, HoodieDataSourceHelpers.listCommitsSince(fs, basePath, "000").size())
|
||||
|
||||
// Snapshot Query
|
||||
val snapshotDF2 = spark.read.format("org.apache.hudi")
|
||||
.load(basePath + "/*/*/*/*")
|
||||
assertEquals(100, snapshotDF2.count()) // still 100, since we only updated
|
||||
|
||||
// Read Incremental Query
|
||||
// we have 2 commits, try pulling the first commit (which is not the latest)
|
||||
val firstCommit = HoodieDataSourceHelpers.listCommitsSince(fs, basePath, "000").get(0)
|
||||
val hoodieIncViewDF1 = spark.read.format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL)
|
||||
.option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, "000")
|
||||
.option(DataSourceReadOptions.END_INSTANTTIME_OPT_KEY, firstCommit)
|
||||
.load(basePath)
|
||||
assertEquals(100, hoodieIncViewDF1.count()) // 100 initial inserts must be pulled
|
||||
var countsPerCommit = hoodieIncViewDF1.groupBy("_hoodie_commit_time").count().collect()
|
||||
assertEquals(1, countsPerCommit.length)
|
||||
assertEquals(firstCommit, countsPerCommit(0).get(0))
|
||||
|
||||
// Upsert an empty dataFrame
|
||||
val emptyRecords = recordsToStrings(dataGen.generateUpdates("002", 0)).toList
|
||||
val emptyDF = spark.read.json(spark.sparkContext.parallelize(emptyRecords, 1))
|
||||
emptyDF.write.format("org.apache.hudi")
|
||||
.options(commonOpts)
|
||||
.mode(SaveMode.Append)
|
||||
.save(basePath)
|
||||
|
||||
// pull the latest commit
|
||||
val hoodieIncViewDF2 = spark.read.format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL)
|
||||
.option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, commitInstantTime1)
|
||||
.load(basePath)
|
||||
|
||||
assertEquals(uniqueKeyCnt, hoodieIncViewDF2.count()) // 100 records must be pulled
|
||||
countsPerCommit = hoodieIncViewDF2.groupBy("_hoodie_commit_time").count().collect()
|
||||
assertEquals(1, countsPerCommit.length)
|
||||
assertEquals(commitInstantTime2, countsPerCommit(0).get(0))
|
||||
|
||||
// pull the latest commit within certain partitions
|
||||
val hoodieIncViewDF3 = spark.read.format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL)
|
||||
.option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, commitInstantTime1)
|
||||
.option(DataSourceReadOptions.INCR_PATH_GLOB_OPT_KEY, "/2016/*/*/*")
|
||||
.load(basePath)
|
||||
assertEquals(hoodieIncViewDF2.filter(col("_hoodie_partition_path").contains("2016")).count(), hoodieIncViewDF3.count())
|
||||
|
||||
val timeTravelDF = spark.read.format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL)
|
||||
.option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, "000")
|
||||
.option(DataSourceReadOptions.END_INSTANTTIME_OPT_KEY, firstCommit)
|
||||
.load(basePath)
|
||||
assertEquals(100, timeTravelDF.count()) // 100 initial inserts must be pulled
|
||||
}
|
||||
|
||||
@Test def testOverWriteModeUseReplaceAction(): Unit = {
|
||||
val records1 = recordsToStrings(dataGen.generateInserts("001", 5)).toList
|
||||
val inputDF1 = spark.read.json(spark.sparkContext.parallelize(records1, 2))
|
||||
inputDF1.write.format("org.apache.hudi")
|
||||
.options(commonOpts)
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
|
||||
.mode(SaveMode.Append)
|
||||
.save(basePath)
|
||||
|
||||
val records2 = recordsToStrings(dataGen.generateInserts("002", 5)).toList
|
||||
val inputDF2 = spark.read.json(spark.sparkContext.parallelize(records2, 2))
|
||||
inputDF2.write.format("org.apache.hudi")
|
||||
.options(commonOpts)
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
|
||||
.mode(SaveMode.Overwrite)
|
||||
.save(basePath)
|
||||
|
||||
val metaClient = new HoodieTableMetaClient(spark.sparkContext.hadoopConfiguration, basePath, true)
|
||||
val commits = metaClient.getActiveTimeline.filterCompletedInstants().getInstants.toArray
|
||||
.map(instant => (instant.asInstanceOf[HoodieInstant]).getAction)
|
||||
assertEquals(2, commits.size)
|
||||
assertEquals("commit", commits(0))
|
||||
assertEquals("replacecommit", commits(1))
|
||||
}
|
||||
|
||||
@Test def testOverWriteModeUseReplaceActionOnDisJointPartitions(): Unit = {
|
||||
// step1: Write 5 records to hoodie table for partition1 DEFAULT_FIRST_PARTITION_PATH
|
||||
val records1 = recordsToStrings(dataGen.generateInsertsForPartition("001", 5, HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH)).toList
|
||||
val inputDF1 = spark.read.json(spark.sparkContext.parallelize(records1, 2))
|
||||
inputDF1.write.format("org.apache.hudi")
|
||||
.options(commonOpts)
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
|
||||
.mode(SaveMode.Append)
|
||||
.save(basePath)
|
||||
|
||||
// step2: Write 7 more rectestOverWriteModeUseReplaceActionords using SaveMode.Overwrite for partition2 DEFAULT_SECOND_PARTITION_PATH
|
||||
val records2 = recordsToStrings(dataGen.generateInsertsForPartition("002", 7, HoodieTestDataGenerator.DEFAULT_SECOND_PARTITION_PATH)).toList
|
||||
val inputDF2 = spark.read.json(spark.sparkContext.parallelize(records2, 2))
|
||||
inputDF2.write.format("org.apache.hudi")
|
||||
.options(commonOpts)
|
||||
.mode(SaveMode.Overwrite)
|
||||
.save(basePath)
|
||||
|
||||
val allRecords = spark.read.format("org.apache.hudi").load(basePath + "/*/*/*")
|
||||
allRecords.registerTempTable("tmpTable")
|
||||
|
||||
spark.sql(String.format("select count(*) from tmpTable")).show()
|
||||
|
||||
// step3: Query the rows count from hoodie table for partition1 DEFAULT_FIRST_PARTITION_PATH
|
||||
val recordCountForParititon1 = spark.sql(String.format("select count(*) from tmpTable where partition = '%s'", HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH)).collect()
|
||||
assertEquals("0", recordCountForParititon1(0).get(0).toString)
|
||||
|
||||
// step4: Query the rows count from hoodie table for partition1 DEFAULT_SECOND_PARTITION_PATH
|
||||
val recordCountForParititon2 = spark.sql(String.format("select count(*) from tmpTable where partition = '%s'", HoodieTestDataGenerator.DEFAULT_SECOND_PARTITION_PATH)).collect()
|
||||
assertEquals("7", recordCountForParititon2(0).get(0).toString)
|
||||
|
||||
// step5: Query the rows count from hoodie table
|
||||
val recordCount = spark.sql(String.format("select count(*) from tmpTable")).collect()
|
||||
assertEquals("7", recordCountForParititon2(0).get(0).toString)
|
||||
|
||||
// step6: Query the rows count from hoodie table for partition1 DEFAULT_SECOND_PARTITION_PATH using spark.collect and then filter mode
|
||||
val recordsForPartitionColumn = spark.sql(String.format("select partition from tmpTable")).collect()
|
||||
val filterSecondPartitionCount = recordsForPartitionColumn.filter(row => row.get(0).equals(HoodieTestDataGenerator.DEFAULT_SECOND_PARTITION_PATH)).size
|
||||
assertEquals(7,filterSecondPartitionCount)
|
||||
|
||||
val metaClient = new HoodieTableMetaClient(spark.sparkContext.hadoopConfiguration, basePath, true)
|
||||
val commits = metaClient.getActiveTimeline.filterCompletedInstants().getInstants.toArray
|
||||
.map(instant => instant.asInstanceOf[HoodieInstant].getAction)
|
||||
assertEquals(2, commits.size)
|
||||
assertEquals("commit", commits(0))
|
||||
assertEquals("replacecommit", commits(1))
|
||||
}
|
||||
|
||||
@Test def testDropInsertDup(): Unit = {
|
||||
val insert1Cnt = 10
|
||||
val insert2DupKeyCnt = 9
|
||||
val insert2NewKeyCnt = 2
|
||||
|
||||
val totalUniqueKeyToGenerate = insert1Cnt + insert2NewKeyCnt
|
||||
val allRecords = dataGen.generateInserts("001", totalUniqueKeyToGenerate)
|
||||
val inserts1 = allRecords.subList(0, insert1Cnt)
|
||||
val inserts2New = dataGen.generateSameKeyInserts("002", allRecords.subList(insert1Cnt, insert1Cnt + insert2NewKeyCnt))
|
||||
val inserts2Dup = dataGen.generateSameKeyInserts("002", inserts1.subList(0, insert2DupKeyCnt))
|
||||
|
||||
val records1 = recordsToStrings(inserts1).toList
|
||||
val inputDF1 = spark.read.json(spark.sparkContext.parallelize(records1, 2))
|
||||
inputDF1.write.format("org.apache.hudi")
|
||||
.options(commonOpts)
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
|
||||
.mode(SaveMode.Overwrite)
|
||||
.save(basePath)
|
||||
val hoodieROViewDF1 = spark.read.format("org.apache.hudi")
|
||||
.load(basePath + "/*/*/*/*")
|
||||
assertEquals(insert1Cnt, hoodieROViewDF1.count())
|
||||
|
||||
val commitInstantTime1 = HoodieDataSourceHelpers.latestCommit(fs, basePath)
|
||||
val records2 = recordsToStrings(inserts2Dup ++ inserts2New).toList
|
||||
val inputDF2 = spark.read.json(spark.sparkContext.parallelize(records2, 2))
|
||||
inputDF2.write.format("org.apache.hudi")
|
||||
.options(commonOpts)
|
||||
.option(DataSourceWriteOptions.INSERT_DROP_DUPS_OPT_KEY, "true")
|
||||
.mode(SaveMode.Append)
|
||||
.save(basePath)
|
||||
val hoodieROViewDF2 = spark.read.format("org.apache.hudi")
|
||||
.load(basePath + "/*/*/*/*")
|
||||
assertEquals(hoodieROViewDF2.count(), totalUniqueKeyToGenerate)
|
||||
|
||||
val hoodieIncViewDF2 = spark.read.format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL)
|
||||
.option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, commitInstantTime1)
|
||||
.load(basePath)
|
||||
assertEquals(hoodieIncViewDF2.count(), insert2NewKeyCnt)
|
||||
}
|
||||
|
||||
@Test def testComplexDataTypeWriteAndReadConsistency(): Unit = {
|
||||
val schema = StructType(StructField("_row_key", StringType, true) :: StructField("name", StringType, true)
|
||||
:: StructField("timeStampValue", TimestampType, true) :: StructField("dateValue", DateType, true)
|
||||
:: StructField("decimalValue", DataTypes.createDecimalType(15, 10), true) :: StructField("timestamp", IntegerType, true)
|
||||
:: StructField("partition", IntegerType, true) :: Nil)
|
||||
|
||||
val records = Seq(Row("11", "Andy", Timestamp.valueOf("1970-01-01 13:31:24"), Date.valueOf("1991-11-07"), BigDecimal.valueOf(1.0), 11, 1),
|
||||
Row("22", "lisi", Timestamp.valueOf("1970-01-02 13:31:24"), Date.valueOf("1991-11-08"), BigDecimal.valueOf(2.0), 11, 1),
|
||||
Row("33", "zhangsan", Timestamp.valueOf("1970-01-03 13:31:24"), Date.valueOf("1991-11-09"), BigDecimal.valueOf(3.0), 11, 1))
|
||||
val rdd = jsc.parallelize(records)
|
||||
val recordsDF = spark.createDataFrame(rdd, schema)
|
||||
recordsDF.write.format("org.apache.hudi")
|
||||
.options(commonOpts)
|
||||
.mode(SaveMode.Append)
|
||||
.save(basePath)
|
||||
|
||||
val recordsReadDF = spark.read.format("org.apache.hudi")
|
||||
.load(basePath + "/*/*")
|
||||
recordsReadDF.printSchema()
|
||||
recordsReadDF.schema.foreach(f => {
|
||||
f.name match {
|
||||
case "timeStampValue" =>
|
||||
assertEquals(f.dataType, org.apache.spark.sql.types.TimestampType)
|
||||
case "dateValue" =>
|
||||
assertEquals(f.dataType, org.apache.spark.sql.types.DateType)
|
||||
case "decimalValue" =>
|
||||
assertEquals(f.dataType, org.apache.spark.sql.types.DecimalType(15, 10))
|
||||
case _ =>
|
||||
}
|
||||
})
|
||||
}
|
||||
|
||||
@Test def testWithAutoCommitOn(): Unit = {
|
||||
val records1 = recordsToStrings(dataGen.generateInserts("000", 100)).toList
|
||||
val inputDF1 = spark.read.json(spark.sparkContext.parallelize(records1, 2))
|
||||
inputDF1.write.format("org.apache.hudi")
|
||||
.options(commonOpts)
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
|
||||
.option(HoodieWriteConfig.HOODIE_AUTO_COMMIT_PROP, "true")
|
||||
.mode(SaveMode.Overwrite)
|
||||
.save(basePath)
|
||||
|
||||
assertTrue(HoodieDataSourceHelpers.hasNewCommits(fs, basePath, "000"))
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,630 @@
|
||||
/*
|
||||
* 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.functional
|
||||
|
||||
import java.time.Instant
|
||||
import java.util.Collections
|
||||
|
||||
import collection.JavaConverters._
|
||||
import org.apache.hadoop.fs.FileSystem
|
||||
import org.apache.hudi.bootstrap.SparkParquetBootstrapDataProvider
|
||||
import org.apache.hudi.client.TestBootstrap
|
||||
import org.apache.hudi.client.bootstrap.selector.FullRecordBootstrapModeSelector
|
||||
import org.apache.hudi.{DataSourceReadOptions, DataSourceWriteOptions, HoodieDataSourceHelpers}
|
||||
import org.apache.hudi.common.fs.FSUtils
|
||||
import org.apache.hudi.common.table.timeline.HoodieTimeline
|
||||
import org.apache.hudi.config.{HoodieBootstrapConfig, HoodieCompactionConfig, HoodieWriteConfig}
|
||||
import org.apache.hudi.keygen.SimpleKeyGenerator
|
||||
import org.apache.spark.api.java.JavaSparkContext
|
||||
import org.apache.spark.sql.functions.{col, lit}
|
||||
import org.apache.spark.sql.{SaveMode, SparkSession}
|
||||
import org.junit.jupiter.api.Assertions.assertEquals
|
||||
import org.junit.jupiter.api.{AfterEach, BeforeEach, Test}
|
||||
import org.junit.jupiter.api.io.TempDir
|
||||
|
||||
class TestDataSourceForBootstrap {
|
||||
|
||||
var spark: SparkSession = _
|
||||
val commonOpts = Map(
|
||||
HoodieWriteConfig.INSERT_PARALLELISM -> "4",
|
||||
HoodieWriteConfig.UPSERT_PARALLELISM -> "4",
|
||||
HoodieWriteConfig.DELETE_PARALLELISM -> "4",
|
||||
HoodieWriteConfig.BULKINSERT_PARALLELISM -> "4",
|
||||
HoodieWriteConfig.FINALIZE_WRITE_PARALLELISM -> "4",
|
||||
HoodieBootstrapConfig.BOOTSTRAP_PARALLELISM -> "4",
|
||||
DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY -> "_row_key",
|
||||
DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY -> "partition",
|
||||
DataSourceWriteOptions.PRECOMBINE_FIELD_OPT_KEY -> "timestamp",
|
||||
HoodieWriteConfig.TABLE_NAME -> "hoodie_test"
|
||||
)
|
||||
var basePath: String = _
|
||||
var srcPath: String = _
|
||||
var fs: FileSystem = _
|
||||
|
||||
@BeforeEach def initialize(@TempDir tempDir: java.nio.file.Path) {
|
||||
spark = SparkSession.builder
|
||||
.appName("Hoodie Datasource test")
|
||||
.master("local[2]")
|
||||
.config("spark.serializer", "org.apache.spark.serializer.KryoSerializer")
|
||||
.getOrCreate
|
||||
basePath = tempDir.toAbsolutePath.toString + "/base"
|
||||
srcPath = tempDir.toAbsolutePath.toString + "/src"
|
||||
fs = FSUtils.getFs(basePath, spark.sparkContext.hadoopConfiguration)
|
||||
}
|
||||
|
||||
@AfterEach def tearDown(): Unit ={
|
||||
// Close spark session
|
||||
if (spark != null) {
|
||||
spark.stop()
|
||||
spark = null
|
||||
}
|
||||
|
||||
// Close file system
|
||||
if (fs != null) {
|
||||
fs.close()
|
||||
fs = null
|
||||
}
|
||||
}
|
||||
|
||||
@Test def testMetadataBootstrapCOWNonPartitioned(): Unit = {
|
||||
val timestamp = Instant.now.toEpochMilli
|
||||
val numRecords = 100
|
||||
val jsc = JavaSparkContext.fromSparkContext(spark.sparkContext)
|
||||
|
||||
val sourceDF = TestBootstrap.generateTestRawTripDataset(timestamp, 0, numRecords, Collections.emptyList(), jsc,
|
||||
spark.sqlContext)
|
||||
|
||||
// Write source data non-partitioned
|
||||
sourceDF.write
|
||||
.format("parquet")
|
||||
.mode(SaveMode.Overwrite)
|
||||
.save(srcPath)
|
||||
|
||||
// Perform bootstrap
|
||||
val bootstrapDF = spark.emptyDataFrame
|
||||
bootstrapDF.write
|
||||
.format("hudi")
|
||||
.options(commonOpts)
|
||||
.option(HoodieWriteConfig.TABLE_NAME, "hoodie_test")
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.BOOTSTRAP_OPERATION_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY, "_row_key")
|
||||
.option(HoodieBootstrapConfig.BOOTSTRAP_BASE_PATH_PROP, srcPath)
|
||||
.option(HoodieBootstrapConfig.BOOTSTRAP_KEYGEN_CLASS, "org.apache.hudi.keygen.NonpartitionedKeyGenerator")
|
||||
.mode(SaveMode.Overwrite)
|
||||
.save(basePath)
|
||||
|
||||
val commitInstantTime1: String = HoodieDataSourceHelpers.latestCommit(fs, basePath)
|
||||
assertEquals(HoodieTimeline.METADATA_BOOTSTRAP_INSTANT_TS, commitInstantTime1)
|
||||
|
||||
// Read bootstrapped table and verify count
|
||||
var hoodieROViewDF1 = spark.read.format("hudi").load(basePath + "/*")
|
||||
assertEquals(numRecords, hoodieROViewDF1.count())
|
||||
|
||||
// Perform upsert
|
||||
val updateTimestamp = Instant.now.toEpochMilli
|
||||
val numRecordsUpdate = 10
|
||||
val updateDF = TestBootstrap.generateTestRawTripDataset(updateTimestamp, 0, numRecordsUpdate,
|
||||
Collections.emptyList(), jsc, spark.sqlContext)
|
||||
|
||||
updateDF.write
|
||||
.format("hudi")
|
||||
.options(commonOpts)
|
||||
.option(HoodieWriteConfig.TABLE_NAME, "hoodie_test")
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.UPSERT_OPERATION_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY, DataSourceWriteOptions.COW_TABLE_TYPE_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY, "_row_key")
|
||||
.option(DataSourceWriteOptions.PRECOMBINE_FIELD_OPT_KEY, "timestamp")
|
||||
.option(DataSourceWriteOptions.KEYGENERATOR_CLASS_OPT_KEY, "org.apache.hudi.keygen.NonpartitionedKeyGenerator")
|
||||
.mode(SaveMode.Append)
|
||||
.save(basePath)
|
||||
|
||||
val commitInstantTime2: String = HoodieDataSourceHelpers.latestCommit(fs, basePath)
|
||||
assertEquals(1, HoodieDataSourceHelpers.listCommitsSince(fs, basePath, commitInstantTime1).size())
|
||||
|
||||
// Read table after upsert and verify count
|
||||
hoodieROViewDF1 = spark.read.format("hudi").load(basePath + "/*")
|
||||
assertEquals(numRecords, hoodieROViewDF1.count())
|
||||
assertEquals(numRecordsUpdate, hoodieROViewDF1.filter(s"timestamp == $updateTimestamp").count())
|
||||
|
||||
// incrementally pull only changes in the bootstrap commit, which would pull all the initial records written
|
||||
// during bootstrap
|
||||
val hoodieIncViewDF1 = spark.read.format("hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL)
|
||||
.option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, "000")
|
||||
.option(DataSourceReadOptions.END_INSTANTTIME_OPT_KEY, commitInstantTime1)
|
||||
.load(basePath)
|
||||
|
||||
assertEquals(numRecords, hoodieIncViewDF1.count())
|
||||
var countsPerCommit = hoodieIncViewDF1.groupBy("_hoodie_commit_time").count().collect();
|
||||
assertEquals(1, countsPerCommit.length)
|
||||
assertEquals(commitInstantTime1, countsPerCommit(0).get(0))
|
||||
|
||||
// incrementally pull only changes in the latest commit, which would pull only the updated records in the
|
||||
// latest commit
|
||||
val hoodieIncViewDF2 = spark.read.format("hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL)
|
||||
.option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, commitInstantTime1)
|
||||
.load(basePath);
|
||||
|
||||
assertEquals(numRecordsUpdate, hoodieIncViewDF2.count())
|
||||
countsPerCommit = hoodieIncViewDF2.groupBy("_hoodie_commit_time").count().collect();
|
||||
assertEquals(1, countsPerCommit.length)
|
||||
assertEquals(commitInstantTime2, countsPerCommit(0).get(0))
|
||||
}
|
||||
|
||||
@Test def testMetadataBootstrapCOWHiveStylePartitioned(): Unit = {
|
||||
val timestamp = Instant.now.toEpochMilli
|
||||
val numRecords = 100
|
||||
val partitionPaths = List("2020-04-01", "2020-04-02", "2020-04-03")
|
||||
val jsc = JavaSparkContext.fromSparkContext(spark.sparkContext)
|
||||
|
||||
val sourceDF = TestBootstrap.generateTestRawTripDataset(timestamp, 0, numRecords, partitionPaths.asJava, jsc,
|
||||
spark.sqlContext)
|
||||
|
||||
// Write source data hive style partitioned
|
||||
sourceDF.write
|
||||
.partitionBy("datestr")
|
||||
.format("parquet")
|
||||
.mode(SaveMode.Overwrite)
|
||||
.save(srcPath)
|
||||
|
||||
// Perform bootstrap
|
||||
val bootstrapDF = spark.emptyDataFrame
|
||||
bootstrapDF.write
|
||||
.format("hudi")
|
||||
.options(commonOpts)
|
||||
.option(HoodieWriteConfig.TABLE_NAME, "hoodie_test")
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.BOOTSTRAP_OPERATION_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY, "_row_key")
|
||||
.option(HoodieBootstrapConfig.BOOTSTRAP_BASE_PATH_PROP, srcPath)
|
||||
.option(HoodieBootstrapConfig.BOOTSTRAP_KEYGEN_CLASS, "org.apache.hudi.keygen.SimpleKeyGenerator")
|
||||
.mode(SaveMode.Overwrite)
|
||||
.save(basePath)
|
||||
|
||||
val commitInstantTime1: String = HoodieDataSourceHelpers.latestCommit(fs, basePath)
|
||||
assertEquals(HoodieTimeline.METADATA_BOOTSTRAP_INSTANT_TS, commitInstantTime1)
|
||||
|
||||
// Read bootstrapped table and verify count
|
||||
val hoodieROViewDF1 = spark.read.format("hudi").load(basePath + "/*")
|
||||
assertEquals(numRecords, hoodieROViewDF1.count())
|
||||
|
||||
// Perform upsert
|
||||
val updateTimestamp = Instant.now.toEpochMilli
|
||||
val numRecordsUpdate = 10
|
||||
val updateDF = TestBootstrap.generateTestRawTripDataset(updateTimestamp, 0, numRecordsUpdate, partitionPaths.asJava,
|
||||
jsc, spark.sqlContext)
|
||||
|
||||
updateDF.write
|
||||
.format("hudi")
|
||||
.options(commonOpts)
|
||||
.option(HoodieWriteConfig.TABLE_NAME, "hoodie_test")
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.UPSERT_OPERATION_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY, DataSourceWriteOptions.COW_TABLE_TYPE_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY, "_row_key")
|
||||
.option(DataSourceWriteOptions.PRECOMBINE_FIELD_OPT_KEY, "timestamp")
|
||||
.option(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY, "datestr")
|
||||
// Required because source data is hive style partitioned
|
||||
.option(DataSourceWriteOptions.HIVE_STYLE_PARTITIONING_OPT_KEY, "true")
|
||||
.mode(SaveMode.Append)
|
||||
.save(basePath)
|
||||
|
||||
val commitInstantTime2: String = HoodieDataSourceHelpers.latestCommit(fs, basePath)
|
||||
assertEquals(1, HoodieDataSourceHelpers.listCommitsSince(fs, basePath, commitInstantTime1).size())
|
||||
|
||||
// Read table after upsert and verify count
|
||||
val hoodieROViewDF2 = spark.read.format("hudi").load(basePath + "/*")
|
||||
assertEquals(numRecords, hoodieROViewDF2.count())
|
||||
assertEquals(numRecordsUpdate, hoodieROViewDF2.filter(s"timestamp == $updateTimestamp").count())
|
||||
|
||||
// incrementally pull only changes in the bootstrap commit, which would pull all the initial records written
|
||||
// during bootstrap
|
||||
val hoodieIncViewDF1 = spark.read.format("hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL)
|
||||
.option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, "000")
|
||||
.option(DataSourceReadOptions.END_INSTANTTIME_OPT_KEY, commitInstantTime1)
|
||||
.load(basePath)
|
||||
|
||||
assertEquals(numRecords, hoodieIncViewDF1.count())
|
||||
var countsPerCommit = hoodieIncViewDF1.groupBy("_hoodie_commit_time").count().collect();
|
||||
assertEquals(1, countsPerCommit.length)
|
||||
assertEquals(commitInstantTime1, countsPerCommit(0).get(0))
|
||||
|
||||
// incrementally pull only changes in the latest commit, which would pull only the updated records in the
|
||||
// latest commit
|
||||
val hoodieIncViewDF2 = spark.read.format("hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL)
|
||||
.option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, commitInstantTime1)
|
||||
.load(basePath);
|
||||
|
||||
assertEquals(numRecordsUpdate, hoodieIncViewDF2.count())
|
||||
countsPerCommit = hoodieIncViewDF2.groupBy("_hoodie_commit_time").count().collect();
|
||||
assertEquals(1, countsPerCommit.length)
|
||||
assertEquals(commitInstantTime2, countsPerCommit(0).get(0))
|
||||
|
||||
// pull the latest commit within certain partitions
|
||||
val hoodieIncViewDF3 = spark.read.format("hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL)
|
||||
.option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, commitInstantTime1)
|
||||
.option(DataSourceReadOptions.INCR_PATH_GLOB_OPT_KEY, "/datestr=2020-04-02/*")
|
||||
.load(basePath)
|
||||
|
||||
assertEquals(hoodieIncViewDF2.filter(col("_hoodie_partition_path").contains("2020-04-02")).count(),
|
||||
hoodieIncViewDF3.count())
|
||||
}
|
||||
|
||||
@Test def testMetadataBootstrapCOWPartitioned(): Unit = {
|
||||
val timestamp = Instant.now.toEpochMilli
|
||||
val numRecords = 100
|
||||
val partitionPaths = List("2020-04-01", "2020-04-02", "2020-04-03")
|
||||
val jsc = JavaSparkContext.fromSparkContext(spark.sparkContext)
|
||||
|
||||
var sourceDF = TestBootstrap.generateTestRawTripDataset(timestamp, 0, numRecords, partitionPaths.asJava, jsc,
|
||||
spark.sqlContext)
|
||||
|
||||
// Writing data for each partition instead of using partitionBy to avoid hive-style partitioning and hence
|
||||
// have partitioned columns stored in the data file
|
||||
partitionPaths.foreach(partitionPath => {
|
||||
sourceDF
|
||||
.filter(sourceDF("datestr").equalTo(lit(partitionPath)))
|
||||
.write
|
||||
.format("parquet")
|
||||
.mode(SaveMode.Overwrite)
|
||||
.save(srcPath + "/" + partitionPath)
|
||||
})
|
||||
|
||||
// Perform bootstrap
|
||||
val bootstrapDF = spark.emptyDataFrame
|
||||
bootstrapDF.write
|
||||
.format("hudi")
|
||||
.options(commonOpts)
|
||||
.option(HoodieWriteConfig.TABLE_NAME, "hoodie_test")
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.BOOTSTRAP_OPERATION_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY, "_row_key")
|
||||
.option(HoodieBootstrapConfig.BOOTSTRAP_BASE_PATH_PROP, srcPath)
|
||||
.option(HoodieBootstrapConfig.BOOTSTRAP_KEYGEN_CLASS, "org.apache.hudi.keygen.SimpleKeyGenerator")
|
||||
.mode(SaveMode.Overwrite)
|
||||
.save(basePath)
|
||||
|
||||
val commitInstantTime1: String = HoodieDataSourceHelpers.latestCommit(fs, basePath)
|
||||
assertEquals(HoodieTimeline.METADATA_BOOTSTRAP_INSTANT_TS, commitInstantTime1)
|
||||
|
||||
// Read bootstrapped table and verify count
|
||||
val hoodieROViewDF1 = spark.read.format("hudi").load(basePath + "/*")
|
||||
assertEquals(numRecords, hoodieROViewDF1.count())
|
||||
|
||||
// Perform upsert
|
||||
val updateTimestamp = Instant.now.toEpochMilli
|
||||
val numRecordsUpdate = 10
|
||||
var updateDF = TestBootstrap.generateTestRawTripDataset(updateTimestamp, 0, numRecordsUpdate, partitionPaths.asJava,
|
||||
jsc, spark.sqlContext)
|
||||
|
||||
updateDF.write
|
||||
.format("hudi")
|
||||
.options(commonOpts)
|
||||
.option(HoodieWriteConfig.TABLE_NAME, "hoodie_test")
|
||||
.option("hoodie.upsert.shuffle.parallelism", "4")
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.UPSERT_OPERATION_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY, DataSourceWriteOptions.COW_TABLE_TYPE_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY, "_row_key")
|
||||
.option(DataSourceWriteOptions.PRECOMBINE_FIELD_OPT_KEY, "timestamp")
|
||||
.option(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY, "datestr")
|
||||
.mode(SaveMode.Append)
|
||||
.save(basePath)
|
||||
|
||||
val commitInstantTime2: String = HoodieDataSourceHelpers.latestCommit(fs, basePath)
|
||||
assertEquals(1, HoodieDataSourceHelpers.listCommitsSince(fs, basePath, commitInstantTime1).size())
|
||||
|
||||
// Read table after upsert and verify count
|
||||
val hoodieROViewDF2 = spark.read.format("hudi").load(basePath + "/*")
|
||||
assertEquals(numRecords, hoodieROViewDF2.count())
|
||||
assertEquals(numRecordsUpdate, hoodieROViewDF2.filter(s"timestamp == $updateTimestamp").count())
|
||||
|
||||
// incrementally pull only changes in the bootstrap commit, which would pull all the initial records written
|
||||
// during bootstrap
|
||||
val hoodieIncViewDF1 = spark.read.format("hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL)
|
||||
.option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, "000")
|
||||
.option(DataSourceReadOptions.END_INSTANTTIME_OPT_KEY, commitInstantTime1)
|
||||
.load(basePath)
|
||||
|
||||
assertEquals(numRecords, hoodieIncViewDF1.count())
|
||||
var countsPerCommit = hoodieIncViewDF1.groupBy("_hoodie_commit_time").count().collect();
|
||||
assertEquals(1, countsPerCommit.length)
|
||||
assertEquals(commitInstantTime1, countsPerCommit(0).get(0))
|
||||
|
||||
// incrementally pull only changes in the latest commit, which would pull only the updated records in the
|
||||
// latest commit
|
||||
val hoodieIncViewDF2 = spark.read.format("hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL)
|
||||
.option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, commitInstantTime1)
|
||||
.load(basePath);
|
||||
|
||||
assertEquals(numRecordsUpdate, hoodieIncViewDF2.count())
|
||||
countsPerCommit = hoodieIncViewDF2.groupBy("_hoodie_commit_time").count().collect();
|
||||
assertEquals(1, countsPerCommit.length)
|
||||
assertEquals(commitInstantTime2, countsPerCommit(0).get(0))
|
||||
|
||||
// pull the latest commit within certain partitions
|
||||
val hoodieIncViewDF3 = spark.read.format("hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL)
|
||||
.option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, commitInstantTime1)
|
||||
.option(DataSourceReadOptions.INCR_PATH_GLOB_OPT_KEY, "/2020-04-02/*")
|
||||
.load(basePath)
|
||||
|
||||
assertEquals(hoodieIncViewDF2.filter(col("_hoodie_partition_path").contains("2020-04-02")).count(),
|
||||
hoodieIncViewDF3.count())
|
||||
}
|
||||
|
||||
@Test def testMetadataBootstrapMORPartitionedInlineCompactionOn(): Unit = {
|
||||
val timestamp = Instant.now.toEpochMilli
|
||||
val numRecords = 100
|
||||
val partitionPaths = List("2020-04-01", "2020-04-02", "2020-04-03")
|
||||
val jsc = JavaSparkContext.fromSparkContext(spark.sparkContext)
|
||||
|
||||
val sourceDF = TestBootstrap.generateTestRawTripDataset(timestamp, 0, numRecords, partitionPaths.asJava, jsc,
|
||||
spark.sqlContext)
|
||||
|
||||
// Writing data for each partition instead of using partitionBy to avoid hive-style partitioning and hence
|
||||
// have partitioned columns stored in the data file
|
||||
partitionPaths.foreach(partitionPath => {
|
||||
sourceDF
|
||||
.filter(sourceDF("datestr").equalTo(lit(partitionPath)))
|
||||
.write
|
||||
.format("parquet")
|
||||
.mode(SaveMode.Overwrite)
|
||||
.save(srcPath + "/" + partitionPath)
|
||||
})
|
||||
|
||||
// Perform bootstrap
|
||||
val bootstrapDF = spark.emptyDataFrame
|
||||
bootstrapDF.write
|
||||
.format("hudi")
|
||||
.options(commonOpts)
|
||||
.option(HoodieWriteConfig.TABLE_NAME, "hoodie_test")
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.BOOTSTRAP_OPERATION_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY, DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY, "_row_key")
|
||||
.option(HoodieBootstrapConfig.BOOTSTRAP_BASE_PATH_PROP, srcPath)
|
||||
.option(HoodieBootstrapConfig.BOOTSTRAP_KEYGEN_CLASS, "org.apache.hudi.keygen.SimpleKeyGenerator")
|
||||
.mode(SaveMode.Overwrite)
|
||||
.save(basePath)
|
||||
|
||||
val commitInstantTime1: String = HoodieDataSourceHelpers.latestCommit(fs, basePath)
|
||||
assertEquals(HoodieTimeline.METADATA_BOOTSTRAP_INSTANT_TS, commitInstantTime1)
|
||||
|
||||
// Read bootstrapped table and verify count
|
||||
val hoodieROViewDF1 = spark.read.format("hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY,
|
||||
DataSourceReadOptions.QUERY_TYPE_READ_OPTIMIZED_OPT_VAL)
|
||||
.load(basePath + "/*")
|
||||
assertEquals(numRecords, hoodieROViewDF1.count())
|
||||
|
||||
// Perform upsert
|
||||
val updateTimestamp = Instant.now.toEpochMilli
|
||||
val numRecordsUpdate = 10
|
||||
val updateDF = TestBootstrap.generateTestRawTripDataset(updateTimestamp, 0, numRecordsUpdate, partitionPaths.asJava,
|
||||
jsc, spark.sqlContext)
|
||||
|
||||
updateDF.write
|
||||
.format("hudi")
|
||||
.options(commonOpts)
|
||||
.option(HoodieWriteConfig.TABLE_NAME, "hoodie_test")
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.UPSERT_OPERATION_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY, DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY, "_row_key")
|
||||
.option(DataSourceWriteOptions.PRECOMBINE_FIELD_OPT_KEY, "timestamp")
|
||||
.option(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY, "datestr")
|
||||
.option(HoodieCompactionConfig.INLINE_COMPACT_PROP, "true")
|
||||
.option(HoodieCompactionConfig.INLINE_COMPACT_NUM_DELTA_COMMITS_PROP, "1")
|
||||
.mode(SaveMode.Append)
|
||||
.save(basePath)
|
||||
|
||||
// Expect 2 new commits since meta bootstrap - delta commit and compaction commit (due to inline compaction)
|
||||
assertEquals(2, HoodieDataSourceHelpers.listCommitsSince(fs, basePath, commitInstantTime1).size())
|
||||
|
||||
// Read table after upsert and verify count. Since we have inline compaction enabled the RO view will have
|
||||
// the updated rows.
|
||||
val hoodieROViewDF2 = spark.read.format("hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY,
|
||||
DataSourceReadOptions.QUERY_TYPE_READ_OPTIMIZED_OPT_VAL)
|
||||
.load(basePath + "/*")
|
||||
assertEquals(numRecords, hoodieROViewDF2.count())
|
||||
assertEquals(numRecordsUpdate, hoodieROViewDF2.filter(s"timestamp == $updateTimestamp").count())
|
||||
}
|
||||
|
||||
@Test def testMetadataBootstrapMORPartitioned(): Unit = {
|
||||
val timestamp = Instant.now.toEpochMilli
|
||||
val numRecords = 100
|
||||
val partitionPaths = List("2020-04-01", "2020-04-02", "2020-04-03")
|
||||
val jsc = JavaSparkContext.fromSparkContext(spark.sparkContext)
|
||||
|
||||
val sourceDF = TestBootstrap.generateTestRawTripDataset(timestamp, 0, numRecords, partitionPaths.asJava, jsc,
|
||||
spark.sqlContext)
|
||||
|
||||
// Writing data for each partition instead of using partitionBy to avoid hive-style partitioning and hence
|
||||
// have partitioned columns stored in the data file
|
||||
partitionPaths.foreach(partitionPath => {
|
||||
sourceDF
|
||||
.filter(sourceDF("datestr").equalTo(lit(partitionPath)))
|
||||
.write
|
||||
.format("parquet")
|
||||
.mode(SaveMode.Overwrite)
|
||||
.save(srcPath + "/" + partitionPath)
|
||||
})
|
||||
|
||||
// Perform bootstrap
|
||||
val bootstrapDF = spark.emptyDataFrame
|
||||
bootstrapDF.write
|
||||
.format("hudi")
|
||||
.options(commonOpts)
|
||||
.option(HoodieWriteConfig.TABLE_NAME, "hoodie_test")
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.BOOTSTRAP_OPERATION_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY, DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY, "_row_key")
|
||||
.option(HoodieBootstrapConfig.BOOTSTRAP_BASE_PATH_PROP, srcPath)
|
||||
.option(HoodieBootstrapConfig.BOOTSTRAP_KEYGEN_CLASS, "org.apache.hudi.keygen.SimpleKeyGenerator")
|
||||
.mode(SaveMode.Overwrite)
|
||||
.save(basePath)
|
||||
|
||||
val commitInstantTime1: String = HoodieDataSourceHelpers.latestCommit(fs, basePath)
|
||||
assertEquals(HoodieTimeline.METADATA_BOOTSTRAP_INSTANT_TS, commitInstantTime1)
|
||||
|
||||
// Read bootstrapped table and verify count
|
||||
val hoodieROViewDF1 = spark.read.format("hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY,
|
||||
DataSourceReadOptions.QUERY_TYPE_READ_OPTIMIZED_OPT_VAL)
|
||||
.load(basePath + "/*")
|
||||
assertEquals(numRecords, hoodieROViewDF1.count())
|
||||
|
||||
// Perform upsert
|
||||
val updateTimestamp = Instant.now.toEpochMilli
|
||||
val numRecordsUpdate = 10
|
||||
val updateDF = TestBootstrap.generateTestRawTripDataset(updateTimestamp, 0, numRecordsUpdate,
|
||||
partitionPaths.asJava, jsc, spark.sqlContext)
|
||||
|
||||
updateDF.write
|
||||
.format("hudi")
|
||||
.options(commonOpts)
|
||||
.option(HoodieWriteConfig.TABLE_NAME, "hoodie_test")
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.UPSERT_OPERATION_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY, DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY, "_row_key")
|
||||
.option(DataSourceWriteOptions.PRECOMBINE_FIELD_OPT_KEY, "timestamp")
|
||||
.option(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY, "datestr")
|
||||
.mode(SaveMode.Append)
|
||||
.save(basePath)
|
||||
|
||||
// Expect 1 new commit since meta bootstrap - delta commit (because inline compaction is off)
|
||||
assertEquals(1, HoodieDataSourceHelpers.listCommitsSince(fs, basePath, commitInstantTime1).size())
|
||||
|
||||
// Read table after upsert and verify count. Since we have inline compaction off the RO view will have
|
||||
// no updated rows.
|
||||
val hoodieROViewDF2 = spark.read.format("hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY,
|
||||
DataSourceReadOptions.QUERY_TYPE_READ_OPTIMIZED_OPT_VAL)
|
||||
.load(basePath + "/*")
|
||||
assertEquals(numRecords, hoodieROViewDF2.count())
|
||||
assertEquals(0, hoodieROViewDF2.filter(s"timestamp == $updateTimestamp").count())
|
||||
}
|
||||
|
||||
@Test def testFullBootstrapCOWPartitioned(): Unit = {
|
||||
val timestamp = Instant.now.toEpochMilli
|
||||
val numRecords = 100
|
||||
val partitionPaths = List("2020-04-01", "2020-04-02", "2020-04-03")
|
||||
val jsc = JavaSparkContext.fromSparkContext(spark.sparkContext)
|
||||
|
||||
val sourceDF = TestBootstrap.generateTestRawTripDataset(timestamp, 0, numRecords, partitionPaths.asJava, jsc,
|
||||
spark.sqlContext)
|
||||
|
||||
// Writing data for each partition instead of using partitionBy to avoid hive-style partitioning and hence
|
||||
// have partitioned columns stored in the data file
|
||||
partitionPaths.foreach(partitionPath => {
|
||||
sourceDF
|
||||
.filter(sourceDF("datestr").equalTo(lit(partitionPath)))
|
||||
.write
|
||||
.format("parquet")
|
||||
.mode(SaveMode.Overwrite)
|
||||
.save(srcPath + "/" + partitionPath)
|
||||
})
|
||||
|
||||
// Perform bootstrap
|
||||
val bootstrapDF = spark.emptyDataFrame
|
||||
bootstrapDF.write
|
||||
.format("hudi")
|
||||
.options(commonOpts)
|
||||
.option(HoodieWriteConfig.TABLE_NAME, "hoodie_test")
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.BOOTSTRAP_OPERATION_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY, "_row_key")
|
||||
.option(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY, "datestr")
|
||||
.option(DataSourceWriteOptions.PRECOMBINE_FIELD_OPT_KEY, "timestamp")
|
||||
.option(HoodieBootstrapConfig.BOOTSTRAP_BASE_PATH_PROP, srcPath)
|
||||
.option(HoodieBootstrapConfig.BOOTSTRAP_KEYGEN_CLASS, classOf[SimpleKeyGenerator].getName)
|
||||
.option(HoodieBootstrapConfig.BOOTSTRAP_MODE_SELECTOR, classOf[FullRecordBootstrapModeSelector].getName)
|
||||
.option(HoodieBootstrapConfig.FULL_BOOTSTRAP_INPUT_PROVIDER, classOf[SparkParquetBootstrapDataProvider].getName)
|
||||
.mode(SaveMode.Overwrite)
|
||||
.save(basePath)
|
||||
|
||||
val commitInstantTime1: String = HoodieDataSourceHelpers.latestCommit(fs, basePath)
|
||||
assertEquals(HoodieTimeline.FULL_BOOTSTRAP_INSTANT_TS, commitInstantTime1)
|
||||
|
||||
// Read bootstrapped table and verify count
|
||||
val hoodieROViewDF1 = spark.read.format("hudi").load(basePath + "/*")
|
||||
assertEquals(numRecords, hoodieROViewDF1.count())
|
||||
|
||||
// Perform upsert
|
||||
val updateTimestamp = Instant.now.toEpochMilli
|
||||
val numRecordsUpdate = 10
|
||||
val updateDF = TestBootstrap.generateTestRawTripDataset(updateTimestamp, 0, numRecordsUpdate, partitionPaths.asJava,
|
||||
jsc, spark.sqlContext)
|
||||
|
||||
updateDF.write
|
||||
.format("hudi")
|
||||
.options(commonOpts)
|
||||
.option(HoodieWriteConfig.TABLE_NAME, "hoodie_test")
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.UPSERT_OPERATION_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY, DataSourceWriteOptions.COW_TABLE_TYPE_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY, "_row_key")
|
||||
.option(DataSourceWriteOptions.PRECOMBINE_FIELD_OPT_KEY, "timestamp")
|
||||
.option(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY, "datestr")
|
||||
.mode(SaveMode.Append)
|
||||
.save(basePath)
|
||||
|
||||
val commitInstantTime2: String = HoodieDataSourceHelpers.latestCommit(fs, basePath)
|
||||
assertEquals(1, HoodieDataSourceHelpers.listCommitsSince(fs, basePath, commitInstantTime1).size())
|
||||
|
||||
// Read table after upsert and verify count
|
||||
val hoodieROViewDF2 = spark.read.format("hudi").load(basePath + "/*")
|
||||
assertEquals(numRecords, hoodieROViewDF2.count())
|
||||
assertEquals(numRecordsUpdate, hoodieROViewDF2.filter(s"timestamp == $updateTimestamp").count())
|
||||
|
||||
// incrementally pull only changes in the bootstrap commit, which would pull all the initial records written
|
||||
// during bootstrap
|
||||
val hoodieIncViewDF1 = spark.read.format("hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL)
|
||||
.option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, "000")
|
||||
.option(DataSourceReadOptions.END_INSTANTTIME_OPT_KEY, commitInstantTime1)
|
||||
.load(basePath)
|
||||
|
||||
assertEquals(numRecords, hoodieIncViewDF1.count())
|
||||
var countsPerCommit = hoodieIncViewDF1.groupBy("_hoodie_commit_time").count().collect();
|
||||
assertEquals(1, countsPerCommit.length)
|
||||
assertEquals(commitInstantTime1, countsPerCommit(0).get(0))
|
||||
|
||||
// incrementally pull only changes in the latest commit, which would pull only the updated records in the
|
||||
// latest commit
|
||||
val hoodieIncViewDF2 = spark.read.format("hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL)
|
||||
.option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, commitInstantTime1)
|
||||
.load(basePath);
|
||||
|
||||
assertEquals(numRecordsUpdate, hoodieIncViewDF2.count())
|
||||
countsPerCommit = hoodieIncViewDF2.groupBy("_hoodie_commit_time").count().collect();
|
||||
assertEquals(1, countsPerCommit.length)
|
||||
assertEquals(commitInstantTime2, countsPerCommit(0).get(0))
|
||||
|
||||
// pull the latest commit within certain partitions
|
||||
val hoodieIncViewDF3 = spark.read.format("hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL)
|
||||
.option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, commitInstantTime1)
|
||||
.option(DataSourceReadOptions.INCR_PATH_GLOB_OPT_KEY, "/2020-04-02/*")
|
||||
.load(basePath)
|
||||
|
||||
assertEquals(hoodieIncViewDF2.filter(col("_hoodie_partition_path").contains("2020-04-02")).count(),
|
||||
hoodieIncViewDF3.count())
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,391 @@
|
||||
/*
|
||||
* 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.functional
|
||||
|
||||
import org.apache.hudi.common.fs.FSUtils
|
||||
import org.apache.hudi.common.testutils.HoodieTestDataGenerator
|
||||
import org.apache.hudi.config.HoodieWriteConfig
|
||||
import org.apache.hudi.{DataSourceReadOptions, DataSourceWriteOptions, HoodieDataSourceHelpers}
|
||||
import org.apache.hudi.common.testutils.RawTripTestPayload.recordsToStrings
|
||||
import org.apache.hudi.testutils.HoodieClientTestBase
|
||||
import org.apache.log4j.LogManager
|
||||
import org.apache.spark.sql._
|
||||
import org.apache.spark.sql.functions._
|
||||
import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue}
|
||||
import org.junit.jupiter.api.{AfterEach, BeforeEach, Test}
|
||||
|
||||
import scala.collection.JavaConversions._
|
||||
|
||||
/**
|
||||
* Tests on Spark DataSource for MOR table.
|
||||
*/
|
||||
class TestMORDataSource extends HoodieClientTestBase {
|
||||
|
||||
var spark: SparkSession = null
|
||||
private val log = LogManager.getLogger(classOf[TestMORDataSource])
|
||||
val commonOpts = Map(
|
||||
"hoodie.insert.shuffle.parallelism" -> "4",
|
||||
"hoodie.upsert.shuffle.parallelism" -> "4",
|
||||
DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY -> "_row_key",
|
||||
DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY -> "partition",
|
||||
DataSourceWriteOptions.PRECOMBINE_FIELD_OPT_KEY -> "timestamp",
|
||||
HoodieWriteConfig.TABLE_NAME -> "hoodie_test"
|
||||
)
|
||||
|
||||
@BeforeEach override def setUp() {
|
||||
initPath()
|
||||
initSparkContexts()
|
||||
spark = sqlContext.sparkSession
|
||||
initTestDataGenerator()
|
||||
initFileSystem()
|
||||
}
|
||||
|
||||
@AfterEach override def tearDown() = {
|
||||
cleanupSparkContexts()
|
||||
cleanupTestDataGenerator()
|
||||
cleanupFileSystem()
|
||||
}
|
||||
|
||||
@Test def testMergeOnReadStorage() {
|
||||
|
||||
val fs = FSUtils.getFs(basePath, spark.sparkContext.hadoopConfiguration)
|
||||
// Bulk Insert Operation
|
||||
val records1 = recordsToStrings(dataGen.generateInserts("001", 100)).toList
|
||||
val inputDF1: Dataset[Row] = spark.read.json(spark.sparkContext.parallelize(records1, 2))
|
||||
inputDF1.write.format("org.apache.hudi")
|
||||
.options(commonOpts)
|
||||
.option("hoodie.compact.inline", "false") // else fails due to compaction & deltacommit instant times being same
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY, DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL)
|
||||
.mode(SaveMode.Overwrite)
|
||||
.save(basePath)
|
||||
|
||||
assertTrue(HoodieDataSourceHelpers.hasNewCommits(fs, basePath, "000"))
|
||||
|
||||
// Read RO View
|
||||
val hudiRODF1 = spark.read.format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_READ_OPTIMIZED_OPT_VAL)
|
||||
.load(basePath + "/*/*/*/*")
|
||||
assertEquals(100, hudiRODF1.count()) // still 100, since we only updated
|
||||
val insertCommitTime = HoodieDataSourceHelpers.latestCommit(fs, basePath)
|
||||
val insertCommitTimes = hudiRODF1.select("_hoodie_commit_time").distinct().collectAsList().map(r => r.getString(0)).toList
|
||||
assertEquals(List(insertCommitTime), insertCommitTimes)
|
||||
|
||||
// Upsert operation
|
||||
val records2 = recordsToStrings(dataGen.generateUniqueUpdates("002", 100)).toList
|
||||
val inputDF2: Dataset[Row] = spark.read.json(spark.sparkContext.parallelize(records2, 2))
|
||||
inputDF2.write.format("org.apache.hudi")
|
||||
.options(commonOpts)
|
||||
.mode(SaveMode.Append)
|
||||
.save(basePath)
|
||||
|
||||
// Read Snapshot query
|
||||
val updateCommitTime = HoodieDataSourceHelpers.latestCommit(fs, basePath)
|
||||
val hudiSnapshotDF2 = spark.read.format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL)
|
||||
.load(basePath + "/*/*/*/*")
|
||||
val updateCommitTimes = hudiSnapshotDF2.select("_hoodie_commit_time").distinct().collectAsList().map(r => r.getString(0)).toList
|
||||
assertEquals(List(updateCommitTime), updateCommitTimes)
|
||||
}
|
||||
|
||||
@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 inputDF1 = spark.read.json(spark.sparkContext.parallelize(records1, 2))
|
||||
inputDF1.write.format("org.apache.hudi")
|
||||
.options(commonOpts)
|
||||
.option("hoodie.compact.inline", "false") // else fails due to compaction & deltacommit instant times being same
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY, DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL)
|
||||
.mode(SaveMode.Overwrite)
|
||||
.save(basePath)
|
||||
assertTrue(HoodieDataSourceHelpers.hasNewCommits(fs, basePath, "000"))
|
||||
val hudiSnapshotDF1 = spark.read.format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL)
|
||||
.load(basePath + "/*/*/*/*")
|
||||
assertEquals(100, hudiSnapshotDF1.count()) // still 100, since we only updated
|
||||
|
||||
// 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 inputDF2: Dataset[Row] = spark.read.json(spark.sparkContext.parallelize(records2, 2))
|
||||
inputDF2.write.format("org.apache.hudi")
|
||||
.options(commonOpts)
|
||||
.mode(SaveMode.Append)
|
||||
.save(basePath)
|
||||
val hudiSnapshotDF2 = spark.read.format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL)
|
||||
.load(basePath + "/*/*/*/*")
|
||||
assertEquals(100, hudiSnapshotDF2.count()) // still 100, since we only updated
|
||||
val commit1Time = hudiSnapshotDF1.select("_hoodie_commit_time").head().get(0).toString
|
||||
val commit2Time = hudiSnapshotDF2.select("_hoodie_commit_time").head().get(0).toString
|
||||
assertEquals(hudiSnapshotDF2.select("_hoodie_commit_time").distinct().count(), 1)
|
||||
assertTrue(commit2Time > commit1Time)
|
||||
assertEquals(100, hudiSnapshotDF2.join(hudiSnapshotDF1, Seq("_hoodie_record_key"), "left").count())
|
||||
|
||||
// Unmerge
|
||||
val hudiSnapshotSkipMergeDF2 = spark.read.format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL)
|
||||
.option(DataSourceReadOptions.REALTIME_MERGE_OPT_KEY, DataSourceReadOptions.REALTIME_SKIP_MERGE_OPT_VAL)
|
||||
.load(basePath + "/*/*/*/*")
|
||||
assertEquals(200, hudiSnapshotSkipMergeDF2.count())
|
||||
assertEquals(100, hudiSnapshotSkipMergeDF2.select("_hoodie_record_key").distinct().count())
|
||||
assertEquals(200, hudiSnapshotSkipMergeDF2.join(hudiSnapshotDF2, Seq("_hoodie_record_key"), "left").count())
|
||||
|
||||
// Test Read Optimized Query on MOR table
|
||||
val hudiRODF2 = spark.read.format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_READ_OPTIMIZED_OPT_VAL)
|
||||
.load(basePath + "/*/*/*/*")
|
||||
assertEquals(100, hudiRODF2.count())
|
||||
|
||||
// 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 inputDF3: Dataset[Row] = spark.read.json(spark.sparkContext.parallelize(records3, 2))
|
||||
inputDF3.write.format("org.apache.hudi")
|
||||
.options(commonOpts)
|
||||
.mode(SaveMode.Append)
|
||||
.save(basePath)
|
||||
val hudiSnapshotDF3 = spark.read.format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL)
|
||||
.load(basePath + "/*/*/*/*")
|
||||
// still 100, because we only updated the existing records
|
||||
assertEquals(100, hudiSnapshotDF3.count())
|
||||
|
||||
// 50 from commit2, 50 from commit3
|
||||
assertEquals(hudiSnapshotDF3.select("_hoodie_commit_time").distinct().count(), 2)
|
||||
assertEquals(50, hudiSnapshotDF3.filter(col("_hoodie_commit_time") > commit2Time).count())
|
||||
assertEquals(50,
|
||||
hudiSnapshotDF3.join(hudiSnapshotDF2, Seq("_hoodie_record_key", "_hoodie_commit_time"), "inner").count())
|
||||
|
||||
// Fourth Operation:
|
||||
// Insert records to a new partition. Produced a new parquet file.
|
||||
// SNAPSHOT view should read the latest log files from the default partition and parquet from the new partition.
|
||||
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 inputDF4: Dataset[Row] = spark.read.json(spark.sparkContext.parallelize(records4, 2))
|
||||
inputDF4.write.format("org.apache.hudi")
|
||||
.options(commonOpts)
|
||||
.mode(SaveMode.Append)
|
||||
.save(basePath)
|
||||
val hudiSnapshotDF4 = spark.read.format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL)
|
||||
.load(basePath + "/*/*/*/*")
|
||||
// 200, because we insert 100 records to a new partition
|
||||
assertEquals(200, hudiSnapshotDF4.count())
|
||||
assertEquals(100,
|
||||
hudiSnapshotDF1.join(hudiSnapshotDF4, Seq("_hoodie_record_key"), "inner").count())
|
||||
|
||||
// Fifth Operation:
|
||||
// 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.generateUpdates("005", 100)).toList
|
||||
val inputDF5: Dataset[Row] = spark.read.json(spark.sparkContext.parallelize(records5, 2))
|
||||
inputDF5.write.format("org.apache.hudi")
|
||||
.options(commonOpts)
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
|
||||
.mode(SaveMode.Append)
|
||||
.save(basePath)
|
||||
val hudiSnapshotDF5 = spark.read.format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL)
|
||||
.load(basePath + "/*/*/*/*")
|
||||
assertEquals(200, hudiSnapshotDF5.count())
|
||||
}
|
||||
|
||||
@Test
|
||||
def testPayloadDelete() {
|
||||
// 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 inputDF1 = spark.read.json(spark.sparkContext.parallelize(records1, 2))
|
||||
inputDF1.write.format("org.apache.hudi")
|
||||
.options(commonOpts)
|
||||
.option("hoodie.compact.inline", "false") // else fails due to compaction & deltacommit instant times being same
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY, DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL)
|
||||
.mode(SaveMode.Overwrite)
|
||||
.save(basePath)
|
||||
assertTrue(HoodieDataSourceHelpers.hasNewCommits(fs, basePath, "000"))
|
||||
val hudiSnapshotDF1 = spark.read.format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL)
|
||||
.load(basePath + "/*/*/*/*")
|
||||
assertEquals(100, hudiSnapshotDF1.count()) // still 100, since we only updated
|
||||
|
||||
// Second Operation:
|
||||
// Upsert 50 delete records
|
||||
// Snopshot view should only read 50 records
|
||||
val records2 = recordsToStrings(dataGen.generateUniqueDeleteRecords("002", 50)).toList
|
||||
val inputDF2: Dataset[Row] = spark.read.json(spark.sparkContext.parallelize(records2, 2))
|
||||
inputDF2.write.format("org.apache.hudi")
|
||||
.options(commonOpts)
|
||||
.mode(SaveMode.Append)
|
||||
.save(basePath)
|
||||
val hudiSnapshotDF2 = spark.read.format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL)
|
||||
.load(basePath + "/*/*/*/*")
|
||||
assertEquals(50, hudiSnapshotDF2.count()) // 50 records were deleted
|
||||
assertEquals(hudiSnapshotDF2.select("_hoodie_commit_time").distinct().count(), 1)
|
||||
val commit1Time = hudiSnapshotDF1.select("_hoodie_commit_time").head().get(0).toString
|
||||
val commit2Time = hudiSnapshotDF2.select("_hoodie_commit_time").head().get(0).toString
|
||||
assertTrue(commit1Time.equals(commit2Time))
|
||||
assertEquals(50, hudiSnapshotDF2.join(hudiSnapshotDF1, Seq("_hoodie_record_key"), "left").count())
|
||||
|
||||
// unmerge query, skip the delete records
|
||||
val hudiSnapshotDF2Unmerge = spark.read.format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL)
|
||||
.option(DataSourceReadOptions.REALTIME_MERGE_OPT_KEY, DataSourceReadOptions.REALTIME_SKIP_MERGE_OPT_VAL)
|
||||
.load(basePath + "/*/*/*/*")
|
||||
assertEquals(100, hudiSnapshotDF2Unmerge.count())
|
||||
|
||||
// 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 inputDF3: Dataset[Row] = spark.read.json(spark.sparkContext.parallelize(records3, 2))
|
||||
inputDF3.write.format("org.apache.hudi")
|
||||
.options(commonOpts)
|
||||
.mode(SaveMode.Append)
|
||||
.save(basePath)
|
||||
val hudiSnapshotDF3 = spark.read.format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL)
|
||||
.load(basePath + "/*/*/*/*")
|
||||
assertEquals(0, hudiSnapshotDF3.count()) // 100 records were deleted, 0 record to load
|
||||
}
|
||||
|
||||
@Test
|
||||
def testPrunedFiltered() {
|
||||
// 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 inputDF1 = spark.read.json(spark.sparkContext.parallelize(records1, 2))
|
||||
inputDF1.write.format("org.apache.hudi")
|
||||
.options(commonOpts)
|
||||
.option("hoodie.compact.inline", "false") // else fails due to compaction & deltacommit instant times being same
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY, DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL)
|
||||
.mode(SaveMode.Overwrite)
|
||||
.save(basePath)
|
||||
val hudiSnapshotDF1 = spark.read.format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL)
|
||||
.load(basePath + "/*/*/*/*")
|
||||
assertEquals(100, hudiSnapshotDF1.count())
|
||||
// select nested columns with order different from the actual schema
|
||||
assertEquals("amount,currency,tip_history,_hoodie_commit_seqno",
|
||||
hudiSnapshotDF1
|
||||
.select("fare.amount", "fare.currency", "tip_history", "_hoodie_commit_seqno")
|
||||
.orderBy(desc("_hoodie_commit_seqno"))
|
||||
.columns.mkString(","))
|
||||
|
||||
// Second Operation:
|
||||
// Upsert 50 update records
|
||||
// Snopshot view should read 100 records
|
||||
val records2 = recordsToStrings(dataGen.generateUniqueUpdates("002", 50))
|
||||
.toList
|
||||
val inputDF2: Dataset[Row] = spark.read.json(spark.sparkContext.parallelize(records2, 2))
|
||||
inputDF2.write.format("org.apache.hudi")
|
||||
.options(commonOpts)
|
||||
.mode(SaveMode.Append)
|
||||
.save(basePath)
|
||||
val hudiSnapshotDF2 = spark.read.format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL)
|
||||
.load(basePath + "/*/*/*/*")
|
||||
|
||||
val commit1Time = hudiSnapshotDF1.select("_hoodie_commit_time").head().get(0).toString
|
||||
|
||||
// filter first commit and only read log records
|
||||
assertEquals(50, hudiSnapshotDF2.select("_hoodie_commit_seqno", "fare.amount", "fare.currency", "tip_history")
|
||||
.filter(col("_hoodie_commit_time") > commit1Time).count())
|
||||
|
||||
// select nested columns with order different from the actual schema
|
||||
assertEquals("amount,currency,tip_history,_hoodie_commit_seqno",
|
||||
hudiSnapshotDF2
|
||||
.select("fare.amount", "fare.currency", "tip_history", "_hoodie_commit_seqno")
|
||||
.orderBy(desc("_hoodie_commit_seqno"))
|
||||
.columns.mkString(","))
|
||||
|
||||
// Correctly loading type
|
||||
val sampleRow = hudiSnapshotDF2
|
||||
.select("begin_lat", "current_date", "fare.currency", "tip_history", "nation")
|
||||
.orderBy(desc("_hoodie_commit_time"))
|
||||
.head()
|
||||
assertEquals(sampleRow.getDouble(0), sampleRow.get(0))
|
||||
assertEquals(sampleRow.getLong(1), sampleRow.get(1))
|
||||
assertEquals(sampleRow.getString(2), sampleRow.get(2))
|
||||
assertEquals(sampleRow.getSeq(3), sampleRow.get(3))
|
||||
assertEquals(sampleRow.getStruct(4), sampleRow.get(4))
|
||||
|
||||
// make sure show() work
|
||||
hudiSnapshotDF1.show(1)
|
||||
hudiSnapshotDF2.show(1)
|
||||
}
|
||||
|
||||
@Test
|
||||
def testVectorizedReader() {
|
||||
spark.conf.set("spark.sql.parquet.enableVectorizedReader", true)
|
||||
assertTrue(spark.conf.get("spark.sql.parquet.enableVectorizedReader").toBoolean)
|
||||
// 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 inputDF1 = spark.read.json(spark.sparkContext.parallelize(records1, 2))
|
||||
inputDF1.write.format("org.apache.hudi")
|
||||
.options(commonOpts)
|
||||
.option("hoodie.compact.inline", "false") // else fails due to compaction & deltacommit instant times being same
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY, DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL)
|
||||
.mode(SaveMode.Overwrite)
|
||||
.save(basePath)
|
||||
val hudiSnapshotDF1 = spark.read.format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL)
|
||||
.load(basePath + "/*/*/*/*")
|
||||
assertEquals(100, hudiSnapshotDF1.count())
|
||||
|
||||
val records2 = recordsToStrings(dataGen.generateUniqueUpdatesAsPerSchema("002", 50, schema))
|
||||
.toList
|
||||
val inputDF2: Dataset[Row] = spark.read.json(spark.sparkContext.parallelize(records2, 2))
|
||||
inputDF2.write.format("org.apache.hudi")
|
||||
.options(commonOpts)
|
||||
.mode(SaveMode.Append)
|
||||
.save(basePath)
|
||||
val hudiSnapshotDF2 = spark.read.format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL)
|
||||
.load(basePath + "/*/*/*/*")
|
||||
assertEquals(100, hudiSnapshotDF2.count())
|
||||
|
||||
// loading correct type
|
||||
val sampleRow = hudiSnapshotDF2
|
||||
.select("fare", "driver", "_hoodie_is_deleted")
|
||||
.head()
|
||||
assertEquals(sampleRow.getDouble(0), sampleRow.get(0))
|
||||
assertEquals(sampleRow.getString(1), sampleRow.get(1))
|
||||
assertEquals(sampleRow.getBoolean(2), sampleRow.get(2))
|
||||
|
||||
// test show()
|
||||
hudiSnapshotDF1.show(1)
|
||||
hudiSnapshotDF2.show(1)
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,180 @@
|
||||
/*
|
||||
* 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.functional
|
||||
|
||||
import org.apache.hadoop.fs.{FileSystem, Path}
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient
|
||||
import org.apache.hudi.common.testutils.RawTripTestPayload.recordsToStrings
|
||||
import org.apache.hudi.config.HoodieWriteConfig
|
||||
import org.apache.hudi.exception.TableNotFoundException
|
||||
import org.apache.hudi.testutils.HoodieClientTestBase
|
||||
import org.apache.hudi.{DataSourceReadOptions, DataSourceWriteOptions, HoodieDataSourceHelpers}
|
||||
import org.apache.log4j.LogManager
|
||||
import org.apache.spark.sql._
|
||||
import org.apache.spark.sql.streaming.{OutputMode, Trigger}
|
||||
import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue}
|
||||
import org.junit.jupiter.api.{AfterEach, BeforeEach, Test}
|
||||
|
||||
import scala.collection.JavaConversions._
|
||||
import scala.concurrent.ExecutionContext.Implicits.global
|
||||
import scala.concurrent.duration.Duration
|
||||
import scala.concurrent.{Await, Future}
|
||||
|
||||
/**
|
||||
* Basic tests on the spark datasource for structured streaming sink
|
||||
*/
|
||||
class TestStructuredStreaming extends HoodieClientTestBase {
|
||||
private val log = LogManager.getLogger(getClass)
|
||||
var spark: SparkSession = null
|
||||
val commonOpts = Map(
|
||||
"hoodie.insert.shuffle.parallelism" -> "4",
|
||||
"hoodie.upsert.shuffle.parallelism" -> "4",
|
||||
DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY -> "_row_key",
|
||||
DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY -> "partition",
|
||||
DataSourceWriteOptions.PRECOMBINE_FIELD_OPT_KEY -> "timestamp",
|
||||
HoodieWriteConfig.TABLE_NAME -> "hoodie_test"
|
||||
)
|
||||
|
||||
@BeforeEach override def setUp() {
|
||||
initPath()
|
||||
initSparkContexts()
|
||||
spark = sqlContext.sparkSession
|
||||
initTestDataGenerator()
|
||||
initFileSystem()
|
||||
}
|
||||
|
||||
@AfterEach override def tearDown() = {
|
||||
cleanupSparkContexts()
|
||||
cleanupTestDataGenerator()
|
||||
cleanupFileSystem()
|
||||
}
|
||||
|
||||
@Test
|
||||
def testStructuredStreaming(): Unit = {
|
||||
fs.delete(new Path(basePath), true)
|
||||
val sourcePath = basePath + "/source"
|
||||
val destPath = basePath + "/dest"
|
||||
fs.mkdirs(new Path(sourcePath))
|
||||
|
||||
// First chunk of data
|
||||
val records1 = recordsToStrings(dataGen.generateInserts("000", 100)).toList
|
||||
val inputDF1 = spark.read.json(spark.sparkContext.parallelize(records1, 2))
|
||||
|
||||
// Second chunk of data
|
||||
val records2 = recordsToStrings(dataGen.generateUpdates("001", 100)).toList
|
||||
val inputDF2 = spark.read.json(spark.sparkContext.parallelize(records2, 2))
|
||||
val uniqueKeyCnt = inputDF2.select("_row_key").distinct().count()
|
||||
|
||||
// define the source of streaming
|
||||
val streamingInput =
|
||||
spark.readStream
|
||||
.schema(inputDF1.schema)
|
||||
.json(sourcePath)
|
||||
|
||||
val f1 = Future {
|
||||
println("streaming starting")
|
||||
//'writeStream' can be called only on streaming Dataset/DataFrame
|
||||
streamingInput
|
||||
.writeStream
|
||||
.format("org.apache.hudi")
|
||||
.options(commonOpts)
|
||||
.trigger(Trigger.ProcessingTime(100))
|
||||
.option("checkpointLocation", basePath + "/checkpoint")
|
||||
.outputMode(OutputMode.Append)
|
||||
.start(destPath)
|
||||
.awaitTermination(10000)
|
||||
println("streaming ends")
|
||||
}
|
||||
|
||||
val f2 = Future {
|
||||
inputDF1.coalesce(1).write.mode(SaveMode.Append).json(sourcePath)
|
||||
// wait for spark streaming to process one microbatch
|
||||
val currNumCommits = waitTillAtleastNCommits(fs, destPath, 1, 120, 5)
|
||||
assertTrue(HoodieDataSourceHelpers.hasNewCommits(fs, destPath, "000"))
|
||||
val commitInstantTime1 = HoodieDataSourceHelpers.latestCommit(fs, destPath)
|
||||
// Read RO View
|
||||
val hoodieROViewDF1 = spark.read.format("org.apache.hudi")
|
||||
.load(destPath + "/*/*/*/*")
|
||||
assert(hoodieROViewDF1.count() == 100)
|
||||
|
||||
inputDF2.coalesce(1).write.mode(SaveMode.Append).json(sourcePath)
|
||||
// wait for spark streaming to process one microbatch
|
||||
waitTillAtleastNCommits(fs, destPath, currNumCommits + 1, 120, 5)
|
||||
val commitInstantTime2 = HoodieDataSourceHelpers.latestCommit(fs, destPath)
|
||||
assertEquals(2, HoodieDataSourceHelpers.listCommitsSince(fs, destPath, "000").size())
|
||||
// Read RO View
|
||||
val hoodieROViewDF2 = spark.read.format("org.apache.hudi")
|
||||
.load(destPath + "/*/*/*/*")
|
||||
assertEquals(100, hoodieROViewDF2.count()) // still 100, since we only updated
|
||||
|
||||
|
||||
// Read Incremental View
|
||||
// we have 2 commits, try pulling the first commit (which is not the latest)
|
||||
val firstCommit = HoodieDataSourceHelpers.listCommitsSince(fs, destPath, "000").get(0)
|
||||
val hoodieIncViewDF1 = spark.read.format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL)
|
||||
.option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, "000")
|
||||
.option(DataSourceReadOptions.END_INSTANTTIME_OPT_KEY, firstCommit)
|
||||
.load(destPath)
|
||||
assertEquals(100, hoodieIncViewDF1.count())
|
||||
// 100 initial inserts must be pulled
|
||||
var countsPerCommit = hoodieIncViewDF1.groupBy("_hoodie_commit_time").count().collect()
|
||||
assertEquals(1, countsPerCommit.length)
|
||||
assertEquals(firstCommit, countsPerCommit(0).get(0))
|
||||
|
||||
// pull the latest commit
|
||||
val hoodieIncViewDF2 = spark.read.format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL)
|
||||
.option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, commitInstantTime1)
|
||||
.load(destPath)
|
||||
|
||||
assertEquals(uniqueKeyCnt, hoodieIncViewDF2.count()) // 100 records must be pulled
|
||||
countsPerCommit = hoodieIncViewDF2.groupBy("_hoodie_commit_time").count().collect()
|
||||
assertEquals(1, countsPerCommit.length)
|
||||
assertEquals(commitInstantTime2, countsPerCommit(0).get(0))
|
||||
}
|
||||
Await.result(Future.sequence(Seq(f1, f2)), Duration.Inf)
|
||||
}
|
||||
|
||||
@throws[InterruptedException]
|
||||
private def waitTillAtleastNCommits(fs: FileSystem, tablePath: String,
|
||||
numCommits: Int, timeoutSecs: Int, sleepSecsAfterEachRun: Int) = {
|
||||
val beginTime = System.currentTimeMillis
|
||||
var currTime = beginTime
|
||||
val timeoutMsecs = timeoutSecs * 1000
|
||||
var numInstants = 0
|
||||
var success = false
|
||||
while ({!success && (currTime - beginTime) < timeoutMsecs}) try {
|
||||
val timeline = HoodieDataSourceHelpers.allCompletedCommitsCompactions(fs, tablePath)
|
||||
log.info("Timeline :" + timeline.getInstants.toArray)
|
||||
if (timeline.countInstants >= numCommits) {
|
||||
numInstants = timeline.countInstants
|
||||
success = true
|
||||
}
|
||||
val metaClient = new HoodieTableMetaClient(fs.getConf, tablePath, true)
|
||||
} catch {
|
||||
case te: TableNotFoundException =>
|
||||
log.info("Got table not found exception. Retrying")
|
||||
} finally {
|
||||
Thread.sleep(sleepSecsAfterEachRun * 1000)
|
||||
currTime = System.currentTimeMillis
|
||||
}
|
||||
if (!success) throw new IllegalStateException("Timed-out waiting for " + numCommits + " commits to appear in " + tablePath)
|
||||
numInstants
|
||||
}
|
||||
}
|
||||
Reference in New Issue
Block a user