[RFC-33] [HUDI-2429][Stacked on HUDI-2560] Support full Schema evolution for Spark (#4910)
* [HUDI-2560] introduce id_based schema to support full schema evolution. * add test for FileBasedInternalSchemaStorageManger and rebase code * add support for change column type and fix some test case * fix some bugs encountered in the production env and delete useless code * fix test error * rebase code * fixed some nested schema change bugs * [HUDI-2429][Stacked On HUDI-2560]Support full schema evolution for spark * [use dummyInternalSchema instead of null] * add support for spark3.1.x * remove support for spark3.1.x , sicne some compile fail * support spark3.1.x * rebase and prepare solve all comments * address all comments * rebase code * fixed the count(*) bug * try to get internalSchema by parser commit file/history file directly, not use metaclient which is time cost address some comments * fixed all comments * fix new comments * rebase code,fix UT failed * fixed mistake * rebase code ,fixed new comments * rebase code , and prepare for address new comments * address commits * address new comments * fix new issues * control fallback original write logical
This commit is contained in:
@@ -16,4 +16,5 @@
|
||||
# limitations under the License.
|
||||
|
||||
|
||||
org.apache.hudi.DefaultSource
|
||||
org.apache.hudi.DefaultSource
|
||||
org.apache.spark.sql.execution.datasources.parquet.SparkHoodieParquetFileFormat
|
||||
@@ -65,6 +65,7 @@ class BaseFileOnlyRelation(sqlContext: SQLContext,
|
||||
tableSchema: HoodieTableSchema,
|
||||
requiredSchema: HoodieTableSchema,
|
||||
filters: Array[Filter]): HoodieUnsafeRDD = {
|
||||
|
||||
val baseFileReader = createBaseFileReader(
|
||||
spark = sparkSession,
|
||||
partitionSchema = partitionSchema,
|
||||
@@ -74,7 +75,7 @@ class BaseFileOnlyRelation(sqlContext: SQLContext,
|
||||
options = optParams,
|
||||
// NOTE: We have to fork the Hadoop Config here as Spark will be modifying it
|
||||
// to configure Parquet reader appropriately
|
||||
hadoopConf = new Configuration(conf)
|
||||
hadoopConf = HoodieDataSourceHelper.getConfigurationWithInternalSchema(new Configuration(conf), requiredSchema.internalSchema, metaClient.getBasePath, validCommits)
|
||||
)
|
||||
|
||||
new HoodieFileScanRDD(sparkSession, baseFileReader, fileSplits)
|
||||
|
||||
@@ -21,12 +21,12 @@ import org.apache.hadoop.fs.Path
|
||||
import org.apache.hudi.DataSourceReadOptions._
|
||||
import org.apache.hudi.DataSourceWriteOptions.{BOOTSTRAP_OPERATION_OPT_VAL, OPERATION}
|
||||
import org.apache.hudi.common.fs.FSUtils
|
||||
import org.apache.hudi.common.model.HoodieRecord
|
||||
import org.apache.hudi.common.model.{HoodieFileFormat, HoodieRecord}
|
||||
import org.apache.hudi.common.model.HoodieTableType.{COPY_ON_WRITE, MERGE_ON_READ}
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant
|
||||
import org.apache.hudi.common.table.{HoodieTableMetaClient, TableSchemaResolver}
|
||||
import org.apache.hudi.exception.HoodieException
|
||||
import org.apache.log4j.LogManager
|
||||
import org.apache.spark.sql.execution.datasources.DataSource
|
||||
import org.apache.spark.sql.execution.streaming.{Sink, Source}
|
||||
import org.apache.spark.sql.hudi.streaming.HoodieStreamSource
|
||||
import org.apache.spark.sql.sources._
|
||||
@@ -46,6 +46,7 @@ class DefaultSource extends RelationProvider
|
||||
with DataSourceRegister
|
||||
with StreamSinkProvider
|
||||
with StreamSourceProvider
|
||||
with SparkAdapterSupport
|
||||
with Serializable {
|
||||
|
||||
SparkSession.getActiveSession.foreach { spark =>
|
||||
@@ -108,7 +109,6 @@ class DefaultSource extends RelationProvider
|
||||
(COPY_ON_WRITE, QUERY_TYPE_READ_OPTIMIZED_OPT_VAL, false) |
|
||||
(MERGE_ON_READ, QUERY_TYPE_READ_OPTIMIZED_OPT_VAL, false) =>
|
||||
new BaseFileOnlyRelation(sqlContext, metaClient, parameters, userSchema, globPaths)
|
||||
|
||||
case (COPY_ON_WRITE, QUERY_TYPE_INCREMENTAL_OPT_VAL, _) =>
|
||||
new IncrementalRelation(sqlContext, parameters, userSchema, metaClient)
|
||||
|
||||
@@ -128,6 +128,11 @@ class DefaultSource extends RelationProvider
|
||||
}
|
||||
}
|
||||
|
||||
def getValidCommits(metaClient: HoodieTableMetaClient): String = {
|
||||
metaClient
|
||||
.getCommitsAndCompactionTimeline.filterCompletedInstants.getInstants.toArray().map(_.asInstanceOf[HoodieInstant].getFileName).mkString(",")
|
||||
}
|
||||
|
||||
/**
|
||||
* This DataSource API is used for writing the DataFrame at the destination. For now, we are returning a dummy
|
||||
* relation here because Spark does not really make use of the relation returned, and just returns an empty
|
||||
|
||||
@@ -33,6 +33,9 @@ import org.apache.hudi.common.table.view.HoodieTableFileSystemView
|
||||
import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient, TableSchemaResolver}
|
||||
import org.apache.hudi.common.util.StringUtils
|
||||
import org.apache.hudi.common.util.ValidationUtils.checkState
|
||||
import org.apache.hudi.hadoop.HoodieROTablePathFilter
|
||||
import org.apache.hudi.internal.schema.InternalSchema
|
||||
import org.apache.hudi.internal.schema.convert.AvroInternalSchemaConverter
|
||||
import org.apache.hudi.io.storage.HoodieHFileReader
|
||||
import org.apache.hudi.metadata.HoodieTableMetadata
|
||||
import org.apache.spark.TaskContext
|
||||
@@ -54,7 +57,7 @@ import scala.util.Try
|
||||
|
||||
trait HoodieFileSplit {}
|
||||
|
||||
case class HoodieTableSchema(structTypeSchema: StructType, avroSchemaStr: String)
|
||||
case class HoodieTableSchema(structTypeSchema: StructType, avroSchemaStr: String, internalSchema: InternalSchema = InternalSchema.getEmptyInternalSchema)
|
||||
|
||||
case class HoodieTableState(tablePath: String,
|
||||
latestCommitTimestamp: String,
|
||||
@@ -114,9 +117,9 @@ abstract class HoodieBaseRelation(val sqlContext: SQLContext,
|
||||
optParams.get(DataSourceReadOptions.TIME_TRAVEL_AS_OF_INSTANT.key)
|
||||
.map(HoodieSqlCommonUtils.formatQueryInstant)
|
||||
|
||||
protected lazy val tableAvroSchema: Schema = {
|
||||
protected lazy val (tableAvroSchema: Schema, internalSchema: InternalSchema) = {
|
||||
val schemaUtil = new TableSchemaResolver(metaClient)
|
||||
Try(schemaUtil.getTableAvroSchema).getOrElse(
|
||||
val avroSchema = Try(schemaUtil.getTableAvroSchema).getOrElse(
|
||||
// If there is no commit in the table, we can't get the schema
|
||||
// t/h [[TableSchemaResolver]], fallback to the provided [[userSchema]] instead.
|
||||
userSchema match {
|
||||
@@ -124,6 +127,13 @@ abstract class HoodieBaseRelation(val sqlContext: SQLContext,
|
||||
case _ => throw new IllegalArgumentException("User-provided schema is required in case the table is empty")
|
||||
}
|
||||
)
|
||||
// try to find internalSchema
|
||||
val internalSchemaFromMeta = try {
|
||||
schemaUtil.getTableInternalSchemaFromCommitMetadata.orElse(InternalSchema.getEmptyInternalSchema)
|
||||
} catch {
|
||||
case _ => InternalSchema.getEmptyInternalSchema
|
||||
}
|
||||
(avroSchema, internalSchemaFromMeta)
|
||||
}
|
||||
|
||||
protected val tableStructSchema: StructType = AvroConversionUtils.convertAvroSchemaToStructType(tableAvroSchema)
|
||||
@@ -155,6 +165,8 @@ abstract class HoodieBaseRelation(val sqlContext: SQLContext,
|
||||
// NOTE: We're including compaction here since it's not considering a "commit" operation
|
||||
metaClient.getCommitsAndCompactionTimeline.filterCompletedInstants
|
||||
|
||||
protected val validCommits = timeline.getInstants.toArray().map(_.asInstanceOf[HoodieInstant].getFileName).mkString(",")
|
||||
|
||||
protected def latestInstant: Option[HoodieInstant] =
|
||||
toScalaOption(timeline.lastInstant())
|
||||
|
||||
@@ -189,8 +201,8 @@ abstract class HoodieBaseRelation(val sqlContext: SQLContext,
|
||||
// filtered out upstream
|
||||
val fetchedColumns: Array[String] = appendMandatoryColumns(requiredColumns)
|
||||
|
||||
val (requiredAvroSchema, requiredStructSchema) =
|
||||
HoodieSparkUtils.getRequiredSchema(tableAvroSchema, fetchedColumns)
|
||||
val (requiredAvroSchema, requiredStructSchema, requiredInternalSchema) =
|
||||
HoodieSparkUtils.getRequiredSchema(tableAvroSchema, fetchedColumns, internalSchema)
|
||||
|
||||
val filterExpressions = convertToExpressions(filters)
|
||||
val (partitionFilters, dataFilters) = filterExpressions.partition(isPartitionPredicate)
|
||||
@@ -198,8 +210,8 @@ abstract class HoodieBaseRelation(val sqlContext: SQLContext,
|
||||
val fileSplits = collectFileSplits(partitionFilters, dataFilters)
|
||||
|
||||
val partitionSchema = StructType(Nil)
|
||||
val tableSchema = HoodieTableSchema(tableStructSchema, tableAvroSchema.toString)
|
||||
val requiredSchema = HoodieTableSchema(requiredStructSchema, requiredAvroSchema.toString)
|
||||
val tableSchema = HoodieTableSchema(tableStructSchema, if (internalSchema.isEmptySchema) tableAvroSchema.toString else AvroInternalSchemaConverter.convert(internalSchema, tableAvroSchema.getName).toString, internalSchema)
|
||||
val requiredSchema = HoodieTableSchema(requiredStructSchema, requiredAvroSchema.toString, requiredInternalSchema)
|
||||
|
||||
// Here we rely on a type erasure, to workaround inherited API restriction and pass [[RDD[InternalRow]]] back as [[RDD[Row]]]
|
||||
// Please check [[needConversion]] scala-doc for more details
|
||||
|
||||
@@ -20,6 +20,9 @@ package org.apache.hudi
|
||||
|
||||
import org.apache.hadoop.conf.Configuration
|
||||
import org.apache.hadoop.fs.FileStatus
|
||||
import org.apache.hudi.client.utils.SparkInternalSchemaConverter
|
||||
import org.apache.hudi.internal.schema.InternalSchema
|
||||
import org.apache.hudi.internal.schema.utils.SerDeHelper
|
||||
import org.apache.spark.sql.SparkSession
|
||||
import org.apache.spark.sql.catalyst.InternalRow
|
||||
import org.apache.spark.sql.catalyst.expressions.{PredicateHelper, SpecificInternalRow, UnsafeProjection}
|
||||
@@ -31,7 +34,7 @@ import org.apache.spark.sql.vectorized.ColumnarBatch
|
||||
|
||||
import scala.collection.JavaConverters._
|
||||
|
||||
object HoodieDataSourceHelper extends PredicateHelper {
|
||||
object HoodieDataSourceHelper extends PredicateHelper with SparkAdapterSupport {
|
||||
|
||||
|
||||
/**
|
||||
@@ -46,7 +49,7 @@ object HoodieDataSourceHelper extends PredicateHelper {
|
||||
options: Map[String, String],
|
||||
hadoopConf: Configuration): PartitionedFile => Iterator[InternalRow] = {
|
||||
|
||||
val readParquetFile: PartitionedFile => Iterator[Any] = new ParquetFileFormat().buildReaderWithPartitionValues(
|
||||
val readParquetFile: PartitionedFile => Iterator[Any] = sparkAdapter.createHoodieParquetFileFormat().get.buildReaderWithPartitionValues(
|
||||
sparkSession = sparkSession,
|
||||
dataSchema = dataSchema,
|
||||
partitionSchema = partitionSchema,
|
||||
@@ -78,4 +81,19 @@ object HoodieDataSourceHelper extends PredicateHelper {
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Set internalSchema evolution parameters to configuration.
|
||||
* spark will broadcast them to each executor, we use those parameters to do schema evolution.
|
||||
*
|
||||
* @param conf hadoop conf.
|
||||
* @param internalSchema internalschema for query.
|
||||
* @param tablePath hoodie table base path.
|
||||
* @param validCommits valid commits, using give validCommits to validate all legal histroy Schema files, and return the latest one.
|
||||
*/
|
||||
def getConfigurationWithInternalSchema(conf: Configuration, internalSchema: InternalSchema, tablePath: String, validCommits: String): Configuration = {
|
||||
conf.set(SparkInternalSchemaConverter.HOODIE_QUERY_SCHEMA, SerDeHelper.toJson(internalSchema))
|
||||
conf.set(SparkInternalSchemaConverter.HOODIE_TABLE_PATH, tablePath)
|
||||
conf.set(SparkInternalSchemaConverter.HOODIE_VALID_COMMITS_LIST, validCommits)
|
||||
conf
|
||||
}
|
||||
}
|
||||
|
||||
@@ -39,6 +39,7 @@ import org.apache.hudi.hadoop.config.HoodieRealtimeConfig
|
||||
import org.apache.hudi.hadoop.utils.HoodieRealtimeRecordReaderUtils.getMaxCompactionMemoryInBytes
|
||||
import org.apache.hudi.metadata.HoodieTableMetadata.getDataTableBasePathFromMetadataTable
|
||||
import org.apache.hudi.metadata.{HoodieBackedTableMetadata, HoodieTableMetadata}
|
||||
import org.apache.hudi.internal.schema.InternalSchema
|
||||
import org.apache.spark.rdd.RDD
|
||||
import org.apache.spark.sql.avro.HoodieAvroDeserializer
|
||||
import org.apache.spark.sql.catalyst.InternalRow
|
||||
@@ -165,9 +166,10 @@ class HoodieMergeOnReadRDD(@transient sc: SparkContext,
|
||||
// be stored in non-columnar formats like Avro, HFile, etc)
|
||||
private val requiredSchemaFieldOrdinals: List[Int] = collectFieldOrdinals(requiredAvroSchema, logFileReaderAvroSchema)
|
||||
|
||||
// TODO: now logScanner with internalSchema support column project, we may no need projectAvroUnsafe
|
||||
private var logScanner =
|
||||
HoodieMergeOnReadRDD.scanLog(split.logFiles, getPartitionPath(split), logFileReaderAvroSchema, tableState,
|
||||
maxCompactionMemoryInBytes, config)
|
||||
maxCompactionMemoryInBytes, config, tableSchema.internalSchema)
|
||||
|
||||
private val logRecords = logScanner.getRecords.asScala
|
||||
|
||||
@@ -305,7 +307,7 @@ private object HoodieMergeOnReadRDD {
|
||||
logSchema: Schema,
|
||||
tableState: HoodieTableState,
|
||||
maxCompactionMemoryInBytes: Long,
|
||||
hadoopConf: Configuration): HoodieMergedLogRecordScanner = {
|
||||
hadoopConf: Configuration, internalSchema: InternalSchema = InternalSchema.getEmptyInternalSchema): HoodieMergedLogRecordScanner = {
|
||||
val tablePath = tableState.tablePath
|
||||
val fs = FSUtils.getFs(tablePath, hadoopConf)
|
||||
|
||||
@@ -333,6 +335,7 @@ private object HoodieMergeOnReadRDD {
|
||||
HoodieRealtimeConfig.DEFAULT_COMPACTION_LAZY_BLOCK_READ_ENABLED).toBoolean)
|
||||
.getOrElse(false))
|
||||
.withReverseReader(false)
|
||||
.withInternalSchema(internalSchema)
|
||||
.withBufferSize(
|
||||
hadoopConf.getInt(HoodieRealtimeConfig.MAX_DFS_STREAM_BUFFER_SIZE_PROP,
|
||||
HoodieRealtimeConfig.DEFAULT_MAX_DFS_STREAM_BUFFER_SIZE))
|
||||
|
||||
@@ -51,6 +51,11 @@ import org.apache.spark.sql._
|
||||
import org.apache.spark.sql.internal.StaticSQLConf
|
||||
import org.apache.spark.sql.types.StructType
|
||||
import org.apache.spark.{SPARK_VERSION, SparkContext}
|
||||
import org.apache.spark.SparkContext
|
||||
|
||||
import java.util.Properties
|
||||
import org.apache.hudi.internal.schema.InternalSchema
|
||||
import org.apache.hudi.internal.schema.utils.{AvroSchemaEvolutionUtils, SerDeHelper}
|
||||
|
||||
import scala.collection.JavaConversions._
|
||||
import scala.collection.mutable
|
||||
@@ -184,9 +189,10 @@ object HoodieSparkSqlWriter {
|
||||
}
|
||||
|
||||
// Create a HoodieWriteClient & issue the delete.
|
||||
val internalSchemaOpt = getLatestTableInternalSchema(fs, basePath, sparkContext)
|
||||
val client = hoodieWriteClient.getOrElse(DataSourceUtils.createHoodieClient(jsc,
|
||||
null, path, tblName,
|
||||
mapAsJavaMap(parameters - HoodieWriteConfig.AUTO_COMMIT_ENABLE.key)))
|
||||
mapAsJavaMap(addSchemaEvolutionParameters(parameters, internalSchemaOpt) - HoodieWriteConfig.AUTO_COMMIT_ENABLE.key)))
|
||||
.asInstanceOf[SparkRDDWriteClient[HoodieRecordPayload[Nothing]]]
|
||||
|
||||
if (isAsyncCompactionEnabled(client, tableConfig, parameters, jsc.hadoopConfiguration())) {
|
||||
@@ -231,8 +237,17 @@ object HoodieSparkSqlWriter {
|
||||
Array(classOf[org.apache.avro.generic.GenericData],
|
||||
classOf[org.apache.avro.Schema]))
|
||||
var schema = AvroConversionUtils.convertStructTypeToAvroSchema(df.schema, structName, nameSpace)
|
||||
val lastestSchema = getLatestTableSchema(fs, basePath, sparkContext, schema)
|
||||
val internalSchemaOpt = getLatestTableInternalSchema(fs, basePath, sparkContext)
|
||||
if (reconcileSchema) {
|
||||
schema = getLatestTableSchema(fs, basePath, sparkContext, schema)
|
||||
schema = lastestSchema
|
||||
}
|
||||
if (internalSchemaOpt.isDefined) {
|
||||
schema = {
|
||||
val newSparkSchema = AvroConversionUtils.convertAvroSchemaToStructType(AvroSchemaEvolutionUtils.canonicalizeColumnNullability(schema, lastestSchema))
|
||||
AvroConversionUtils.convertStructTypeToAvroSchema(newSparkSchema, structName, nameSpace)
|
||||
|
||||
}
|
||||
}
|
||||
validateSchemaForHoodieIsDeleted(schema)
|
||||
sparkContext.getConf.registerAvroSchemas(schema)
|
||||
@@ -264,8 +279,9 @@ object HoodieSparkSqlWriter {
|
||||
|
||||
val writeSchema = if (dropPartitionColumns) generateSchemaWithoutPartitionColumns(partitionColumns, schema) else schema
|
||||
// Create a HoodieWriteClient & issue the write.
|
||||
|
||||
val client = hoodieWriteClient.getOrElse(DataSourceUtils.createHoodieClient(jsc, writeSchema.toString, path,
|
||||
tblName, mapAsJavaMap(parameters - HoodieWriteConfig.AUTO_COMMIT_ENABLE.key)
|
||||
tblName, mapAsJavaMap(addSchemaEvolutionParameters(parameters, internalSchemaOpt) - HoodieWriteConfig.AUTO_COMMIT_ENABLE.key)
|
||||
)).asInstanceOf[SparkRDDWriteClient[HoodieRecordPayload[Nothing]]]
|
||||
|
||||
if (isAsyncCompactionEnabled(client, tableConfig, parameters, jsc.hadoopConfiguration())) {
|
||||
@@ -315,6 +331,36 @@ object HoodieSparkSqlWriter {
|
||||
processedRecord
|
||||
}
|
||||
|
||||
def addSchemaEvolutionParameters(parameters: Map[String, String], internalSchemaOpt: Option[InternalSchema]): Map[String, String] = {
|
||||
val schemaEvolutionEnable = if (internalSchemaOpt.isDefined) "true" else "false"
|
||||
parameters ++ Map(HoodieWriteConfig.INTERNAL_SCHEMA_STRING.key() -> SerDeHelper.toJson(internalSchemaOpt.getOrElse(null)),
|
||||
HoodieWriteConfig.SCHEMA_EVOLUTION_ENABLE.key() -> schemaEvolutionEnable)
|
||||
}
|
||||
|
||||
/**
|
||||
* get latest internalSchema from table
|
||||
*
|
||||
* @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 getLatestTableInternalSchema(fs: FileSystem, basePath: Path, sparkContext: SparkContext): Option[InternalSchema] = {
|
||||
try {
|
||||
if (FSUtils.isTableExists(basePath.toString, fs)) {
|
||||
val tableMetaClient = HoodieTableMetaClient.builder.setConf(sparkContext.hadoopConfiguration).setBasePath(basePath.toString).build()
|
||||
val tableSchemaResolver = new TableSchemaResolver(tableMetaClient)
|
||||
val internalSchemaOpt = tableSchemaResolver.getTableInternalSchemaFromCommitMetadata
|
||||
if (internalSchemaOpt.isPresent) Some(internalSchemaOpt.get()) else None
|
||||
} else {
|
||||
None
|
||||
}
|
||||
} catch {
|
||||
case _ => None
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Checks if schema needs upgrade (if incoming record's write schema is old while table schema got evolved).
|
||||
*
|
||||
|
||||
@@ -18,6 +18,7 @@
|
||||
package org.apache.hudi
|
||||
|
||||
import java.util.Properties
|
||||
import org.apache.hudi.config.HoodieWriteConfig
|
||||
import org.apache.hudi.DataSourceOptionsHelper.allAlternatives
|
||||
import org.apache.hudi.DataSourceWriteOptions._
|
||||
import org.apache.hudi.common.config.HoodieMetadataConfig.ENABLE
|
||||
@@ -163,6 +164,13 @@ object HoodieWriterUtils {
|
||||
diffConfigs.insert(0, "\nConfig conflict(key\tcurrent value\texisting value):\n")
|
||||
throw new HoodieException(diffConfigs.toString.trim)
|
||||
}
|
||||
// Check schema evolution for bootstrap table.
|
||||
// now we do not support bootstrap table.
|
||||
if (params.get(OPERATION.key).contains(BOOTSTRAP_OPERATION_OPT_VAL)
|
||||
&& params.getOrElse(HoodieWriteConfig.SCHEMA_EVOLUTION_ENABLE.key(), "false").toBoolean) {
|
||||
throw new HoodieException(String
|
||||
.format("now schema evolution cannot support bootstrap table, pls set %s to false", HoodieWriteConfig.SCHEMA_EVOLUTION_ENABLE.key()))
|
||||
}
|
||||
}
|
||||
|
||||
private def getStringFromTableConfigWithAlternatives(tableConfig: HoodieConfig, key: String): String = {
|
||||
|
||||
@@ -18,17 +18,20 @@
|
||||
package org.apache.hudi
|
||||
|
||||
import org.apache.avro.Schema
|
||||
import org.apache.hudi.common.model.{HoodieCommitMetadata, HoodieRecord, HoodieReplaceCommitMetadata}
|
||||
import org.apache.hudi.common.model.{HoodieCommitMetadata, HoodieFileFormat, HoodieRecord, HoodieReplaceCommitMetadata}
|
||||
import org.apache.hudi.common.table.{HoodieTableMetaClient, TableSchemaResolver}
|
||||
|
||||
import java.util.stream.Collectors
|
||||
|
||||
import org.apache.hadoop.fs.{GlobPattern, Path}
|
||||
import org.apache.hudi.client.common.HoodieSparkEngineContext
|
||||
import org.apache.hudi.client.utils.SparkInternalSchemaConverter
|
||||
import org.apache.hudi.common.fs.FSUtils
|
||||
import org.apache.hudi.common.table.timeline.{HoodieInstant, HoodieTimeline}
|
||||
import org.apache.hudi.common.util.HoodieTimer
|
||||
import org.apache.hudi.common.util.{HoodieTimer, InternalSchemaCache}
|
||||
import org.apache.hudi.config.HoodieWriteConfig
|
||||
import org.apache.hudi.exception.HoodieException
|
||||
import org.apache.hudi.internal.schema.InternalSchema
|
||||
import org.apache.hudi.internal.schema.utils.SerDeHelper
|
||||
import org.apache.hudi.table.HoodieSparkTable
|
||||
import org.apache.log4j.LogManager
|
||||
import org.apache.spark.api.java.JavaSparkContext
|
||||
@@ -82,10 +85,17 @@ class IncrementalRelation(val sqlContext: SQLContext,
|
||||
private val commitsToReturn = commitsTimelineToReturn.getInstants.iterator().toList
|
||||
|
||||
// use schema from a file produced in the end/latest instant
|
||||
val usedSchema: StructType = {
|
||||
|
||||
val (usedSchema, internalSchema) = {
|
||||
log.info("Inferring schema..")
|
||||
val schemaResolver = new TableSchemaResolver(metaClient)
|
||||
val tableSchema = if (useEndInstantSchema) {
|
||||
val iSchema = if (useEndInstantSchema && !commitsToReturn.isEmpty) {
|
||||
InternalSchemaCache.searchSchemaAndCache(commitsToReturn.last.getTimestamp.toLong, metaClient, hoodieTable.getConfig.getInternalSchemaCacheEnable)
|
||||
} else {
|
||||
schemaResolver.getTableInternalSchemaFromCommitMetadata.orElse(null)
|
||||
}
|
||||
|
||||
val tableSchema = if (useEndInstantSchema && iSchema.isEmptySchema) {
|
||||
if (commitsToReturn.isEmpty) schemaResolver.getTableAvroSchemaWithoutMetadataFields() else
|
||||
schemaResolver.getTableAvroSchemaWithoutMetadataFields(commitsToReturn.last)
|
||||
} else {
|
||||
@@ -93,10 +103,15 @@ class IncrementalRelation(val sqlContext: SQLContext,
|
||||
}
|
||||
if (tableSchema.getType == Schema.Type.NULL) {
|
||||
// if there is only one commit in the table and is an empty commit without schema, return empty RDD here
|
||||
StructType(Nil)
|
||||
(StructType(Nil), InternalSchema.getEmptyInternalSchema)
|
||||
} else {
|
||||
val dataSchema = AvroConversionUtils.convertAvroSchemaToStructType(tableSchema)
|
||||
StructType(skeletonSchema.fields ++ dataSchema.fields)
|
||||
if (iSchema != null && !iSchema.isEmptySchema) {
|
||||
// if internalSchema is ready, dataSchema will contains skeletonSchema
|
||||
(dataSchema, iSchema)
|
||||
} else {
|
||||
(StructType(skeletonSchema.fields ++ dataSchema.fields), InternalSchema.getEmptyInternalSchema)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -161,6 +176,16 @@ class IncrementalRelation(val sqlContext: SQLContext,
|
||||
}
|
||||
// unset the path filter, otherwise if end_instant_time is not the latest instant, path filter set for RO view
|
||||
// will filter out all the files incorrectly.
|
||||
// pass internalSchema to hadoopConf, so it can be used in executors.
|
||||
val validCommits = metaClient
|
||||
.getCommitsAndCompactionTimeline.filterCompletedInstants.getInstants.toArray().map(_.asInstanceOf[HoodieInstant].getFileName).mkString(",")
|
||||
sqlContext.sparkContext.hadoopConfiguration.set(SparkInternalSchemaConverter.HOODIE_QUERY_SCHEMA, SerDeHelper.toJson(internalSchema))
|
||||
sqlContext.sparkContext.hadoopConfiguration.set(SparkInternalSchemaConverter.HOODIE_TABLE_PATH, metaClient.getBasePath)
|
||||
sqlContext.sparkContext.hadoopConfiguration.set(SparkInternalSchemaConverter.HOODIE_VALID_COMMITS_LIST, validCommits)
|
||||
val formatClassName = metaClient.getTableConfig.getBaseFileFormat match {
|
||||
case HoodieFileFormat.PARQUET => if (!internalSchema.isEmptySchema) "HoodieParquet" else "parquet"
|
||||
case HoodieFileFormat.ORC => "orc"
|
||||
}
|
||||
sqlContext.sparkContext.hadoopConfiguration.unset("mapreduce.input.pathFilter.class")
|
||||
val sOpts = optParams.filter(p => !p._1.equalsIgnoreCase("path"))
|
||||
if (filteredRegularFullPaths.isEmpty && filteredMetaBootstrapFullPaths.isEmpty) {
|
||||
@@ -216,8 +241,8 @@ class IncrementalRelation(val sqlContext: SQLContext,
|
||||
|
||||
if (regularFileIdToFullPath.nonEmpty) {
|
||||
df = df.union(sqlContext.read.options(sOpts)
|
||||
.schema(usedSchema)
|
||||
.parquet(filteredRegularFullPaths.toList: _*)
|
||||
.schema(usedSchema).format(formatClassName)
|
||||
.load(filteredRegularFullPaths.toList: _*)
|
||||
.filter(String.format("%s >= '%s'", HoodieRecord.COMMIT_TIME_METADATA_FIELD,
|
||||
commitsToReturn.head.getTimestamp))
|
||||
.filter(String.format("%s <= '%s'", HoodieRecord.COMMIT_TIME_METADATA_FIELD,
|
||||
|
||||
@@ -80,7 +80,7 @@ class MergeOnReadIncrementalRelation(sqlContext: SQLContext,
|
||||
options = optParams,
|
||||
// NOTE: We have to fork the Hadoop Config here as Spark will be modifying it
|
||||
// to configure Parquet reader appropriately
|
||||
hadoopConf = new Configuration(conf)
|
||||
hadoopConf = HoodieDataSourceHelper.getConfigurationWithInternalSchema(new Configuration(conf), internalSchema, metaClient.getBasePath, validCommits)
|
||||
)
|
||||
|
||||
val requiredSchemaParquetReader = createBaseFileReader(
|
||||
@@ -92,7 +92,7 @@ class MergeOnReadIncrementalRelation(sqlContext: SQLContext,
|
||||
options = optParams,
|
||||
// NOTE: We have to fork the Hadoop Config here as Spark will be modifying it
|
||||
// to configure Parquet reader appropriately
|
||||
hadoopConf = new Configuration(conf)
|
||||
hadoopConf = HoodieDataSourceHelper.getConfigurationWithInternalSchema(new Configuration(conf), requiredSchema.internalSchema, metaClient.getBasePath, validCommits)
|
||||
)
|
||||
|
||||
val hoodieTableState = getTableState
|
||||
|
||||
@@ -79,7 +79,7 @@ class MergeOnReadSnapshotRelation(sqlContext: SQLContext,
|
||||
options = optParams,
|
||||
// NOTE: We have to fork the Hadoop Config here as Spark will be modifying it
|
||||
// to configure Parquet reader appropriately
|
||||
hadoopConf = new Configuration(conf)
|
||||
hadoopConf = HoodieDataSourceHelper.getConfigurationWithInternalSchema(new Configuration(conf), internalSchema, metaClient.getBasePath, validCommits)
|
||||
)
|
||||
|
||||
val requiredSchemaParquetReader = createBaseFileReader(
|
||||
@@ -91,7 +91,7 @@ class MergeOnReadSnapshotRelation(sqlContext: SQLContext,
|
||||
options = optParams,
|
||||
// NOTE: We have to fork the Hadoop Config here as Spark will be modifying it
|
||||
// to configure Parquet reader appropriately
|
||||
hadoopConf = new Configuration(conf)
|
||||
hadoopConf = HoodieDataSourceHelper.getConfigurationWithInternalSchema(new Configuration(conf), requiredSchema.internalSchema, metaClient.getBasePath, validCommits)
|
||||
)
|
||||
|
||||
val tableState = getTableState
|
||||
|
||||
@@ -0,0 +1,48 @@
|
||||
/*
|
||||
* 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.spark.sql.execution.datasources.parquet
|
||||
|
||||
import org.apache.hadoop.conf.Configuration
|
||||
import org.apache.hudi.SparkAdapterSupport
|
||||
import org.apache.spark.sql.SparkSession
|
||||
import org.apache.spark.sql.catalyst.InternalRow
|
||||
import org.apache.spark.sql.execution.datasources.PartitionedFile
|
||||
import org.apache.spark.sql.sources.Filter
|
||||
import org.apache.spark.sql.types.StructType
|
||||
|
||||
|
||||
class SparkHoodieParquetFileFormat extends ParquetFileFormat with SparkAdapterSupport {
|
||||
override def shortName(): String = "HoodieParquet"
|
||||
|
||||
override def toString: String = "HoodieParquet"
|
||||
|
||||
override def buildReaderWithPartitionValues(
|
||||
sparkSession: SparkSession,
|
||||
dataSchema: StructType,
|
||||
partitionSchema: StructType,
|
||||
requiredSchema: StructType,
|
||||
filters: Seq[Filter],
|
||||
options: Map[String, String],
|
||||
hadoopConf: Configuration): PartitionedFile => Iterator[InternalRow] = {
|
||||
sparkAdapter
|
||||
.createHoodieParquetFileFormat().get
|
||||
.buildReaderWithPartitionValues(sparkSession, dataSchema, partitionSchema, requiredSchema, filters, options, hadoopConf)
|
||||
}
|
||||
}
|
||||
|
||||
@@ -110,7 +110,7 @@ object AlterHoodieTableAddColumnsCommand {
|
||||
HoodieWriterUtils.parametersWithWriteDefaults(hoodieCatalogTable.catalogProperties).asJava
|
||||
)
|
||||
|
||||
val commitActionType = CommitUtils.getCommitActionType(WriteOperationType.INSERT, hoodieCatalogTable.tableType)
|
||||
val commitActionType = CommitUtils.getCommitActionType(WriteOperationType.ALTER_SCHEMA, hoodieCatalogTable.tableType)
|
||||
val instantTime = HoodieActiveTimeline.createNewInstantTime
|
||||
client.startCommitWithTime(instantTime, commitActionType)
|
||||
|
||||
@@ -118,7 +118,7 @@ object AlterHoodieTableAddColumnsCommand {
|
||||
val timeLine = hoodieTable.getActiveTimeline
|
||||
val requested = new HoodieInstant(State.REQUESTED, commitActionType, instantTime)
|
||||
val metadata = new HoodieCommitMetadata
|
||||
metadata.setOperationType(WriteOperationType.INSERT)
|
||||
metadata.setOperationType(WriteOperationType.ALTER_SCHEMA)
|
||||
timeLine.transitionRequestedToInflight(requested, Option.of(metadata.toJsonString.getBytes(StandardCharsets.UTF_8)))
|
||||
|
||||
client.commit(instantTime, jsc.emptyRDD)
|
||||
|
||||
@@ -39,6 +39,10 @@ class HoodieSparkSessionExtension extends (SparkSessionExtensions => Unit)
|
||||
}
|
||||
}
|
||||
|
||||
extensions.injectResolutionRule { session =>
|
||||
sparkAdapter.createResolveHudiAlterTableCommand(session)
|
||||
}
|
||||
|
||||
HoodieAnalysis.customPostHocResolutionRules().foreach { rule =>
|
||||
extensions.injectPostHocResolutionRule { session =>
|
||||
rule(session)
|
||||
|
||||
@@ -221,7 +221,7 @@ class TestHoodieSparkUtils {
|
||||
|
||||
val tableAvroSchema = new Schema.Parser().parse(avroSchemaString)
|
||||
|
||||
val (requiredAvroSchema, requiredStructSchema) =
|
||||
val (requiredAvroSchema, requiredStructSchema, _) =
|
||||
HoodieSparkUtils.getRequiredSchema(tableAvroSchema, Array("ts"))
|
||||
|
||||
assertEquals("timestamp-millis",
|
||||
|
||||
@@ -331,7 +331,7 @@ class TestParquetColumnProjection extends SparkClientFunctionalTestHarness with
|
||||
}
|
||||
|
||||
val readColumns = targetColumns ++ relation.mandatoryColumns
|
||||
val (_, projectedStructType) = HoodieSparkUtils.getRequiredSchema(tableState.schema, readColumns)
|
||||
val (_, projectedStructType, _) = HoodieSparkUtils.getRequiredSchema(tableState.schema, readColumns)
|
||||
|
||||
val row: InternalRow = rows.take(1).head
|
||||
|
||||
|
||||
@@ -102,6 +102,22 @@ class TestHoodieSqlBase extends FunSuite with BeforeAndAfterAll {
|
||||
assertResult(expects.map(row => Row(row: _*)).toArray.sortBy(_.toString()))(spark.sql(sql).collect().sortBy(_.toString()))
|
||||
}
|
||||
|
||||
protected def checkAnswer(array: Array[Row])(expects: Seq[Any]*): Unit = {
|
||||
assertResult(expects.map(row => Row(row: _*)).toArray)(array)
|
||||
}
|
||||
|
||||
protected def checkExceptions(sql: String)(errorMsgs: Seq[String]): Unit = {
|
||||
var hasException = false
|
||||
try {
|
||||
spark.sql(sql)
|
||||
} catch {
|
||||
case e: Throwable =>
|
||||
assertResult(errorMsgs.contains(e.getMessage.split("\n")(0)))(true)
|
||||
hasException = true
|
||||
}
|
||||
assertResult(true)(hasException)
|
||||
}
|
||||
|
||||
protected def checkException(sql: String)(errorMsg: String): Unit = {
|
||||
var hasException = false
|
||||
try {
|
||||
|
||||
@@ -0,0 +1,472 @@
|
||||
/*
|
||||
* 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.spark.sql.hudi
|
||||
|
||||
import org.apache.hadoop.fs.Path
|
||||
import org.apache.hudi.common.model.HoodieRecord
|
||||
import org.apache.hudi.config.{HoodieClusteringConfig, HoodieWriteConfig}
|
||||
import org.apache.hudi.{DataSourceWriteOptions, HoodieSparkUtils}
|
||||
import org.apache.spark.sql.catalyst.TableIdentifier
|
||||
import org.apache.spark.sql.{DataFrame, Row, SaveMode, SparkSession}
|
||||
|
||||
import scala.collection.JavaConversions._
|
||||
import scala.collection.JavaConverters._
|
||||
|
||||
class TestSpark3DDL extends TestHoodieSqlBase {
|
||||
|
||||
def createTestResult(tableName: String): Array[Row] = {
|
||||
spark.sql(s"select * from ${tableName} order by id")
|
||||
.drop("_hoodie_commit_time", "_hoodie_commit_seqno", "_hoodie_record_key", "_hoodie_partition_path", "_hoodie_file_name").collect()
|
||||
}
|
||||
|
||||
def createAndPreparePartitionTable(spark: SparkSession, tableName: String, tablePath: String, tableType: String): Unit = {
|
||||
// try to clean tablePath
|
||||
spark.sql(
|
||||
s"""
|
||||
|create table $tableName (
|
||||
| id int, comb int, col0 int, col1 bigint, col2 float, col3 double, col4 decimal(10,4), col5 string, col6 date, col7 timestamp, col8 boolean, col9 binary, par date
|
||||
|) using hudi
|
||||
| location '$tablePath'
|
||||
| options (
|
||||
| type = '$tableType',
|
||||
| primaryKey = 'id',
|
||||
| preCombineField = 'comb'
|
||||
| )
|
||||
| partitioned by (par)
|
||||
""".stripMargin)
|
||||
spark.sql(
|
||||
s"""
|
||||
| insert into $tableName values
|
||||
| (1,1,11,100001,101.01,1001.0001,100001.0001,'a000001','2021-12-25','2021-12-25 12:01:01',true,'a01','2021-12-25'),
|
||||
| (2,2,12,100002,102.02,1002.0002,100002.0002,'a000002','2021-12-25','2021-12-25 12:02:02',true,'a02','2021-12-25'),
|
||||
| (3,3,13,100003,103.03,1003.0003,100003.0003,'a000003','2021-12-25','2021-12-25 12:03:03',false,'a03','2021-12-25'),
|
||||
| (4,4,14,100004,104.04,1004.0004,100004.0004,'a000004','2021-12-26','2021-12-26 12:04:04',true,'a04','2021-12-26'),
|
||||
| (5,5,15,100005,105.05,1005.0005,100005.0005,'a000005','2021-12-26','2021-12-26 12:05:05',false,'a05','2021-12-26')
|
||||
|""".stripMargin)
|
||||
}
|
||||
|
||||
test("Test multi change data type") {
|
||||
withTempDir { tmp =>
|
||||
Seq("cow", "mor").foreach { tableType =>
|
||||
val tableName = generateTableName
|
||||
val tablePath = s"${new Path(tmp.getCanonicalPath, tableName).toUri.toString}"
|
||||
if (HoodieSparkUtils.gteqSpark3_1) {
|
||||
spark.sql("set hoodie.schema.on.read.enable=true")
|
||||
createAndPreparePartitionTable(spark, tableName, tablePath, tableType)
|
||||
// date -> string -> date
|
||||
spark.sql(s"alter table $tableName alter column col6 type String")
|
||||
checkAnswer(spark.sql(s"select col6 from $tableName where id = 1").collect())(
|
||||
Seq("2021-12-25")
|
||||
)
|
||||
spark.sql(
|
||||
s"""
|
||||
| insert into $tableName values
|
||||
| (1,1,13.0,100001,101.01,1001.0001,100001.0001,'a000001','2021-12-26','2021-12-25 12:01:01',true,'a01','2021-12-25')
|
||||
|""".stripMargin)
|
||||
spark.sql(s"alter table $tableName alter column col6 type date")
|
||||
checkAnswer(spark.sql(s"select col6 from $tableName where id = 1 or id = 5 order by id").collect())(
|
||||
Seq(java.sql.Date.valueOf("2021-12-26")), // value from new file
|
||||
Seq(java.sql.Date.valueOf("2021-12-26")) // value from old file
|
||||
)
|
||||
// int -> double -> decimal
|
||||
spark.sql(s"alter table $tableName alter column col0 type double")
|
||||
spark.sql(
|
||||
s"""
|
||||
| insert into $tableName values
|
||||
| (1,1,13.0,100001,101.01,1001.0001,100001.0001,'a000001','2021-12-25','2021-12-25 12:01:01',true,'a01','2021-12-25'),
|
||||
| (6,1,14.0,100001,101.01,1001.0001,100001.0001,'a000001','2021-12-25','2021-12-25 12:01:01',true,'a01','2021-12-25')
|
||||
|""".stripMargin)
|
||||
spark.sql(s"alter table $tableName alter column col0 type decimal(16, 4)")
|
||||
checkAnswer(spark.sql(s"select col0 from $tableName where id = 1 or id = 6 order by id").collect())(
|
||||
Seq(new java.math.BigDecimal("13.0000")),
|
||||
Seq(new java.math.BigDecimal("14.0000"))
|
||||
)
|
||||
// float -> double -> decimal
|
||||
spark.sql(s"alter table $tableName alter column col2 type double")
|
||||
spark.sql(
|
||||
s"""
|
||||
| insert into $tableName values
|
||||
| (1,1,13.0,100001,901.01,1001.0001,100001.0001,'a000001','2021-12-25','2021-12-25 12:01:01',true,'a01','2021-12-25'),
|
||||
| (6,1,14.0,100001,601.01,1001.0001,100001.0001,'a000001','2021-12-25','2021-12-25 12:01:01',true,'a01','2021-12-25')
|
||||
|""".stripMargin)
|
||||
spark.sql(s"alter table $tableName alter column col2 type decimal(16, 4)")
|
||||
checkAnswer(spark.sql(s"select col0, col2 from $tableName where id = 1 or id = 6 order by id").collect())(
|
||||
Seq(new java.math.BigDecimal("13.0000"), new java.math.BigDecimal("901.0100")),
|
||||
Seq(new java.math.BigDecimal("14.0000"), new java.math.BigDecimal("601.0100"))
|
||||
)
|
||||
// long -> double -> decimal
|
||||
spark.sql(s"alter table $tableName alter column col1 type double")
|
||||
spark.sql(
|
||||
s"""
|
||||
| insert into $tableName values
|
||||
| (1,1,13.0,700001.0,901.01,1001.0001,100001.0001,'a000001','2021-12-25','2021-12-25 12:01:01',true,'a01','2021-12-25')
|
||||
|""".stripMargin)
|
||||
spark.sql(s"alter table $tableName alter column col1 type decimal(16, 4)")
|
||||
checkAnswer(spark.sql(s"select col0, col2, col1 from $tableName where id = 1 or id = 6 order by id").collect())(
|
||||
Seq(new java.math.BigDecimal("13.0000"), new java.math.BigDecimal("901.0100"), new java.math.BigDecimal("700001.0000")),
|
||||
Seq(new java.math.BigDecimal("14.0000"), new java.math.BigDecimal("601.0100"), new java.math.BigDecimal("100001.0000"))
|
||||
)
|
||||
spark.sessionState.catalog.dropTable(TableIdentifier(tableName), true, true)
|
||||
spark.sessionState.catalog.refreshTable(TableIdentifier(tableName))
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
test("Test multi change data type2") {
|
||||
withTempDir { tmp =>
|
||||
Seq("cow", "mor").foreach { tableType =>
|
||||
val tableName = generateTableName
|
||||
val tablePath = s"${new Path(tmp.getCanonicalPath, tableName).toUri.toString}"
|
||||
if (HoodieSparkUtils.gteqSpark3_1) {
|
||||
spark.sql("set hoodie.schema.on.read.enable=true")
|
||||
createAndPreparePartitionTable(spark, tableName, tablePath, tableType)
|
||||
// float -> double -> decimal -> String
|
||||
spark.sql(s"alter table $tableName alter column col2 type double")
|
||||
spark.sql(s"alter table $tableName alter column col2 type decimal(16, 4)")
|
||||
spark.sql(s"alter table $tableName alter column col2 type String")
|
||||
checkAnswer(spark.sql(s"select col2 from $tableName where id = 1").collect())(
|
||||
Seq("101.01")
|
||||
)
|
||||
// long -> double -> decimal -> string
|
||||
spark.sql(s"alter table $tableName alter column col1 type double")
|
||||
spark.sql(s"alter table $tableName alter column col1 type decimal(16, 4)")
|
||||
spark.sql(s"alter table $tableName alter column col1 type String")
|
||||
checkAnswer(spark.sql(s"select col1 from $tableName where id = 1").collect())(
|
||||
Seq("100001")
|
||||
)
|
||||
// int -> double -> decimal -> String
|
||||
spark.sql(s"alter table $tableName alter column col0 type double")
|
||||
spark.sql(s"alter table $tableName alter column col0 type decimal(16, 4)")
|
||||
spark.sql(s"alter table $tableName alter column col0 type String")
|
||||
checkAnswer(spark.sql(s"select col0 from $tableName where id = 1").collect())(
|
||||
Seq("11")
|
||||
)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
test("Test Partition Table alter ") {
|
||||
withTempDir { tmp =>
|
||||
Seq("cow", "mor").foreach { tableType =>
|
||||
val tableName = generateTableName
|
||||
val tablePath = s"${new Path(tmp.getCanonicalPath, tableName).toUri.toString}"
|
||||
if (HoodieSparkUtils.gteqSpark3_1) {
|
||||
spark.sql("set hoodie.schema.on.read.enable=true")
|
||||
createAndPreparePartitionTable(spark, tableName, tablePath, tableType)
|
||||
|
||||
// test set properties
|
||||
spark.sql(s"alter table $tableName set tblproperties(comment='it is a hudi table', 'key1'='value1', 'key2'='value2')")
|
||||
val meta = spark.sessionState.catalog.getTableMetadata(TableIdentifier(tableName))
|
||||
assert(meta.comment.get.equals("it is a hudi table"))
|
||||
assert(Seq("key1", "key2").filter(meta.properties.contains(_)).size == 2)
|
||||
// test unset propertes
|
||||
spark.sql(s"alter table $tableName unset tblproperties(comment, 'key1', 'key2')")
|
||||
val unsetMeta = spark.sessionState.catalog.getTableMetadata(TableIdentifier(tableName))
|
||||
assert(Seq("key1", "key2").filter(unsetMeta.properties.contains(_)).size == 0)
|
||||
assert(unsetMeta.comment.isEmpty)
|
||||
// test forbidden operation.
|
||||
checkException(s"Alter table $tableName add columns(col_new1 int first)")("forbid adjust top-level columns position by using through first syntax")
|
||||
HoodieRecord.HOODIE_META_COLUMNS.subList(0, HoodieRecord.HOODIE_META_COLUMNS.size - 2).asScala.foreach {f =>
|
||||
checkException(s"Alter table $tableName add columns(col_new1 int after $f)")("forbid adjust the position of ordinary columns between meta columns")
|
||||
}
|
||||
Seq("id", "comb", "par").foreach { col =>
|
||||
checkException(s"alter table $tableName drop column $col")("cannot support apply changes for primaryKey/CombineKey/partitionKey")
|
||||
checkException(s"alter table $tableName rename column $col to ${col + col}")("cannot support apply changes for primaryKey/CombineKey/partitionKey")
|
||||
}
|
||||
// check duplicate add or rename
|
||||
// keep consistent with hive, column names insensitive
|
||||
checkExceptions(s"alter table $tableName rename column col0 to col9")(Seq("cannot rename column: col0 to a existing name",
|
||||
"Cannot rename column, because col9 already exists in root"))
|
||||
checkExceptions(s"alter table $tableName rename column col0 to COL9")(Seq("cannot rename column: col0 to a existing name", "Cannot rename column, because COL9 already exists in root"))
|
||||
checkExceptions(s"alter table $tableName add columns(col9 int first)")(Seq("cannot add column: col9 which already exist", "Cannot add column, because col9 already exists in root"))
|
||||
checkExceptions(s"alter table $tableName add columns(COL9 int first)")(Seq("cannot add column: COL9 which already exist", "Cannot add column, because COL9 already exists in root"))
|
||||
// test add comment for columns / alter columns comment
|
||||
spark.sql(s"alter table $tableName add columns(col1_new int comment 'add new columns col1_new after id' after id)")
|
||||
spark.sql(s"alter table $tableName alter column col9 comment 'col9 desc'")
|
||||
val schema = spark.sessionState.catalog.getTableMetadata(TableIdentifier(tableName)).schema
|
||||
assert(schema.filter(p => p.name.equals("col1_new")).get(0).getComment().get == "add new columns col1_new after id")
|
||||
assert(schema.filter(p => p.name.equals("col9")).get(0).getComment().get == "col9 desc")
|
||||
// test change column type float to double
|
||||
spark.sql(s"alter table $tableName alter column col2 type double")
|
||||
spark.sql(s"select id, col1_new, col2 from $tableName where id = 1 or id = 2 order by id").show(false)
|
||||
spark.sql(
|
||||
s"""
|
||||
| insert into $tableName values
|
||||
| (1,3,1,11,100001,101.01,1001.0001,100001.0001,'a000001','2021-12-25','2021-12-25 12:01:01',true,'a01','2021-12-25'),
|
||||
| (6,6,5,15,100005,105.05,1005.0005,100005.0005,'a000005','2021-12-26','2021-12-26 12:05:05',false,'a05','2021-12-26')
|
||||
|""".stripMargin)
|
||||
|
||||
spark.sql(s"select id, col1_new, col2 from $tableName where id = 1 or id = 6 or id = 2 order by id").show(false)
|
||||
// try schedule compact
|
||||
if (tableType == "mor") spark.sql(s"schedule compaction on $tableName")
|
||||
// test change column type decimal(10,4) 为decimal(18,8)
|
||||
spark.sql(s"alter table $tableName alter column col4 type decimal(18, 8)")
|
||||
spark.sql(s"select id, col1_new, col2 from $tableName where id = 1 or id = 2 order by id").show(false)
|
||||
spark.sql(
|
||||
s"""
|
||||
| insert into $tableName values
|
||||
| (5,6,5,15,100005,105.05,1005.0005,100005.0005,'a000005','2021-12-26','2021-12-26 12:05:05',false,'a05','2021-12-26')
|
||||
|""".stripMargin)
|
||||
|
||||
spark.sql(s"select id, col1_new, col4 from $tableName where id = 1 or id = 6 or id = 2 order by id").show(false)
|
||||
// test change column type float to double
|
||||
spark.sql(s"alter table $tableName alter column col2 type string")
|
||||
spark.sql(s"select id, col1_new, col2 from $tableName where id = 1 or id = 2 order by id").show(false)
|
||||
spark.sql(
|
||||
s"""
|
||||
| insert into $tableName values
|
||||
| (1,3,1,11,100001,'101.01',1001.0001,100001.0001,'a000001','2021-12-25','2021-12-25 12:01:01',true,'a01','2021-12-25'),
|
||||
| (6,6,5,15,100005,'105.05',1005.0005,100005.0005,'a000005','2021-12-26','2021-12-26 12:05:05',false,'a05','2021-12-26')
|
||||
|""".stripMargin)
|
||||
|
||||
spark.sql(s"select id, col1_new, col2 from $tableName where id = 1 or id = 6 or id = 2 order by id").show(false)
|
||||
// try schedule compact
|
||||
if (tableType == "mor") spark.sql(s"schedule compaction on $tableName")
|
||||
// if tableType is mor, check compaction
|
||||
if (tableType == "mor") {
|
||||
val compactionRows = spark.sql(s"show compaction on $tableName limit 10").collect()
|
||||
val timestamps = compactionRows.map(_.getString(0))
|
||||
assertResult(2)(timestamps.length)
|
||||
spark.sql(s"run compaction on $tableName at ${timestamps(1)}")
|
||||
spark.sql(s"run compaction on $tableName at ${timestamps(0)}")
|
||||
}
|
||||
spark.sql(
|
||||
s"""
|
||||
| insert into $tableName values
|
||||
| (1,3,1,11,100001,'101.01',1001.0001,100009.0001,'a000008','2021-12-25','2021-12-25 12:01:01',true,'a01','2021-12-25'),
|
||||
| (11,3,1,11,100001,'101.01',1001.0001,100011.0001,'a000008','2021-12-25','2021-12-25 12:01:01',true,'a01','2021-12-25'),
|
||||
| (6,6,5,15,100005,'105.05',1005.0005,100007.0005,'a000009','2021-12-26','2021-12-26 12:05:05',false,'a05','2021-12-26')
|
||||
|""".stripMargin)
|
||||
|
||||
spark.sql(s"select id, col1_new, col2 from $tableName where id = 1 or id = 6 or id = 2 or id = 11 order by id").show(false)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
test("Test Chinese table ") {
|
||||
withTempDir { tmp =>
|
||||
Seq("cow", "mor").foreach { tableType =>
|
||||
val tableName = generateTableName
|
||||
val tablePath = s"${new Path(tmp.getCanonicalPath, tableName).toUri.toString}"
|
||||
if (HoodieSparkUtils.gteqSpark3_1) {
|
||||
spark.sql("set hoodie.schema.on.read.enable=true")
|
||||
spark.sql(
|
||||
s"""
|
||||
|create table $tableName (
|
||||
| id int, comb int, `名字` string, col9 string, `成绩` int, `身高` float, `体重` double, `上次更新时间` date, par date
|
||||
|) using hudi
|
||||
| location '$tablePath'
|
||||
| options (
|
||||
| type = '$tableType',
|
||||
| primaryKey = 'id',
|
||||
| preCombineField = 'comb'
|
||||
| )
|
||||
| partitioned by (par)
|
||||
""".stripMargin)
|
||||
spark.sql(
|
||||
s"""
|
||||
| insert into $tableName values
|
||||
| (1,3,'李明', '读书', 100,180.0001,99.0001,'2021-12-25', '2021-12-26')
|
||||
|""".stripMargin)
|
||||
spark.sql(s"alter table $tableName rename column col9 to `爱好_Best`")
|
||||
|
||||
// update current table to produce log files for mor
|
||||
spark.sql(
|
||||
s"""
|
||||
| insert into $tableName values
|
||||
| (1,3,'李明', '读书', 100,180.0001,99.0001,'2021-12-26', '2021-12-26')
|
||||
|""".stripMargin)
|
||||
|
||||
// alter date to string
|
||||
spark.sql(s"alter table $tableName alter column `上次更新时间` type string ")
|
||||
checkAnswer(spark.sql(s"select `上次更新时间` from $tableName").collect())(
|
||||
Seq("2021-12-26")
|
||||
)
|
||||
// alter string to date
|
||||
spark.sql(s"alter table $tableName alter column `上次更新时间` type date ")
|
||||
spark.sql(s"select `上次更新时间` from $tableName").collect()
|
||||
checkAnswer(spark.sql(s"select `上次更新时间` from $tableName").collect())(
|
||||
Seq(java.sql.Date.valueOf("2021-12-26"))
|
||||
)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
test("Test Alter Table") {
|
||||
withTempDir { tmp =>
|
||||
Seq("cow", "mor").foreach { tableType =>
|
||||
val tableName = generateTableName
|
||||
val tablePath = s"${new Path(tmp.getCanonicalPath, tableName).toUri.toString}"
|
||||
if (HoodieSparkUtils.gteqSpark3_1) {
|
||||
spark.sql("set hoodie.schema.on.read.enable=true")
|
||||
spark.sql(
|
||||
s"""
|
||||
|create table $tableName (
|
||||
| id int,
|
||||
| name string,
|
||||
| price double,
|
||||
| ts long
|
||||
|) using hudi
|
||||
| location '$tablePath'
|
||||
| options (
|
||||
| type = '$tableType',
|
||||
| primaryKey = 'id',
|
||||
| preCombineField = 'ts'
|
||||
| )
|
||||
""".stripMargin)
|
||||
spark.sql(s"show create table ${tableName}").show(false)
|
||||
spark.sql(s"insert into ${tableName} values (1, 'jack', 0.9, 1000)")
|
||||
spark.sql(s"update ${tableName} set price = 1.9 where id = 1")
|
||||
|
||||
spark.sql(s"alter table ${tableName} alter column id type long")
|
||||
checkAnswer(createTestResult(tableName))(
|
||||
Seq(1, "jack", 1.9, 1000)
|
||||
)
|
||||
// test add action, include position change
|
||||
spark.sql(s"alter table ${tableName} add columns(ext1 string comment 'add ext1' after name)")
|
||||
spark.sql(s"insert into ${tableName} values (2, 'jack', 'exx1', 0.9, 1000)")
|
||||
checkAnswer(createTestResult(tableName))(
|
||||
Seq(1, "jack", null, 1.9, 1000), Seq(2, "jack","exx1", 0.9, 1000)
|
||||
)
|
||||
// test rename
|
||||
spark.sql(s"alter table ${tableName} rename column price to newprice")
|
||||
checkAnswer(createTestResult(tableName))(
|
||||
Seq(1, "jack", null, 1.9, 1000), Seq(2, "jack","exx1", 0.9, 1000)
|
||||
)
|
||||
spark.sql(s"update ${tableName} set ext1 = 'haha' where id = 1 ")
|
||||
checkAnswer(createTestResult(tableName))(
|
||||
Seq(1, "jack", "haha", 1.9, 1000), Seq(2, "jack","exx1", 0.9, 1000)
|
||||
)
|
||||
// drop column newprice
|
||||
|
||||
spark.sql(s"alter table ${tableName} drop column newprice")
|
||||
checkAnswer(createTestResult(tableName))(
|
||||
Seq(1, "jack", "haha", 1000), Seq(2, "jack","exx1", 1000)
|
||||
)
|
||||
// add newprice back
|
||||
spark.sql(s"alter table ${tableName} add columns(newprice string comment 'add newprice back' after ext1)")
|
||||
checkAnswer(createTestResult(tableName))(
|
||||
Seq(1, "jack", "haha", null, 1000), Seq(2, "jack","exx1", null, 1000)
|
||||
)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
test("Test Alter Table complex") {
|
||||
withTempDir { tmp =>
|
||||
Seq("cow", "mor").foreach { tableType =>
|
||||
val tableName = generateTableName
|
||||
val tablePath = s"${new Path(tmp.getCanonicalPath, tableName).toUri.toString}"
|
||||
if (HoodieSparkUtils.gteqSpark3_1) {
|
||||
spark.sql("set hoodie.schema.on.read.enable=true")
|
||||
spark.sql(
|
||||
s"""
|
||||
|create table $tableName (
|
||||
| id int,
|
||||
| name string,
|
||||
| members map<String, struct<n:string, a:int>>,
|
||||
| user struct<name:string, age:int, score: int>,
|
||||
| ts long
|
||||
|) using hudi
|
||||
| location '$tablePath'
|
||||
| options (
|
||||
| type = '$tableType',
|
||||
| primaryKey = 'id',
|
||||
| preCombineField = 'ts'
|
||||
| )
|
||||
""".stripMargin)
|
||||
|
||||
spark.sql(s"alter table $tableName alter column members.value.a first")
|
||||
|
||||
spark.sql(s"insert into ${tableName} values(1, 'jack', map('k1', struct('v1', 100), 'k2', struct('v2', 200)), struct('jackStruct', 29, 100), 1000)")
|
||||
|
||||
// rename column
|
||||
spark.sql(s"alter table ${tableName} rename column user to userx")
|
||||
|
||||
checkAnswer(spark.sql(s"select ts, userx.score, id, userx.age, name from ${tableName}").collect())(
|
||||
Seq(1000, 100, 1, 29, "jack")
|
||||
)
|
||||
|
||||
// drop column
|
||||
spark.sql(s"alter table ${tableName} drop columns(name, userx.name, userx.score)")
|
||||
|
||||
spark.sql(s"select * from ${tableName}").show(false)
|
||||
|
||||
// add cols back, and adjust cols position
|
||||
spark.sql(s"alter table ${tableName} add columns(name string comment 'add name back' after userx," +
|
||||
s" userx.name string comment 'add userx.name back' first, userx.score int comment 'add userx.score back' after age)")
|
||||
|
||||
// query new columns: name, userx.name, userx.score, those field should not be readed.
|
||||
checkAnswer(spark.sql(s"select name, userx.name, userx.score from ${tableName}").collect())(Seq(null, null, null))
|
||||
|
||||
// insert again
|
||||
spark.sql(s"insert into ${tableName} values(2 , map('k1', struct('v1', 100), 'k2', struct('v2', 200)), struct('jackStructNew', 291 , 101), 'jacknew', 1000)")
|
||||
|
||||
// check again
|
||||
checkAnswer(spark.sql(s"select name, userx.name as uxname, userx.score as uxs from ${tableName} order by id").collect())(
|
||||
Seq(null, null, null),
|
||||
Seq("jacknew", "jackStructNew", 101))
|
||||
|
||||
|
||||
spark.sql(s"alter table ${tableName} alter column userx.age type long")
|
||||
|
||||
spark.sql(s"select userx.age, id, name from ${tableName}")
|
||||
checkAnswer(spark.sql(s"select userx.age, id, name from ${tableName} order by id").collect())(
|
||||
Seq(29, 1, null),
|
||||
Seq(291, 2, "jacknew"))
|
||||
// test map value type change
|
||||
spark.sql(s"alter table ${tableName} add columns(mxp map<String, int>)")
|
||||
spark.sql(s"insert into ${tableName} values(2 , map('k1', struct('v1', 100), 'k2', struct('v2', 200)), struct('jackStructNew', 291 , 101), 'jacknew', 1000, map('t1', 9))")
|
||||
spark.sql(s"alter table ${tableName} alter column mxp.value type double")
|
||||
spark.sql(s"insert into ${tableName} values(2 , map('k1', struct('v1', 100), 'k2', struct('v2', 200)), struct('jackStructNew', 291 , 101), 'jacknew', 1000, map('t1', 10))")
|
||||
spark.sql(s"select * from $tableName").show(false)
|
||||
checkAnswer(spark.sql(s"select mxp from ${tableName} order by id").collect())(
|
||||
Seq(null),
|
||||
Seq(Map("t1" -> 10.0d))
|
||||
)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private def performClustering(writeDf: DataFrame, basePath: String, tableName: String, tableType: String): Unit = {
|
||||
writeDf.write.format("org.apache.hudi")
|
||||
.option(DataSourceWriteOptions.TABLE_TYPE.key(), tableType)
|
||||
.option("hoodie.upsert.shuffle.parallelism", "1")
|
||||
.option(DataSourceWriteOptions.RECORDKEY_FIELD.key(), "id")
|
||||
.option(DataSourceWriteOptions.PRECOMBINE_FIELD.key(), "comb")
|
||||
.option(DataSourceWriteOptions.PARTITIONPATH_FIELD.key(), "par")
|
||||
.option(HoodieWriteConfig.TBL_NAME.key, tableName)
|
||||
.option("hoodie.schema.on.read.enable", "true")
|
||||
// option for clustering
|
||||
.option("hoodie.clustering.inline", "true")
|
||||
.option("hoodie.clustering.inline.max.commits", "1")
|
||||
.option("hoodie.clustering.plan.strategy.small.file.limit", String.valueOf(2*1024*1024L))
|
||||
.option("hoodie.clustering.plan.strategy.max.bytes.per.group", String.valueOf(10*1024*1024L))
|
||||
.option("hoodie.clustering.plan.strategy.target.file.max.bytes", String.valueOf(4 * 1024* 1024L))
|
||||
.option(HoodieClusteringConfig.PLAN_STRATEGY_SORT_COLUMNS.key, "col1, col2")
|
||||
.mode(SaveMode.Append)
|
||||
.save(basePath)
|
||||
}
|
||||
}
|
||||
@@ -1,3 +1,4 @@
|
||||
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
@@ -27,7 +28,9 @@ import org.apache.spark.sql.catalyst.expressions.{Expression, Like}
|
||||
import org.apache.spark.sql.catalyst.parser.ParserInterface
|
||||
import org.apache.spark.sql.catalyst.plans.JoinType
|
||||
import org.apache.spark.sql.catalyst.plans.logical.{InsertIntoTable, Join, LogicalPlan}
|
||||
import org.apache.spark.sql.catalyst.rules.Rule
|
||||
import org.apache.spark.sql.catalyst.{AliasIdentifier, TableIdentifier}
|
||||
import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat
|
||||
import org.apache.spark.sql.execution.datasources.{FilePartition, PartitionedFile, Spark2ParsePartitionUtil, SparkParsePartitionUtil}
|
||||
import org.apache.spark.sql.hudi.SparkAdapter
|
||||
import org.apache.spark.sql.hudi.parser.HoodieSpark2ExtendedSqlParser
|
||||
@@ -155,4 +158,14 @@ class Spark2Adapter extends SparkAdapter {
|
||||
override def getRelationTimeTravel(plan: LogicalPlan): Option[(LogicalPlan, Option[Expression], Option[String])] = {
|
||||
throw new IllegalStateException(s"Should not call getRelationTimeTravel for spark2")
|
||||
}
|
||||
|
||||
override def createResolveHudiAlterTableCommand(sparkSession: SparkSession): Rule[LogicalPlan] = {
|
||||
new Rule[LogicalPlan] {
|
||||
override def apply(plan: LogicalPlan): LogicalPlan = plan
|
||||
}
|
||||
}
|
||||
|
||||
override def createHoodieParquetFileFormat(): Option[ParquetFileFormat] = {
|
||||
Some(new ParquetFileFormat)
|
||||
}
|
||||
}
|
||||
|
||||
@@ -19,6 +19,7 @@ package org.apache.spark.sql.adapter
|
||||
|
||||
import org.apache.hudi.Spark3RowSerDe
|
||||
import org.apache.hudi.client.utils.SparkRowSerDe
|
||||
import org.apache.spark.SPARK_VERSION
|
||||
import org.apache.hudi.spark3.internal.ReflectUtil
|
||||
import org.apache.spark.sql.avro.{HoodieAvroSchemaConverters, HoodieSparkAvroSchemaConverters}
|
||||
import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
|
||||
@@ -27,11 +28,13 @@ import org.apache.spark.sql.catalyst.expressions.{Expression, Like}
|
||||
import org.apache.spark.sql.catalyst.parser.ParserInterface
|
||||
import org.apache.spark.sql.catalyst.plans.JoinType
|
||||
import org.apache.spark.sql.catalyst.plans.logical.{InsertIntoStatement, Join, JoinHint, LogicalPlan}
|
||||
import org.apache.spark.sql.catalyst.rules.Rule
|
||||
import org.apache.spark.sql.catalyst.{AliasIdentifier, TableIdentifier}
|
||||
import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._
|
||||
import org.apache.spark.sql.connector.catalog.Table
|
||||
import org.apache.spark.sql.execution.datasources._
|
||||
import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
|
||||
import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat
|
||||
import org.apache.spark.sql.hudi.SparkAdapter
|
||||
import org.apache.spark.sql.internal.SQLConf
|
||||
import org.apache.spark.sql.{Row, SparkSession}
|
||||
@@ -131,4 +134,19 @@ abstract class BaseSpark3Adapter extends SparkAdapter {
|
||||
override def getRelationTimeTravel(plan: LogicalPlan): Option[(LogicalPlan, Option[Expression], Option[String])] = {
|
||||
throw new IllegalStateException(s"Should not call getRelationTimeTravel for spark3.1.x")
|
||||
}
|
||||
override def createExtendedSparkParser: Option[(SparkSession, ParserInterface) => ParserInterface] = {
|
||||
// since spark3.2.1 support datasourceV2, so we need to a new SqlParser to deal DDL statment
|
||||
if (SPARK_VERSION.startsWith("3.1")) {
|
||||
val loadClassName = "org.apache.spark.sql.parser.HoodieSpark312ExtendedSqlParser"
|
||||
Some {
|
||||
(spark: SparkSession, delegate: ParserInterface) => {
|
||||
val clazz = Class.forName(loadClassName, true, Thread.currentThread().getContextClassLoader)
|
||||
val ctor = clazz.getConstructors.head
|
||||
ctor.newInstance(spark, delegate).asInstanceOf[ParserInterface]
|
||||
}
|
||||
}
|
||||
} else {
|
||||
None
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -46,6 +46,7 @@
|
||||
<configuration>
|
||||
<args>
|
||||
<arg>-nobootcp</arg>
|
||||
<arg>-target:jvm-1.8</arg>
|
||||
</args>
|
||||
<checkMultipleScalaVersions>false</checkMultipleScalaVersions>
|
||||
</configuration>
|
||||
@@ -157,7 +158,7 @@
|
||||
<dependency>
|
||||
<groupId>org.apache.spark</groupId>
|
||||
<artifactId>spark-sql_2.12</artifactId>
|
||||
<version>${spark3.version}</version>
|
||||
<version>${spark31.version}</version>
|
||||
<optional>true</optional>
|
||||
</dependency>
|
||||
|
||||
@@ -181,16 +182,36 @@
|
||||
<groupId>org.apache.hudi</groupId>
|
||||
<artifactId>hudi-spark-client</artifactId>
|
||||
<version>${project.version}</version>
|
||||
<exclusions>
|
||||
<exclusion>
|
||||
<groupId>org.apache.spark</groupId>
|
||||
<artifactId>*</artifactId>
|
||||
</exclusion>
|
||||
</exclusions>
|
||||
</dependency>
|
||||
|
||||
<dependency>
|
||||
<groupId>org.apache.hudi</groupId>
|
||||
<artifactId>hudi-spark-common_${scala.binary.version}</artifactId>
|
||||
<version>${project.version}</version>
|
||||
<exclusions>
|
||||
<exclusion>
|
||||
<groupId>org.apache.spark</groupId>
|
||||
<artifactId>*</artifactId>
|
||||
</exclusion>
|
||||
</exclusions>
|
||||
</dependency>
|
||||
|
||||
<dependency>
|
||||
<groupId>org.apache.hudi</groupId>
|
||||
<artifactId>hudi-spark3-common</artifactId>
|
||||
<version>${project.version}</version>
|
||||
<exclusions>
|
||||
<exclusion>
|
||||
<groupId>org.apache.spark</groupId>
|
||||
<artifactId>*</artifactId>
|
||||
</exclusion>
|
||||
</exclusions>
|
||||
</dependency>
|
||||
|
||||
<!-- Hoodie - Test -->
|
||||
@@ -202,6 +223,7 @@
|
||||
<type>test-jar</type>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
|
||||
<dependency>
|
||||
<groupId>org.apache.hudi</groupId>
|
||||
<artifactId>hudi-spark-client</artifactId>
|
||||
@@ -209,7 +231,14 @@
|
||||
<classifier>tests</classifier>
|
||||
<type>test-jar</type>
|
||||
<scope>test</scope>
|
||||
<exclusions>
|
||||
<exclusion>
|
||||
<groupId>org.apache.spark</groupId>
|
||||
<artifactId>*</artifactId>
|
||||
</exclusion>
|
||||
</exclusions>
|
||||
</dependency>
|
||||
|
||||
<dependency>
|
||||
<groupId>org.apache.hudi</groupId>
|
||||
<artifactId>hudi-common</artifactId>
|
||||
@@ -218,6 +247,7 @@
|
||||
<type>test-jar</type>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
|
||||
<dependency>
|
||||
<groupId>org.apache.hudi</groupId>
|
||||
<artifactId>hudi-spark-common_${scala.binary.version}</artifactId>
|
||||
@@ -225,6 +255,12 @@
|
||||
<classifier>tests</classifier>
|
||||
<type>test-jar</type>
|
||||
<scope>test</scope>
|
||||
<exclusions>
|
||||
<exclusion>
|
||||
<groupId>org.apache.spark</groupId>
|
||||
<artifactId>*</artifactId>
|
||||
</exclusion>
|
||||
</exclusions>
|
||||
</dependency>
|
||||
|
||||
<dependency>
|
||||
|
||||
@@ -0,0 +1,188 @@
|
||||
/*
|
||||
* 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.spark.sql.execution.datasources.parquet;
|
||||
|
||||
import org.apache.hadoop.mapreduce.InputSplit;
|
||||
import org.apache.hadoop.mapreduce.TaskAttemptContext;
|
||||
import org.apache.hudi.client.utils.SparkInternalSchemaConverter;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.spark.memory.MemoryMode;
|
||||
import org.apache.spark.sql.catalyst.InternalRow;
|
||||
import org.apache.spark.sql.execution.vectorized.OffHeapColumnVector;
|
||||
import org.apache.spark.sql.execution.vectorized.OnHeapColumnVector;
|
||||
import org.apache.spark.sql.execution.vectorized.WritableColumnVector;
|
||||
import org.apache.spark.sql.types.DataType;
|
||||
import org.apache.spark.sql.types.StructType;
|
||||
import org.apache.spark.sql.vectorized.ColumnarBatch;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.time.ZoneId;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
|
||||
public class Spark312HoodieVectorizedParquetRecordReader extends VectorizedParquetRecordReader {
|
||||
|
||||
// save the col type change info.
|
||||
private Map<Integer, Pair<DataType, DataType>> typeChangeInfos;
|
||||
|
||||
private ColumnarBatch columnarBatch;
|
||||
|
||||
private Map<Integer, WritableColumnVector> idToColumnVectors;
|
||||
|
||||
private WritableColumnVector[] columnVectors;
|
||||
|
||||
// The capacity of vectorized batch.
|
||||
private int capacity;
|
||||
|
||||
// If true, this class returns batches instead of rows.
|
||||
private boolean returnColumnarBatch;
|
||||
|
||||
// The memory mode of the columnarBatch.
|
||||
private final MemoryMode memoryMode;
|
||||
|
||||
/**
|
||||
* Batch of rows that we assemble and the current index we've returned. Every time this
|
||||
* batch is used up (batchIdx == numBatched), we populated the batch.
|
||||
*/
|
||||
private int batchIdx = 0;
|
||||
private int numBatched = 0;
|
||||
|
||||
public Spark312HoodieVectorizedParquetRecordReader(
|
||||
ZoneId convertTz,
|
||||
String datetimeRebaseMode,
|
||||
String int96RebaseMode,
|
||||
boolean useOffHeap,
|
||||
int capacity,
|
||||
Map<Integer, Pair<DataType, DataType>> typeChangeInfos) {
|
||||
super(convertTz, datetimeRebaseMode, int96RebaseMode, useOffHeap, capacity);
|
||||
memoryMode = useOffHeap ? MemoryMode.OFF_HEAP : MemoryMode.ON_HEAP;
|
||||
this.typeChangeInfos = typeChangeInfos;
|
||||
this.capacity = capacity;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void initBatch(StructType partitionColumns, InternalRow partitionValues) {
|
||||
super.initBatch(partitionColumns, partitionValues);
|
||||
if (columnVectors == null) {
|
||||
columnVectors = new WritableColumnVector[sparkSchema.length() + partitionColumns.length()];
|
||||
}
|
||||
if (idToColumnVectors == null) {
|
||||
idToColumnVectors = new HashMap<>();
|
||||
typeChangeInfos.entrySet()
|
||||
.stream()
|
||||
.forEach(f -> {
|
||||
WritableColumnVector vector =
|
||||
memoryMode == MemoryMode.OFF_HEAP ? new OffHeapColumnVector(capacity, f.getValue().getLeft()) : new OnHeapColumnVector(capacity, f.getValue().getLeft());
|
||||
idToColumnVectors.put(f.getKey(), vector);
|
||||
});
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void initialize(InputSplit inputSplit, TaskAttemptContext taskAttemptContext) throws IOException, InterruptedException, UnsupportedOperationException {
|
||||
super.initialize(inputSplit, taskAttemptContext);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
super.close();
|
||||
for (Map.Entry<Integer, WritableColumnVector> e : idToColumnVectors.entrySet()) {
|
||||
e.getValue().close();
|
||||
}
|
||||
idToColumnVectors = null;
|
||||
columnarBatch = null;
|
||||
columnVectors = null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ColumnarBatch resultBatch() {
|
||||
ColumnarBatch currentColumnBatch = super.resultBatch();
|
||||
boolean changed = false;
|
||||
for (Map.Entry<Integer, Pair<DataType, DataType>> entry : typeChangeInfos.entrySet()) {
|
||||
boolean rewrite = SparkInternalSchemaConverter
|
||||
.convertColumnVectorType((WritableColumnVector) currentColumnBatch.column(entry.getKey()),
|
||||
idToColumnVectors.get(entry.getKey()), currentColumnBatch.numRows());
|
||||
if (rewrite) {
|
||||
changed = true;
|
||||
columnVectors[entry.getKey()] = idToColumnVectors.get(entry.getKey());
|
||||
}
|
||||
}
|
||||
if (changed) {
|
||||
if (columnarBatch == null) {
|
||||
// fill other vector
|
||||
for (int i = 0; i < columnVectors.length; i++) {
|
||||
if (columnVectors[i] == null) {
|
||||
columnVectors[i] = (WritableColumnVector) currentColumnBatch.column(i);
|
||||
}
|
||||
}
|
||||
columnarBatch = new ColumnarBatch(columnVectors);
|
||||
}
|
||||
columnarBatch.setNumRows(currentColumnBatch.numRows());
|
||||
return columnarBatch;
|
||||
} else {
|
||||
return currentColumnBatch;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean nextBatch() throws IOException {
|
||||
boolean result = super.nextBatch();
|
||||
if (idToColumnVectors != null) {
|
||||
idToColumnVectors.entrySet().stream().forEach(e -> e.getValue().reset());
|
||||
}
|
||||
numBatched = resultBatch().numRows();
|
||||
batchIdx = 0;
|
||||
return result;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void enableReturningBatches() {
|
||||
returnColumnarBatch = true;
|
||||
super.enableReturningBatches();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object getCurrentValue() {
|
||||
if (typeChangeInfos == null || typeChangeInfos.isEmpty()) {
|
||||
return super.getCurrentValue();
|
||||
}
|
||||
|
||||
if (returnColumnarBatch) {
|
||||
return columnarBatch == null ? super.getCurrentValue() : columnarBatch;
|
||||
}
|
||||
|
||||
return columnarBatch == null ? super.getCurrentValue() : columnarBatch.getRow(batchIdx - 1);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean nextKeyValue() throws IOException {
|
||||
resultBatch();
|
||||
|
||||
if (returnColumnarBatch) {
|
||||
return nextBatch();
|
||||
}
|
||||
|
||||
if (batchIdx >= numBatched) {
|
||||
if (!nextBatch()) {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
++batchIdx;
|
||||
return true;
|
||||
}
|
||||
}
|
||||
@@ -23,6 +23,11 @@ import org.apache.spark.sql.avro.{HoodieAvroDeserializer, HoodieAvroSchemaConver
|
||||
import org.apache.spark.sql.hudi.SparkAdapter
|
||||
import org.apache.spark.sql.types.DataType
|
||||
import org.apache.spark.sql.{HoodieCatalystExpressionUtils, HoodieSpark3_1CatalystExpressionUtils}
|
||||
import org.apache.spark.SPARK_VERSION
|
||||
import org.apache.spark.sql.catalyst.rules.Rule
|
||||
import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat
|
||||
import org.apache.spark.sql.catalyst.plans.logical._
|
||||
import org.apache.spark.sql.{HoodieCatalystExpressionUtils, HoodieSpark3_1CatalystExpressionUtils, SparkSession}
|
||||
|
||||
/**
|
||||
* Implementation of [[SparkAdapter]] for Spark 3.1.x
|
||||
@@ -37,4 +42,27 @@ class Spark3_1Adapter extends BaseSpark3Adapter {
|
||||
override def createAvroDeserializer(rootAvroType: Schema, rootCatalystType: DataType): HoodieAvroDeserializer =
|
||||
new HoodieSpark3_1AvroDeserializer(rootAvroType, rootCatalystType)
|
||||
|
||||
override def createResolveHudiAlterTableCommand(sparkSession: SparkSession): Rule[LogicalPlan] = {
|
||||
if (SPARK_VERSION.startsWith("3.1")) {
|
||||
val loadClassName = "org.apache.spark.sql.hudi.ResolveHudiAlterTableCommand312"
|
||||
val clazz = Class.forName(loadClassName, true, Thread.currentThread().getContextClassLoader)
|
||||
val ctor = clazz.getConstructors.head
|
||||
ctor.newInstance(sparkSession).asInstanceOf[Rule[LogicalPlan]]
|
||||
} else {
|
||||
new Rule[LogicalPlan] {
|
||||
override def apply(plan: LogicalPlan): LogicalPlan = plan
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
override def createHoodieParquetFileFormat(): Option[ParquetFileFormat] = {
|
||||
if (SPARK_VERSION.startsWith("3.1")) {
|
||||
val loadClassName = "org.apache.spark.sql.execution.datasources.parquet.Spark312HoodieParquetFileFormat"
|
||||
val clazz = Class.forName(loadClassName, true, Thread.currentThread().getContextClassLoader)
|
||||
val ctor = clazz.getConstructors.head
|
||||
Some(ctor.newInstance().asInstanceOf[ParquetFileFormat])
|
||||
} else {
|
||||
None
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -0,0 +1,66 @@
|
||||
/*
|
||||
* 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.spark.sql.catalyst.plans.logical
|
||||
|
||||
import org.apache.spark.sql.connector.catalog.TableChange.ColumnPosition
|
||||
import org.apache.spark.sql.types.DataType
|
||||
|
||||
/**
|
||||
* ALTER TABLE ... ADD COLUMNS command, as parsed from SQL.
|
||||
*/
|
||||
case class HoodieAlterTableAddColumnsStatement(
|
||||
tableName: Seq[String],
|
||||
columnsToAdd: Seq[QualifiedColType]) extends ParsedStatement
|
||||
|
||||
/**
|
||||
* ALTER TABLE ... CHANGE COLUMN command, as parsed from SQL.
|
||||
*/
|
||||
case class HoodieAlterTableAlterColumnStatement(
|
||||
tableName: Seq[String],
|
||||
column: Seq[String],
|
||||
dataType: Option[DataType],
|
||||
nullable: Option[Boolean],
|
||||
comment: Option[String],
|
||||
position: Option[ColumnPosition]) extends ParsedStatement
|
||||
|
||||
|
||||
/**
|
||||
* ALTER TABLE ... RENAME COLUMN command, as parsed from SQL.
|
||||
*/
|
||||
case class HoodieAlterTableRenameColumnStatement(
|
||||
tableName: Seq[String],
|
||||
column: Seq[String],
|
||||
newName: String) extends ParsedStatement
|
||||
|
||||
/**
|
||||
* ALTER TABLE ... DROP COLUMNS command, as parsed from SQL.
|
||||
*/
|
||||
case class HoodieAlterTableDropColumnsStatement(
|
||||
tableName: Seq[String], columnsToDrop: Seq[Seq[String]]) extends ParsedStatement
|
||||
|
||||
/**
|
||||
* ALTER TABLE ... SET TBLPROPERTIES command, as parsed from SQL.
|
||||
*/
|
||||
case class HoodieAlterTableSetPropertiesStatement(
|
||||
tableName: Seq[String], properties: Map[String, String]) extends ParsedStatement
|
||||
|
||||
/**
|
||||
* ALTER TABLE ... UNSET TBLPROPERTIES command, as parsed from SQL.
|
||||
*/
|
||||
case class HoodieAlterTableUnsetPropertiesStatement(
|
||||
tableName: Seq[String], propertyKeys: Seq[String], ifExists: Boolean) extends ParsedStatement
|
||||
@@ -0,0 +1,365 @@
|
||||
/*
|
||||
* 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.spark.sql.execution.datasources.parquet
|
||||
|
||||
import java.net.URI
|
||||
import java.util
|
||||
import org.apache.hadoop.conf.Configuration
|
||||
import org.apache.hadoop.fs.Path
|
||||
import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl
|
||||
import org.apache.hadoop.mapreduce.{JobID, TaskAttemptID, TaskID, TaskType}
|
||||
import org.apache.hudi.client.utils.SparkInternalSchemaConverter
|
||||
import org.apache.hudi.common.fs.FSUtils
|
||||
import org.apache.hudi.HoodieSparkUtils
|
||||
import org.apache.hudi.common.util.InternalSchemaCache
|
||||
import org.apache.hudi.common.util.collection.Pair
|
||||
import org.apache.hudi.internal.schema.InternalSchema
|
||||
import org.apache.hudi.internal.schema.utils.{InternalSchemaUtils, SerDeHelper}
|
||||
import org.apache.hudi.internal.schema.action.InternalSchemaMerger
|
||||
import org.apache.parquet.filter2.compat.FilterCompat
|
||||
import org.apache.parquet.filter2.predicate.FilterApi
|
||||
import org.apache.parquet.format.converter.ParquetMetadataConverter.SKIP_ROW_GROUPS
|
||||
import org.apache.parquet.hadoop.{ParquetFileReader, ParquetInputFormat, ParquetRecordReader}
|
||||
|
||||
import org.apache.spark.TaskContext
|
||||
import org.apache.spark.sql.SparkSession
|
||||
import org.apache.spark.sql.catalyst.InternalRow
|
||||
import org.apache.spark.sql.catalyst.expressions.{Cast, JoinedRow}
|
||||
import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection
|
||||
import org.apache.spark.sql.catalyst.util.DateTimeUtils
|
||||
import org.apache.spark.sql.execution.datasources.{DataSourceUtils, PartitionedFile, RecordReaderIterator}
|
||||
import org.apache.spark.sql.execution.datasources.parquet._
|
||||
import org.apache.spark.sql.internal.SQLConf
|
||||
import org.apache.spark.sql.sources._
|
||||
import org.apache.spark.sql.types.{AtomicType, DataType, StructField, StructType}
|
||||
import org.apache.spark.util.SerializableConfiguration
|
||||
|
||||
class Spark312HoodieParquetFileFormat extends ParquetFileFormat {
|
||||
|
||||
// reference ParquetFileFormat from spark project
|
||||
override def buildReaderWithPartitionValues(
|
||||
sparkSession: SparkSession,
|
||||
dataSchema: StructType,
|
||||
partitionSchema: StructType,
|
||||
requiredSchema: StructType,
|
||||
filters: Seq[Filter],
|
||||
options: Map[String, String],
|
||||
hadoopConf: Configuration): PartitionedFile => Iterator[InternalRow] = {
|
||||
if (hadoopConf.get(SparkInternalSchemaConverter.HOODIE_QUERY_SCHEMA, "").isEmpty) {
|
||||
// fallback to origin parquet File read
|
||||
super.buildReaderWithPartitionValues(sparkSession, dataSchema, partitionSchema, requiredSchema, filters, options, hadoopConf)
|
||||
} else {
|
||||
hadoopConf.set(ParquetInputFormat.READ_SUPPORT_CLASS, classOf[ParquetReadSupport].getName)
|
||||
hadoopConf.set(
|
||||
ParquetReadSupport.SPARK_ROW_REQUESTED_SCHEMA,
|
||||
requiredSchema.json)
|
||||
hadoopConf.set(
|
||||
ParquetWriteSupport.SPARK_ROW_SCHEMA,
|
||||
requiredSchema.json)
|
||||
hadoopConf.set(
|
||||
SQLConf.SESSION_LOCAL_TIMEZONE.key,
|
||||
sparkSession.sessionState.conf.sessionLocalTimeZone)
|
||||
hadoopConf.setBoolean(
|
||||
SQLConf.NESTED_SCHEMA_PRUNING_ENABLED.key,
|
||||
sparkSession.sessionState.conf.nestedSchemaPruningEnabled)
|
||||
hadoopConf.setBoolean(
|
||||
SQLConf.CASE_SENSITIVE.key,
|
||||
sparkSession.sessionState.conf.caseSensitiveAnalysis)
|
||||
|
||||
ParquetWriteSupport.setSchema(requiredSchema, hadoopConf)
|
||||
|
||||
// Sets flags for `ParquetToSparkSchemaConverter`
|
||||
hadoopConf.setBoolean(
|
||||
SQLConf.PARQUET_BINARY_AS_STRING.key,
|
||||
sparkSession.sessionState.conf.isParquetBinaryAsString)
|
||||
hadoopConf.setBoolean(
|
||||
SQLConf.PARQUET_INT96_AS_TIMESTAMP.key,
|
||||
sparkSession.sessionState.conf.isParquetINT96AsTimestamp)
|
||||
// for dataSource v1, we have no method to do project for spark physical plan.
|
||||
// it's safe to do cols project here.
|
||||
val internalSchemaString = hadoopConf.get(SparkInternalSchemaConverter.HOODIE_QUERY_SCHEMA)
|
||||
val querySchemaOption = SerDeHelper.fromJson(internalSchemaString)
|
||||
if (querySchemaOption.isPresent && !requiredSchema.isEmpty) {
|
||||
val prunedSchema = SparkInternalSchemaConverter.convertAndPruneStructTypeToInternalSchema(requiredSchema, querySchemaOption.get())
|
||||
hadoopConf.set(SparkInternalSchemaConverter.HOODIE_QUERY_SCHEMA, SerDeHelper.toJson(prunedSchema))
|
||||
}
|
||||
val broadcastedHadoopConf =
|
||||
sparkSession.sparkContext.broadcast(new SerializableConfiguration(hadoopConf))
|
||||
|
||||
// TODO: if you move this into the closure it reverts to the default values.
|
||||
// If true, enable using the custom RecordReader for parquet. This only works for
|
||||
// a subset of the types (no complex types).
|
||||
val resultSchema = StructType(partitionSchema.fields ++ requiredSchema.fields)
|
||||
val sqlConf = sparkSession.sessionState.conf
|
||||
val enableOffHeapColumnVector = sqlConf.offHeapColumnVectorEnabled
|
||||
val enableVectorizedReader: Boolean =
|
||||
sqlConf.parquetVectorizedReaderEnabled &&
|
||||
resultSchema.forall(_.dataType.isInstanceOf[AtomicType])
|
||||
val enableRecordFilter: Boolean = sqlConf.parquetRecordFilterEnabled
|
||||
val timestampConversion: Boolean = sqlConf.isParquetINT96TimestampConversion
|
||||
val capacity = sqlConf.parquetVectorizedReaderBatchSize
|
||||
val enableParquetFilterPushDown: Boolean = sqlConf.parquetFilterPushDown
|
||||
// Whole stage codegen (PhysicalRDD) is able to deal with batches directly
|
||||
val returningBatch = supportBatch(sparkSession, resultSchema)
|
||||
val pushDownDate = sqlConf.parquetFilterPushDownDate
|
||||
val pushDownTimestamp = sqlConf.parquetFilterPushDownTimestamp
|
||||
val pushDownDecimal = sqlConf.parquetFilterPushDownDecimal
|
||||
val pushDownStringStartWith = sqlConf.parquetFilterPushDownStringStartWith
|
||||
val pushDownInFilterThreshold = sqlConf.parquetFilterPushDownInFilterThreshold
|
||||
val isCaseSensitive = sqlConf.caseSensitiveAnalysis
|
||||
|
||||
(file: PartitionedFile) => {
|
||||
assert(file.partitionValues.numFields == partitionSchema.size)
|
||||
val filePath = new Path(new URI(file.filePath))
|
||||
val split =
|
||||
new org.apache.parquet.hadoop.ParquetInputSplit(
|
||||
filePath,
|
||||
file.start,
|
||||
file.start + file.length,
|
||||
file.length,
|
||||
Array.empty,
|
||||
null)
|
||||
val sharedConf = broadcastedHadoopConf.value.value
|
||||
// do deal with internalSchema
|
||||
val internalSchemaString = sharedConf.get(SparkInternalSchemaConverter.HOODIE_QUERY_SCHEMA)
|
||||
// querySchema must be a pruned schema.
|
||||
val querySchemaOption = SerDeHelper.fromJson(internalSchemaString)
|
||||
val internalSchemaChangeEnabled = if (internalSchemaString.isEmpty || !querySchemaOption.isPresent) false else true
|
||||
val tablePath = sharedConf.get(SparkInternalSchemaConverter.HOODIE_TABLE_PATH)
|
||||
val commitInstantTime = FSUtils.getCommitTime(filePath.getName).toLong;
|
||||
val fileSchema = if (internalSchemaChangeEnabled) {
|
||||
val validCommits = sharedConf.get(SparkInternalSchemaConverter.HOODIE_VALID_COMMITS_LIST)
|
||||
InternalSchemaCache.getInternalSchemaByVersionId(commitInstantTime, tablePath, sharedConf, if (validCommits == null) "" else validCommits)
|
||||
} else {
|
||||
// this should not happened, searchSchemaAndCache will deal with correctly.
|
||||
null
|
||||
}
|
||||
|
||||
lazy val footerFileMetaData =
|
||||
ParquetFileReader.readFooter(sharedConf, filePath, SKIP_ROW_GROUPS).getFileMetaData
|
||||
val datetimeRebaseMode = DataSourceUtils.datetimeRebaseMode(
|
||||
footerFileMetaData.getKeyValueMetaData.get,
|
||||
SQLConf.get.getConf(SQLConf.LEGACY_PARQUET_REBASE_MODE_IN_READ))
|
||||
// Try to push down filters when filter push-down is enabled.
|
||||
val pushed = if (enableParquetFilterPushDown) {
|
||||
val parquetSchema = footerFileMetaData.getSchema
|
||||
val parquetFilters = if (HoodieSparkUtils.gteqSpark3_1_3) {
|
||||
Spark312HoodieParquetFileFormat.createParquetFilters(
|
||||
parquetSchema,
|
||||
pushDownDate,
|
||||
pushDownTimestamp,
|
||||
pushDownDecimal,
|
||||
pushDownStringStartWith,
|
||||
pushDownInFilterThreshold,
|
||||
isCaseSensitive,
|
||||
datetimeRebaseMode)
|
||||
} else {
|
||||
Spark312HoodieParquetFileFormat.createParquetFilters(
|
||||
parquetSchema,
|
||||
pushDownDate,
|
||||
pushDownTimestamp,
|
||||
pushDownDecimal,
|
||||
pushDownStringStartWith,
|
||||
pushDownInFilterThreshold,
|
||||
isCaseSensitive)
|
||||
}
|
||||
filters.map(Spark312HoodieParquetFileFormat.rebuildFilterFromParquet(_, fileSchema, querySchemaOption.get()))
|
||||
// Collects all converted Parquet filter predicates. Notice that not all predicates can be
|
||||
// converted (`ParquetFilters.createFilter` returns an `Option`). That's why a `flatMap`
|
||||
// is used here.
|
||||
.flatMap(parquetFilters.createFilter(_))
|
||||
.reduceOption(FilterApi.and)
|
||||
} else {
|
||||
None
|
||||
}
|
||||
|
||||
// PARQUET_INT96_TIMESTAMP_CONVERSION says to apply timezone conversions to int96 timestamps'
|
||||
// *only* if the file was created by something other than "parquet-mr", so check the actual
|
||||
// writer here for this file. We have to do this per-file, as each file in the table may
|
||||
// have different writers.
|
||||
// Define isCreatedByParquetMr as function to avoid unnecessary parquet footer reads.
|
||||
def isCreatedByParquetMr: Boolean =
|
||||
footerFileMetaData.getCreatedBy().startsWith("parquet-mr")
|
||||
|
||||
val convertTz =
|
||||
if (timestampConversion && !isCreatedByParquetMr) {
|
||||
Some(DateTimeUtils.getZoneId(sharedConf.get(SQLConf.SESSION_LOCAL_TIMEZONE.key)))
|
||||
} else {
|
||||
None
|
||||
}
|
||||
val int96RebaseMode = DataSourceUtils.int96RebaseMode(
|
||||
footerFileMetaData.getKeyValueMetaData.get,
|
||||
SQLConf.get.getConf(SQLConf.LEGACY_PARQUET_INT96_REBASE_MODE_IN_READ))
|
||||
|
||||
val attemptId = new TaskAttemptID(new TaskID(new JobID(), TaskType.MAP, 0), 0)
|
||||
// use new conf
|
||||
val hadoopAttempConf = new Configuration(broadcastedHadoopConf.value.value)
|
||||
//
|
||||
// reset request schema
|
||||
var typeChangeInfos: java.util.Map[Integer, Pair[DataType, DataType]] = new java.util.HashMap()
|
||||
if (internalSchemaChangeEnabled) {
|
||||
val mergedInternalSchema = new InternalSchemaMerger(fileSchema, querySchemaOption.get(), true, true).mergeSchema()
|
||||
val mergedSchema = SparkInternalSchemaConverter.constructSparkSchemaFromInternalSchema(mergedInternalSchema)
|
||||
typeChangeInfos = SparkInternalSchemaConverter.collectTypeChangedCols(querySchemaOption.get(), mergedInternalSchema)
|
||||
hadoopAttempConf.set(ParquetReadSupport.SPARK_ROW_REQUESTED_SCHEMA, mergedSchema.json)
|
||||
}
|
||||
val hadoopAttemptContext =
|
||||
new TaskAttemptContextImpl(hadoopAttempConf, attemptId)
|
||||
|
||||
// Try to push down filters when filter push-down is enabled.
|
||||
// Notice: This push-down is RowGroups level, not individual records.
|
||||
if (pushed.isDefined) {
|
||||
ParquetInputFormat.setFilterPredicate(hadoopAttemptContext.getConfiguration, pushed.get)
|
||||
}
|
||||
val taskContext = Option(TaskContext.get())
|
||||
if (enableVectorizedReader) {
|
||||
val vectorizedReader = new Spark312HoodieVectorizedParquetRecordReader(
|
||||
convertTz.orNull,
|
||||
datetimeRebaseMode.toString,
|
||||
int96RebaseMode.toString,
|
||||
enableOffHeapColumnVector && taskContext.isDefined,
|
||||
capacity, typeChangeInfos)
|
||||
val iter = new RecordReaderIterator(vectorizedReader)
|
||||
// SPARK-23457 Register a task completion listener before `initialization`.
|
||||
taskContext.foreach(_.addTaskCompletionListener[Unit](_ => iter.close()))
|
||||
vectorizedReader.initialize(split, hadoopAttemptContext)
|
||||
logDebug(s"Appending $partitionSchema ${file.partitionValues}")
|
||||
vectorizedReader.initBatch(partitionSchema, file.partitionValues)
|
||||
if (returningBatch) {
|
||||
vectorizedReader.enableReturningBatches()
|
||||
}
|
||||
|
||||
// UnsafeRowParquetRecordReader appends the columns internally to avoid another copy.
|
||||
iter.asInstanceOf[Iterator[InternalRow]]
|
||||
} else {
|
||||
logDebug(s"Falling back to parquet-mr")
|
||||
// ParquetRecordReader returns InternalRow
|
||||
val readSupport = new ParquetReadSupport(
|
||||
convertTz,
|
||||
enableVectorizedReader = false,
|
||||
datetimeRebaseMode,
|
||||
int96RebaseMode)
|
||||
val reader = if (pushed.isDefined && enableRecordFilter) {
|
||||
val parquetFilter = FilterCompat.get(pushed.get, null)
|
||||
new ParquetRecordReader[InternalRow](readSupport, parquetFilter)
|
||||
} else {
|
||||
new ParquetRecordReader[InternalRow](readSupport)
|
||||
}
|
||||
val iter = new RecordReaderIterator[InternalRow](reader)
|
||||
// SPARK-23457 Register a task completion listener before `initialization`.
|
||||
taskContext.foreach(_.addTaskCompletionListener[Unit](_ => iter.close()))
|
||||
reader.initialize(split, hadoopAttemptContext)
|
||||
|
||||
val fullSchema = requiredSchema.toAttributes ++ partitionSchema.toAttributes
|
||||
val unsafeProjection = if (typeChangeInfos.isEmpty) {
|
||||
GenerateUnsafeProjection.generate(fullSchema, fullSchema)
|
||||
} else {
|
||||
// find type changed.
|
||||
val newFullSchema = new StructType(requiredSchema.fields.zipWithIndex.map { case (f, i) =>
|
||||
if (typeChangeInfos.containsKey(i)) {
|
||||
StructField(f.name, typeChangeInfos.get(i).getRight, f.nullable, f.metadata)
|
||||
} else f
|
||||
}).toAttributes ++ partitionSchema.toAttributes
|
||||
val castSchema = newFullSchema.zipWithIndex.map { case (attr, i) =>
|
||||
if (typeChangeInfos.containsKey(i)) {
|
||||
Cast(attr, typeChangeInfos.get(i).getLeft)
|
||||
} else attr
|
||||
}
|
||||
GenerateUnsafeProjection.generate(castSchema, newFullSchema)
|
||||
}
|
||||
|
||||
if (partitionSchema.length == 0) {
|
||||
// There is no partition columns
|
||||
iter.map(unsafeProjection)
|
||||
} else {
|
||||
val joinedRow = new JoinedRow()
|
||||
iter.map(d => unsafeProjection(joinedRow(d, file.partitionValues)))
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
object Spark312HoodieParquetFileFormat {
|
||||
|
||||
val PARQUET_FILTERS_CLASS_NAME = "org.apache.spark.sql.execution.datasources.parquet.ParquetFilters"
|
||||
|
||||
private def createParquetFilters(arg: Any*): ParquetFilters = {
|
||||
val clazz = Class.forName(PARQUET_FILTERS_CLASS_NAME, true, Thread.currentThread().getContextClassLoader)
|
||||
val ctor = clazz.getConstructors.head
|
||||
ctor.newInstance(arg.map(_.asInstanceOf[AnyRef]): _*).asInstanceOf[ParquetFilters]
|
||||
}
|
||||
|
||||
private def rebuildFilterFromParquet(oldFilter: Filter, fileSchema: InternalSchema, querySchema: InternalSchema): Filter = {
|
||||
if (fileSchema == null || querySchema == null) {
|
||||
oldFilter
|
||||
} else {
|
||||
oldFilter match {
|
||||
case eq: EqualTo =>
|
||||
val newAttribute = InternalSchemaUtils.reBuildFilterName(eq.attribute, fileSchema, querySchema)
|
||||
if (newAttribute.isEmpty) AlwaysTrue else eq.copy(attribute = newAttribute)
|
||||
case eqs: EqualNullSafe =>
|
||||
val newAttribute = InternalSchemaUtils.reBuildFilterName(eqs.attribute, fileSchema, querySchema)
|
||||
if (newAttribute.isEmpty) AlwaysTrue else eqs.copy(attribute = newAttribute)
|
||||
case gt: GreaterThan =>
|
||||
val newAttribute = InternalSchemaUtils.reBuildFilterName(gt.attribute, fileSchema, querySchema)
|
||||
if (newAttribute.isEmpty) AlwaysTrue else gt.copy(attribute = newAttribute)
|
||||
case gtr: GreaterThanOrEqual =>
|
||||
val newAttribute = InternalSchemaUtils.reBuildFilterName(gtr.attribute, fileSchema, querySchema)
|
||||
if (newAttribute.isEmpty) AlwaysTrue else gtr.copy(attribute = newAttribute)
|
||||
case lt: LessThan =>
|
||||
val newAttribute = InternalSchemaUtils.reBuildFilterName(lt.attribute, fileSchema, querySchema)
|
||||
if (newAttribute.isEmpty) AlwaysTrue else lt.copy(attribute = newAttribute)
|
||||
case lte: LessThanOrEqual =>
|
||||
val newAttribute = InternalSchemaUtils.reBuildFilterName(lte.attribute, fileSchema, querySchema)
|
||||
if (newAttribute.isEmpty) AlwaysTrue else lte.copy(attribute = newAttribute)
|
||||
case i: In =>
|
||||
val newAttribute = InternalSchemaUtils.reBuildFilterName(i.attribute, fileSchema, querySchema)
|
||||
if (newAttribute.isEmpty) AlwaysTrue else i.copy(attribute = newAttribute)
|
||||
case isn: IsNull =>
|
||||
val newAttribute = InternalSchemaUtils.reBuildFilterName(isn.attribute, fileSchema, querySchema)
|
||||
if (newAttribute.isEmpty) AlwaysTrue else isn.copy(attribute = newAttribute)
|
||||
case isnn: IsNotNull =>
|
||||
val newAttribute = InternalSchemaUtils.reBuildFilterName(isnn.attribute, fileSchema, querySchema)
|
||||
if (newAttribute.isEmpty) AlwaysTrue else isnn.copy(attribute = newAttribute)
|
||||
case And(left, right) =>
|
||||
And(rebuildFilterFromParquet(left, fileSchema, querySchema), rebuildFilterFromParquet(right, fileSchema, querySchema))
|
||||
case Or(left, right) =>
|
||||
Or(rebuildFilterFromParquet(left, fileSchema, querySchema), rebuildFilterFromParquet(right, fileSchema, querySchema))
|
||||
case Not(child) =>
|
||||
Not(rebuildFilterFromParquet(child, fileSchema, querySchema))
|
||||
case ssw: StringStartsWith =>
|
||||
val newAttribute = InternalSchemaUtils.reBuildFilterName(ssw.attribute, fileSchema, querySchema)
|
||||
if (newAttribute.isEmpty) AlwaysTrue else ssw.copy(attribute = newAttribute)
|
||||
case ses: StringEndsWith =>
|
||||
val newAttribute = InternalSchemaUtils.reBuildFilterName(ses.attribute, fileSchema, querySchema)
|
||||
if (newAttribute.isEmpty) AlwaysTrue else ses.copy(attribute = newAttribute)
|
||||
case sc: StringContains =>
|
||||
val newAttribute = InternalSchemaUtils.reBuildFilterName(sc.attribute, fileSchema, querySchema)
|
||||
if (newAttribute.isEmpty) AlwaysTrue else sc.copy(attribute = newAttribute)
|
||||
case AlwaysTrue =>
|
||||
AlwaysTrue
|
||||
case AlwaysFalse =>
|
||||
AlwaysFalse
|
||||
case _ =>
|
||||
AlwaysTrue
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,282 @@
|
||||
/*
|
||||
* 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.spark.sql.hudi
|
||||
|
||||
import java.util.Locale
|
||||
import org.apache.hudi.config.HoodieWriteConfig
|
||||
import org.apache.hudi.internal.schema.action.TableChange.ColumnChangeID
|
||||
import org.apache.spark.sql.{AnalysisException, SparkSession}
|
||||
import org.apache.spark.sql.catalyst.TableIdentifier
|
||||
import org.apache.spark.sql.catalyst.catalog.CatalogTable
|
||||
import org.apache.spark.sql.connector.catalog.{CatalogV2Util, TableChange}
|
||||
import org.apache.spark.sql.catalyst.plans.logical._
|
||||
import org.apache.spark.sql.catalyst.rules.Rule
|
||||
import org.apache.spark.sql.connector.catalog.CatalogV2Util.failNullType
|
||||
import org.apache.spark.sql.connector.catalog.TableChange._
|
||||
import org.apache.spark.sql.hudi.command.AlterTableCommand312
|
||||
import org.apache.spark.sql.types.StructType
|
||||
|
||||
import scala.collection.mutable
|
||||
|
||||
/**
|
||||
* Rule to mostly resolve, normalize and rewrite column names based on case sensitivity
|
||||
* for alter table column commands.
|
||||
* TODO: we should remove this file when we support datasourceV2 for hoodie on spark3.1x
|
||||
*/
|
||||
case class ResolveHudiAlterTableCommand312(sparkSession: SparkSession) extends Rule[LogicalPlan] {
|
||||
import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._
|
||||
def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsUp {
|
||||
case add @ HoodieAlterTableAddColumnsStatement(asTable(table), cols) =>
|
||||
if (isHoodieTable(table) && schemaEvolutionEnabled){
|
||||
cols.foreach(c => CatalogV2Util.failNullType(c.dataType))
|
||||
val changes = cols.map { col =>
|
||||
TableChange.addColumn(
|
||||
col.name.toArray,
|
||||
col.dataType,
|
||||
col.nullable,
|
||||
col.comment.orNull,
|
||||
col.position.orNull)
|
||||
}
|
||||
val newChanges = normalizeChanges(changes, table.schema)
|
||||
AlterTableCommand312(table, newChanges, ColumnChangeID.ADD)
|
||||
} else {
|
||||
// throw back to spark
|
||||
AlterTableAddColumnsStatement(add.tableName, add.columnsToAdd)
|
||||
}
|
||||
case a @ HoodieAlterTableAlterColumnStatement(asTable(table), _, _, _, _, _) =>
|
||||
if (isHoodieTable(table) && schemaEvolutionEnabled){
|
||||
a.dataType.foreach(failNullType)
|
||||
val colName = a.column.toArray
|
||||
val typeChange = a.dataType.map { newDataType =>
|
||||
TableChange.updateColumnType(colName, newDataType)
|
||||
}
|
||||
val nullabilityChange = a.nullable.map { nullable =>
|
||||
TableChange.updateColumnNullability(colName, nullable)
|
||||
}
|
||||
val commentChange = a.comment.map { newComment =>
|
||||
TableChange.updateColumnComment(colName, newComment)
|
||||
}
|
||||
val positionChange = a.position.map { newPosition =>
|
||||
TableChange.updateColumnPosition(colName, newPosition)
|
||||
}
|
||||
AlterTableCommand312(table, normalizeChanges(typeChange.toSeq ++ nullabilityChange ++ commentChange ++ positionChange, table.schema), ColumnChangeID.UPDATE)
|
||||
} else {
|
||||
// throw back to spark
|
||||
AlterTableAlterColumnStatement(a.tableName, a.column, a.dataType, a.nullable, a.comment, a.position)
|
||||
}
|
||||
case rename @ HoodieAlterTableRenameColumnStatement(asTable(table), col, newName) =>
|
||||
if (isHoodieTable(table) && schemaEvolutionEnabled){
|
||||
val changes = Seq(TableChange.renameColumn(col.toArray, newName))
|
||||
AlterTableCommand312(table, normalizeChanges(changes, table.schema), ColumnChangeID.UPDATE)
|
||||
} else {
|
||||
// throw back to spark
|
||||
AlterTableRenameColumnStatement(rename.tableName, rename.column, rename.newName)
|
||||
}
|
||||
case drop @ HoodieAlterTableDropColumnsStatement(asTable(table), cols) =>
|
||||
if (isHoodieTable(table) && schemaEvolutionEnabled) {
|
||||
val changes = cols.map(col => TableChange.deleteColumn(col.toArray))
|
||||
AlterTableCommand312(table, normalizeChanges(changes, table.schema), ColumnChangeID.DELETE)
|
||||
} else {
|
||||
// throw back to spark
|
||||
AlterTableDropColumnsStatement(drop.tableName, drop.columnsToDrop)
|
||||
}
|
||||
case set @ HoodieAlterTableSetPropertiesStatement(asTable(table), props) =>
|
||||
if (isHoodieTable(table) && schemaEvolutionEnabled) {
|
||||
val changes = props.map { case (key, value) =>
|
||||
TableChange.setProperty(key, value)
|
||||
}.toSeq
|
||||
AlterTableCommand312(table, normalizeChanges(changes, table.schema), ColumnChangeID.PROPERTY_CHANGE)
|
||||
} else {
|
||||
// throw back to spark
|
||||
AlterTableSetPropertiesStatement(set.tableName, set.properties)
|
||||
}
|
||||
case unset @ HoodieAlterTableUnsetPropertiesStatement(asTable(table), keys, _) =>
|
||||
if (isHoodieTable(table) && schemaEvolutionEnabled) {
|
||||
val changes = keys.map(key => TableChange.removeProperty(key))
|
||||
AlterTableCommand312(table, normalizeChanges(changes, table.schema), ColumnChangeID.PROPERTY_CHANGE)
|
||||
} else {
|
||||
// throw back to spark
|
||||
AlterTableUnsetPropertiesStatement(unset.tableName, unset.propertyKeys, unset.ifExists)
|
||||
}
|
||||
}
|
||||
|
||||
private def schemaEvolutionEnabled(): Boolean = sparkSession
|
||||
.sessionState.conf.getConfString(HoodieWriteConfig.SCHEMA_EVOLUTION_ENABLE.key(), "false").toBoolean
|
||||
|
||||
private def isHoodieTable(table: CatalogTable): Boolean = table.provider.map(_.toLowerCase(Locale.ROOT)).orNull == "hudi"
|
||||
|
||||
def normalizeChanges(changes: Seq[TableChange], schema: StructType): Seq[TableChange] = {
|
||||
val colsToAdd = mutable.Map.empty[Seq[String], Seq[String]]
|
||||
changes.flatMap {
|
||||
case add: AddColumn =>
|
||||
def addColumn(parentSchema: StructType, parentName: String, normalizedParentName: Seq[String]): TableChange = {
|
||||
val fieldsAdded = colsToAdd.getOrElse(normalizedParentName, Nil)
|
||||
val pos = findColumnPosition(add.position(), parentName, parentSchema, fieldsAdded)
|
||||
val field = add.fieldNames().last
|
||||
colsToAdd(normalizedParentName) = fieldsAdded :+ field
|
||||
TableChange.addColumn(
|
||||
(normalizedParentName :+ field).toArray,
|
||||
add.dataType(),
|
||||
add.isNullable,
|
||||
add.comment,
|
||||
pos)
|
||||
}
|
||||
val parent = add.fieldNames().init
|
||||
if (parent.nonEmpty) {
|
||||
// Adding a nested field, need to normalize the parent column and position
|
||||
val target = schema.findNestedField(parent, includeCollections = true, conf.resolver)
|
||||
if (target.isEmpty) {
|
||||
// Leave unresolved. Throws error in CheckAnalysis
|
||||
Some(add)
|
||||
} else {
|
||||
val (normalizedName, sf) = target.get
|
||||
sf.dataType match {
|
||||
case struct: StructType =>
|
||||
Some(addColumn(struct, parent.quoted, normalizedName :+ sf.name))
|
||||
case other =>
|
||||
Some(add)
|
||||
}
|
||||
}
|
||||
} else {
|
||||
// Adding to the root. Just need to normalize position
|
||||
Some(addColumn(schema, "root", Nil))
|
||||
}
|
||||
|
||||
case typeChange: UpdateColumnType =>
|
||||
// Hive style syntax provides the column type, even if it may not have changed
|
||||
val fieldOpt = schema.findNestedField(
|
||||
typeChange.fieldNames(), includeCollections = true, conf.resolver)
|
||||
|
||||
if (fieldOpt.isEmpty) {
|
||||
// We couldn't resolve the field. Leave it to CheckAnalysis
|
||||
Some(typeChange)
|
||||
} else {
|
||||
val (fieldNames, field) = fieldOpt.get
|
||||
if (field.dataType == typeChange.newDataType()) {
|
||||
// The user didn't want the field to change, so remove this change
|
||||
None
|
||||
} else {
|
||||
Some(TableChange.updateColumnType(
|
||||
(fieldNames :+ field.name).toArray, typeChange.newDataType()))
|
||||
}
|
||||
}
|
||||
case n: UpdateColumnNullability =>
|
||||
// Need to resolve column
|
||||
resolveFieldNames(
|
||||
schema,
|
||||
n.fieldNames(),
|
||||
TableChange.updateColumnNullability(_, n.nullable())).orElse(Some(n))
|
||||
|
||||
case position: UpdateColumnPosition =>
|
||||
position.position() match {
|
||||
case after: After =>
|
||||
// Need to resolve column as well as position reference
|
||||
val fieldOpt = schema.findNestedField(
|
||||
position.fieldNames(), includeCollections = true, conf.resolver)
|
||||
|
||||
if (fieldOpt.isEmpty) {
|
||||
Some(position)
|
||||
} else {
|
||||
val (normalizedPath, field) = fieldOpt.get
|
||||
val targetCol = schema.findNestedField(
|
||||
normalizedPath :+ after.column(), includeCollections = true, conf.resolver)
|
||||
if (targetCol.isEmpty) {
|
||||
// Leave unchanged to CheckAnalysis
|
||||
Some(position)
|
||||
} else {
|
||||
Some(TableChange.updateColumnPosition(
|
||||
(normalizedPath :+ field.name).toArray,
|
||||
ColumnPosition.after(targetCol.get._2.name)))
|
||||
}
|
||||
}
|
||||
case _ =>
|
||||
// Need to resolve column
|
||||
resolveFieldNames(
|
||||
schema,
|
||||
position.fieldNames(),
|
||||
TableChange.updateColumnPosition(_, position.position())).orElse(Some(position))
|
||||
}
|
||||
|
||||
case comment: UpdateColumnComment =>
|
||||
resolveFieldNames(
|
||||
schema,
|
||||
comment.fieldNames(),
|
||||
TableChange.updateColumnComment(_, comment.newComment())).orElse(Some(comment))
|
||||
|
||||
case rename: RenameColumn =>
|
||||
resolveFieldNames(
|
||||
schema,
|
||||
rename.fieldNames(),
|
||||
TableChange.renameColumn(_, rename.newName())).orElse(Some(rename))
|
||||
|
||||
case delete: DeleteColumn =>
|
||||
resolveFieldNames(schema, delete.fieldNames(), TableChange.deleteColumn)
|
||||
.orElse(Some(delete))
|
||||
|
||||
case column: ColumnChange =>
|
||||
// This is informational for future developers
|
||||
throw new UnsupportedOperationException(
|
||||
"Please add an implementation for a column change here")
|
||||
case other => Some(other)
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the table change if the field can be resolved, returns None if the column is not
|
||||
* found. An error will be thrown in CheckAnalysis for columns that can't be resolved.
|
||||
*/
|
||||
private def resolveFieldNames(
|
||||
schema: StructType,
|
||||
fieldNames: Array[String],
|
||||
copy: Array[String] => TableChange): Option[TableChange] = {
|
||||
val fieldOpt = schema.findNestedField(
|
||||
fieldNames, includeCollections = true, conf.resolver)
|
||||
fieldOpt.map { case (path, field) => copy((path :+ field.name).toArray) }
|
||||
}
|
||||
|
||||
private def findColumnPosition(
|
||||
position: ColumnPosition,
|
||||
parentName: String,
|
||||
struct: StructType,
|
||||
fieldsAdded: Seq[String]): ColumnPosition = {
|
||||
position match {
|
||||
case null => null
|
||||
case after: After =>
|
||||
(struct.fieldNames ++ fieldsAdded).find(n => conf.resolver(n, after.column())) match {
|
||||
case Some(colName) =>
|
||||
ColumnPosition.after(colName)
|
||||
case None =>
|
||||
throw new AnalysisException("Couldn't find the reference column for " +
|
||||
s"$after at $parentName")
|
||||
}
|
||||
case other => other
|
||||
}
|
||||
}
|
||||
|
||||
object asTable {
|
||||
def unapply(parts: Seq[String]): Option[CatalogTable] = {
|
||||
val identifier = parts match {
|
||||
case Seq(tblName) => TableIdentifier(tblName)
|
||||
case Seq(dbName, tblName) => TableIdentifier(tblName, Some(dbName))
|
||||
case _ =>
|
||||
throw new AnalysisException(
|
||||
s"${parts} is not a valid TableIdentifier as it has more than 2 name parts.")
|
||||
}
|
||||
Some(sparkSession.sessionState.catalog.getTableMetadata(identifier))
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,324 @@
|
||||
/*
|
||||
* 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.spark.sql.hudi.command
|
||||
|
||||
import java.net.URI
|
||||
import java.nio.charset.StandardCharsets
|
||||
import java.util
|
||||
import java.util.concurrent.atomic.AtomicInteger
|
||||
import org.apache.hadoop.conf.Configuration
|
||||
import org.apache.hadoop.fs.Path
|
||||
import org.apache.hudi.DataSourceWriteOptions._
|
||||
import org.apache.hudi.client.utils.SparkInternalSchemaConverter
|
||||
import org.apache.hudi.common.model.{HoodieCommitMetadata, WriteOperationType}
|
||||
import org.apache.hudi.{AvroConversionUtils, DataSourceOptionsHelper, DataSourceUtils}
|
||||
import org.apache.hudi.common.table.timeline.{HoodieActiveTimeline, HoodieInstant}
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant.State
|
||||
import org.apache.hudi.common.table.{HoodieTableMetaClient, TableSchemaResolver}
|
||||
import org.apache.hudi.common.util.{CommitUtils, Option}
|
||||
import org.apache.hudi.config.HoodieWriteConfig
|
||||
import org.apache.hudi.internal.schema.InternalSchema
|
||||
import org.apache.hudi.internal.schema.action.TableChange.ColumnChangeID
|
||||
import org.apache.hudi.internal.schema.action.TableChanges
|
||||
import org.apache.hudi.internal.schema.convert.AvroInternalSchemaConverter
|
||||
import org.apache.hudi.internal.schema.utils.{SchemaChangeUtils, SerDeHelper}
|
||||
import org.apache.hudi.internal.schema.io.FileBasedInternalSchemaStorageManager
|
||||
import org.apache.hudi.table.HoodieSparkTable
|
||||
import org.apache.spark.api.java.JavaSparkContext
|
||||
import org.apache.spark.internal.Logging
|
||||
import org.apache.spark.sql.{Row, SparkSession}
|
||||
import org.apache.spark.sql.catalyst.catalog.{CatalogTable, CatalogTableType}
|
||||
import org.apache.spark.sql.connector.catalog.{TableCatalog, TableChange}
|
||||
import org.apache.spark.sql.connector.catalog.TableChange.{AddColumn, DeleteColumn, RemoveProperty, SetProperty}
|
||||
import org.apache.spark.sql.execution.command.RunnableCommand
|
||||
import org.apache.spark.sql.types.StructType
|
||||
|
||||
import scala.collection.JavaConverters._
|
||||
import scala.util.control.NonFatal
|
||||
|
||||
// TODO: we should remove this file when we support datasourceV2 for hoodie on spark3.1x
|
||||
case class AlterTableCommand312(table: CatalogTable, changes: Seq[TableChange], changeType: ColumnChangeID) extends RunnableCommand with Logging {
|
||||
override def run(sparkSession: SparkSession): Seq[Row] = {
|
||||
changeType match {
|
||||
case ColumnChangeID.ADD => applyAddAction(sparkSession)
|
||||
case ColumnChangeID.DELETE => applyDeleteAction(sparkSession)
|
||||
case ColumnChangeID.UPDATE => applyUpdateAction(sparkSession)
|
||||
case ColumnChangeID.PROPERTY_CHANGE if (changes.filter(_.isInstanceOf[SetProperty]).size == changes.size) =>
|
||||
applyPropertySet(sparkSession)
|
||||
case ColumnChangeID.PROPERTY_CHANGE if (changes.filter(_.isInstanceOf[RemoveProperty]).size == changes.size) =>
|
||||
applyPropertyUnset(sparkSession)
|
||||
case other => throw new RuntimeException(s"find unsupported alter command type: ${other}")
|
||||
}
|
||||
Seq.empty[Row]
|
||||
}
|
||||
|
||||
def applyAddAction(sparkSession: SparkSession): Unit = {
|
||||
val (oldSchema, historySchema) = getInternalSchemaAndHistorySchemaStr(sparkSession)
|
||||
val addChange = TableChanges.ColumnAddChange.get(oldSchema)
|
||||
changes.map(_.asInstanceOf[AddColumn]).foreach { addColumn =>
|
||||
val names = addColumn.fieldNames()
|
||||
val parentName = AlterTableCommand312.getParentName(names)
|
||||
// add col change
|
||||
val colType = SparkInternalSchemaConverter.buildTypeFromStructType(addColumn.dataType(), true, new AtomicInteger(0))
|
||||
addChange.addColumns(parentName, names.last, colType, addColumn.comment())
|
||||
// add position change
|
||||
addColumn.position() match {
|
||||
case after: TableChange.After =>
|
||||
addChange.addPositionChange(names.mkString("."),
|
||||
if (parentName.isEmpty) after.column() else parentName + "." + after.column(), "after")
|
||||
case _: TableChange.First =>
|
||||
addChange.addPositionChange(names.mkString("."), "", "first")
|
||||
case _ =>
|
||||
}
|
||||
}
|
||||
val newSchema = SchemaChangeUtils.applyTableChanges2Schema(oldSchema, addChange)
|
||||
val verifiedHistorySchema = if (historySchema == null || historySchema.isEmpty) {
|
||||
SerDeHelper.inheritSchemas(oldSchema, "")
|
||||
} else {
|
||||
historySchema
|
||||
}
|
||||
AlterTableCommand312.commitWithSchema(newSchema, verifiedHistorySchema, table, sparkSession)
|
||||
logInfo("column add finished")
|
||||
}
|
||||
|
||||
def applyDeleteAction(sparkSession: SparkSession): Unit = {
|
||||
val (oldSchema, historySchema) = getInternalSchemaAndHistorySchemaStr(sparkSession)
|
||||
val deleteChange = TableChanges.ColumnDeleteChange.get(oldSchema)
|
||||
changes.map(_.asInstanceOf[DeleteColumn]).foreach { c =>
|
||||
val originalColName = c.fieldNames().mkString(".");
|
||||
AlterTableCommand312.checkSchemaChange(Seq(originalColName), table)
|
||||
deleteChange.deleteColumn(originalColName)
|
||||
}
|
||||
val newSchema = SchemaChangeUtils.applyTableChanges2Schema(oldSchema, deleteChange)
|
||||
// delete action should not change the getMaxColumnId field.
|
||||
newSchema.setMaxColumnId(oldSchema.getMaxColumnId)
|
||||
val verifiedHistorySchema = if (historySchema == null || historySchema.isEmpty) {
|
||||
SerDeHelper.inheritSchemas(oldSchema, "")
|
||||
} else {
|
||||
historySchema
|
||||
}
|
||||
AlterTableCommand312.commitWithSchema(newSchema, verifiedHistorySchema, table, sparkSession)
|
||||
logInfo("column delete finished")
|
||||
}
|
||||
|
||||
def applyUpdateAction(sparkSession: SparkSession): Unit = {
|
||||
val (oldSchema, historySchema) = getInternalSchemaAndHistorySchemaStr(sparkSession)
|
||||
val updateChange = TableChanges.ColumnUpdateChange.get(oldSchema)
|
||||
changes.foreach { change =>
|
||||
change match {
|
||||
case updateType: TableChange.UpdateColumnType =>
|
||||
val newType = SparkInternalSchemaConverter.buildTypeFromStructType(updateType.newDataType(), true, new AtomicInteger(0))
|
||||
updateChange.updateColumnType(updateType.fieldNames().mkString("."), newType)
|
||||
case updateComment: TableChange.UpdateColumnComment =>
|
||||
updateChange.updateColumnComment(updateComment.fieldNames().mkString("."), updateComment.newComment())
|
||||
case updateName: TableChange.RenameColumn =>
|
||||
val originalColName = updateName.fieldNames().mkString(".")
|
||||
AlterTableCommand312.checkSchemaChange(Seq(originalColName), table)
|
||||
updateChange.renameColumn(originalColName, updateName.newName())
|
||||
case updateNullAbility: TableChange.UpdateColumnNullability =>
|
||||
updateChange.updateColumnNullability(updateNullAbility.fieldNames().mkString("."), updateNullAbility.nullable())
|
||||
case updatePosition: TableChange.UpdateColumnPosition =>
|
||||
val names = updatePosition.fieldNames()
|
||||
val parentName = AlterTableCommand312.getParentName(names)
|
||||
updatePosition.position() match {
|
||||
case after: TableChange.After =>
|
||||
updateChange.addPositionChange(names.mkString("."),
|
||||
if (parentName.isEmpty) after.column() else parentName + "." + after.column(), "after")
|
||||
case _: TableChange.First =>
|
||||
updateChange.addPositionChange(names.mkString("."), "", "first")
|
||||
case _ =>
|
||||
}
|
||||
}
|
||||
}
|
||||
val newSchema = SchemaChangeUtils.applyTableChanges2Schema(oldSchema, updateChange)
|
||||
val verifiedHistorySchema = if (historySchema == null || historySchema.isEmpty) {
|
||||
SerDeHelper.inheritSchemas(oldSchema, "")
|
||||
} else {
|
||||
historySchema
|
||||
}
|
||||
AlterTableCommand312.commitWithSchema(newSchema, verifiedHistorySchema, table, sparkSession)
|
||||
logInfo("column update finished")
|
||||
}
|
||||
|
||||
// to do support unset default value to columns, and apply them to internalSchema
|
||||
def applyPropertyUnset(sparkSession: SparkSession): Unit = {
|
||||
val catalog = sparkSession.sessionState.catalog
|
||||
val propKeys = changes.map(_.asInstanceOf[RemoveProperty]).map(_.property())
|
||||
// ignore NonExist unset
|
||||
propKeys.foreach { k =>
|
||||
if (!table.properties.contains(k) && k != TableCatalog.PROP_COMMENT) {
|
||||
logWarning(s"find non exist unset property: ${k} , ignore it")
|
||||
}
|
||||
}
|
||||
val tableComment = if (propKeys.contains(TableCatalog.PROP_COMMENT)) None else table.comment
|
||||
val newProperties = table.properties.filter { case (k, _) => !propKeys.contains(k) }
|
||||
val newTable = table.copy(properties = newProperties, comment = tableComment)
|
||||
catalog.alterTable(newTable)
|
||||
logInfo("table properties change finished")
|
||||
}
|
||||
|
||||
// to do support set default value to columns, and apply them to internalSchema
|
||||
def applyPropertySet(sparkSession: SparkSession): Unit = {
|
||||
val catalog = sparkSession.sessionState.catalog
|
||||
val properties = changes.map(_.asInstanceOf[SetProperty]).map(f => f.property -> f.value).toMap
|
||||
// This overrides old properties and update the comment parameter of CatalogTable
|
||||
// with the newly added/modified comment since CatalogTable also holds comment as its
|
||||
// direct property.
|
||||
val newTable = table.copy(
|
||||
properties = table.properties ++ properties,
|
||||
comment = properties.get(TableCatalog.PROP_COMMENT).orElse(table.comment))
|
||||
catalog.alterTable(newTable)
|
||||
logInfo("table properties change finished")
|
||||
}
|
||||
|
||||
def getInternalSchemaAndHistorySchemaStr(sparkSession: SparkSession): (InternalSchema, String) = {
|
||||
val path = AlterTableCommand312.getTableLocation(table, sparkSession)
|
||||
val hadoopConf = sparkSession.sessionState.newHadoopConf()
|
||||
val metaClient = HoodieTableMetaClient.builder().setBasePath(path)
|
||||
.setConf(hadoopConf).build()
|
||||
val schemaUtil = new TableSchemaResolver(metaClient)
|
||||
|
||||
val schema = schemaUtil.getTableInternalSchemaFromCommitMetadata().orElse {
|
||||
AvroInternalSchemaConverter.convert(schemaUtil.getTableAvroSchema)
|
||||
}
|
||||
|
||||
val historySchemaStr = schemaUtil.getTableHistorySchemaStrFromCommitMetadata.orElse("")
|
||||
(schema, historySchemaStr)
|
||||
}
|
||||
}
|
||||
|
||||
object AlterTableCommand312 extends Logging {
|
||||
|
||||
/**
|
||||
* Generate an commit with new schema to change the table's schema.
|
||||
* @param internalSchema new schema after change
|
||||
* @param historySchemaStr history schemas
|
||||
* @param table The hoodie table.
|
||||
* @param sparkSession The spark session.
|
||||
*/
|
||||
def commitWithSchema(internalSchema: InternalSchema, historySchemaStr: String, table: CatalogTable, sparkSession: SparkSession): Unit = {
|
||||
val schema = AvroInternalSchemaConverter.convert(internalSchema, table.identifier.table)
|
||||
val path = getTableLocation(table, sparkSession)
|
||||
|
||||
val jsc = new JavaSparkContext(sparkSession.sparkContext)
|
||||
val client = DataSourceUtils.createHoodieClient(jsc, schema.toString,
|
||||
path, table.identifier.table, parametersWithWriteDefaults(table.storage.properties).asJava)
|
||||
|
||||
val hadoopConf = sparkSession.sessionState.newHadoopConf()
|
||||
val metaClient = HoodieTableMetaClient.builder().setBasePath(path).setConf(hadoopConf).build()
|
||||
|
||||
val commitActionType = CommitUtils.getCommitActionType(WriteOperationType.ALTER_SCHEMA, metaClient.getTableType)
|
||||
val instantTime = HoodieActiveTimeline.createNewInstantTime
|
||||
client.startCommitWithTime(instantTime, commitActionType)
|
||||
|
||||
val hoodieTable = HoodieSparkTable.create(client.getConfig, client.getEngineContext)
|
||||
val timeLine = hoodieTable.getActiveTimeline
|
||||
val requested = new HoodieInstant(State.REQUESTED, commitActionType, instantTime)
|
||||
val metadata = new HoodieCommitMetadata
|
||||
metadata.setOperationType(WriteOperationType.ALTER_SCHEMA)
|
||||
timeLine.transitionRequestedToInflight(requested, Option.of(metadata.toJsonString.getBytes(StandardCharsets.UTF_8)))
|
||||
val extraMeta = new util.HashMap[String, String]()
|
||||
extraMeta.put(SerDeHelper.LATEST_SCHEMA, SerDeHelper.toJson(internalSchema.setSchemaId(instantTime.toLong)))
|
||||
val schemaManager = new FileBasedInternalSchemaStorageManager(metaClient)
|
||||
schemaManager.persistHistorySchemaStr(instantTime, SerDeHelper.inheritSchemas(internalSchema, historySchemaStr))
|
||||
client.commit(instantTime, jsc.emptyRDD, Option.of(extraMeta))
|
||||
val existRoTable = sparkSession.catalog.tableExists(table.identifier.unquotedString + "_ro")
|
||||
val existRtTable = sparkSession.catalog.tableExists(table.identifier.unquotedString + "_rt")
|
||||
try {
|
||||
sparkSession.catalog.refreshTable(table.identifier.unquotedString)
|
||||
// try to refresh ro/rt table
|
||||
if (existRoTable) sparkSession.catalog.refreshTable(table.identifier.unquotedString + "_ro")
|
||||
if (existRoTable) sparkSession.catalog.refreshTable(table.identifier.unquotedString + "_rt")
|
||||
} catch {
|
||||
case NonFatal(e) =>
|
||||
log.error(s"Exception when attempting to refresh table ${table.identifier.quotedString}", e)
|
||||
}
|
||||
// try to sync to hive
|
||||
// drop partition field before call alter table
|
||||
val fullSparkSchema = SparkInternalSchemaConverter.constructSparkSchemaFromInternalSchema(internalSchema)
|
||||
val dataSparkSchema = new StructType(fullSparkSchema.fields.filter(p => !table.partitionColumnNames.exists(f => sparkSession.sessionState.conf.resolver(f, p.name))))
|
||||
alterTableDataSchema(sparkSession, table.identifier.database.getOrElse("default"), table.identifier.table, dataSparkSchema)
|
||||
if (existRoTable) alterTableDataSchema(sparkSession, table.identifier.database.getOrElse("default"), table.identifier.table + "_ro", dataSparkSchema)
|
||||
if (existRtTable) alterTableDataSchema(sparkSession, table.identifier.database.getOrElse("default"), table.identifier.table + "_rt", dataSparkSchema)
|
||||
}
|
||||
|
||||
def alterTableDataSchema(sparkSession: SparkSession, db: String, tableName: String, dataSparkSchema: StructType): Unit = {
|
||||
sparkSession.sessionState.catalog
|
||||
.externalCatalog
|
||||
.alterTableDataSchema(db, tableName, dataSparkSchema)
|
||||
}
|
||||
|
||||
def getTableLocation(table: CatalogTable, sparkSession: SparkSession): String = {
|
||||
val uri = if (table.tableType == CatalogTableType.MANAGED) {
|
||||
Some(sparkSession.sessionState.catalog.defaultTablePath(table.identifier))
|
||||
} else {
|
||||
table.storage.locationUri
|
||||
}
|
||||
val conf = sparkSession.sessionState.newHadoopConf()
|
||||
uri.map(makePathQualified(_, conf))
|
||||
.map(removePlaceHolder)
|
||||
.getOrElse(throw new IllegalArgumentException(s"Missing location for ${table.identifier}"))
|
||||
}
|
||||
|
||||
private def removePlaceHolder(path: String): String = {
|
||||
if (path == null || path.length == 0) {
|
||||
path
|
||||
} else if (path.endsWith("-__PLACEHOLDER__")) {
|
||||
path.substring(0, path.length() - 16)
|
||||
} else {
|
||||
path
|
||||
}
|
||||
}
|
||||
|
||||
def makePathQualified(path: URI, hadoopConf: Configuration): String = {
|
||||
val hadoopPath = new Path(path)
|
||||
val fs = hadoopPath.getFileSystem(hadoopConf)
|
||||
fs.makeQualified(hadoopPath).toUri.toString
|
||||
}
|
||||
|
||||
def getParentName(names: Array[String]): String = {
|
||||
if (names.size > 1) {
|
||||
names.dropRight(1).mkString(".")
|
||||
} else ""
|
||||
}
|
||||
|
||||
def parametersWithWriteDefaults(parameters: Map[String, String]): Map[String, String] = {
|
||||
Map(OPERATION.key -> OPERATION.defaultValue,
|
||||
TABLE_TYPE.key -> TABLE_TYPE.defaultValue,
|
||||
PRECOMBINE_FIELD.key -> PRECOMBINE_FIELD.defaultValue,
|
||||
HoodieWriteConfig.WRITE_PAYLOAD_CLASS_NAME.key -> HoodieWriteConfig.DEFAULT_WRITE_PAYLOAD_CLASS,
|
||||
INSERT_DROP_DUPS.key -> INSERT_DROP_DUPS.defaultValue,
|
||||
ASYNC_COMPACT_ENABLE.key -> ASYNC_COMPACT_ENABLE.defaultValue,
|
||||
INLINE_CLUSTERING_ENABLE.key -> INLINE_CLUSTERING_ENABLE.defaultValue,
|
||||
ASYNC_CLUSTERING_ENABLE.key -> ASYNC_CLUSTERING_ENABLE.defaultValue
|
||||
) ++ DataSourceOptionsHelper.translateConfigurations(parameters)
|
||||
}
|
||||
|
||||
def checkSchemaChange(colNames: Seq[String], catalogTable: CatalogTable): Unit = {
|
||||
val primaryKeys = catalogTable.storage.properties.getOrElse("primaryKey", catalogTable.properties.getOrElse("primaryKey", "keyid")).split(",").map(_.trim)
|
||||
val preCombineKey = Seq(catalogTable.storage.properties.getOrElse("preCombineField", catalogTable.properties.getOrElse("preCombineField", "ts"))).map(_.trim)
|
||||
val partitionKey = catalogTable.partitionColumnNames.map(_.trim)
|
||||
val checkNames = primaryKeys ++ preCombineKey ++ partitionKey
|
||||
colNames.foreach { col =>
|
||||
if (checkNames.contains(col)) {
|
||||
throw new UnsupportedOperationException("cannot support apply changes for primaryKey/CombineKey/partitionKey")
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -0,0 +1,29 @@
|
||||
/*
|
||||
* 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.spark.sql.parser
|
||||
|
||||
import org.apache.spark.internal.Logging
|
||||
import org.apache.spark.sql.SparkSession
|
||||
import org.apache.spark.sql.catalyst.parser.ParserInterface
|
||||
import org.apache.spark.sql.execution.{SparkSqlAstBuilder, SparkSqlParser}
|
||||
|
||||
// TODO: we should remove this file when we support datasourceV2 for hoodie on spark3.1x
|
||||
class HoodieSpark312ExtendedSqlParser(session: SparkSession, delegate: ParserInterface) extends SparkSqlParser with Logging {
|
||||
override val astBuilder: SparkSqlAstBuilder = new HoodieSpark312SqlAstBuilder(session)
|
||||
}
|
||||
|
||||
@@ -0,0 +1,125 @@
|
||||
/*
|
||||
* 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.spark.sql.parser
|
||||
|
||||
import org.apache.spark.sql.SparkSession
|
||||
import org.apache.spark.sql.catalyst.parser.ParserUtils.withOrigin
|
||||
import org.apache.spark.sql.catalyst.parser.SqlBaseParser._
|
||||
import org.apache.spark.sql.catalyst.plans.logical._
|
||||
import org.apache.spark.sql.execution.SparkSqlAstBuilder
|
||||
|
||||
// TODO: we should remove this file when we support datasourceV2 for hoodie on spark3.1x
|
||||
class HoodieSpark312SqlAstBuilder(sparkSession: SparkSession) extends SparkSqlAstBuilder {
|
||||
|
||||
/**
|
||||
* Parse a [[AlterTableAlterColumnStatement]] command to alter a column's property.
|
||||
*
|
||||
* For example:
|
||||
* {{{
|
||||
* ALTER TABLE table1 ALTER COLUMN a.b.c TYPE bigint
|
||||
* ALTER TABLE table1 ALTER COLUMN a.b.c SET NOT NULL
|
||||
* ALTER TABLE table1 ALTER COLUMN a.b.c DROP NOT NULL
|
||||
* ALTER TABLE table1 ALTER COLUMN a.b.c COMMENT 'new comment'
|
||||
* ALTER TABLE table1 ALTER COLUMN a.b.c FIRST
|
||||
* ALTER TABLE table1 ALTER COLUMN a.b.c AFTER x
|
||||
* }}}
|
||||
*/
|
||||
override def visitAlterTableAlterColumn(ctx: AlterTableAlterColumnContext): LogicalPlan = withOrigin(ctx) {
|
||||
val alter = super.visitAlterTableAlterColumn(ctx).asInstanceOf[AlterTableAlterColumnStatement]
|
||||
HoodieAlterTableAlterColumnStatement(alter.tableName, alter.column, alter.dataType, alter.nullable, alter.comment, alter.position)
|
||||
}
|
||||
|
||||
/**
|
||||
* Parse a [[org.apache.spark.sql.catalyst.plans.logical.AlterTableAddColumnsStatement]] command.
|
||||
*
|
||||
* For example:
|
||||
* {{{
|
||||
* ALTER TABLE table1
|
||||
* ADD COLUMNS (col_name data_type [COMMENT col_comment], ...);
|
||||
* }}}
|
||||
*/
|
||||
override def visitAddTableColumns(ctx: AddTableColumnsContext): LogicalPlan = withOrigin(ctx) {
|
||||
val add = super.visitAddTableColumns(ctx).asInstanceOf[AlterTableAddColumnsStatement]
|
||||
HoodieAlterTableAddColumnsStatement(add.tableName, add.columnsToAdd)
|
||||
}
|
||||
|
||||
/**
|
||||
* Parse a [[org.apache.spark.sql.catalyst.plans.logical.AlterTableRenameColumnStatement]] command.
|
||||
*
|
||||
* For example:
|
||||
* {{{
|
||||
* ALTER TABLE table1 RENAME COLUMN a.b.c TO x
|
||||
* }}}
|
||||
*/
|
||||
override def visitRenameTableColumn(
|
||||
ctx: RenameTableColumnContext): LogicalPlan = withOrigin(ctx) {
|
||||
val rename = super.visitRenameTableColumn(ctx).asInstanceOf[AlterTableRenameColumnStatement]
|
||||
HoodieAlterTableRenameColumnStatement(rename.tableName, rename.column, rename.newName)
|
||||
}
|
||||
|
||||
/**
|
||||
* Parse a [[AlterTableDropColumnsStatement]] command.
|
||||
*
|
||||
* For example:
|
||||
* {{{
|
||||
* ALTER TABLE table1 DROP COLUMN a.b.c
|
||||
* ALTER TABLE table1 DROP COLUMNS a.b.c, x, y
|
||||
* }}}
|
||||
*/
|
||||
override def visitDropTableColumns(
|
||||
ctx: DropTableColumnsContext): LogicalPlan = withOrigin(ctx) {
|
||||
val drop = super.visitDropTableColumns(ctx).asInstanceOf[AlterTableDropColumnsStatement]
|
||||
HoodieAlterTableDropColumnsStatement(drop.tableName, drop.columnsToDrop)
|
||||
}
|
||||
|
||||
/**
|
||||
* Parse [[AlterViewSetPropertiesStatement]] or [[AlterTableSetPropertiesStatement]] commands.
|
||||
*
|
||||
* For example:
|
||||
* {{{
|
||||
* ALTER TABLE table SET TBLPROPERTIES ('table_property' = 'property_value');
|
||||
* ALTER VIEW view SET TBLPROPERTIES ('table_property' = 'property_value');
|
||||
* }}}
|
||||
*/
|
||||
override def visitSetTableProperties(
|
||||
ctx: SetTablePropertiesContext): LogicalPlan = withOrigin(ctx) {
|
||||
val set = super.visitSetTableProperties(ctx)
|
||||
set match {
|
||||
case s: AlterTableSetPropertiesStatement => HoodieAlterTableSetPropertiesStatement(s.tableName, s.properties)
|
||||
case other => other
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Parse [[AlterViewUnsetPropertiesStatement]] or [[AlterTableUnsetPropertiesStatement]] commands.
|
||||
*
|
||||
* For example:
|
||||
* {{{
|
||||
* ALTER TABLE table UNSET TBLPROPERTIES [IF EXISTS] ('comment', 'key');
|
||||
* ALTER VIEW view UNSET TBLPROPERTIES [IF EXISTS] ('comment', 'key');
|
||||
* }}}
|
||||
*/
|
||||
override def visitUnsetTableProperties(
|
||||
ctx: UnsetTablePropertiesContext): LogicalPlan = withOrigin(ctx) {
|
||||
val unset = super.visitUnsetTableProperties(ctx)
|
||||
unset match {
|
||||
case us: AlterTableUnsetPropertiesStatement => HoodieAlterTableUnsetPropertiesStatement(us.tableName, us.propertyKeys, us.ifExists)
|
||||
case other => other
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -80,6 +80,12 @@
|
||||
<plugin>
|
||||
<groupId>net.alchim31.maven</groupId>
|
||||
<artifactId>scala-maven-plugin</artifactId>
|
||||
<configuration>
|
||||
<args>
|
||||
<arg>-nobootcp</arg>
|
||||
<arg>-target:jvm-1.8</arg>
|
||||
</args>
|
||||
</configuration>
|
||||
<executions>
|
||||
<execution>
|
||||
<id>scala-compile-first</id>
|
||||
@@ -175,11 +181,33 @@
|
||||
<dependency>
|
||||
<groupId>org.apache.spark</groupId>
|
||||
<artifactId>spark-sql_2.12</artifactId>
|
||||
<version>${spark3.version}</version>
|
||||
<version>${spark32.version}</version>
|
||||
<scope>provided</scope>
|
||||
<optional>true</optional>
|
||||
</dependency>
|
||||
|
||||
<dependency>
|
||||
<groupId>org.apache.spark</groupId>
|
||||
<artifactId>spark-catalyst_2.12</artifactId>
|
||||
<version>${spark32.version}</version>
|
||||
<scope>provided</scope>
|
||||
<optional>true</optional>
|
||||
</dependency>
|
||||
|
||||
<dependency>
|
||||
<groupId>org.apache.spark</groupId>
|
||||
<artifactId>spark-core_2.12</artifactId>
|
||||
<version>${spark32.version}</version>
|
||||
<scope>provided</scope>
|
||||
<optional>true</optional>
|
||||
<exclusions>
|
||||
<exclusion>
|
||||
<groupId>*</groupId>
|
||||
<artifactId>*</artifactId>
|
||||
</exclusion>
|
||||
</exclusions>
|
||||
</dependency>
|
||||
|
||||
<dependency>
|
||||
<groupId>com.fasterxml.jackson.core</groupId>
|
||||
<artifactId>jackson-databind</artifactId>
|
||||
@@ -200,18 +228,49 @@
|
||||
<groupId>org.apache.hudi</groupId>
|
||||
<artifactId>hudi-spark-client</artifactId>
|
||||
<version>${project.version}</version>
|
||||
<exclusions>
|
||||
<exclusion>
|
||||
<groupId>org.apache.spark</groupId>
|
||||
<artifactId>*</artifactId>
|
||||
</exclusion>
|
||||
</exclusions>
|
||||
</dependency>
|
||||
|
||||
<dependency>
|
||||
<groupId>org.apache.hudi</groupId>
|
||||
<artifactId>hudi-spark-common_${scala.binary.version}</artifactId>
|
||||
<version>${project.version}</version>
|
||||
<exclusions>
|
||||
<exclusion>
|
||||
<groupId>org.apache.spark</groupId>
|
||||
<artifactId>*</artifactId>
|
||||
</exclusion>
|
||||
</exclusions>
|
||||
</dependency>
|
||||
|
||||
<dependency>
|
||||
<groupId>org.json4s</groupId>
|
||||
<artifactId>json4s-jackson_${scala.binary.version}</artifactId>
|
||||
<version>3.7.0-M11</version>
|
||||
<exclusions>
|
||||
<exclusion>
|
||||
<groupId>com.fasterxml.jackson.core</groupId>
|
||||
<artifactId>*</artifactId>
|
||||
</exclusion>
|
||||
</exclusions>
|
||||
</dependency>
|
||||
|
||||
<dependency>
|
||||
<groupId>org.apache.hudi</groupId>
|
||||
<artifactId>hudi-spark3-common</artifactId>
|
||||
<version>${project.version}</version>
|
||||
<exclusions>
|
||||
<exclusion>
|
||||
<groupId>org.apache.spark</groupId>
|
||||
<artifactId>*</artifactId>
|
||||
</exclusion>
|
||||
</exclusions>
|
||||
</dependency>
|
||||
|
||||
<!-- Hoodie - Test -->
|
||||
<dependency>
|
||||
<groupId>org.apache.hudi</groupId>
|
||||
@@ -221,6 +280,7 @@
|
||||
<type>test-jar</type>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
|
||||
<dependency>
|
||||
<groupId>org.apache.hudi</groupId>
|
||||
<artifactId>hudi-spark-client</artifactId>
|
||||
@@ -228,7 +288,14 @@
|
||||
<classifier>tests</classifier>
|
||||
<type>test-jar</type>
|
||||
<scope>test</scope>
|
||||
<exclusions>
|
||||
<exclusion>
|
||||
<groupId>org.apache.spark</groupId>
|
||||
<artifactId>*</artifactId>
|
||||
</exclusion>
|
||||
</exclusions>
|
||||
</dependency>
|
||||
|
||||
<dependency>
|
||||
<groupId>org.apache.hudi</groupId>
|
||||
<artifactId>hudi-common</artifactId>
|
||||
@@ -237,6 +304,7 @@
|
||||
<type>test-jar</type>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
|
||||
<dependency>
|
||||
<groupId>org.apache.hudi</groupId>
|
||||
<artifactId>hudi-spark-common_${scala.binary.version}</artifactId>
|
||||
@@ -244,6 +312,12 @@
|
||||
<classifier>tests</classifier>
|
||||
<type>test-jar</type>
|
||||
<scope>test</scope>
|
||||
<exclusions>
|
||||
<exclusion>
|
||||
<groupId>org.apache.spark</groupId>
|
||||
<artifactId>*</artifactId>
|
||||
</exclusion>
|
||||
</exclusions>
|
||||
</dependency>
|
||||
|
||||
<dependency>
|
||||
|
||||
@@ -0,0 +1,191 @@
|
||||
/*
|
||||
* 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.spark.sql.execution.datasources.parquet;
|
||||
|
||||
import org.apache.hadoop.mapreduce.InputSplit;
|
||||
import org.apache.hadoop.mapreduce.TaskAttemptContext;
|
||||
import org.apache.hudi.client.utils.SparkInternalSchemaConverter;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.spark.memory.MemoryMode;
|
||||
import org.apache.spark.sql.catalyst.InternalRow;
|
||||
import org.apache.spark.sql.execution.vectorized.OffHeapColumnVector;
|
||||
import org.apache.spark.sql.execution.vectorized.OnHeapColumnVector;
|
||||
import org.apache.spark.sql.execution.vectorized.WritableColumnVector;
|
||||
import org.apache.spark.sql.types.DataType;
|
||||
import org.apache.spark.sql.types.StructType;
|
||||
import org.apache.spark.sql.vectorized.ColumnarBatch;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.time.ZoneId;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
|
||||
public class Spark32HoodieVectorizedParquetRecordReader extends VectorizedParquetRecordReader {
|
||||
|
||||
// save the col type change info.
|
||||
private Map<Integer, Pair<DataType, DataType>> typeChangeInfos;
|
||||
|
||||
private ColumnarBatch columnarBatch;
|
||||
|
||||
private Map<Integer, WritableColumnVector> idToColumnVectors;
|
||||
|
||||
private WritableColumnVector[] columnVectors;
|
||||
|
||||
// The capacity of vectorized batch.
|
||||
private int capacity;
|
||||
|
||||
// If true, this class returns batches instead of rows.
|
||||
private boolean returnColumnarBatch;
|
||||
|
||||
// The memory mode of the columnarBatch.
|
||||
private final MemoryMode memoryMode;
|
||||
|
||||
/**
|
||||
* Batch of rows that we assemble and the current index we've returned. Every time this
|
||||
* batch is used up (batchIdx == numBatched), we populated the batch.
|
||||
*/
|
||||
private int batchIdx = 0;
|
||||
private int numBatched = 0;
|
||||
|
||||
public Spark32HoodieVectorizedParquetRecordReader(
|
||||
ZoneId convertTz,
|
||||
String datetimeRebaseMode,
|
||||
String datetimeRebaseTz,
|
||||
String int96RebaseMode,
|
||||
String int96RebaseTz,
|
||||
boolean useOffHeap,
|
||||
int capacity,
|
||||
Map<Integer, Pair<DataType, DataType>> typeChangeInfos) {
|
||||
super(convertTz, datetimeRebaseMode, datetimeRebaseTz, int96RebaseMode, int96RebaseTz, useOffHeap, capacity);
|
||||
memoryMode = useOffHeap ? MemoryMode.OFF_HEAP : MemoryMode.ON_HEAP;
|
||||
this.typeChangeInfos = typeChangeInfos;
|
||||
this.capacity = capacity;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void initBatch(StructType partitionColumns, InternalRow partitionValues) {
|
||||
super.initBatch(partitionColumns, partitionValues);
|
||||
if (columnVectors == null) {
|
||||
columnVectors = new WritableColumnVector[sparkSchema.length() + partitionColumns.length()];
|
||||
}
|
||||
if (idToColumnVectors == null) {
|
||||
idToColumnVectors = new HashMap<>();
|
||||
typeChangeInfos.entrySet()
|
||||
.stream()
|
||||
.forEach(f -> {
|
||||
WritableColumnVector vector =
|
||||
memoryMode == MemoryMode.OFF_HEAP ? new OffHeapColumnVector(capacity, f.getValue().getLeft()) : new OnHeapColumnVector(capacity, f.getValue().getLeft());
|
||||
idToColumnVectors.put(f.getKey(), vector);
|
||||
});
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void initialize(InputSplit inputSplit, TaskAttemptContext taskAttemptContext) throws IOException, InterruptedException, UnsupportedOperationException {
|
||||
super.initialize(inputSplit, taskAttemptContext);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
super.close();
|
||||
for (Map.Entry<Integer, WritableColumnVector> e : idToColumnVectors.entrySet()) {
|
||||
e.getValue().close();
|
||||
}
|
||||
idToColumnVectors = null;
|
||||
columnarBatch = null;
|
||||
columnVectors = null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ColumnarBatch resultBatch() {
|
||||
ColumnarBatch currentColumnBatch = super.resultBatch();
|
||||
boolean changed = false;
|
||||
for (Map.Entry<Integer, Pair<DataType, DataType>> entry : typeChangeInfos.entrySet()) {
|
||||
boolean rewrite = SparkInternalSchemaConverter
|
||||
.convertColumnVectorType((WritableColumnVector) currentColumnBatch.column(entry.getKey()),
|
||||
idToColumnVectors.get(entry.getKey()), currentColumnBatch.numRows());
|
||||
if (rewrite) {
|
||||
changed = true;
|
||||
columnVectors[entry.getKey()] = idToColumnVectors.get(entry.getKey());
|
||||
}
|
||||
}
|
||||
if (changed) {
|
||||
if (columnarBatch == null) {
|
||||
// fill other vector
|
||||
for (int i = 0; i < columnVectors.length; i++) {
|
||||
if (columnVectors[i] == null) {
|
||||
columnVectors[i] = (WritableColumnVector) currentColumnBatch.column(i);
|
||||
}
|
||||
}
|
||||
columnarBatch = new ColumnarBatch(columnVectors);
|
||||
}
|
||||
columnarBatch.setNumRows(currentColumnBatch.numRows());
|
||||
return columnarBatch;
|
||||
} else {
|
||||
return currentColumnBatch;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean nextBatch() throws IOException {
|
||||
boolean result = super.nextBatch();
|
||||
if (idToColumnVectors != null) {
|
||||
idToColumnVectors.entrySet().stream().forEach(e -> e.getValue().reset());
|
||||
}
|
||||
numBatched = resultBatch().numRows();
|
||||
batchIdx = 0;
|
||||
return result;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void enableReturningBatches() {
|
||||
returnColumnarBatch = true;
|
||||
super.enableReturningBatches();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object getCurrentValue() {
|
||||
if (typeChangeInfos == null || typeChangeInfos.isEmpty()) {
|
||||
return super.getCurrentValue();
|
||||
}
|
||||
|
||||
if (returnColumnarBatch) {
|
||||
return columnarBatch == null ? super.getCurrentValue() : columnarBatch;
|
||||
}
|
||||
|
||||
return columnarBatch == null ? super.getCurrentValue() : columnarBatch.getRow(batchIdx - 1);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean nextKeyValue() throws IOException {
|
||||
resultBatch();
|
||||
|
||||
if (returnColumnarBatch) {
|
||||
return nextBatch();
|
||||
}
|
||||
|
||||
if (batchIdx >= numBatched) {
|
||||
if (!nextBatch()) {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
++batchIdx;
|
||||
return true;
|
||||
}
|
||||
}
|
||||
|
||||
@@ -22,6 +22,9 @@ import org.apache.spark.sql.avro._
|
||||
import org.apache.spark.sql.catalyst.expressions.Expression
|
||||
import org.apache.spark.sql.catalyst.parser.ParserInterface
|
||||
import org.apache.spark.sql.catalyst.plans.logical._
|
||||
import org.apache.spark.SPARK_VERSION
|
||||
import org.apache.spark.sql.catalyst.rules.Rule
|
||||
import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat
|
||||
import org.apache.spark.sql.parser.HoodieSpark3_2ExtendedSqlParser
|
||||
import org.apache.spark.sql.types.DataType
|
||||
import org.apache.spark.sql.{HoodieCatalystExpressionUtils, HoodieSpark3_2CatalystExpressionUtils, SparkSession}
|
||||
@@ -63,4 +66,28 @@ class Spark3_2Adapter extends BaseSpark3Adapter {
|
||||
(spark: SparkSession, delegate: ParserInterface) => new HoodieSpark3_2ExtendedSqlParser(spark, delegate)
|
||||
)
|
||||
}
|
||||
|
||||
override def createResolveHudiAlterTableCommand(sparkSession: SparkSession): Rule[LogicalPlan] = {
|
||||
if (SPARK_VERSION.startsWith("3.2")) {
|
||||
val loadClassName = "org.apache.spark.sql.hudi.ResolveHudiAlterTableCommandSpark32"
|
||||
val clazz = Class.forName(loadClassName, true, Thread.currentThread().getContextClassLoader)
|
||||
val ctor = clazz.getConstructors.head
|
||||
ctor.newInstance(sparkSession).asInstanceOf[Rule[LogicalPlan]]
|
||||
} else {
|
||||
new Rule[LogicalPlan] {
|
||||
override def apply(plan: LogicalPlan): LogicalPlan = plan
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
override def createHoodieParquetFileFormat(): Option[ParquetFileFormat] = {
|
||||
if (SPARK_VERSION.startsWith("3.2")) {
|
||||
val loadClassName = "org.apache.spark.sql.execution.datasources.parquet.Spark32HoodieParquetFileFormat"
|
||||
val clazz = Class.forName(loadClassName, true, Thread.currentThread().getContextClassLoader)
|
||||
val ctor = clazz.getConstructors.head
|
||||
Some(ctor.newInstance().asInstanceOf[ParquetFileFormat])
|
||||
} else {
|
||||
None
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -0,0 +1,349 @@
|
||||
/*
|
||||
* 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.spark.sql.execution.datasources.parquet
|
||||
|
||||
import java.net.URI
|
||||
|
||||
import org.apache.hadoop.conf.Configuration
|
||||
import org.apache.hadoop.fs.Path
|
||||
import org.apache.hadoop.mapred.FileSplit
|
||||
import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl
|
||||
import org.apache.hadoop.mapreduce.{JobID, TaskAttemptID, TaskID, TaskType}
|
||||
import org.apache.hudi.client.utils.SparkInternalSchemaConverter
|
||||
import org.apache.hudi.common.fs.FSUtils
|
||||
import org.apache.hudi.common.util.InternalSchemaCache
|
||||
import org.apache.hudi.common.util.collection.Pair
|
||||
import org.apache.hudi.internal.schema.InternalSchema
|
||||
import org.apache.hudi.internal.schema.action.InternalSchemaMerger
|
||||
import org.apache.hudi.internal.schema.utils.{InternalSchemaUtils, SerDeHelper}
|
||||
import org.apache.parquet.filter2.compat.FilterCompat
|
||||
import org.apache.parquet.filter2.predicate.FilterApi
|
||||
import org.apache.parquet.format.converter.ParquetMetadataConverter.SKIP_ROW_GROUPS
|
||||
import org.apache.parquet.hadoop.{ParquetFileReader, ParquetInputFormat, ParquetRecordReader}
|
||||
import org.apache.spark.TaskContext
|
||||
import org.apache.spark.sql.SparkSession
|
||||
import org.apache.spark.sql.catalyst.InternalRow
|
||||
import org.apache.spark.sql.catalyst.expressions.{Cast, JoinedRow}
|
||||
import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection
|
||||
import org.apache.spark.sql.catalyst.util.DateTimeUtils
|
||||
import org.apache.spark.sql.execution.datasources.{DataSourceUtils, PartitionedFile, RecordReaderIterator}
|
||||
import org.apache.spark.sql.internal.SQLConf
|
||||
import org.apache.spark.sql.sources._
|
||||
import org.apache.spark.sql.types.{AtomicType, DataType, StructField, StructType}
|
||||
import org.apache.spark.util.SerializableConfiguration
|
||||
|
||||
class Spark32HoodieParquetFileFormat extends ParquetFileFormat {
|
||||
|
||||
// reference ParquetFileFormat from spark project
|
||||
override def buildReaderWithPartitionValues(
|
||||
sparkSession: SparkSession,
|
||||
dataSchema: StructType,
|
||||
partitionSchema: StructType,
|
||||
requiredSchema: StructType,
|
||||
filters: Seq[Filter],
|
||||
options: Map[String, String],
|
||||
hadoopConf: Configuration): PartitionedFile => Iterator[InternalRow] = {
|
||||
if (hadoopConf.get(SparkInternalSchemaConverter.HOODIE_QUERY_SCHEMA, "").isEmpty) {
|
||||
// fallback to origin parquet File read
|
||||
super.buildReaderWithPartitionValues(sparkSession, dataSchema, partitionSchema, requiredSchema, filters, options, hadoopConf)
|
||||
} else {
|
||||
hadoopConf.set(ParquetInputFormat.READ_SUPPORT_CLASS, classOf[ParquetReadSupport].getName)
|
||||
hadoopConf.set(
|
||||
ParquetReadSupport.SPARK_ROW_REQUESTED_SCHEMA,
|
||||
requiredSchema.json)
|
||||
hadoopConf.set(
|
||||
ParquetWriteSupport.SPARK_ROW_SCHEMA,
|
||||
requiredSchema.json)
|
||||
hadoopConf.set(
|
||||
SQLConf.SESSION_LOCAL_TIMEZONE.key,
|
||||
sparkSession.sessionState.conf.sessionLocalTimeZone)
|
||||
hadoopConf.setBoolean(
|
||||
SQLConf.NESTED_SCHEMA_PRUNING_ENABLED.key,
|
||||
sparkSession.sessionState.conf.nestedSchemaPruningEnabled)
|
||||
hadoopConf.setBoolean(
|
||||
SQLConf.CASE_SENSITIVE.key,
|
||||
sparkSession.sessionState.conf.caseSensitiveAnalysis)
|
||||
|
||||
ParquetWriteSupport.setSchema(requiredSchema, hadoopConf)
|
||||
|
||||
// Sets flags for `ParquetToSparkSchemaConverter`
|
||||
hadoopConf.setBoolean(
|
||||
SQLConf.PARQUET_BINARY_AS_STRING.key,
|
||||
sparkSession.sessionState.conf.isParquetBinaryAsString)
|
||||
hadoopConf.setBoolean(
|
||||
SQLConf.PARQUET_INT96_AS_TIMESTAMP.key,
|
||||
sparkSession.sessionState.conf.isParquetINT96AsTimestamp)
|
||||
// for dataSource v1, we have no method to do project for spark physical plan.
|
||||
// it's safe to do cols project here.
|
||||
val internalSchemaString = hadoopConf.get(SparkInternalSchemaConverter.HOODIE_QUERY_SCHEMA)
|
||||
val querySchemaOption = SerDeHelper.fromJson(internalSchemaString)
|
||||
if (querySchemaOption.isPresent && !requiredSchema.isEmpty) {
|
||||
val prunedSchema = SparkInternalSchemaConverter.convertAndPruneStructTypeToInternalSchema(requiredSchema, querySchemaOption.get())
|
||||
hadoopConf.set(SparkInternalSchemaConverter.HOODIE_QUERY_SCHEMA, SerDeHelper.toJson(prunedSchema))
|
||||
}
|
||||
val broadcastedHadoopConf =
|
||||
sparkSession.sparkContext.broadcast(new SerializableConfiguration(hadoopConf))
|
||||
|
||||
// TODO: if you move this into the closure it reverts to the default values.
|
||||
// If true, enable using the custom RecordReader for parquet. This only works for
|
||||
// a subset of the types (no complex types).
|
||||
val resultSchema = StructType(partitionSchema.fields ++ requiredSchema.fields)
|
||||
val sqlConf = sparkSession.sessionState.conf
|
||||
val enableOffHeapColumnVector = sqlConf.offHeapColumnVectorEnabled
|
||||
val enableVectorizedReader: Boolean =
|
||||
sqlConf.parquetVectorizedReaderEnabled &&
|
||||
resultSchema.forall(_.dataType.isInstanceOf[AtomicType])
|
||||
val enableRecordFilter: Boolean = sqlConf.parquetRecordFilterEnabled
|
||||
val timestampConversion: Boolean = sqlConf.isParquetINT96TimestampConversion
|
||||
val capacity = sqlConf.parquetVectorizedReaderBatchSize
|
||||
val enableParquetFilterPushDown: Boolean = sqlConf.parquetFilterPushDown
|
||||
// Whole stage codegen (PhysicalRDD) is able to deal with batches directly
|
||||
val returningBatch = supportBatch(sparkSession, resultSchema)
|
||||
val pushDownDate = sqlConf.parquetFilterPushDownDate
|
||||
val pushDownTimestamp = sqlConf.parquetFilterPushDownTimestamp
|
||||
val pushDownDecimal = sqlConf.parquetFilterPushDownDecimal
|
||||
val pushDownStringStartWith = sqlConf.parquetFilterPushDownStringStartWith
|
||||
val pushDownInFilterThreshold = sqlConf.parquetFilterPushDownInFilterThreshold
|
||||
val isCaseSensitive = sqlConf.caseSensitiveAnalysis
|
||||
val parquetOptions = new ParquetOptions(options, sparkSession.sessionState.conf)
|
||||
val datetimeRebaseModeInRead = parquetOptions.datetimeRebaseModeInRead
|
||||
val int96RebaseModeInread = parquetOptions.int96RebaseModeInRead
|
||||
|
||||
(file: PartitionedFile) => {
|
||||
assert(file.partitionValues.numFields == partitionSchema.size)
|
||||
val filePath = new Path(new URI(file.filePath))
|
||||
val split = new FileSplit(filePath, file.start, file.length, Array.empty[String])
|
||||
val sharedConf = broadcastedHadoopConf.value.value
|
||||
// do deal with internalSchema
|
||||
val internalSchemaString = sharedConf.get(SparkInternalSchemaConverter.HOODIE_QUERY_SCHEMA)
|
||||
// querySchema must be a pruned schema.
|
||||
val querySchemaOption = SerDeHelper.fromJson(internalSchemaString)
|
||||
val internalSchemaChangeEnabled = if (internalSchemaString.isEmpty || !querySchemaOption.isPresent) false else true
|
||||
val tablePath = sharedConf.get(SparkInternalSchemaConverter.HOODIE_TABLE_PATH)
|
||||
val commitInstantTime = FSUtils.getCommitTime(filePath.getName).toLong;
|
||||
val fileSchema = if (internalSchemaChangeEnabled) {
|
||||
val validCommits = sharedConf.get(SparkInternalSchemaConverter.HOODIE_VALID_COMMITS_LIST)
|
||||
InternalSchemaCache.getInternalSchemaByVersionId(commitInstantTime, tablePath, sharedConf, if (validCommits == null) "" else validCommits)
|
||||
} else {
|
||||
// this should not happened, searchSchemaAndCache will deal with correctly.
|
||||
null
|
||||
}
|
||||
|
||||
lazy val footerFileMetaData =
|
||||
ParquetFooterReader.readFooter(sharedConf, filePath, SKIP_ROW_GROUPS).getFileMetaData
|
||||
val datetimeRebaseSpec = DataSourceUtils.datetimeRebaseSpec(
|
||||
footerFileMetaData.getKeyValueMetaData.get, datetimeRebaseModeInRead)
|
||||
// Try to push down filters when filter push-down is enabled.
|
||||
val pushed = if (enableParquetFilterPushDown) {
|
||||
val parquetSchema = footerFileMetaData.getSchema
|
||||
val parquetFilters = new ParquetFilters(
|
||||
parquetSchema,
|
||||
pushDownDate,
|
||||
pushDownTimestamp,
|
||||
pushDownDecimal,
|
||||
pushDownStringStartWith,
|
||||
pushDownInFilterThreshold,
|
||||
isCaseSensitive,
|
||||
datetimeRebaseSpec)
|
||||
filters.map(Spark32HoodieParquetFileFormat.rebuildFilterFromParquet(_, fileSchema, querySchemaOption.get()))
|
||||
// Collects all converted Parquet filter predicates. Notice that not all predicates can be
|
||||
// converted (`ParquetFilters.createFilter` returns an `Option`). That's why a `flatMap`
|
||||
// is used here.
|
||||
.flatMap(parquetFilters.createFilter(_))
|
||||
.reduceOption(FilterApi.and)
|
||||
} else {
|
||||
None
|
||||
}
|
||||
|
||||
// PARQUET_INT96_TIMESTAMP_CONVERSION says to apply timezone conversions to int96 timestamps'
|
||||
// *only* if the file was created by something other than "parquet-mr", so check the actual
|
||||
// writer here for this file. We have to do this per-file, as each file in the table may
|
||||
// have different writers.
|
||||
// Define isCreatedByParquetMr as function to avoid unnecessary parquet footer reads.
|
||||
def isCreatedByParquetMr: Boolean =
|
||||
footerFileMetaData.getCreatedBy().startsWith("parquet-mr")
|
||||
|
||||
val convertTz =
|
||||
if (timestampConversion && !isCreatedByParquetMr) {
|
||||
Some(DateTimeUtils.getZoneId(sharedConf.get(SQLConf.SESSION_LOCAL_TIMEZONE.key)))
|
||||
} else {
|
||||
None
|
||||
}
|
||||
val int96RebaseSpec = DataSourceUtils.int96RebaseSpec(
|
||||
footerFileMetaData.getKeyValueMetaData.get, int96RebaseModeInread)
|
||||
|
||||
val attemptId = new TaskAttemptID(new TaskID(new JobID(), TaskType.MAP, 0), 0)
|
||||
// use new conf
|
||||
val hadoopAttempConf = new Configuration(broadcastedHadoopConf.value.value)
|
||||
//
|
||||
// reset request schema
|
||||
var typeChangeInfos: java.util.Map[Integer, Pair[DataType, DataType]] = new java.util.HashMap()
|
||||
if (internalSchemaChangeEnabled) {
|
||||
val mergedInternalSchema = new InternalSchemaMerger(fileSchema, querySchemaOption.get(), true, true).mergeSchema()
|
||||
val mergedSchema = SparkInternalSchemaConverter.constructSparkSchemaFromInternalSchema(mergedInternalSchema)
|
||||
typeChangeInfos = SparkInternalSchemaConverter.collectTypeChangedCols(querySchemaOption.get(), mergedInternalSchema)
|
||||
hadoopAttempConf.set(ParquetReadSupport.SPARK_ROW_REQUESTED_SCHEMA, mergedSchema.json)
|
||||
}
|
||||
val hadoopAttemptContext =
|
||||
new TaskAttemptContextImpl(hadoopAttempConf, attemptId)
|
||||
|
||||
// Try to push down filters when filter push-down is enabled.
|
||||
// Notice: This push-down is RowGroups level, not individual records.
|
||||
if (pushed.isDefined) {
|
||||
ParquetInputFormat.setFilterPredicate(hadoopAttemptContext.getConfiguration, pushed.get)
|
||||
}
|
||||
val taskContext = Option(TaskContext.get())
|
||||
if (enableVectorizedReader) {
|
||||
val vectorizedReader = new Spark32HoodieVectorizedParquetRecordReader(
|
||||
convertTz.orNull,
|
||||
datetimeRebaseSpec.mode.toString,
|
||||
datetimeRebaseSpec.timeZone,
|
||||
int96RebaseSpec.mode.toString,
|
||||
int96RebaseSpec.timeZone,
|
||||
enableOffHeapColumnVector && taskContext.isDefined,
|
||||
capacity, typeChangeInfos)
|
||||
val iter = new RecordReaderIterator(vectorizedReader)
|
||||
// SPARK-23457 Register a task completion listener before `initialization`.
|
||||
// taskContext.foreach(_.addTaskCompletionListener[Unit](_ => iter.close()))
|
||||
try {
|
||||
vectorizedReader.initialize(split, hadoopAttemptContext)
|
||||
logDebug(s"Appending $partitionSchema ${file.partitionValues}")
|
||||
vectorizedReader.initBatch(partitionSchema, file.partitionValues)
|
||||
if (returningBatch) {
|
||||
vectorizedReader.enableReturningBatches()
|
||||
}
|
||||
|
||||
// UnsafeRowParquetRecordReader appends the columns internally to avoid another copy.
|
||||
iter.asInstanceOf[Iterator[InternalRow]]
|
||||
} catch {
|
||||
case e: Throwable =>
|
||||
// SPARK-23457: In case there is an exception in initialization, close the iterator to
|
||||
// avoid leaking resources.
|
||||
iter.close()
|
||||
throw e
|
||||
}
|
||||
} else {
|
||||
logDebug(s"Falling back to parquet-mr")
|
||||
// ParquetRecordReader returns InternalRow
|
||||
val readSupport = new ParquetReadSupport(
|
||||
convertTz,
|
||||
enableVectorizedReader = false,
|
||||
datetimeRebaseSpec,
|
||||
int96RebaseSpec)
|
||||
val reader = if (pushed.isDefined && enableRecordFilter) {
|
||||
val parquetFilter = FilterCompat.get(pushed.get, null)
|
||||
new ParquetRecordReader[InternalRow](readSupport, parquetFilter)
|
||||
} else {
|
||||
new ParquetRecordReader[InternalRow](readSupport)
|
||||
}
|
||||
val iter = new RecordReaderIterator[InternalRow](reader)
|
||||
// SPARK-23457 Register a task completion listener before `initialization`.
|
||||
taskContext.foreach(_.addTaskCompletionListener[Unit](_ => iter.close()))
|
||||
reader.initialize(split, hadoopAttemptContext)
|
||||
|
||||
val fullSchema = requiredSchema.toAttributes ++ partitionSchema.toAttributes
|
||||
val unsafeProjection = if (typeChangeInfos.isEmpty) {
|
||||
GenerateUnsafeProjection.generate(fullSchema, fullSchema)
|
||||
} else {
|
||||
// find type changed.
|
||||
val newFullSchema = new StructType(requiredSchema.fields.zipWithIndex.map { case (f, i) =>
|
||||
if (typeChangeInfos.containsKey(i)) {
|
||||
StructField(f.name, typeChangeInfos.get(i).getRight, f.nullable, f.metadata)
|
||||
} else f
|
||||
}).toAttributes ++ partitionSchema.toAttributes
|
||||
val castSchema = newFullSchema.zipWithIndex.map { case (attr, i) =>
|
||||
if (typeChangeInfos.containsKey(i)) {
|
||||
Cast(attr, typeChangeInfos.get(i).getLeft)
|
||||
} else attr
|
||||
}
|
||||
GenerateUnsafeProjection.generate(castSchema, newFullSchema)
|
||||
}
|
||||
|
||||
if (partitionSchema.length == 0) {
|
||||
// There is no partition columns
|
||||
iter.map(unsafeProjection)
|
||||
} else {
|
||||
val joinedRow = new JoinedRow()
|
||||
iter.map(d => unsafeProjection(joinedRow(d, file.partitionValues)))
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
object Spark32HoodieParquetFileFormat {
|
||||
|
||||
private def rebuildFilterFromParquet(oldFilter: Filter, fileSchema: InternalSchema, querySchema: InternalSchema): Filter = {
|
||||
if (fileSchema == null || querySchema == null) {
|
||||
oldFilter
|
||||
} else {
|
||||
oldFilter match {
|
||||
case eq: EqualTo =>
|
||||
val newAttribute = InternalSchemaUtils.reBuildFilterName(eq.attribute, fileSchema, querySchema)
|
||||
if (newAttribute.isEmpty) AlwaysTrue else eq.copy(attribute = newAttribute)
|
||||
case eqs: EqualNullSafe =>
|
||||
val newAttribute = InternalSchemaUtils.reBuildFilterName(eqs.attribute, fileSchema, querySchema)
|
||||
if (newAttribute.isEmpty) AlwaysTrue else eqs.copy(attribute = newAttribute)
|
||||
case gt: GreaterThan =>
|
||||
val newAttribute = InternalSchemaUtils.reBuildFilterName(gt.attribute, fileSchema, querySchema)
|
||||
if (newAttribute.isEmpty) AlwaysTrue else gt.copy(attribute = newAttribute)
|
||||
case gtr: GreaterThanOrEqual =>
|
||||
val newAttribute = InternalSchemaUtils.reBuildFilterName(gtr.attribute, fileSchema, querySchema)
|
||||
if (newAttribute.isEmpty) AlwaysTrue else gtr.copy(attribute = newAttribute)
|
||||
case lt: LessThan =>
|
||||
val newAttribute = InternalSchemaUtils.reBuildFilterName(lt.attribute, fileSchema, querySchema)
|
||||
if (newAttribute.isEmpty) AlwaysTrue else lt.copy(attribute = newAttribute)
|
||||
case lte: LessThanOrEqual =>
|
||||
val newAttribute = InternalSchemaUtils.reBuildFilterName(lte.attribute, fileSchema, querySchema)
|
||||
if (newAttribute.isEmpty) AlwaysTrue else lte.copy(attribute = newAttribute)
|
||||
case i: In =>
|
||||
val newAttribute = InternalSchemaUtils.reBuildFilterName(i.attribute, fileSchema, querySchema)
|
||||
if (newAttribute.isEmpty) AlwaysTrue else i.copy(attribute = newAttribute)
|
||||
case isn: IsNull =>
|
||||
val newAttribute = InternalSchemaUtils.reBuildFilterName(isn.attribute, fileSchema, querySchema)
|
||||
if (newAttribute.isEmpty) AlwaysTrue else isn.copy(attribute = newAttribute)
|
||||
case isnn: IsNotNull =>
|
||||
val newAttribute = InternalSchemaUtils.reBuildFilterName(isnn.attribute, fileSchema, querySchema)
|
||||
if (newAttribute.isEmpty) AlwaysTrue else isnn.copy(attribute = newAttribute)
|
||||
case And(left, right) =>
|
||||
And(rebuildFilterFromParquet(left, fileSchema, querySchema), rebuildFilterFromParquet(right, fileSchema, querySchema))
|
||||
case Or(left, right) =>
|
||||
Or(rebuildFilterFromParquet(left, fileSchema, querySchema), rebuildFilterFromParquet(right, fileSchema, querySchema))
|
||||
case Not(child) =>
|
||||
Not(rebuildFilterFromParquet(child, fileSchema, querySchema))
|
||||
case ssw: StringStartsWith =>
|
||||
val newAttribute = InternalSchemaUtils.reBuildFilterName(ssw.attribute, fileSchema, querySchema)
|
||||
if (newAttribute.isEmpty) AlwaysTrue else ssw.copy(attribute = newAttribute)
|
||||
case ses: StringEndsWith =>
|
||||
val newAttribute = InternalSchemaUtils.reBuildFilterName(ses.attribute, fileSchema, querySchema)
|
||||
if (newAttribute.isEmpty) AlwaysTrue else ses.copy(attribute = newAttribute)
|
||||
case sc: StringContains =>
|
||||
val newAttribute = InternalSchemaUtils.reBuildFilterName(sc.attribute, fileSchema, querySchema)
|
||||
if (newAttribute.isEmpty) AlwaysTrue else sc.copy(attribute = newAttribute)
|
||||
case AlwaysTrue =>
|
||||
AlwaysTrue
|
||||
case AlwaysFalse =>
|
||||
AlwaysFalse
|
||||
case _ =>
|
||||
AlwaysTrue
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -0,0 +1,67 @@
|
||||
/*
|
||||
* 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.spark.sql.hudi
|
||||
|
||||
import org.apache.hudi.config.HoodieWriteConfig
|
||||
import org.apache.hudi.internal.schema.action.TableChange.ColumnChangeID
|
||||
import org.apache.spark.sql.catalyst.analysis.ResolvedTable
|
||||
import org.apache.spark.sql.catalyst.catalog.CatalogTable
|
||||
import org.apache.spark.sql.SparkSession
|
||||
import org.apache.spark.sql.catalyst.plans.logical.{AddColumns, AlterColumn, DropColumns, LogicalPlan, RenameColumn, ReplaceColumns, SetTableProperties, UnsetTableProperties}
|
||||
import org.apache.spark.sql.catalyst.rules.Rule
|
||||
import org.apache.spark.sql.hudi.catalog.HoodieInternalV2Table
|
||||
import org.apache.spark.sql.hudi.command.{AlterTableCommand => HudiAlterTableCommand}
|
||||
|
||||
/**
|
||||
* Rule to mostly resolve, normalize and rewrite column names based on case sensitivity.
|
||||
* for alter table column commands.
|
||||
*/
|
||||
class ResolveHudiAlterTableCommandSpark32(sparkSession: SparkSession) extends Rule[LogicalPlan] {
|
||||
|
||||
def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsUp {
|
||||
case set @ SetTableProperties(asTable(table), _) if schemaEvolutionEnabled && set.resolved =>
|
||||
HudiAlterTableCommand(table, set.changes, ColumnChangeID.PROPERTY_CHANGE)
|
||||
case unSet @ UnsetTableProperties(asTable(table), _, _) if schemaEvolutionEnabled && unSet.resolved =>
|
||||
HudiAlterTableCommand(table, unSet.changes, ColumnChangeID.PROPERTY_CHANGE)
|
||||
case drop @ DropColumns(asTable(table), _) if schemaEvolutionEnabled && drop.resolved =>
|
||||
HudiAlterTableCommand(table, drop.changes, ColumnChangeID.DELETE)
|
||||
case add @ AddColumns(asTable(table), _) if schemaEvolutionEnabled && add.resolved =>
|
||||
HudiAlterTableCommand(table, add.changes, ColumnChangeID.ADD)
|
||||
case renameColumn @ RenameColumn(asTable(table), _, _) if schemaEvolutionEnabled && renameColumn.resolved=>
|
||||
HudiAlterTableCommand(table, renameColumn.changes, ColumnChangeID.UPDATE)
|
||||
case alter @ AlterColumn(asTable(table), _, _, _, _, _) if schemaEvolutionEnabled && alter.resolved =>
|
||||
HudiAlterTableCommand(table, alter.changes, ColumnChangeID.UPDATE)
|
||||
case replace @ ReplaceColumns(asTable(table), _) if schemaEvolutionEnabled && replace.resolved =>
|
||||
HudiAlterTableCommand(table, replace.changes, ColumnChangeID.REPLACE)
|
||||
}
|
||||
|
||||
private def schemaEvolutionEnabled(): Boolean = sparkSession
|
||||
.sessionState.conf.getConfString(HoodieWriteConfig.SCHEMA_EVOLUTION_ENABLE.key(), "false").toBoolean
|
||||
|
||||
object asTable {
|
||||
def unapply(a: LogicalPlan): Option[CatalogTable] = {
|
||||
a match {
|
||||
case ResolvedTable(_, _, table: HoodieInternalV2Table, _) =>
|
||||
table.catalogTable
|
||||
case _ =>
|
||||
None
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -0,0 +1,347 @@
|
||||
/*
|
||||
* 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.spark.sql.hudi.command
|
||||
|
||||
import java.net.URI
|
||||
import java.nio.charset.StandardCharsets
|
||||
import java.util
|
||||
import java.util.concurrent.atomic.AtomicInteger
|
||||
import org.apache.hadoop.conf.Configuration
|
||||
import org.apache.hadoop.fs.Path
|
||||
import org.apache.hudi.DataSourceWriteOptions._
|
||||
import org.apache.hudi.client.utils.SparkInternalSchemaConverter
|
||||
import org.apache.hudi.common.model.{HoodieCommitMetadata, WriteOperationType}
|
||||
import org.apache.hudi.{DataSourceOptionsHelper, DataSourceUtils}
|
||||
import org.apache.hudi.common.table.timeline.{HoodieActiveTimeline, HoodieInstant}
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant.State
|
||||
import org.apache.hudi.common.table.{HoodieTableMetaClient, TableSchemaResolver}
|
||||
import org.apache.hudi.common.util.{CommitUtils, Option}
|
||||
import org.apache.hudi.config.HoodieWriteConfig
|
||||
import org.apache.hudi.internal.schema.InternalSchema
|
||||
import org.apache.hudi.internal.schema.action.TableChange.ColumnChangeID
|
||||
import org.apache.hudi.internal.schema.action.TableChanges
|
||||
import org.apache.hudi.internal.schema.convert.AvroInternalSchemaConverter
|
||||
import org.apache.hudi.internal.schema.utils.{SchemaChangeUtils, SerDeHelper}
|
||||
import org.apache.hudi.internal.schema.io.FileBasedInternalSchemaStorageManager
|
||||
import org.apache.hudi.table.HoodieSparkTable
|
||||
import org.apache.spark.api.java.JavaSparkContext
|
||||
import org.apache.spark.internal.Logging
|
||||
import org.apache.spark.sql.{Row, SparkSession}
|
||||
import org.apache.spark.sql.catalyst.catalog.{CatalogTable, CatalogTableType}
|
||||
import org.apache.spark.sql.connector.catalog.{TableCatalog, TableChange}
|
||||
import org.apache.spark.sql.connector.catalog.TableChange.{AddColumn, DeleteColumn, RemoveProperty, SetProperty}
|
||||
import org.apache.spark.sql.types.StructType
|
||||
|
||||
import scala.collection.JavaConverters._
|
||||
import scala.util.control.NonFatal
|
||||
|
||||
case class AlterTableCommand(table: CatalogTable, changes: Seq[TableChange], changeType: ColumnChangeID) extends HoodieLeafRunnableCommand with Logging {
|
||||
override def run(sparkSession: SparkSession): Seq[Row] = {
|
||||
changeType match {
|
||||
case ColumnChangeID.ADD => applyAddAction(sparkSession)
|
||||
case ColumnChangeID.DELETE => applyDeleteAction(sparkSession)
|
||||
case ColumnChangeID.UPDATE => applyUpdateAction(sparkSession)
|
||||
case ColumnChangeID.PROPERTY_CHANGE if (changes.filter(_.isInstanceOf[SetProperty]).size == changes.size) =>
|
||||
applyPropertySet(sparkSession)
|
||||
case ColumnChangeID.PROPERTY_CHANGE if (changes.filter(_.isInstanceOf[RemoveProperty]).size == changes.size) =>
|
||||
applyPropertyUnset(sparkSession)
|
||||
case ColumnChangeID.REPLACE => applyReplaceAction(sparkSession)
|
||||
case other => throw new RuntimeException(s"find unsupported alter command type: ${other}")
|
||||
}
|
||||
Seq.empty[Row]
|
||||
}
|
||||
|
||||
def applyReplaceAction(sparkSession: SparkSession): Unit = {
|
||||
// convert to delete first then add again
|
||||
val deleteChanges = changes.filter(p => p.isInstanceOf[DeleteColumn]).map(_.asInstanceOf[DeleteColumn])
|
||||
val addChanges = changes.filter(p => p.isInstanceOf[AddColumn]).map(_.asInstanceOf[AddColumn])
|
||||
val (oldSchema, historySchema) = getInternalSchemaAndHistorySchemaStr(sparkSession)
|
||||
val newSchema = applyAddAction2Schema(sparkSession, applyDeleteAction2Schema(sparkSession, oldSchema, deleteChanges), addChanges)
|
||||
val verifiedHistorySchema = if (historySchema == null || historySchema.isEmpty) {
|
||||
SerDeHelper.inheritSchemas(oldSchema, "")
|
||||
} else {
|
||||
historySchema
|
||||
}
|
||||
AlterTableCommand.commitWithSchema(newSchema, verifiedHistorySchema, table, sparkSession)
|
||||
logInfo("column replace finished")
|
||||
}
|
||||
|
||||
def applyAddAction2Schema(sparkSession: SparkSession, oldSchema: InternalSchema, addChanges: Seq[AddColumn]): InternalSchema = {
|
||||
val addChange = TableChanges.ColumnAddChange.get(oldSchema)
|
||||
addChanges.foreach { addColumn =>
|
||||
val names = addColumn.fieldNames()
|
||||
val parentName = AlterTableCommand.getParentName(names)
|
||||
// add col change
|
||||
val colType = SparkInternalSchemaConverter.buildTypeFromStructType(addColumn.dataType(), true, new AtomicInteger(0))
|
||||
addChange.addColumns(parentName, names.last, colType, addColumn.comment())
|
||||
// add position change
|
||||
addColumn.position() match {
|
||||
case after: TableChange.After =>
|
||||
addChange.addPositionChange(names.mkString("."),
|
||||
if (parentName.isEmpty) after.column() else parentName + "." + after.column(), "after")
|
||||
case _: TableChange.First =>
|
||||
addChange.addPositionChange(names.mkString("."), "", "first")
|
||||
case _ =>
|
||||
}
|
||||
}
|
||||
SchemaChangeUtils.applyTableChanges2Schema(oldSchema, addChange)
|
||||
}
|
||||
|
||||
def applyDeleteAction2Schema(sparkSession: SparkSession, oldSchema: InternalSchema, deleteChanges: Seq[DeleteColumn]): InternalSchema = {
|
||||
val deleteChange = TableChanges.ColumnDeleteChange.get(oldSchema)
|
||||
deleteChanges.foreach { c =>
|
||||
val originalColName = c.fieldNames().mkString(".")
|
||||
checkSchemaChange(Seq(originalColName), table)
|
||||
deleteChange.deleteColumn(originalColName)
|
||||
}
|
||||
SchemaChangeUtils.applyTableChanges2Schema(oldSchema, deleteChange).setSchemaId(oldSchema.getMaxColumnId)
|
||||
}
|
||||
|
||||
|
||||
def applyAddAction(sparkSession: SparkSession): Unit = {
|
||||
val (oldSchema, historySchema) = getInternalSchemaAndHistorySchemaStr(sparkSession)
|
||||
val newSchema = applyAddAction2Schema(sparkSession, oldSchema, changes.map(_.asInstanceOf[AddColumn]))
|
||||
val verifiedHistorySchema = if (historySchema == null || historySchema.isEmpty) {
|
||||
SerDeHelper.inheritSchemas(oldSchema, "")
|
||||
} else {
|
||||
historySchema
|
||||
}
|
||||
AlterTableCommand.commitWithSchema(newSchema, verifiedHistorySchema, table, sparkSession)
|
||||
logInfo("column add finished")
|
||||
}
|
||||
|
||||
def applyDeleteAction(sparkSession: SparkSession): Unit = {
|
||||
val (oldSchema, historySchema) = getInternalSchemaAndHistorySchemaStr(sparkSession)
|
||||
val newSchema = applyDeleteAction2Schema(sparkSession, oldSchema, changes.map(_.asInstanceOf[DeleteColumn]))
|
||||
// delete action should not change the getMaxColumnId field.
|
||||
newSchema.setMaxColumnId(oldSchema.getMaxColumnId)
|
||||
val verifiedHistorySchema = if (historySchema == null || historySchema.isEmpty) {
|
||||
SerDeHelper.inheritSchemas(oldSchema, "")
|
||||
} else {
|
||||
historySchema
|
||||
}
|
||||
AlterTableCommand.commitWithSchema(newSchema, verifiedHistorySchema, table, sparkSession)
|
||||
logInfo("column delete finished")
|
||||
}
|
||||
|
||||
def applyUpdateAction(sparkSession: SparkSession): Unit = {
|
||||
val (oldSchema, historySchema) = getInternalSchemaAndHistorySchemaStr(sparkSession)
|
||||
val updateChange = TableChanges.ColumnUpdateChange.get(oldSchema)
|
||||
changes.foreach { change =>
|
||||
change match {
|
||||
case updateType: TableChange.UpdateColumnType =>
|
||||
val newType = SparkInternalSchemaConverter.buildTypeFromStructType(updateType.newDataType(), true, new AtomicInteger(0))
|
||||
updateChange.updateColumnType(updateType.fieldNames().mkString("."), newType)
|
||||
case updateComment: TableChange.UpdateColumnComment =>
|
||||
updateChange.updateColumnComment(updateComment.fieldNames().mkString("."), updateComment.newComment())
|
||||
case updateName: TableChange.RenameColumn =>
|
||||
val originalColName = updateName.fieldNames().mkString(".")
|
||||
checkSchemaChange(Seq(originalColName), table)
|
||||
updateChange.renameColumn(originalColName, updateName.newName())
|
||||
case updateNullAbility: TableChange.UpdateColumnNullability =>
|
||||
updateChange.updateColumnNullability(updateNullAbility.fieldNames().mkString("."), updateNullAbility.nullable())
|
||||
case updatePosition: TableChange.UpdateColumnPosition =>
|
||||
val names = updatePosition.fieldNames()
|
||||
val parentName = AlterTableCommand.getParentName(names)
|
||||
updatePosition.position() match {
|
||||
case after: TableChange.After =>
|
||||
updateChange.addPositionChange(names.mkString("."),
|
||||
if (parentName.isEmpty) after.column() else parentName + "." + after.column(), "after")
|
||||
case _: TableChange.First =>
|
||||
updateChange.addPositionChange(names.mkString("."), "", "first")
|
||||
case _ =>
|
||||
}
|
||||
}
|
||||
}
|
||||
val newSchema = SchemaChangeUtils.applyTableChanges2Schema(oldSchema, updateChange)
|
||||
val verifiedHistorySchema = if (historySchema == null || historySchema.isEmpty) {
|
||||
SerDeHelper.inheritSchemas(oldSchema, "")
|
||||
} else {
|
||||
historySchema
|
||||
}
|
||||
AlterTableCommand.commitWithSchema(newSchema, verifiedHistorySchema, table, sparkSession)
|
||||
logInfo("column update finished")
|
||||
}
|
||||
|
||||
// to do support unset default value to columns, and apply them to internalSchema
|
||||
def applyPropertyUnset(sparkSession: SparkSession): Unit = {
|
||||
val catalog = sparkSession.sessionState.catalog
|
||||
val propKeys = changes.map(_.asInstanceOf[RemoveProperty]).map(_.property())
|
||||
// ignore NonExist unset
|
||||
propKeys.foreach { k =>
|
||||
if (!table.properties.contains(k) && k != TableCatalog.PROP_COMMENT) {
|
||||
logWarning(s"find non exist unset property: ${k} , ignore it")
|
||||
}
|
||||
}
|
||||
val tableComment = if (propKeys.contains(TableCatalog.PROP_COMMENT)) None else table.comment
|
||||
val newProperties = table.properties.filter { case (k, _) => !propKeys.contains(k) }
|
||||
val newTable = table.copy(properties = newProperties, comment = tableComment)
|
||||
catalog.alterTable(newTable)
|
||||
logInfo("table properties change finished")
|
||||
}
|
||||
|
||||
// to do support set default value to columns, and apply them to internalSchema
|
||||
def applyPropertySet(sparkSession: SparkSession): Unit = {
|
||||
val catalog = sparkSession.sessionState.catalog
|
||||
val properties = changes.map(_.asInstanceOf[SetProperty]).map(f => f.property -> f.value).toMap
|
||||
// This overrides old properties and update the comment parameter of CatalogTable
|
||||
// with the newly added/modified comment since CatalogTable also holds comment as its
|
||||
// direct property.
|
||||
val newTable = table.copy(
|
||||
properties = table.properties ++ properties,
|
||||
comment = properties.get(TableCatalog.PROP_COMMENT).orElse(table.comment))
|
||||
catalog.alterTable(newTable)
|
||||
logInfo("table properties change finished")
|
||||
}
|
||||
|
||||
def getInternalSchemaAndHistorySchemaStr(sparkSession: SparkSession): (InternalSchema, String) = {
|
||||
val path = AlterTableCommand.getTableLocation(table, sparkSession)
|
||||
val hadoopConf = sparkSession.sessionState.newHadoopConf()
|
||||
val metaClient = HoodieTableMetaClient.builder().setBasePath(path)
|
||||
.setConf(hadoopConf).build()
|
||||
val schemaUtil = new TableSchemaResolver(metaClient)
|
||||
|
||||
val schema = schemaUtil.getTableInternalSchemaFromCommitMetadata().orElse {
|
||||
AvroInternalSchemaConverter.convert(schemaUtil.getTableAvroSchema)
|
||||
}
|
||||
|
||||
val historySchemaStr = schemaUtil.getTableHistorySchemaStrFromCommitMetadata.orElse("")
|
||||
(schema, historySchemaStr)
|
||||
}
|
||||
|
||||
def checkSchemaChange(colNames: Seq[String], catalogTable: CatalogTable): Unit = {
|
||||
val primaryKeys = catalogTable.storage.properties.getOrElse("primaryKey", catalogTable.properties.getOrElse("primaryKey", "keyid")).split(",").map(_.trim)
|
||||
val preCombineKey = Seq(catalogTable.storage.properties.getOrElse("preCombineField", catalogTable.properties.getOrElse("preCombineField", "ts"))).map(_.trim)
|
||||
val partitionKey = catalogTable.partitionColumnNames.map(_.trim)
|
||||
val checkNames = primaryKeys ++ preCombineKey ++ partitionKey
|
||||
colNames.foreach { col =>
|
||||
if (checkNames.contains(col)) {
|
||||
throw new UnsupportedOperationException("cannot support apply changes for primaryKey/CombineKey/partitionKey")
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
object AlterTableCommand extends Logging {
|
||||
|
||||
/**
|
||||
* Generate an commit with new schema to change the table's schema.
|
||||
*
|
||||
* @param internalSchema new schema after change
|
||||
* @param historySchemaStr history schemas
|
||||
* @param table The hoodie table.
|
||||
* @param sparkSession The spark session.
|
||||
*/
|
||||
def commitWithSchema(internalSchema: InternalSchema, historySchemaStr: String, table: CatalogTable, sparkSession: SparkSession): Unit = {
|
||||
val schema = AvroInternalSchemaConverter.convert(internalSchema, table.identifier.table)
|
||||
val path = getTableLocation(table, sparkSession)
|
||||
val jsc = new JavaSparkContext(sparkSession.sparkContext)
|
||||
val client = DataSourceUtils.createHoodieClient(jsc, schema.toString,
|
||||
path, table.identifier.table, parametersWithWriteDefaults(table.storage.properties).asJava)
|
||||
|
||||
val hadoopConf = sparkSession.sessionState.newHadoopConf()
|
||||
val metaClient = HoodieTableMetaClient.builder().setBasePath(path).setConf(hadoopConf).build()
|
||||
|
||||
val commitActionType = CommitUtils.getCommitActionType(WriteOperationType.ALTER_SCHEMA, metaClient.getTableType)
|
||||
val instantTime = HoodieActiveTimeline.createNewInstantTime
|
||||
client.startCommitWithTime(instantTime, commitActionType)
|
||||
|
||||
val hoodieTable = HoodieSparkTable.create(client.getConfig, client.getEngineContext)
|
||||
val timeLine = hoodieTable.getActiveTimeline
|
||||
val requested = new HoodieInstant(State.REQUESTED, commitActionType, instantTime)
|
||||
val metadata = new HoodieCommitMetadata
|
||||
metadata.setOperationType(WriteOperationType.ALTER_SCHEMA)
|
||||
timeLine.transitionRequestedToInflight(requested, Option.of(metadata.toJsonString.getBytes(StandardCharsets.UTF_8)))
|
||||
val extraMeta = new util.HashMap[String, String]()
|
||||
extraMeta.put(SerDeHelper.LATEST_SCHEMA, SerDeHelper.toJson(internalSchema.setSchemaId(instantTime.toLong)))
|
||||
val schemaManager = new FileBasedInternalSchemaStorageManager(metaClient)
|
||||
schemaManager.persistHistorySchemaStr(instantTime, SerDeHelper.inheritSchemas(internalSchema, historySchemaStr))
|
||||
client.commit(instantTime, jsc.emptyRDD, Option.of(extraMeta))
|
||||
val existRoTable = sparkSession.catalog.tableExists(table.identifier.unquotedString + "_ro")
|
||||
val existRtTable = sparkSession.catalog.tableExists(table.identifier.unquotedString + "_rt")
|
||||
try {
|
||||
sparkSession.catalog.refreshTable(table.identifier.unquotedString)
|
||||
// try to refresh ro/rt table
|
||||
if (existRoTable) sparkSession.catalog.refreshTable(table.identifier.unquotedString + "_ro")
|
||||
if (existRoTable) sparkSession.catalog.refreshTable(table.identifier.unquotedString + "_rt")
|
||||
} catch {
|
||||
case NonFatal(e) =>
|
||||
log.error(s"Exception when attempting to refresh table ${table.identifier.quotedString}", e)
|
||||
}
|
||||
// try to sync to hive
|
||||
// drop partition field before call alter table
|
||||
val fullSparkSchema = SparkInternalSchemaConverter.constructSparkSchemaFromInternalSchema(internalSchema)
|
||||
val dataSparkSchema = new StructType(fullSparkSchema.fields.filter(p => !table.partitionColumnNames.exists(f => sparkSession.sessionState.conf.resolver(f, p.name))))
|
||||
alterTableDataSchema(sparkSession, table.identifier.database.getOrElse("default"), table.identifier.table, dataSparkSchema)
|
||||
if (existRoTable) alterTableDataSchema(sparkSession, table.identifier.database.getOrElse("default"), table.identifier.table + "_ro", dataSparkSchema)
|
||||
if (existRtTable) alterTableDataSchema(sparkSession, table.identifier.database.getOrElse("default"), table.identifier.table + "_rt", dataSparkSchema)
|
||||
}
|
||||
|
||||
def alterTableDataSchema(sparkSession: SparkSession, db: String, tableName: String, dataSparkSchema: StructType): Unit = {
|
||||
sparkSession.sessionState.catalog
|
||||
.externalCatalog
|
||||
.alterTableDataSchema(db, tableName, dataSparkSchema)
|
||||
}
|
||||
|
||||
def getTableLocation(table: CatalogTable, sparkSession: SparkSession): String = {
|
||||
val uri = if (table.tableType == CatalogTableType.MANAGED) {
|
||||
Some(sparkSession.sessionState.catalog.defaultTablePath(table.identifier))
|
||||
} else {
|
||||
table.storage.locationUri
|
||||
}
|
||||
val conf = sparkSession.sessionState.newHadoopConf()
|
||||
uri.map(makePathQualified(_, conf))
|
||||
.map(removePlaceHolder)
|
||||
.getOrElse(throw new IllegalArgumentException(s"Missing location for ${table.identifier}"))
|
||||
}
|
||||
|
||||
private def removePlaceHolder(path: String): String = {
|
||||
if (path == null || path.length == 0) {
|
||||
path
|
||||
} else if (path.endsWith("-PLACEHOLDER")) {
|
||||
path.substring(0, path.length() - 16)
|
||||
} else {
|
||||
path
|
||||
}
|
||||
}
|
||||
|
||||
def makePathQualified(path: URI, hadoopConf: Configuration): String = {
|
||||
val hadoopPath = new Path(path)
|
||||
val fs = hadoopPath.getFileSystem(hadoopConf)
|
||||
fs.makeQualified(hadoopPath).toUri.toString
|
||||
}
|
||||
|
||||
def getParentName(names: Array[String]): String = {
|
||||
if (names.size > 1) {
|
||||
names.dropRight(1).mkString(".")
|
||||
} else ""
|
||||
}
|
||||
|
||||
def parametersWithWriteDefaults(parameters: Map[String, String]): Map[String, String] = {
|
||||
Map(OPERATION.key -> OPERATION.defaultValue,
|
||||
TABLE_TYPE.key -> TABLE_TYPE.defaultValue,
|
||||
PRECOMBINE_FIELD.key -> PRECOMBINE_FIELD.defaultValue,
|
||||
HoodieWriteConfig.WRITE_PAYLOAD_CLASS_NAME.key -> HoodieWriteConfig.DEFAULT_WRITE_PAYLOAD_CLASS,
|
||||
INSERT_DROP_DUPS.key -> INSERT_DROP_DUPS.defaultValue,
|
||||
ASYNC_COMPACT_ENABLE.key -> ASYNC_COMPACT_ENABLE.defaultValue,
|
||||
INLINE_CLUSTERING_ENABLE.key -> INLINE_CLUSTERING_ENABLE.defaultValue,
|
||||
ASYNC_CLUSTERING_ENABLE.key -> ASYNC_CLUSTERING_ENABLE.defaultValue
|
||||
) ++ DataSourceOptionsHelper.translateConfigurations(parameters)
|
||||
}
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user