1
0

[HUDI-3018] Adding validation to dataframe scheme to ensure reserved field does not have diff data type (#4852)

This commit is contained in:
Sivabalan Narayanan
2022-02-27 11:59:23 -05:00
committed by GitHub
parent 2f99e8458a
commit d5444ff7ff
5 changed files with 33 additions and 13 deletions

View File

@@ -18,23 +18,21 @@
package org.apache.hudi.functional
import org.apache.hadoop.fs.FileSystem
import org.apache.hudi.common.config.HoodieMetadataConfig
import org.apache.hudi.common.model.HoodieRecord
import org.apache.hudi.common.table.timeline.HoodieInstant
import org.apache.hudi.common.table.{HoodieTableMetaClient, TableSchemaResolver}
import org.apache.hudi.common.testutils.HoodieTestDataGenerator
import org.apache.hudi.common.testutils.RawTripTestPayload.{deleteRecordsToStrings, recordsToStrings}
import org.apache.hudi.config.HoodieWriteConfig
import org.apache.hudi.exception.HoodieUpsertException
import org.apache.hudi.exception.{HoodieException, HoodieUpsertException}
import org.apache.hudi.keygen._
import org.apache.hudi.keygen.constant.KeyGeneratorOptions.Config
import org.apache.hudi.testutils.HoodieClientTestBase
import org.apache.hudi.{AvroConversionUtils, DataSourceReadOptions, DataSourceWriteOptions, HoodieDataSourceHelpers}
import org.apache.hudi.{AvroConversionUtils, DataSourceReadOptions, DataSourceWriteOptions, HoodieDataSourceHelpers, HoodieMergeOnReadRDD}
import org.apache.spark.sql._
import org.apache.spark.sql.functions.{col, concat, lit, udf}
import org.apache.spark.sql.types._
import org.joda.time.DateTime
import org.joda.time.format.DateTimeFormat
import org.junit.jupiter.api.Assertions.{assertEquals, assertThrows, assertTrue, fail}
@@ -44,7 +42,6 @@ import org.junit.jupiter.params.ParameterizedTest
import org.junit.jupiter.params.provider.{CsvSource, ValueSource}
import java.sql.{Date, Timestamp}
import scala.collection.JavaConversions._
import scala.collection.JavaConverters._
@@ -98,6 +95,23 @@ class TestCOWDataSource extends HoodieClientTestBase {
assertTrue(HoodieDataSourceHelpers.hasNewCommits(fs, basePath, "000"))
}
@Test def testHoodieIsDeletedNonBooleanField() {
// Insert Operation
val records = recordsToStrings(dataGen.generateInserts("000", 100)).toList
val inputDF = spark.read.json(spark.sparkContext.parallelize(records, 2))
val df = inputDF.withColumn(HoodieRecord.HOODIE_IS_DELETED, lit("abc"))
assertThrows(classOf[HoodieException], new Executable {
override def execute(): Unit = {
df.write.format("hudi")
.options(commonOpts)
.mode(SaveMode.Overwrite)
.save(basePath)
}
}, "Should have failed since _hoodie_is_deleted is not a BOOLEAN data type")
}
/**
* This tests the case that query by with a specified partition condition on hudi table which is
* different between the value of the partition field and the actual partition path,