1
0

[HUDI-2285][HUDI-2476] Metadata table synchronous design. Rebased and Squashed from pull/3426 (#3590)

* [HUDI-2285] Adding Synchronous updates to metadata before completion of commits in data timelime.

- This patch adds synchronous updates to metadata table. In other words, every write is first committed to metadata table followed by data table. While reading metadata table, we ignore any delta commits that are present only in metadata table and not in data table timeline.
- Compaction of metadata table is fenced by the condition that we trigger compaction only when there are no inflight requests in datatable. This ensures that all base files in metadata table is always in sync with data table(w/o any holes) and only there could be some extra invalid commits among delta log files in metadata table.
- Due to this, archival of data table also fences itself up until compacted instant in metadata table.
All writes to metadata table happens within the datatable lock. So, metadata table works in one writer mode only. This might be tough to loosen since all writers write to same FILES partition and so, will result in a conflict anyways.
- As part of this, have added acquiring locks in data table for those operations which were not before while committing (rollback, clean, compaction, cluster). To note, we were not doing any conflict resolution. All we are doing here is to commit by taking a lock. So that all writes to metadata table is always a single writer. 
- Also added building block to add buckets for partitions, which will be leveraged by other indexes like record level index, etc. For now, FILES partition has only one bucket. In general, any number of buckets per partition is allowed and each partition has a fixed fileId prefix with incremental suffix for each bucket within each partition.
Have fixed [HUDI-2476]. This fix is about retrying a failed compaction if it succeeded in metadata for first time, but failed w/ data table.
- Enabling metadata table by default.
- Adding more tests for metadata table

Co-authored-by: Prashant Wason <pwason@uber.com>
This commit is contained in:
Sivabalan Narayanan
2021-10-06 00:17:52 -04:00
committed by GitHub
parent 46808dcb1f
commit 5f32162a2f
101 changed files with 3329 additions and 2069 deletions

View File

@@ -125,9 +125,6 @@ case class HoodieFileIndex(
properties.put(HoodieMetadataConfig.ENABLE,
sqlConf.getConfString(HoodieMetadataConfig.ENABLE.key(),
HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS.toString))
properties.put(HoodieMetadataConfig.VALIDATE_ENABLE,
sqlConf.getConfString(HoodieMetadataConfig.VALIDATE_ENABLE.key(),
HoodieMetadataConfig.VALIDATE_ENABLE.defaultValue().toString))
properties.putAll(options.asJava)
properties
}

View File

@@ -539,7 +539,6 @@ object HoodieSparkSqlWriter {
hiveSyncConfig.partitionValueExtractorClass = hoodieConfig.getString(HIVE_PARTITION_EXTRACTOR_CLASS)
hiveSyncConfig.useJdbc = hoodieConfig.getBoolean(HIVE_USE_JDBC)
hiveSyncConfig.useFileListingFromMetadata = hoodieConfig.getBoolean(HoodieMetadataConfig.ENABLE)
hiveSyncConfig.verifyMetadataFileListing = hoodieConfig.getBoolean(HoodieMetadataConfig.VALIDATE_ENABLE)
hiveSyncConfig.ignoreExceptions = hoodieConfig.getStringOrDefault(HIVE_IGNORE_EXCEPTIONS).toBoolean
hiveSyncConfig.supportTimestamp = hoodieConfig.getStringOrDefault(HIVE_SUPPORT_TIMESTAMP_TYPE).toBoolean
hiveSyncConfig.autoCreateDatabase = hoodieConfig.getStringOrDefault(HIVE_AUTO_CREATE_DATABASE).toBoolean

View File

@@ -18,13 +18,14 @@
package org.apache.hudi
import org.apache.hudi.DataSourceWriteOptions._
import org.apache.hudi.common.config.HoodieMetadataConfig.{ENABLE, VALIDATE_ENABLE}
import org.apache.hudi.common.config.HoodieMetadataConfig.ENABLE
import org.apache.hudi.common.config.{HoodieConfig, TypedProperties}
import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory
import java.util.Properties
import scala.collection.JavaConversions.mapAsJavaMap
import scala.collection.JavaConverters.{mapAsScalaMapConverter, _}
import scala.collection.JavaConverters.mapAsScalaMapConverter
import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory
/**
* WriterUtils to assist in write path in Datasource and tests.
@@ -50,7 +51,6 @@ object HoodieWriterUtils {
PARTITIONPATH_FIELD.key -> PARTITIONPATH_FIELD.defaultValue,
KEYGENERATOR_CLASS_NAME.key -> DEFAULT_KEYGENERATOR_CLASS_OPT_VAL,
ENABLE.key -> ENABLE.defaultValue.toString,
VALIDATE_ENABLE.key -> VALIDATE_ENABLE.defaultValue.toString,
COMMIT_METADATA_KEYPREFIX.key -> COMMIT_METADATA_KEYPREFIX.defaultValue,
INSERT_DROP_DUPS.key -> INSERT_DROP_DUPS.defaultValue,
STREAMING_RETRY_CNT.key -> STREAMING_RETRY_CNT.defaultValue,

View File

@@ -35,6 +35,7 @@ import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.model.HoodieKey;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieTableType;
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.table.timeline.HoodieInstant.State;
import org.apache.hudi.common.table.timeline.HoodieTimeline;
@@ -251,6 +252,7 @@ public class TestBootstrap extends HoodieClientTestBase {
.withFullBootstrapInputProvider(TestFullBootstrapDataProvider.class.getName())
.withBootstrapParallelism(3)
.withBootstrapModeSelector(bootstrapModeSelectorClass).build())
.withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(false).build())
.build();
SparkRDDWriteClient client = new SparkRDDWriteClient(context, config);
client.bootstrap(Option.empty());
@@ -258,8 +260,9 @@ public class TestBootstrap extends HoodieClientTestBase {
numInstantsAfterBootstrap, timestamp, timestamp, deltaCommit, bootstrapInstants);
// Rollback Bootstrap
FSUtils.deleteInstantFile(metaClient.getFs(), metaClient.getMetaPath(), new HoodieInstant(State.COMPLETED,
HoodieActiveTimeline.deleteInstantFile(metaClient.getFs(), metaClient.getMetaPath(), new HoodieInstant(State.COMPLETED,
deltaCommit ? HoodieTimeline.DELTA_COMMIT_ACTION : HoodieTimeline.COMMIT_ACTION, bootstrapCommitInstantTs));
metaClient.reloadActiveTimeline();
client.rollbackFailedBootstrap();
metaClient.reloadActiveTimeline();
assertEquals(0, metaClient.getCommitsTimeline().countInstants());
@@ -374,8 +377,7 @@ public class TestBootstrap extends HoodieClientTestBase {
reloadInputFormats();
List<GenericRecord> records = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat(
jsc.hadoopConfiguration(),
FSUtils.getAllPartitionPaths(context, basePath, HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS,
HoodieMetadataConfig.VALIDATE_ENABLE.defaultValue(), false).stream()
FSUtils.getAllPartitionPaths(context, basePath, HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS, false).stream()
.map(f -> basePath + "/" + f).collect(Collectors.toList()),
basePath, roJobConf, false, schema, TRIP_HIVE_COLUMN_TYPES, false, new ArrayList<>());
assertEquals(totalRecords, records.size());
@@ -393,8 +395,7 @@ public class TestBootstrap extends HoodieClientTestBase {
seenKeys = new HashSet<>();
records = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat(
jsc.hadoopConfiguration(),
FSUtils.getAllPartitionPaths(context, basePath, HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS,
HoodieMetadataConfig.VALIDATE_ENABLE.defaultValue(), false).stream()
FSUtils.getAllPartitionPaths(context, basePath, HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS, false).stream()
.map(f -> basePath + "/" + f).collect(Collectors.toList()),
basePath, rtJobConf, true, schema, TRIP_HIVE_COLUMN_TYPES, false, new ArrayList<>());
assertEquals(totalRecords, records.size());
@@ -410,8 +411,7 @@ public class TestBootstrap extends HoodieClientTestBase {
reloadInputFormats();
records = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat(
jsc.hadoopConfiguration(),
FSUtils.getAllPartitionPaths(context, basePath, HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS,
HoodieMetadataConfig.VALIDATE_ENABLE.defaultValue(), false).stream()
FSUtils.getAllPartitionPaths(context, basePath, HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS, false).stream()
.map(f -> basePath + "/" + f).collect(Collectors.toList()),
basePath, roJobConf, false, schema, TRIP_HIVE_COLUMN_TYPES,
true, HoodieRecord.HOODIE_META_COLUMNS);
@@ -428,8 +428,7 @@ public class TestBootstrap extends HoodieClientTestBase {
seenKeys = new HashSet<>();
records = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat(
jsc.hadoopConfiguration(),
FSUtils.getAllPartitionPaths(context, basePath, HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS,
HoodieMetadataConfig.VALIDATE_ENABLE.defaultValue(), false).stream()
FSUtils.getAllPartitionPaths(context, basePath, HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS, false).stream()
.map(f -> basePath + "/" + f).collect(Collectors.toList()),
basePath, rtJobConf, true, schema, TRIP_HIVE_COLUMN_TYPES, true,
HoodieRecord.HOODIE_META_COLUMNS);
@@ -444,8 +443,7 @@ public class TestBootstrap extends HoodieClientTestBase {
reloadInputFormats();
records = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat(
jsc.hadoopConfiguration(),
FSUtils.getAllPartitionPaths(context, basePath, HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS,
HoodieMetadataConfig.VALIDATE_ENABLE.defaultValue(), false).stream()
FSUtils.getAllPartitionPaths(context, basePath, HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS, false).stream()
.map(f -> basePath + "/" + f).collect(Collectors.toList()),
basePath, roJobConf, false, schema, TRIP_HIVE_COLUMN_TYPES, true,
Arrays.asList("_row_key"));
@@ -462,8 +460,7 @@ public class TestBootstrap extends HoodieClientTestBase {
seenKeys = new HashSet<>();
records = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat(
jsc.hadoopConfiguration(),
FSUtils.getAllPartitionPaths(context, basePath, HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS,
HoodieMetadataConfig.VALIDATE_ENABLE.defaultValue(), false).stream()
FSUtils.getAllPartitionPaths(context, basePath, HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS, false).stream()
.map(f -> basePath + "/" + f).collect(Collectors.toList()),
basePath, rtJobConf, true, schema, TRIP_HIVE_COLUMN_TYPES, true,
Arrays.asList("_row_key"));
@@ -550,6 +547,7 @@ public class TestBootstrap extends HoodieClientTestBase {
}
}
@Override
public HoodieWriteConfig.Builder getConfigBuilder(String schemaStr) {
HoodieWriteConfig.Builder builder = getConfigBuilder(schemaStr, IndexType.BLOOM)
.withExternalSchemaTrasformation(true);

View File

@@ -19,7 +19,7 @@ package org.apache.hudi
import org.apache.commons.io.FileUtils
import org.apache.hadoop.fs.Path
import org.apache.hudi.DataSourceWriteOptions.{INSERT_DROP_DUPS, INSERT_OPERATION_OPT_VAL, MOR_TABLE_TYPE_OPT_VAL, OPERATION, TABLE_TYPE}
import org.apache.hudi.DataSourceWriteOptions.{INSERT_DROP_DUPS, INSERT_OPERATION_OPT_VAL, KEYGENERATOR_CLASS_NAME, MOR_TABLE_TYPE_OPT_VAL, OPERATION, PAYLOAD_CLASS_NAME, PRECOMBINE_FIELD, RECORDKEY_FIELD, TABLE_TYPE}
import org.apache.hudi.client.SparkRDDWriteClient
import org.apache.hudi.common.config.HoodieConfig
import org.apache.hudi.common.model.{HoodieFileFormat, HoodieRecord, HoodieRecordPayload, HoodieTableType, WriteOperationType}
@@ -445,6 +445,7 @@ class HoodieSparkSqlWriterSuite {
val records = DataSourceTestUtils.generateRandomRows(100)
val recordsSeq = convertRowListToSeq(records)
val df = spark.createDataFrame(sc.parallelize(recordsSeq), structType)
initializeMetaClientForBootstrap(fooTableParams, tableType, false)
val client = spy(DataSourceUtils.createHoodieClient(
new JavaSparkContext(sc), modifiedSchema.toString, tempBasePath, hoodieFooTableName,
mapAsJavaMap(fooTableParams)).asInstanceOf[SparkRDDWriteClient[HoodieRecordPayload[Nothing]]])
@@ -501,8 +502,9 @@ class HoodieSparkSqlWriterSuite {
DataSourceWriteOptions.PARTITIONPATH_FIELD.key -> "partition",
HoodieBootstrapConfig.KEYGEN_CLASS_NAME.key -> classOf[NonpartitionedKeyGenerator].getCanonicalName)
val fooTableParams = HoodieWriterUtils.parametersWithWriteDefaults(fooTableModifier)
initializeMetaClientForBootstrap(fooTableParams, tableType, true)
val client = spy(DataSourceUtils.createHoodieClient(
val client = spy(DataSourceUtils.createHoodieClient(
new JavaSparkContext(sc),
null,
tempBasePath,
@@ -522,6 +524,27 @@ class HoodieSparkSqlWriterSuite {
}
}
def initializeMetaClientForBootstrap(fooTableParams : Map[String, String], tableType: String, addBootstrapPath : Boolean) : Unit = {
// when metadata is enabled, directly instantiating write client using DataSourceUtils.createHoodieClient
// will hit a code which tries to instantiate meta client for data table. if table does not exist, it fails.
// hence doing an explicit instantiation here.
val tableMetaClientBuilder = HoodieTableMetaClient.withPropertyBuilder()
.setTableType(tableType)
.setTableName(hoodieFooTableName)
.setRecordKeyFields(fooTableParams(DataSourceWriteOptions.RECORDKEY_FIELD.key))
.setBaseFileFormat(HoodieTableConfig.BASE_FILE_FORMAT.defaultValue().name())
.setArchiveLogFolder(HoodieTableConfig.ARCHIVELOG_FOLDER.defaultValue())
.setPayloadClassName(fooTableParams(PAYLOAD_CLASS_NAME.key))
.setPreCombineField(fooTableParams(PRECOMBINE_FIELD.key))
.setPartitionFields(fooTableParams(DataSourceWriteOptions.PARTITIONPATH_FIELD.key))
.setKeyGeneratorClassProp(fooTableParams(KEYGENERATOR_CLASS_NAME.key))
if(addBootstrapPath) {
tableMetaClientBuilder
.setBootstrapBasePath(fooTableParams(HoodieBootstrapConfig.BASE_PATH.key))
}
tableMetaClientBuilder.initTable(sc.hadoopConfiguration, tempBasePath)
}
/**
* Test cases for schema evolution in different types of tables.
*

View File

@@ -17,6 +17,7 @@
package org.apache.hudi.functional
import org.apache.hadoop.fs.FileSystem
import org.apache.hudi.common.config.HoodieMetadataConfig
import org.apache.hudi.common.table.timeline.HoodieInstant
import org.apache.hudi.common.table.{HoodieTableMetaClient, TableSchemaResolver}
@@ -74,6 +75,8 @@ class TestCOWDataSource extends HoodieClientTestBase {
cleanupSparkContexts()
cleanupTestDataGenerator()
cleanupFileSystem()
FileSystem.closeAll()
System.gc()
}
@Test def testShortNameStorage() {
@@ -368,7 +371,7 @@ class TestCOWDataSource extends HoodieClientTestBase {
val recordsDF = spark.createDataFrame(rdd, schema)
recordsDF.write.format("org.apache.hudi")
.options(commonOpts)
.mode(SaveMode.Append)
.mode(SaveMode.Overwrite)
.save(basePath)
val recordsReadDF = spark.read.format("org.apache.hudi")
@@ -401,29 +404,35 @@ class TestCOWDataSource extends HoodieClientTestBase {
}
private def getDataFrameWriter(keyGenerator: String): DataFrameWriter[Row] = {
getDataFrameWriter(keyGenerator, true)
}
private def getDataFrameWriter(keyGenerator: String, enableMetadata: Boolean): DataFrameWriter[Row] = {
val records = recordsToStrings(dataGen.generateInserts("000", 100)).toList
val inputDF = spark.read.json(spark.sparkContext.parallelize(records, 2))
val opts = commonOpts ++ Map(HoodieMetadataConfig.ENABLE.key() -> String.valueOf(enableMetadata))
inputDF.write.format("hudi")
.options(commonOpts)
.options(opts)
.option(DataSourceWriteOptions.KEYGENERATOR_CLASS_NAME.key, keyGenerator)
.mode(SaveMode.Overwrite)
}
@Test def testSparkPartitonByWithCustomKeyGenerator(): Unit = {
// Without fieldType, the default is SIMPLE
var writer = getDataFrameWriter(classOf[CustomKeyGenerator].getName)
var writer = getDataFrameWriter(classOf[CustomKeyGenerator].getName, false)
writer.partitionBy("current_ts")
.mode(SaveMode.Overwrite)
.save(basePath)
var recordsReadDF = spark.read.format("org.apache.hudi")
.load(basePath + "/*/*")
assertTrue(recordsReadDF.filter(col("_hoodie_partition_path") =!= col("current_ts").cast("string")).count() == 0)
// Specify fieldType as TIMESTAMP
writer = getDataFrameWriter(classOf[CustomKeyGenerator].getName)
writer = getDataFrameWriter(classOf[CustomKeyGenerator].getName, false)
writer.partitionBy("current_ts:TIMESTAMP")
.option(Config.TIMESTAMP_TYPE_FIELD_PROP, "EPOCHMILLISECONDS")
.option(Config.TIMESTAMP_OUTPUT_DATE_FORMAT_PROP, "yyyyMMdd")
.mode(SaveMode.Overwrite)
.save(basePath)
recordsReadDF = spark.read.format("org.apache.hudi")
.load(basePath + "/*/*")
@@ -431,10 +440,11 @@ class TestCOWDataSource extends HoodieClientTestBase {
assertTrue(recordsReadDF.filter(col("_hoodie_partition_path") =!= udf_date_format(col("current_ts"))).count() == 0)
// Mixed fieldType
writer = getDataFrameWriter(classOf[CustomKeyGenerator].getName)
writer = getDataFrameWriter(classOf[CustomKeyGenerator].getName, false)
writer.partitionBy("driver", "rider:SIMPLE", "current_ts:TIMESTAMP")
.option(Config.TIMESTAMP_TYPE_FIELD_PROP, "EPOCHMILLISECONDS")
.option(Config.TIMESTAMP_OUTPUT_DATE_FORMAT_PROP, "yyyyMMdd")
.mode(SaveMode.Overwrite)
.save(basePath)
recordsReadDF = spark.read.format("org.apache.hudi")
.load(basePath + "/*/*/*")
@@ -442,7 +452,7 @@ class TestCOWDataSource extends HoodieClientTestBase {
concat(col("driver"), lit("/"), col("rider"), lit("/"), udf_date_format(col("current_ts")))).count() == 0)
// Test invalid partitionKeyType
writer = getDataFrameWriter(classOf[CustomKeyGenerator].getName)
writer = getDataFrameWriter(classOf[CustomKeyGenerator].getName, false)
writer = writer.partitionBy("current_ts:DUMMY")
.option(Config.TIMESTAMP_TYPE_FIELD_PROP, "EPOCHMILLISECONDS")
.option(Config.TIMESTAMP_OUTPUT_DATE_FORMAT_PROP, "yyyyMMdd")
@@ -459,9 +469,10 @@ class TestCOWDataSource extends HoodieClientTestBase {
// Use the `driver` field as the partition key
var writer = getDataFrameWriter(classOf[SimpleKeyGenerator].getName)
writer.partitionBy("driver")
.mode(SaveMode.Overwrite)
.save(basePath)
var recordsReadDF = spark.read.format("org.apache.hudi")
.load(basePath + "/*/*")
.load(basePath)
assertTrue(recordsReadDF.filter(col("_hoodie_partition_path") =!= col("driver")).count() == 0)
// Use the `driver,rider` field as the partition key, If no such field exists, the default value `default` is used
@@ -469,7 +480,7 @@ class TestCOWDataSource extends HoodieClientTestBase {
writer.partitionBy("driver", "rider")
.save(basePath)
recordsReadDF = spark.read.format("org.apache.hudi")
.load(basePath + "/*/*")
.load(basePath)
assertTrue(recordsReadDF.filter(col("_hoodie_partition_path") =!= lit("default")).count() == 0)
}
@@ -477,9 +488,10 @@ class TestCOWDataSource extends HoodieClientTestBase {
// Use the `driver` field as the partition key
var writer = getDataFrameWriter(classOf[ComplexKeyGenerator].getName)
writer.partitionBy("driver")
.mode(SaveMode.Overwrite)
.save(basePath)
var recordsReadDF = spark.read.format("org.apache.hudi")
.load(basePath + "/*/*")
.load(basePath)
assertTrue(recordsReadDF.filter(col("_hoodie_partition_path") =!= col("driver")).count() == 0)
// Use the `driver`,`rider` field as the partition key
@@ -487,15 +499,16 @@ class TestCOWDataSource extends HoodieClientTestBase {
writer.partitionBy("driver", "rider")
.save(basePath)
recordsReadDF = spark.read.format("org.apache.hudi")
.load(basePath + "/*/*")
.load(basePath)
assertTrue(recordsReadDF.filter(col("_hoodie_partition_path") =!= concat(col("driver"), lit("/"), col("rider"))).count() == 0)
}
@Test def testSparkPartitonByWithTimestampBasedKeyGenerator() {
val writer = getDataFrameWriter(classOf[TimestampBasedKeyGenerator].getName)
val writer = getDataFrameWriter(classOf[TimestampBasedKeyGenerator].getName, false)
writer.partitionBy("current_ts")
.option(Config.TIMESTAMP_TYPE_FIELD_PROP, "EPOCHMILLISECONDS")
.option(Config.TIMESTAMP_OUTPUT_DATE_FORMAT_PROP, "yyyyMMdd")
.mode(SaveMode.Overwrite)
.save(basePath)
val recordsReadDF = spark.read.format("org.apache.hudi")
@@ -507,10 +520,11 @@ class TestCOWDataSource extends HoodieClientTestBase {
@Test def testSparkPartitonByWithGlobalDeleteKeyGenerator() {
val writer = getDataFrameWriter(classOf[GlobalDeleteKeyGenerator].getName)
writer.partitionBy("driver")
.mode(SaveMode.Overwrite)
.save(basePath)
val recordsReadDF = spark.read.format("org.apache.hudi")
.load(basePath + "/*")
.load(basePath)
assertTrue(recordsReadDF.filter(col("_hoodie_partition_path") =!= lit("")).count() == 0)
}
@@ -518,17 +532,19 @@ class TestCOWDataSource extends HoodieClientTestBase {
// Empty string column
var writer = getDataFrameWriter(classOf[NonpartitionedKeyGenerator].getName)
writer.partitionBy("")
.mode(SaveMode.Overwrite)
.save(basePath)
var recordsReadDF = spark.read.format("org.apache.hudi")
.load(basePath + "/*")
.load(basePath)
assertTrue(recordsReadDF.filter(col("_hoodie_partition_path") =!= lit("")).count() == 0)
// Non-existent column
writer = getDataFrameWriter(classOf[NonpartitionedKeyGenerator].getName)
writer.partitionBy("abc")
.mode(SaveMode.Overwrite)
.save(basePath)
recordsReadDF = spark.read.format("org.apache.hudi")
.load(basePath + "/*")
.load(basePath)
assertTrue(recordsReadDF.filter(col("_hoodie_partition_path") =!= lit("")).count() == 0)
}
@@ -597,7 +613,7 @@ class TestCOWDataSource extends HoodieClientTestBase {
val recordsDF = spark.createDataFrame(rdd, schema1)
recordsDF.write.format("org.apache.hudi")
.options(opts)
.mode(SaveMode.Append)
.mode(SaveMode.Overwrite)
.save(basePath)
// 2. write records with schema2 add column age
@@ -651,10 +667,10 @@ class TestCOWDataSource extends HoodieClientTestBase {
val inputDF = spark.read.schema(schema1.toDDL).json(spark.sparkContext.parallelize(records, 2))
inputDF.write.format("org.apache.hudi")
.options(opts)
.mode(SaveMode.Append)
.mode(SaveMode.Overwrite)
.save(basePath)
val recordsReadDF = spark.read.format("org.apache.hudi")
.load(basePath + "/*/*")
.load(basePath)
val resultSchema = new StructType(recordsReadDF.schema.filter(p=> !p.name.startsWith("_hoodie")).toArray)
assertEquals(resultSchema, schema1)
}