[HUDI-1129] Improving schema evolution support in hudi (#2927)
* Adding support to ingest records with old schema after table's schema is evolved * Rebasing against latest master - Trimming test file to be < 800 lines - Renaming config names * Addressing feedback Co-authored-by: Vinoth Chandar <vinoth@apache.org>
This commit is contained in:
committed by
GitHub
parent
73d898322b
commit
1196736185
@@ -17,6 +17,8 @@
|
||||
|
||||
package org.apache.hudi
|
||||
|
||||
|
||||
import org.apache.avro.Schema
|
||||
import org.apache.avro.generic.GenericRecord
|
||||
import org.apache.hadoop.conf.Configuration
|
||||
import org.apache.hadoop.fs.{FileSystem, Path}
|
||||
@@ -25,9 +27,10 @@ import org.apache.hudi.DataSourceWriteOptions._
|
||||
import org.apache.hudi.avro.HoodieAvroUtils
|
||||
import org.apache.hudi.client.{HoodieWriteResult, SparkRDDWriteClient}
|
||||
import org.apache.hudi.common.config.{HoodieConfig, HoodieMetadataConfig, TypedProperties}
|
||||
import org.apache.hudi.common.fs.FSUtils
|
||||
import org.apache.hudi.common.model.{HoodieRecordPayload, HoodieTableType, WriteOperationType}
|
||||
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline
|
||||
import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient}
|
||||
import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient, TableSchemaResolver}
|
||||
import org.apache.hudi.common.util.{CommitUtils, ReflectionUtils}
|
||||
import org.apache.hudi.config.HoodieBootstrapConfig.{BOOTSTRAP_BASE_PATH_PROP, BOOTSTRAP_INDEX_CLASS_PROP}
|
||||
import org.apache.hudi.config.{HoodieInternalConfig, HoodieWriteConfig}
|
||||
@@ -158,13 +161,17 @@ object HoodieSparkSqlWriter {
|
||||
sparkContext.getConf.registerKryoClasses(
|
||||
Array(classOf[org.apache.avro.generic.GenericData],
|
||||
classOf[org.apache.avro.Schema]))
|
||||
val schema = AvroConversionUtils.convertStructTypeToAvroSchema(df.schema, structName, nameSpace)
|
||||
var schema = AvroConversionUtils.convertStructTypeToAvroSchema(df.schema, structName, nameSpace)
|
||||
val reconcileSchema = parameters(DataSourceWriteOptions.RECONCILE_SCHEMA.key()).toBoolean
|
||||
if (reconcileSchema) {
|
||||
schema = getLatestTableSchema(fs, basePath, sparkContext, schema)
|
||||
}
|
||||
sparkContext.getConf.registerAvroSchemas(schema)
|
||||
log.info(s"Registered avro schema : ${schema.toString(true)}")
|
||||
|
||||
// Convert to RDD[HoodieRecord]
|
||||
val genericRecords: RDD[GenericRecord] = HoodieSparkUtils.createRdd(df, schema, structName, nameSpace)
|
||||
|
||||
val genericRecords: RDD[GenericRecord] = HoodieSparkUtils.createRdd(df, structName, nameSpace, reconcileSchema,
|
||||
org.apache.hudi.common.util.Option.of(schema))
|
||||
val shouldCombine = parameters(INSERT_DROP_DUPS.key()).toBoolean ||
|
||||
operation.equals(WriteOperationType.UPSERT) ||
|
||||
parameters.getOrElse(HoodieWriteConfig.COMBINE_BEFORE_INSERT_PROP.key(),
|
||||
@@ -212,7 +219,8 @@ object HoodieSparkSqlWriter {
|
||||
classOf[org.apache.avro.Schema]))
|
||||
|
||||
// Convert to RDD[HoodieKey]
|
||||
val genericRecords: RDD[GenericRecord] = HoodieSparkUtils.createRdd(df, structName, nameSpace)
|
||||
val genericRecords: RDD[GenericRecord] = HoodieSparkUtils.createRdd(df, structName, nameSpace,
|
||||
parameters(DataSourceWriteOptions.RECONCILE_SCHEMA.key()).toBoolean)
|
||||
val hoodieKeysToDelete = genericRecords.map(gr => keyGenerator.getKey(gr)).toJavaRDD()
|
||||
|
||||
if (!tableExists) {
|
||||
@@ -249,6 +257,25 @@ object HoodieSparkSqlWriter {
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Checks if schema needs upgrade (if incoming record's write schema is old while table schema got evolved).
|
||||
*
|
||||
* @param fs instance of FileSystem.
|
||||
* @param basePath base path.
|
||||
* @param sparkContext instance of spark context.
|
||||
* @param schema incoming record's schema.
|
||||
* @return Pair of(boolean, table schema), where first entry will be true only if schema conversion is required.
|
||||
*/
|
||||
def getLatestTableSchema(fs: FileSystem, basePath: Path, sparkContext: SparkContext, schema: Schema): Schema = {
|
||||
var latestSchema: Schema = schema
|
||||
if (FSUtils.isTableExists(basePath.toString, fs)) {
|
||||
val tableMetaClient = HoodieTableMetaClient.builder.setConf(sparkContext.hadoopConfiguration).setBasePath(basePath.toString).build()
|
||||
val tableSchemaResolver = new TableSchemaResolver(tableMetaClient)
|
||||
latestSchema = tableSchemaResolver.getLatestSchema(schema, false, null);
|
||||
}
|
||||
latestSchema
|
||||
}
|
||||
|
||||
def bootstrap(sqlContext: SQLContext,
|
||||
mode: SaveMode,
|
||||
parameters: Map[String, String],
|
||||
|
||||
@@ -17,19 +17,16 @@
|
||||
|
||||
package org.apache.hudi
|
||||
|
||||
import java.util.Properties
|
||||
|
||||
import scala.collection.JavaConverters._
|
||||
import org.apache.hudi.DataSourceWriteOptions._
|
||||
import org.apache.hudi.common.config.HoodieMetadataConfig.{METADATA_ENABLE_PROP, METADATA_VALIDATE_PROP}
|
||||
import org.apache.hudi.common.config.{HoodieConfig, TypedProperties}
|
||||
|
||||
import scala.collection.JavaConversions.mapAsJavaMap
|
||||
import scala.collection.JavaConverters.mapAsScalaMapConverter
|
||||
import org.apache.hudi.common.config.HoodieMetadataConfig.METADATA_ENABLE_PROP
|
||||
import org.apache.hudi.common.config.HoodieMetadataConfig.METADATA_VALIDATE_PROP
|
||||
import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory
|
||||
import org.apache.hudi.keygen.{BaseKeyGenerator, CustomAvroKeyGenerator, CustomKeyGenerator, KeyGenerator}
|
||||
|
||||
import java.util.Properties
|
||||
import scala.collection.JavaConversions.mapAsJavaMap
|
||||
import scala.collection.JavaConverters.{mapAsScalaMapConverter, _}
|
||||
|
||||
/**
|
||||
* WriterUtils to assist in write path in Datasource and tests.
|
||||
*/
|
||||
@@ -78,7 +75,8 @@ object HoodieWriterUtils {
|
||||
ASYNC_COMPACT_ENABLE.key -> ASYNC_COMPACT_ENABLE.defaultValue,
|
||||
INLINE_CLUSTERING_ENABLE.key -> INLINE_CLUSTERING_ENABLE.defaultValue,
|
||||
ASYNC_CLUSTERING_ENABLE.key -> ASYNC_CLUSTERING_ENABLE.defaultValue,
|
||||
ENABLE_ROW_WRITER.key -> ENABLE_ROW_WRITER.defaultValue
|
||||
ENABLE_ROW_WRITER.key -> ENABLE_ROW_WRITER.defaultValue,
|
||||
RECONCILE_SCHEMA.key -> RECONCILE_SCHEMA.defaultValue.toString
|
||||
) ++ DataSourceOptionsHelper.translateConfigurations(parameters)
|
||||
}
|
||||
|
||||
|
||||
Reference in New Issue
Block a user