diff --git a/core/src/main/java/org/apache/iceberg/avro/ValueWriters.java b/core/src/main/java/org/apache/iceberg/avro/ValueWriters.java index b99b67fe7f4b..1658db96c21a 100644 --- a/core/src/main/java/org/apache/iceberg/avro/ValueWriters.java +++ b/core/src/main/java/org/apache/iceberg/avro/ValueWriters.java @@ -35,6 +35,7 @@ import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.types.TypeUtil; import org.apache.iceberg.util.DecimalUtil; +import org.apache.iceberg.util.UUIDUtil; public class ValueWriters { private ValueWriters() {} @@ -263,12 +264,8 @@ private UUIDWriter() {} @Override @SuppressWarnings("ByteBufferBackingArray") public void write(UUID uuid, Encoder encoder) throws IOException { - // TODO: direct conversion from string to byte buffer - ByteBuffer buffer = BUFFER.get(); - buffer.rewind(); - buffer.putLong(uuid.getMostSignificantBits()); - buffer.putLong(uuid.getLeastSignificantBits()); - encoder.writeFixed(buffer.array()); + // calling array() is safe because the buffer is always allocated by the thread-local + encoder.writeFixed(UUIDUtil.convertToByteBuffer(uuid, BUFFER.get()).array()); } } diff --git a/spark/v3.1/spark/src/main/java/org/apache/iceberg/spark/data/SparkValueWriters.java b/spark/v3.1/spark/src/main/java/org/apache/iceberg/spark/data/SparkValueWriters.java index 5f2e2c054888..bb8218bd83df 100644 --- a/spark/v3.1/spark/src/main/java/org/apache/iceberg/spark/data/SparkValueWriters.java +++ b/spark/v3.1/spark/src/main/java/org/apache/iceberg/spark/data/SparkValueWriters.java @@ -29,6 +29,7 @@ import org.apache.iceberg.avro.ValueWriter; import org.apache.iceberg.types.TypeUtil; import org.apache.iceberg.util.DecimalUtil; +import org.apache.iceberg.util.UUIDUtil; import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.catalyst.util.ArrayData; import org.apache.spark.sql.catalyst.util.MapData; @@ -102,11 +103,8 @@ private UUIDWriter() {} public void write(UTF8String s, Encoder encoder) throws IOException { // TODO: direct conversion from string to byte buffer UUID uuid = UUID.fromString(s.toString()); - ByteBuffer buffer = BUFFER.get(); - buffer.rewind(); - buffer.putLong(uuid.getMostSignificantBits()); - buffer.putLong(uuid.getLeastSignificantBits()); - encoder.writeFixed(buffer.array()); + // calling array() is safe because the buffer is always allocated by the thread-local + encoder.writeFixed(UUIDUtil.convertToByteBuffer(uuid, BUFFER.get()).array()); } } diff --git a/spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/data/SparkValueWriters.java b/spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/data/SparkValueWriters.java index 5f2e2c054888..bb8218bd83df 100644 --- a/spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/data/SparkValueWriters.java +++ b/spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/data/SparkValueWriters.java @@ -29,6 +29,7 @@ import org.apache.iceberg.avro.ValueWriter; import org.apache.iceberg.types.TypeUtil; import org.apache.iceberg.util.DecimalUtil; +import org.apache.iceberg.util.UUIDUtil; import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.catalyst.util.ArrayData; import org.apache.spark.sql.catalyst.util.MapData; @@ -102,11 +103,8 @@ private UUIDWriter() {} public void write(UTF8String s, Encoder encoder) throws IOException { // TODO: direct conversion from string to byte buffer UUID uuid = UUID.fromString(s.toString()); - ByteBuffer buffer = BUFFER.get(); - buffer.rewind(); - buffer.putLong(uuid.getMostSignificantBits()); - buffer.putLong(uuid.getLeastSignificantBits()); - encoder.writeFixed(buffer.array()); + // calling array() is safe because the buffer is always allocated by the thread-local + encoder.writeFixed(UUIDUtil.convertToByteBuffer(uuid, BUFFER.get()).array()); } } diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/data/SparkValueWriters.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/data/SparkValueWriters.java index 5f2e2c054888..bb8218bd83df 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/data/SparkValueWriters.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/data/SparkValueWriters.java @@ -29,6 +29,7 @@ import org.apache.iceberg.avro.ValueWriter; import org.apache.iceberg.types.TypeUtil; import org.apache.iceberg.util.DecimalUtil; +import org.apache.iceberg.util.UUIDUtil; import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.catalyst.util.ArrayData; import org.apache.spark.sql.catalyst.util.MapData; @@ -102,11 +103,8 @@ private UUIDWriter() {} public void write(UTF8String s, Encoder encoder) throws IOException { // TODO: direct conversion from string to byte buffer UUID uuid = UUID.fromString(s.toString()); - ByteBuffer buffer = BUFFER.get(); - buffer.rewind(); - buffer.putLong(uuid.getMostSignificantBits()); - buffer.putLong(uuid.getLeastSignificantBits()); - encoder.writeFixed(buffer.array()); + // calling array() is safe because the buffer is always allocated by the thread-local + encoder.writeFixed(UUIDUtil.convertToByteBuffer(uuid, BUFFER.get()).array()); } } diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/data/SparkValueWriters.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/data/SparkValueWriters.java index 5f2e2c054888..bb8218bd83df 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/data/SparkValueWriters.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/data/SparkValueWriters.java @@ -29,6 +29,7 @@ import org.apache.iceberg.avro.ValueWriter; import org.apache.iceberg.types.TypeUtil; import org.apache.iceberg.util.DecimalUtil; +import org.apache.iceberg.util.UUIDUtil; import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.catalyst.util.ArrayData; import org.apache.spark.sql.catalyst.util.MapData; @@ -102,11 +103,8 @@ private UUIDWriter() {} public void write(UTF8String s, Encoder encoder) throws IOException { // TODO: direct conversion from string to byte buffer UUID uuid = UUID.fromString(s.toString()); - ByteBuffer buffer = BUFFER.get(); - buffer.rewind(); - buffer.putLong(uuid.getMostSignificantBits()); - buffer.putLong(uuid.getLeastSignificantBits()); - encoder.writeFixed(buffer.array()); + // calling array() is safe because the buffer is always allocated by the thread-local + encoder.writeFixed(UUIDUtil.convertToByteBuffer(uuid, BUFFER.get()).array()); } }