Skip to content
Closed
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 @@ -195,6 +195,8 @@ private[sql] class AvroDeserializer(
case b: ByteBuffer =>
val bytes = new Array[Byte](b.remaining)
b.get(bytes)
// Do not forget to reset the position
b.rewind()
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think you're right but can you explain why this is needed?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

HeapBuffer.get(bytes) puts the data from POS to the end into bytes, and sets POS as the end. The next call will return empty bytes. You can take a look at added unit test. The second call of deserializer will return an InternalRow with empty binary column.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Sounds good. I wonder why this never surfaced before? seems like it would mean any binary cols in Avro don't work?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Maybe this is not common to call this twice to deserialize one avro data object

bytes
case b: Array[Byte] => b
case other =>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -360,4 +360,25 @@ class AvroCatalystDataConversionSuite extends SparkFunSuite
None,
new OrderedFilters(Seq(Not(EqualTo("Age", 39))), sqlSchema))
}

test("AvroDeserializer with binary type") {
val jsonFormatSchema =
"""
|{
| "type": "record",
| "name": "record",
| "fields" : [
| {"name": "a", "type": "bytes"}
| ]
|}
""".stripMargin
val avroSchema = new Schema.Parser().parse(jsonFormatSchema)
val avroRecord = new GenericData.Record(avroSchema)
val bb = java.nio.ByteBuffer.wrap(Array[Byte](97, 48, 53))
avroRecord.put("a", bb)

val expected = InternalRow(Array[Byte](97, 48, 53))
checkDeserialization(avroSchema, avroRecord, Some(expected))
checkDeserialization(avroSchema, avroRecord, Some(expected))
}
}