Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -28,12 +28,15 @@ import org.apache.avro.Schema.Type._
import org.apache.avro.generic.GenericData.{Fixed, Record}
import org.apache.avro.generic.{GenericData, GenericFixed, GenericRecord}
import org.apache.avro.{LogicalTypes, Schema}

import org.apache.spark.sql.Row
import org.apache.spark.sql.avro.{IncompatibleSchemaException, SchemaConverters}
import org.apache.spark.sql.avro.SchemaConverters
import org.apache.spark.sql.catalyst.expressions.GenericRow
import org.apache.spark.sql.catalyst.util.DateTimeUtils
import org.apache.spark.sql.types._

import org.apache.hudi.AvroConversionUtils._
import org.apache.hudi.exception.HoodieIncompatibleSchemaException

import scala.collection.JavaConverters._

Expand Down Expand Up @@ -131,7 +134,7 @@ object AvroConversionHelper {
case null =>
new Timestamp(item.asInstanceOf[Long])
case other =>
throw new IncompatibleSchemaException(
throw new HoodieIncompatibleSchemaException(
s"Cannot convert Avro logical type $other to Catalyst Timestamp type.")
}
}
Expand All @@ -149,7 +152,7 @@ object AvroConversionHelper {
converters(i) = converter
avroFieldIndexes(i) = avroField.pos()
} else if (!sqlField.nullable) {
throw new IncompatibleSchemaException(
throw new HoodieIncompatibleSchemaException(
s"Cannot find non-nullable field ${sqlField.name} at path ${path.mkString(".")} " +
"in Avro schema\n" +
s"Source Avro schema: $sourceAvroSchema.\n" +
Expand Down Expand Up @@ -254,7 +257,7 @@ object AvroConversionHelper {
converted(i) = fieldConverters(i)(item)
new GenericRow(converted)
}
case _ => throw new IncompatibleSchemaException(
case _ => throw new HoodieIncompatibleSchemaException(
s"Cannot convert Avro schema to catalyst type because schema at path " +
s"${path.mkString(".")} is not compatible " +
s"(avroType = $other, sqlType = $sqlType). \n" +
Expand All @@ -263,7 +266,7 @@ object AvroConversionHelper {
}
}
case (left, right) =>
throw new IncompatibleSchemaException(
throw new HoodieIncompatibleSchemaException(
s"Cannot convert Avro schema to catalyst type because schema at path " +
s"${path.mkString(".")} is not compatible (avroType = $left, sqlType = $right). \n" +
s"Source Avro schema: $sourceAvroSchema.\n" +
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -87,4 +87,9 @@ trait SparkAdapter extends Serializable {
* Create Like expression.
*/
def createLike(left: Expression, right: Expression): Expression

/**
* ParserInterface#parseMultipartIdentifier is supported since spark3, for spark2 this should not be called.
*/
def parseMultipartIdentifier(parser: ParserInterface, sqlText: String): Seq[String]
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,33 @@
/*
* 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.hudi.exception;

/**
* Exception for incompatible schema.
*/
public class HoodieIncompatibleSchemaException extends Exception {

public HoodieIncompatibleSchemaException(String msg, Throwable e) {
super(msg, e);
}

public HoodieIncompatibleSchemaException(String msg) {
super(msg);
}
}
8 changes: 7 additions & 1 deletion hudi-spark-datasource/hudi-spark/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -503,6 +503,13 @@
<scope>test</scope>
</dependency>

<dependency>
<groupId>org.slf4j</groupId>
<artifactId>slf4j-api</artifactId>
<version>${slf4j.version}</version>
<scope>test</scope>
</dependency>

<dependency>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-hdfs</artifactId>
Expand All @@ -524,6 +531,5 @@
</exclusion>
</exclusions>
</dependency>

</dependencies>
</project>
Original file line number Diff line number Diff line change
Expand Up @@ -28,7 +28,7 @@ import org.apache.hudi.exception.HoodieException
import org.apache.hudi.hadoop.config.HoodieRealtimeConfig
import org.apache.hudi.hadoop.utils.HoodieInputFormatUtils.HOODIE_RECORD_KEY_COL_POS
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.avro.{AvroDeserializer, AvroSerializer}
import org.apache.spark.sql.avro.{HoodieAvroSerializer, HooodieAvroDeserializer}
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions.{SpecificInternalRow, UnsafeProjection}
import org.apache.spark.sql.execution.datasources.PartitionedFile
Expand Down Expand Up @@ -119,7 +119,7 @@ class HoodieMergeOnReadRDD(@transient sc: SparkContext,
tableState.requiredStructSchema
.map(f => tableAvroSchema.getField(f.name).pos()).toList
private val recordBuilder = new GenericRecordBuilder(requiredAvroSchema)
private val deserializer = new AvroDeserializer(requiredAvroSchema, tableState.requiredStructSchema)
private val deserializer = HooodieAvroDeserializer(requiredAvroSchema, tableState.requiredStructSchema)
private val unsafeProjection = UnsafeProjection.create(tableState.requiredStructSchema)
private val logRecords = HoodieMergeOnReadRDD.scanLog(split, tableAvroSchema, config).getRecords
private val logRecordsKeyIterator = logRecords.keySet().iterator().asScala
Expand All @@ -135,7 +135,7 @@ class HoodieMergeOnReadRDD(@transient sc: SparkContext,
} else {
val requiredAvroRecord = AvroConversionUtils
.buildAvroRecordBySchema(curAvroRecord.get(), requiredAvroSchema, requiredFieldPosition, recordBuilder)
recordToLoad = unsafeProjection(deserializer.deserialize(requiredAvroRecord).asInstanceOf[InternalRow])
recordToLoad = unsafeProjection(deserializer.deserializeData(requiredAvroRecord).asInstanceOf[InternalRow])
true
}
} else {
Expand All @@ -158,7 +158,7 @@ class HoodieMergeOnReadRDD(@transient sc: SparkContext,
tableState.requiredStructSchema
.map(f => tableAvroSchema.getField(f.name).pos()).toList
private val recordBuilder = new GenericRecordBuilder(requiredAvroSchema)
private val deserializer = new AvroDeserializer(requiredAvroSchema, tableState.requiredStructSchema)
private val deserializer = HooodieAvroDeserializer(requiredAvroSchema, tableState.requiredStructSchema)
private val unsafeProjection = UnsafeProjection.create(tableState.requiredStructSchema)
private val logRecords = HoodieMergeOnReadRDD.scanLog(split, tableAvroSchema, config).getRecords
private val logRecordsKeyIterator = logRecords.keySet().iterator().asScala
Expand All @@ -180,7 +180,7 @@ class HoodieMergeOnReadRDD(@transient sc: SparkContext,
} else {
val requiredAvroRecord = AvroConversionUtils
.buildAvroRecordBySchema(curAvroRecord.get(), requiredAvroSchema, requiredFieldPosition, recordBuilder)
recordToLoad = unsafeProjection(deserializer.deserialize(requiredAvroRecord).asInstanceOf[InternalRow])
recordToLoad = unsafeProjection(deserializer.deserializeData(requiredAvroRecord).asInstanceOf[InternalRow])
true
}
} else {
Expand All @@ -203,8 +203,8 @@ class HoodieMergeOnReadRDD(@transient sc: SparkContext,
private val requiredFieldPosition =
tableState.requiredStructSchema
.map(f => tableAvroSchema.getField(f.name).pos()).toList
private val serializer = new AvroSerializer(tableState.tableStructSchema, tableAvroSchema, false)
private val requiredDeserializer = new AvroDeserializer(requiredAvroSchema, tableState.requiredStructSchema)
private val serializer = HoodieAvroSerializer(tableState.tableStructSchema, tableAvroSchema, false)
private val requiredDeserializer = HooodieAvroDeserializer(requiredAvroSchema, tableState.requiredStructSchema)
private val recordBuilder = new GenericRecordBuilder(requiredAvroSchema)
private val unsafeProjection = UnsafeProjection.create(tableState.requiredStructSchema)
private val logRecords = HoodieMergeOnReadRDD.scanLog(split, tableAvroSchema, config).getRecords
Expand Down Expand Up @@ -236,7 +236,7 @@ class HoodieMergeOnReadRDD(@transient sc: SparkContext,
recordBuilder
)
recordToLoad = unsafeProjection(requiredDeserializer
.deserialize(requiredAvroRecord).asInstanceOf[InternalRow])
.deserializeData(requiredAvroRecord).asInstanceOf[InternalRow])
true
}
} else {
Expand Down Expand Up @@ -264,7 +264,7 @@ class HoodieMergeOnReadRDD(@transient sc: SparkContext,
recordBuilder
)
recordToLoad = unsafeProjection(requiredDeserializer
.deserialize(requiredAvroRecord).asInstanceOf[InternalRow])
.deserializeData(requiredAvroRecord).asInstanceOf[InternalRow])
true
}
}
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,28 @@
/*
* 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.avro

import org.apache.avro.Schema
import org.apache.spark.sql.types.DataType

/**
* As AvroSerializer cannot be access out of the spark.sql.avro package since spark 3.1, we define
* this class to be accessed by other class.
*/
case class HoodieAvroSerializer(rootCatalystType: DataType, rootAvroType: Schema, nullable: Boolean)
extends AvroSerializer(rootCatalystType, rootAvroType, nullable)
Original file line number Diff line number Diff line change
@@ -0,0 +1,36 @@
/*
* 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.avro

import org.apache.avro.Schema
import org.apache.spark.sql.types.DataType

/**
* This is to be compatible with the type returned by Spark 3.1
* and other spark versions for AvroDeserializer
*/
case class HooodieAvroDeserializer(rootAvroType: Schema, rootCatalystType: DataType)
extends AvroDeserializer(rootAvroType, rootCatalystType) {

def deserializeData(data: Any): Any = {
super.deserialize(data) match {
case Some(r) => r // spark 3.1 return type is Option, we fetch the data.
case o => o // for other spark version, return the data directly.
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -31,7 +31,7 @@ import org.apache.hudi.common.util.{ValidationUtils, Option => HOption}
import org.apache.hudi.config.HoodieWriteConfig
import org.apache.hudi.io.HoodieWriteHandle
import org.apache.hudi.sql.IExpressionEvaluator
import org.apache.spark.sql.avro.{AvroSerializer, SchemaConverters}
import org.apache.spark.sql.avro.{AvroSerializer, HoodieAvroSerializer, SchemaConverters}
import org.apache.spark.sql.catalyst.expressions.Expression
import org.apache.spark.sql.hudi.SerDeUtils
import org.apache.spark.sql.hudi.command.payload.ExpressionPayload.getEvaluator
Expand Down Expand Up @@ -310,7 +310,7 @@ object ExpressionPayload {
val conditionEvaluator = ExpressionCodeGen.doCodeGen(Seq(condition), conditionSerializer)

val assignSqlType = SchemaConverters.toSqlType(writeSchema).dataType.asInstanceOf[StructType]
val assignSerializer = new AvroSerializer(assignSqlType, writeSchema, false)
val assignSerializer = new HoodieAvroSerializer(assignSqlType, writeSchema, false)
val assignmentEvaluator = ExpressionCodeGen.doCodeGen(assignments, assignSerializer)
conditionEvaluator -> assignmentEvaluator
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,7 @@ package org.apache.spark.sql.hudi.command.payload

import org.apache.avro.generic.IndexedRecord
import org.apache.avro.Schema
import org.apache.spark.sql.avro.{AvroDeserializer, SchemaConverters}
import org.apache.spark.sql.avro.{HooodieAvroDeserializer, SchemaConverters}
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.types._

Expand All @@ -29,8 +29,8 @@ import org.apache.spark.sql.types._
class SqlTypedRecord(val record: IndexedRecord) extends IndexedRecord {

private lazy val sqlType = SchemaConverters.toSqlType(getSchema).dataType.asInstanceOf[StructType]
private lazy val avroDeserializer = new AvroDeserializer(record.getSchema, sqlType)
private lazy val sqlRow = avroDeserializer.deserialize(record).asInstanceOf[InternalRow]
private lazy val avroDeserializer = HooodieAvroDeserializer(record.getSchema, sqlType)
private lazy val sqlRow = avroDeserializer.deserializeData(record).asInstanceOf[InternalRow]

override def put(i: Int, v: Any): Unit = {
record.put(i, v)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -62,7 +62,7 @@ class HoodieCommonSqlParser(session: SparkSession, delegate: ParserInterface)
}

def parseMultipartIdentifier(sqlText: String): Seq[String] = {
throw new UnsupportedOperationException(s"Unsupported parseMultipartIdentifier method")
sparkAdapter.parseMultipartIdentifier(delegate, sqlText)
}

protected def parse[T](command: String)(toResult: HoodieSqlCommonParser => T): T = {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -137,7 +137,8 @@ class TestDataSourceForBootstrap {
verifyIncrementalViewResult(commitInstantTime1, commitInstantTime2, isPartitioned = false, isHiveStylePartitioned = false)
}

@Test def testMetadataBootstrapCOWHiveStylePartitioned(): Unit = {
@Test
def testMetadataBootstrapCOWHiveStylePartitioned(): Unit = {
val timestamp = Instant.now.toEpochMilli
val jsc = JavaSparkContext.fromSparkContext(spark.sparkContext)

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -46,8 +46,8 @@ class TestAlterTableDropPartition extends TestHoodieSqlBase {
// insert data
spark.sql(s"""insert into $tableName values (1, "z3", "v1", "2021-10-01"), (2, "l4", "v1", "2021-10-02")""")

checkException(s"alter table $tableName drop partition (dt='2021-10-01')")(
s"dt is not a valid partition column in table `default`.`${tableName}`.;")
checkExceptionContain(s"alter table $tableName drop partition (dt='2021-10-01')")(
s"dt is not a valid partition column in table `default`.`$tableName`.")
}

Seq(false, true).foreach { urlencode =>
Expand Down Expand Up @@ -115,12 +115,8 @@ class TestAlterTableDropPartition extends TestHoodieSqlBase {
spark.sql(s"""insert into $tableName values (1, "z3", "v1", "2021-10-01"), (2, "l4", "v1", "2021-10-02")""")

// specify duplicate partition columns
try {
spark.sql(s"alter table $tableName drop partition (dt='2021-10-01', dt='2021-10-02')")
} catch {
case NonFatal(e) =>
assert(e.getMessage.contains("Found duplicate keys 'dt'"))
}
checkExceptionContain(s"alter table $tableName drop partition (dt='2021-10-01', dt='2021-10-02')")(
"Found duplicate keys 'dt'")

// drop 2021-10-01 partition
spark.sql(s"alter table $tableName drop partition (dt='2021-10-01')")
Expand Down Expand Up @@ -164,8 +160,8 @@ class TestAlterTableDropPartition extends TestHoodieSqlBase {
|""".stripMargin)

// not specified all partition column
checkException(s"alter table $tableName drop partition (year='2021', month='10')")(
"All partition columns need to be specified for Hoodie's dropping partition;"
checkExceptionContain(s"alter table $tableName drop partition (year='2021', month='10')")(
"All partition columns need to be specified for Hoodie's dropping partition"
)
// drop 2021-10-01 partition
spark.sql(s"alter table $tableName drop partition (year='2021', month='10', day='01')")
Expand Down
Loading