-
Notifications
You must be signed in to change notification settings - Fork 29.2k
[SPARK-25104][SQL]Avro: Validate user specified output schema #22094
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Closed
Closed
Changes from all commits
Commits
Show all changes
2 commits
Select commit
Hold shift + click to select a range
File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -26,6 +26,7 @@ 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 | ||
|
|
@@ -72,62 +73,70 @@ class AvroSerializer(rootCatalystType: DataType, rootAvroType: Schema, nullable: | |
| private lazy val decimalConversions = new DecimalConversion() | ||
|
|
||
| private def newConverter(catalystType: DataType, avroType: Schema): Converter = { | ||
| catalystType match { | ||
| case NullType => | ||
| (catalystType, avroType.getType) match { | ||
| case (NullType, NULL) => | ||
| (getter, ordinal) => null | ||
| case BooleanType => | ||
| case (BooleanType, BOOLEAN) => | ||
| (getter, ordinal) => getter.getBoolean(ordinal) | ||
| case ByteType => | ||
| case (ByteType, INT) => | ||
| (getter, ordinal) => getter.getByte(ordinal).toInt | ||
| case ShortType => | ||
| case (ShortType, INT) => | ||
| (getter, ordinal) => getter.getShort(ordinal).toInt | ||
| case IntegerType => | ||
| case (IntegerType, INT) => | ||
| (getter, ordinal) => getter.getInt(ordinal) | ||
| case LongType => | ||
| case (LongType, LONG) => | ||
| (getter, ordinal) => getter.getLong(ordinal) | ||
| case FloatType => | ||
| case (FloatType, FLOAT) => | ||
| (getter, ordinal) => getter.getFloat(ordinal) | ||
| case DoubleType => | ||
| case (DoubleType, DOUBLE) => | ||
| (getter, ordinal) => getter.getDouble(ordinal) | ||
| case d: DecimalType => | ||
| 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 StringType => avroType.getType match { | ||
| case Type.ENUM => | ||
| import scala.collection.JavaConverters._ | ||
| 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 _ => | ||
| (getter, ordinal) => new Utf8(getter.getUTF8String(ordinal).getBytes) | ||
| } | ||
| case BinaryType => avroType.getType match { | ||
| case Type.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 _ => | ||
| (getter, ordinal) => ByteBuffer.wrap(getter.getBinary(ordinal)) | ||
| } | ||
| case DateType => | ||
| 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 => avroType.getLogicalType match { | ||
|
|
||
| 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, | ||
|
|
@@ -137,7 +146,7 @@ class AvroSerializer(rootCatalystType: DataType, rootAvroType: Schema, nullable: | |
| s"Cannot convert Catalyst Timestamp type to Avro logical type ${other}") | ||
| } | ||
|
|
||
| case ArrayType(et, containsNull) => | ||
| case (ArrayType(et, containsNull), ARRAY) => | ||
| val elementConverter = newConverter( | ||
| et, resolveNullableType(avroType.getElementType, containsNull)) | ||
| (getter, ordinal) => { | ||
|
|
@@ -158,12 +167,12 @@ class AvroSerializer(rootCatalystType: DataType, rootAvroType: Schema, nullable: | |
| java.util.Arrays.asList(result: _*) | ||
| } | ||
|
|
||
| case st: StructType => | ||
| 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) if kt == StringType => | ||
| case (MapType(kt, vt, valueContainsNull), MAP) if kt == StringType => | ||
| val valueConverter = newConverter( | ||
| vt, resolveNullableType(avroType.getValueType, valueContainsNull)) | ||
| (getter, ordinal) => | ||
|
|
@@ -185,12 +194,17 @@ class AvroSerializer(rootCatalystType: DataType, rootAvroType: Schema, nullable: | |
| result | ||
|
|
||
| case other => | ||
| throw new IncompatibleSchemaException(s"Unexpected type: $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) { | ||
| throw new IncompatibleSchemaException(s"Cannot convert Catalyst type $catalystStruct to " + | ||
| s"Avro type $avroStruct.") | ||
| } | ||
| val avroFields = avroStruct.getFields | ||
| assert(avroFields.size() == catalystStruct.length) | ||
| val fieldConverters = catalystStruct.zip(avroFields.asScala).map { | ||
|
|
@@ -212,7 +226,7 @@ class AvroSerializer(rootCatalystType: DataType, rootAvroType: Schema, nullable: | |
| } | ||
|
|
||
| private def resolveNullableType(avroType: Schema, nullable: Boolean): Schema = { | ||
| if (nullable) { | ||
| if (nullable && avroType.getType != NULL) { | ||
|
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This fixes a trivial bug if |
||
| // avro uses union to represent nullable type. | ||
| val fields = avroType.getTypes.asScala | ||
| assert(fields.length == 2) | ||
|
|
||
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
Uh oh!
There was an error while loading. Please reload this page.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Do we want to allow users to do casting up from catalystType to avroType? For example, catalystType float to avroType double. If so, this can be done in different PR.
Uh oh!
There was an error while loading. Please reload this page.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Personally I would like to keep it simple as this PR proposes.
If data type casting needed, users can always do it in DataFrame before writing Avro files.
But if the casting is important, we can work on it.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Yeah, if someone feels it's important, let's do it in different PR.