1
0

[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:
xiarixiaoyao
2022-04-02 04:20:24 +08:00
committed by GitHub
parent 9275b8fc7e
commit 444ff496a4
89 changed files with 10352 additions and 106 deletions

View File

@@ -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;
}
}

View File

@@ -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
}
}
}

View File

@@ -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
}
}
}
}

View File

@@ -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
}
}
}
}

View File

@@ -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)
}
}