1
0

[HUDI-3549] Removing dependency on "spark-avro" (#4955)

Hudi will be taking on promise for it bundles to stay compatible with Spark minor versions (for ex 2.4, 3.1, 3.2): meaning that single build of Hudi (for ex "hudi-spark3.2-bundle") will be compatible with ALL patch versions in that minor branch (in that case 3.2.1, 3.2.0, etc)

To achieve that we'll have to remove (and ban) "spark-avro" as a dependency, which on a few occasions was the root-cause of incompatibility b/w consecutive Spark patch versions (most recently 3.2.1 and 3.2.0, due to this PR).

Instead of bundling "spark-avro" as dependency, we will be copying over some of the classes Hudi depends on and maintain them along the Hudi code-base to make sure we're able to provide for the aforementioned guarantee. To workaround arising compatibility issues we will be applying local patches to guarantee compatibility of Hudi bundles w/in the Spark minor version branches.

Following Hudi modules to Spark minor branches is currently maintained:

"hudi-spark3" -> 3.2.x
"hudi-spark3.1.x" -> 3.1.x
"hudi-spark2" -> 2.4.x
Following classes hierarchies (borrowed from "spark-avro") are maintained w/in these Spark-specific modules to guarantee compatibility with respective minor version branches:

AvroSerializer
AvroDeserializer
AvroUtils
Each of these classes has been correspondingly copied from Spark 3.2.1 (for 3.2.x branch), 3.1.2 (for 3.1.x branch), 2.4.4 (for 2.4.x branch) into their respective modules.

SchemaConverters class in turn is shared across all those modules given its relative stability (there're only cosmetical changes from 2.4.4 to 3.2.1).
All of the aforementioned classes have their corresponding scope of visibility limited to corresponding packages (org.apache.spark.sql.avro, org.apache.spark.sql) to make sure broader code-base does not become dependent on them and instead relies on facades abstracting them.

Additionally, given that Hudi plans on supporting all the patch versions of Spark w/in aforementioned minor versions branches of Spark, additional build steps were added to validate that Hudi could be properly compiled against those versions. Testing, however, is performed against the most recent patch versions of Spark with the help of Azure CI.

Brief change log:
- Removing spark-avro bundling from Hudi by default
- Scaffolded Spark 3.2.x hierarchy
- Bootstrapped Spark 3.1.x Avro serializer/deserializer hierarchy
- Bootstrapped Spark 2.4.x Avro serializer/deserializer hierarchy
- Moved ExpressionCodeGen,ExpressionPayload into hudi-spark module
- Fixed AvroDeserializer to stay compatible w/ both Spark 3.2.1 and 3.2.0
- Modified bot.yml to build full matrix of support Spark versions
- Removed "spark-avro" dependency from all modules
- Fixed relocation of spark-avro classes in bundles to assist in running integ-tests.
This commit is contained in:
Alexey Kudinkin
2022-03-29 11:44:47 -07:00
committed by GitHub
parent 0802510ca9
commit e5a2baeed0
54 changed files with 2665 additions and 278 deletions

View File

@@ -20,7 +20,7 @@ package org.apache.spark.sql.adapter
import org.apache.avro.Schema
import org.apache.hudi.Spark2RowSerDe
import org.apache.hudi.client.utils.SparkRowSerDe
import org.apache.spark.sql.avro.{HoodieAvroDeserializer, HoodieAvroSerializer, HoodieSpark2AvroDeserializer, HoodieSparkAvroSerializer}
import org.apache.spark.sql.avro.{HoodieAvroDeserializer, HoodieAvroSchemaConverters, HoodieAvroSerializer, HoodieSpark2_4AvroDeserializer, HoodieSpark2_4AvroSerializer, HoodieSparkAvroSchemaConverters}
import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder
import org.apache.spark.sql.catalyst.expressions.{Expression, Like}
@@ -38,17 +38,19 @@ import org.apache.spark.sql.{HoodieCatalystExpressionUtils, HoodieSpark2Catalyst
import scala.collection.mutable.ArrayBuffer
/**
* The adapter for spark2.
* Implementation of [[SparkAdapter]] for Spark 2.4.x
*/
class Spark2Adapter extends SparkAdapter {
override def createCatalystExpressionUtils(): HoodieCatalystExpressionUtils = HoodieSpark2CatalystExpressionUtils
override def createAvroSerializer(rootCatalystType: DataType, rootAvroType: Schema, nullable: Boolean): HoodieAvroSerializer =
new HoodieSparkAvroSerializer(rootCatalystType, rootAvroType, nullable)
new HoodieSpark2_4AvroSerializer(rootCatalystType, rootAvroType, nullable)
override def createAvroDeserializer(rootAvroType: Schema, rootCatalystType: DataType): HoodieAvroDeserializer =
new HoodieSpark2AvroDeserializer(rootAvroType, rootCatalystType)
new HoodieSpark2_4AvroDeserializer(rootAvroType, rootCatalystType)
override def getAvroSchemaConverters: HoodieAvroSchemaConverters = HoodieSparkAvroSchemaConverters
override def createSparkRowSerDe(encoder: ExpressionEncoder[Row]): SparkRowSerDe = {
new Spark2RowSerDe(encoder)

View File

@@ -37,10 +37,16 @@ import scala.collection.mutable.ArrayBuffer
/**
* A deserializer to deserialize data in avro format to data in catalyst format.
*
* NOTE: This is a version of {@code AvroDeserializer} impl from Spark 2.4.4 w/ the fix for SPARK-30267
* NOTE: This code is borrowed from Spark 2.4.4
* This code is borrowed, so that we can better control compatibility w/in Spark minor
* branches (3.2.x, 3.1.x, etc)
*
* PLEASE REFRAIN MAKING ANY CHANGES TO THIS CODE UNLESS ABSOLUTELY NECESSARY
*
* NOTE: This is a version of [[AvroDeserializer]] impl from Spark 2.4.4 w/ the fix for SPARK-30267
* applied on top of it
*/
class PatchedAvroDeserializer(rootAvroType: Schema, rootCatalystType: DataType) {
class AvroDeserializer(rootAvroType: Schema, rootCatalystType: DataType) {
private lazy val decimalConversions = new DecimalConversion()
private val converter: Any => Any = rootCatalystType match {

View File

@@ -0,0 +1,244 @@
/*
* 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 java.nio.ByteBuffer
import scala.collection.JavaConverters._
import org.apache.avro.{LogicalTypes, Schema}
import org.apache.avro.Conversions.DecimalConversion
import org.apache.avro.LogicalTypes.{TimestampMicros, TimestampMillis}
import org.apache.avro.Schema
import org.apache.avro.Schema.Type
import org.apache.avro.Schema.Type._
import org.apache.avro.generic.GenericData.{EnumSymbol, Fixed, Record}
import org.apache.avro.generic.GenericData.Record
import org.apache.avro.util.Utf8
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions.{SpecializedGetters, SpecificInternalRow}
import org.apache.spark.sql.types._
/**
* A serializer to serialize data in catalyst format to data in avro format.
*
* NOTE: This code is borrowed from Spark 2.4.4
* This code is borrowed, so that we can better control compatibility w/in Spark minor
* branches (3.2.x, 3.1.x, etc)
*
* PLEASE REFRAIN MAKING ANY CHANGES TO THIS CODE UNLESS ABSOLUTELY NECESSARY
*/
class AvroSerializer(rootCatalystType: DataType, rootAvroType: Schema, nullable: Boolean) {
def serialize(catalystData: Any): Any = {
converter.apply(catalystData)
}
private val converter: Any => Any = {
val actualAvroType = resolveNullableType(rootAvroType, nullable)
val baseConverter = rootCatalystType match {
case st: StructType =>
newStructConverter(st, actualAvroType).asInstanceOf[Any => Any]
case _ =>
val tmpRow = new SpecificInternalRow(Seq(rootCatalystType))
val converter = newConverter(rootCatalystType, actualAvroType)
(data: Any) =>
tmpRow.update(0, data)
converter.apply(tmpRow, 0)
}
if (nullable) {
(data: Any) =>
if (data == null) {
null
} else {
baseConverter.apply(data)
}
} else {
baseConverter
}
}
private type Converter = (SpecializedGetters, Int) => Any
private lazy val decimalConversions = new DecimalConversion()
private def newConverter(catalystType: DataType, avroType: Schema): Converter = {
(catalystType, avroType.getType) match {
case (NullType, NULL) =>
(getter, ordinal) => null
case (BooleanType, BOOLEAN) =>
(getter, ordinal) => getter.getBoolean(ordinal)
case (ByteType, INT) =>
(getter, ordinal) => getter.getByte(ordinal).toInt
case (ShortType, INT) =>
(getter, ordinal) => getter.getShort(ordinal).toInt
case (IntegerType, INT) =>
(getter, ordinal) => getter.getInt(ordinal)
case (LongType, LONG) =>
(getter, ordinal) => getter.getLong(ordinal)
case (FloatType, FLOAT) =>
(getter, ordinal) => getter.getFloat(ordinal)
case (DoubleType, DOUBLE) =>
(getter, ordinal) => getter.getDouble(ordinal)
case (d: DecimalType, FIXED)
if avroType.getLogicalType == LogicalTypes.decimal(d.precision, d.scale) =>
(getter, ordinal) =>
val decimal = getter.getDecimal(ordinal, d.precision, d.scale)
decimalConversions.toFixed(decimal.toJavaBigDecimal, avroType,
LogicalTypes.decimal(d.precision, d.scale))
case (d: DecimalType, BYTES)
if avroType.getLogicalType == LogicalTypes.decimal(d.precision, d.scale) =>
(getter, ordinal) =>
val decimal = getter.getDecimal(ordinal, d.precision, d.scale)
decimalConversions.toBytes(decimal.toJavaBigDecimal, avroType,
LogicalTypes.decimal(d.precision, d.scale))
case (StringType, ENUM) =>
val enumSymbols: Set[String] = avroType.getEnumSymbols.asScala.toSet
(getter, ordinal) =>
val data = getter.getUTF8String(ordinal).toString
if (!enumSymbols.contains(data)) {
throw new IncompatibleSchemaException(
"Cannot write \"" + data + "\" since it's not defined in enum \"" +
enumSymbols.mkString("\", \"") + "\"")
}
new EnumSymbol(avroType, data)
case (StringType, STRING) =>
(getter, ordinal) => new Utf8(getter.getUTF8String(ordinal).getBytes)
case (BinaryType, FIXED) =>
val size = avroType.getFixedSize()
(getter, ordinal) =>
val data: Array[Byte] = getter.getBinary(ordinal)
if (data.length != size) {
throw new IncompatibleSchemaException(
s"Cannot write ${data.length} ${if (data.length > 1) "bytes" else "byte"} of " +
"binary data into FIXED Type with size of " +
s"$size ${if (size > 1) "bytes" else "byte"}")
}
new Fixed(avroType, data)
case (BinaryType, BYTES) =>
(getter, ordinal) => ByteBuffer.wrap(getter.getBinary(ordinal))
case (DateType, INT) =>
(getter, ordinal) => getter.getInt(ordinal)
case (TimestampType, LONG) => avroType.getLogicalType match {
case _: TimestampMillis => (getter, ordinal) => getter.getLong(ordinal) / 1000
case _: TimestampMicros => (getter, ordinal) => getter.getLong(ordinal)
// For backward compatibility, if the Avro type is Long and it is not logical type,
// output the timestamp value as with millisecond precision.
case null => (getter, ordinal) => getter.getLong(ordinal) / 1000
case other => throw new IncompatibleSchemaException(
s"Cannot convert Catalyst Timestamp type to Avro logical type ${other}")
}
case (ArrayType(et, containsNull), ARRAY) =>
val elementConverter = newConverter(
et, resolveNullableType(avroType.getElementType, containsNull))
(getter, ordinal) => {
val arrayData = getter.getArray(ordinal)
val len = arrayData.numElements()
val result = new Array[Any](len)
var i = 0
while (i < len) {
if (containsNull && arrayData.isNullAt(i)) {
result(i) = null
} else {
result(i) = elementConverter(arrayData, i)
}
i += 1
}
// avro writer is expecting a Java Collection, so we convert it into
// `ArrayList` backed by the specified array without data copying.
java.util.Arrays.asList(result: _*)
}
case (st: StructType, RECORD) =>
val structConverter = newStructConverter(st, avroType)
val numFields = st.length
(getter, ordinal) => structConverter(getter.getStruct(ordinal, numFields))
case (MapType(kt, vt, valueContainsNull), MAP) if kt == StringType =>
val valueConverter = newConverter(
vt, resolveNullableType(avroType.getValueType, valueContainsNull))
(getter, ordinal) =>
val mapData = getter.getMap(ordinal)
val len = mapData.numElements()
val result = new java.util.HashMap[String, Any](len)
val keyArray = mapData.keyArray()
val valueArray = mapData.valueArray()
var i = 0
while (i < len) {
val key = keyArray.getUTF8String(i).toString
if (valueContainsNull && valueArray.isNullAt(i)) {
result.put(key, null)
} else {
result.put(key, valueConverter(valueArray, i))
}
i += 1
}
result
case other =>
throw new IncompatibleSchemaException(s"Cannot convert Catalyst type $catalystType to " +
s"Avro type $avroType.")
}
}
private def newStructConverter(
catalystStruct: StructType, avroStruct: Schema): InternalRow => Record = {
if (avroStruct.getType != RECORD || avroStruct.getFields.size() != catalystStruct.length) {
throw new IncompatibleSchemaException(s"Cannot convert Catalyst type $catalystStruct to " +
s"Avro type $avroStruct.")
}
val fieldConverters = catalystStruct.zip(avroStruct.getFields.asScala).map {
case (f1, f2) => newConverter(f1.dataType, resolveNullableType(f2.schema(), f1.nullable))
}
val numFields = catalystStruct.length
(row: InternalRow) =>
val result = new Record(avroStruct)
var i = 0
while (i < numFields) {
if (row.isNullAt(i)) {
result.put(i, null)
} else {
result.put(i, fieldConverters(i).apply(row, i))
}
i += 1
}
result
}
private def resolveNullableType(avroType: Schema, nullable: Boolean): Schema = {
if (nullable && avroType.getType != NULL) {
// avro uses union to represent nullable type.
val fields = avroType.getTypes.asScala
assert(fields.length == 2)
val actualType = fields.filter(_.getType != Type.NULL)
assert(actualType.length == 1)
actualType.head
} else {
avroType
}
}
}

View File

@@ -20,14 +20,10 @@ package org.apache.spark.sql.avro
import org.apache.avro.Schema
import org.apache.spark.sql.types.DataType
/**
* This is Spark 2 implementation for the [[HoodieAvroDeserializer]] leveraging [[PatchedAvroDeserializer]],
* which is just copied over version of [[AvroDeserializer]] from Spark 2.4.4 w/ SPARK-30267 being back-ported to it
*/
class HoodieSpark2AvroDeserializer(rootAvroType: Schema, rootCatalystType: DataType)
class HoodieSpark2_4AvroDeserializer(rootAvroType: Schema, rootCatalystType: DataType)
extends HoodieAvroDeserializer {
private val avroDeserializer = new PatchedAvroDeserializer(rootAvroType, rootCatalystType)
private val avroDeserializer = new AvroDeserializer(rootAvroType, rootCatalystType)
// As of Spark 3.1, this will return data wrapped with Option, so we make sure these interfaces
// are aligned across Spark versions

View File

@@ -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.avro
import org.apache.avro.Schema
import org.apache.spark.sql.types.DataType
class HoodieSpark2_4AvroSerializer(rootCatalystType: DataType, rootAvroType: Schema, nullable: Boolean)
extends HoodieAvroSerializer {
val avroSerializer = new AvroSerializer(rootCatalystType, rootAvroType, nullable)
override def serialize(catalystData: Any): Any = avroSerializer.serialize(catalystData)
}