diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/UnsafeHelper.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/UnsafeHelper.java new file mode 100644 index 000000000000..6b5b3b71aaab --- /dev/null +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/UnsafeHelper.java @@ -0,0 +1,54 @@ +/* + * 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.unsafe; + +import java.nio.ByteBuffer; + +public final class UnsafeHelper { + private UnsafeHelper() {} + + public static void writeTo( + ByteBuffer buffer, + Object baseObject, + long baseOffset, + int sizeInBytes) { + assert(buffer.hasArray()); + byte[] target = buffer.array(); + int offset = buffer.arrayOffset(); + int pos = buffer.position(); + writeToMemory(baseObject, baseOffset, target, + Platform.BYTE_ARRAY_OFFSET + offset + pos, sizeInBytes); + buffer.position(pos + sizeInBytes); + } + + public static void writeToMemory( + Object baseObject, + long baseOffset, + Object target, + long targetOffset, + int sizeInBytes) { + Platform.copyMemory(baseObject, baseOffset, target, targetOffset, sizeInBytes); + } + + public static byte[] copyToMemory(Object baseObject, long baseOffset, int sizeInBytes) { + final byte[] arrayDataCopy = new byte[sizeInBytes]; + Platform.copyMemory(baseObject, baseOffset, arrayDataCopy, Platform.BYTE_ARRAY_OFFSET, + sizeInBytes); + return arrayDataCopy; + } +} diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java index 3a3bfc4a94bb..3f848251a50a 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java @@ -32,6 +32,7 @@ import com.google.common.primitives.Ints; import org.apache.spark.unsafe.Platform; +import org.apache.spark.unsafe.UnsafeHelper; import org.apache.spark.unsafe.array.ByteArrayMethods; import org.apache.spark.unsafe.hash.Murmur3_x86_32; @@ -168,16 +169,11 @@ public UTF8String() { * bytes in this string. */ public void writeToMemory(Object target, long targetOffset) { - Platform.copyMemory(base, offset, target, targetOffset, numBytes); + UnsafeHelper.writeToMemory(base, offset, target, targetOffset, numBytes); } public void writeTo(ByteBuffer buffer) { - assert(buffer.hasArray()); - byte[] target = buffer.array(); - int offset = buffer.arrayOffset(); - int pos = buffer.position(); - writeToMemory(target, Platform.BYTE_ARRAY_OFFSET + offset + pos); - buffer.position(pos + numBytes); + UnsafeHelper.writeTo(buffer, base, offset, numBytes); } /** diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/SqlTypesUnsafeHelper.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/SqlTypesUnsafeHelper.java new file mode 100644 index 000000000000..9018832f6de2 --- /dev/null +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/SqlTypesUnsafeHelper.java @@ -0,0 +1,104 @@ +/* + * 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.catalyst.expressions; + +import org.apache.spark.sql.types.Decimal; +import org.apache.spark.unsafe.Platform; +import org.apache.spark.unsafe.types.CalendarInterval; +import org.apache.spark.unsafe.types.UTF8String; + +import java.math.BigDecimal; +import java.math.BigInteger; + +public final class SqlTypesUnsafeHelper { + private SqlTypesUnsafeHelper() {} + + public static byte[] getBinary(long offsetAndSize, Object baseObject, long baseOffset) { + final int offset = getOffsetFromOffsetAndSize(offsetAndSize); + final int size = getSizeFromOffsetAndSize(offsetAndSize); + final byte[] bytes = new byte[size]; + Platform.copyMemory(baseObject, baseOffset + offset, bytes, Platform.BYTE_ARRAY_OFFSET, size); + return bytes; + } + + public static UTF8String getUTF8String(long offsetAndSize, Object baseObject, long baseOffset) { + final int offset = getOffsetFromOffsetAndSize(offsetAndSize); + final int size = getSizeFromOffsetAndSize(offsetAndSize); + return UTF8String.fromAddress(baseObject, baseOffset + offset, size); + } + + public static CalendarInterval getInterval( + long offsetAndSize, + Object baseObject, + long baseOffset) { + final int offset = getOffsetFromOffsetAndSize(offsetAndSize); + final int months = (int) Platform.getLong(baseObject, baseOffset + offset); + final long microseconds = Platform.getLong(baseObject, baseOffset + offset + 8); + return new CalendarInterval(months, microseconds); + } + + public static Decimal getDecimalExceedingLong( + byte[] bytes, + int precision, + int scale, + boolean wrapWithScalaBigDecimal) { + final BigInteger bigInteger = new BigInteger(bytes); + BigDecimal decimal = new BigDecimal(bigInteger, scale); + if (wrapWithScalaBigDecimal) { + return Decimal.apply(new scala.math.BigDecimal(decimal), precision, scale); + } else { + return Decimal.apply(decimal, precision, scale); + } + } + + public static UnsafeRow getStruct( + long offsetAndSize, + Object baseObject, + long baseOffset, + int numFields) { + final int offset = getOffsetFromOffsetAndSize(offsetAndSize); + final int size = getSizeFromOffsetAndSize(offsetAndSize); + final UnsafeRow row = new UnsafeRow(numFields); + row.pointTo(baseObject, baseOffset + offset, size); + return row; + } + + public static UnsafeArrayData getArray(long offsetAndSize, Object baseObject, long baseOffset) { + final int offset = getOffsetFromOffsetAndSize(offsetAndSize); + final int size = getSizeFromOffsetAndSize(offsetAndSize); + final UnsafeArrayData array = new UnsafeArrayData(); + array.pointTo(baseObject, baseOffset + offset, size); + return array; + } + + public static UnsafeMapData getMap(long offsetAndSize, Object baseObject, long baseOffset) { + final int offset = getOffsetFromOffsetAndSize(offsetAndSize); + final int size = getSizeFromOffsetAndSize(offsetAndSize); + final UnsafeMapData map = new UnsafeMapData(); + map.pointTo(baseObject, baseOffset + offset, size); + return map; + } + + public static int getOffsetFromOffsetAndSize(long offsetAndSize) { + return (int) (offsetAndSize >> 32); + } + + public static int getSizeFromOffsetAndSize(long offsetAndSize) { + return (int) offsetAndSize; + } +} diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeArrayData.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeArrayData.java index d5f679fe23d4..bec3f89f5686 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeArrayData.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeArrayData.java @@ -17,13 +17,12 @@ package org.apache.spark.sql.catalyst.expressions; -import java.math.BigDecimal; -import java.math.BigInteger; import java.nio.ByteBuffer; import org.apache.spark.sql.catalyst.util.ArrayData; import org.apache.spark.sql.types.*; import org.apache.spark.unsafe.Platform; +import org.apache.spark.unsafe.UnsafeHelper; import org.apache.spark.unsafe.array.ByteArrayMethods; import org.apache.spark.unsafe.bitset.BitSetMethods; import org.apache.spark.unsafe.hash.Murmur3_x86_32; @@ -227,74 +226,45 @@ public Decimal getDecimal(int ordinal, int precision, int scale) { if (precision <= Decimal.MAX_LONG_DIGITS()) { return Decimal.apply(getLong(ordinal), precision, scale); } else { - final byte[] bytes = getBinary(ordinal); - final BigInteger bigInteger = new BigInteger(bytes); - final BigDecimal javaDecimal = new BigDecimal(bigInteger, scale); - return Decimal.apply(new scala.math.BigDecimal(javaDecimal), precision, scale); + return SqlTypesUnsafeHelper.getDecimalExceedingLong(getBinary(ordinal), precision, scale, + true); } } @Override public UTF8String getUTF8String(int ordinal) { if (isNullAt(ordinal)) return null; - final long offsetAndSize = getLong(ordinal); - final int offset = (int) (offsetAndSize >> 32); - final int size = (int) offsetAndSize; - return UTF8String.fromAddress(baseObject, baseOffset + offset, size); + return SqlTypesUnsafeHelper.getUTF8String(getLong(ordinal), baseObject, baseOffset); } @Override public byte[] getBinary(int ordinal) { if (isNullAt(ordinal)) return null; - final long offsetAndSize = getLong(ordinal); - final int offset = (int) (offsetAndSize >> 32); - final int size = (int) offsetAndSize; - final byte[] bytes = new byte[size]; - Platform.copyMemory(baseObject, baseOffset + offset, bytes, Platform.BYTE_ARRAY_OFFSET, size); - return bytes; + return SqlTypesUnsafeHelper.getBinary(getLong(ordinal), baseObject, baseOffset); } @Override public CalendarInterval getInterval(int ordinal) { if (isNullAt(ordinal)) return null; - final long offsetAndSize = getLong(ordinal); - final int offset = (int) (offsetAndSize >> 32); - final int months = (int) Platform.getLong(baseObject, baseOffset + offset); - final long microseconds = Platform.getLong(baseObject, baseOffset + offset + 8); - return new CalendarInterval(months, microseconds); + return SqlTypesUnsafeHelper.getInterval(getLong(ordinal), baseObject, baseOffset); } @Override public UnsafeRow getStruct(int ordinal, int numFields) { if (isNullAt(ordinal)) return null; - final long offsetAndSize = getLong(ordinal); - final int offset = (int) (offsetAndSize >> 32); - final int size = (int) offsetAndSize; - final UnsafeRow row = new UnsafeRow(numFields); - row.pointTo(baseObject, baseOffset + offset, size); - return row; + return SqlTypesUnsafeHelper.getStruct(getLong(ordinal), baseObject, baseOffset, numFields); } @Override public UnsafeArrayData getArray(int ordinal) { if (isNullAt(ordinal)) return null; - final long offsetAndSize = getLong(ordinal); - final int offset = (int) (offsetAndSize >> 32); - final int size = (int) offsetAndSize; - final UnsafeArrayData array = new UnsafeArrayData(); - array.pointTo(baseObject, baseOffset + offset, size); - return array; + return SqlTypesUnsafeHelper.getArray(getLong(ordinal), baseObject, baseOffset); } @Override public UnsafeMapData getMap(int ordinal) { if (isNullAt(ordinal)) return null; - final long offsetAndSize = getLong(ordinal); - final int offset = (int) (offsetAndSize >> 32); - final int size = (int) offsetAndSize; - final UnsafeMapData map = new UnsafeMapData(); - map.pointTo(baseObject, baseOffset + offset, size); - return map; + return SqlTypesUnsafeHelper.getMap(getLong(ordinal), baseObject, baseOffset); } @Override @@ -363,25 +333,14 @@ public boolean equals(Object other) { return false; } - public void writeToMemory(Object target, long targetOffset) { - Platform.copyMemory(baseObject, baseOffset, target, targetOffset, sizeInBytes); - } - public void writeTo(ByteBuffer buffer) { - assert(buffer.hasArray()); - byte[] target = buffer.array(); - int offset = buffer.arrayOffset(); - int pos = buffer.position(); - writeToMemory(target, Platform.BYTE_ARRAY_OFFSET + offset + pos); - buffer.position(pos + sizeInBytes); + UnsafeHelper.writeTo(buffer, baseObject, baseOffset, sizeInBytes); } @Override public UnsafeArrayData copy() { UnsafeArrayData arrayCopy = new UnsafeArrayData(); - final byte[] arrayDataCopy = new byte[sizeInBytes]; - Platform.copyMemory( - baseObject, baseOffset, arrayDataCopy, Platform.BYTE_ARRAY_OFFSET, sizeInBytes); + final byte[] arrayDataCopy = UnsafeHelper.copyToMemory(baseObject, baseOffset, sizeInBytes); arrayCopy.pointTo(arrayDataCopy, Platform.BYTE_ARRAY_OFFSET, sizeInBytes); return arrayCopy; } @@ -447,14 +406,7 @@ public static UnsafeArrayData fromPrimitiveArray( final long headerInBytes = calculateHeaderPortionInBytes(length); final long valueRegionInBytes = (long)elementSize * length; final long totalSizeInLongs = (headerInBytes + valueRegionInBytes + 7) / 8; - if (totalSizeInLongs > Integer.MAX_VALUE / 8) { - throw new UnsupportedOperationException("Cannot convert this array to unsafe format as " + - "it's too big."); - } - - final long[] data = new long[(int)totalSizeInLongs]; - - Platform.putLong(data, Platform.LONG_ARRAY_OFFSET, length); + final long[] data = initializeUnderlyingArray(length, totalSizeInLongs); if (arr != null) { Platform.copyMemory(arr, offset, data, Platform.LONG_ARRAY_OFFSET + headerInBytes, valueRegionInBytes); @@ -466,28 +418,32 @@ public static UnsafeArrayData fromPrimitiveArray( } public static UnsafeArrayData createFreshArray(int length, int elementSize) { - final long headerInBytes = calculateHeaderPortionInBytes(length); - final long valueRegionInBytes = (long)elementSize * length; - final long totalSizeInLongs = (headerInBytes + valueRegionInBytes + 7) / 8; + final long totalSizeInLongs = getTotalSize(elementSize, length); + final long[] data = initializeUnderlyingArray(length, totalSizeInLongs); + UnsafeArrayData result = new UnsafeArrayData(); + result.pointTo(data, Platform.LONG_ARRAY_OFFSET, (int)totalSizeInLongs * 8); + return result; + } + + private static long[] initializeUnderlyingArray(int length, long totalSizeInLongs) { if (totalSizeInLongs > Integer.MAX_VALUE / 8) { throw new UnsupportedOperationException("Cannot convert this array to unsafe format as " + - "it's too big."); + "it's too big."); } final long[] data = new long[(int)totalSizeInLongs]; - Platform.putLong(data, Platform.LONG_ARRAY_OFFSET, length); - - UnsafeArrayData result = new UnsafeArrayData(); - result.pointTo(data, Platform.LONG_ARRAY_OFFSET, (int)totalSizeInLongs * 8); - return result; + return data; } - public static boolean shouldUseGenericArrayData(int elementSize, long length) { + private static long getTotalSize(int elementSize, long length) { final long headerInBytes = calculateHeaderPortionInBytes(length); final long valueRegionInBytes = elementSize * length; - final long totalSizeInLongs = (headerInBytes + valueRegionInBytes + 7) / 8; - return totalSizeInLongs > Integer.MAX_VALUE / 8; + return (headerInBytes + valueRegionInBytes + 7) / 8; + } + + public static boolean shouldUseGenericArrayData(int elementSize, long length) { + return getTotalSize(elementSize, length) > Integer.MAX_VALUE / 8; } public static UnsafeArrayData fromPrimitiveArray(boolean[] arr) { diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeMapData.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeMapData.java index a0833a6df8bb..d01d4aca1a2b 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeMapData.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeMapData.java @@ -21,6 +21,7 @@ import org.apache.spark.sql.catalyst.util.MapData; import org.apache.spark.unsafe.Platform; +import org.apache.spark.unsafe.UnsafeHelper; /** * An Unsafe implementation of Map which is backed by raw memory instead of Java objects. @@ -101,25 +102,14 @@ public UnsafeArrayData valueArray() { return values; } - public void writeToMemory(Object target, long targetOffset) { - Platform.copyMemory(baseObject, baseOffset, target, targetOffset, sizeInBytes); - } - public void writeTo(ByteBuffer buffer) { - assert(buffer.hasArray()); - byte[] target = buffer.array(); - int offset = buffer.arrayOffset(); - int pos = buffer.position(); - writeToMemory(target, Platform.BYTE_ARRAY_OFFSET + offset + pos); - buffer.position(pos + sizeInBytes); + UnsafeHelper.writeTo(buffer, baseObject, baseOffset, sizeInBytes); } @Override public UnsafeMapData copy() { UnsafeMapData mapCopy = new UnsafeMapData(); - final byte[] mapDataCopy = new byte[sizeInBytes]; - Platform.copyMemory( - baseObject, baseOffset, mapDataCopy, Platform.BYTE_ARRAY_OFFSET, sizeInBytes); + final byte[] mapDataCopy = UnsafeHelper.copyToMemory(baseObject, baseOffset, sizeInBytes); mapCopy.pointTo(mapDataCopy, Platform.BYTE_ARRAY_OFFSET, sizeInBytes); return mapCopy; } diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeRow.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeRow.java index 9bf9452855f5..9d60e7020b45 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeRow.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeRow.java @@ -18,7 +18,6 @@ package org.apache.spark.sql.catalyst.expressions; import java.io.*; -import java.math.BigDecimal; import java.math.BigInteger; import java.nio.ByteBuffer; import java.util.Arrays; @@ -34,6 +33,7 @@ import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.types.*; import org.apache.spark.unsafe.Platform; +import org.apache.spark.unsafe.UnsafeHelper; import org.apache.spark.unsafe.array.ByteArrayMethods; import org.apache.spark.unsafe.bitset.BitSetMethods; import org.apache.spark.unsafe.hash.Murmur3_x86_32; @@ -397,20 +397,15 @@ public Decimal getDecimal(int ordinal, int precision, int scale) { if (precision <= Decimal.MAX_LONG_DIGITS()) { return Decimal.createUnsafe(getLong(ordinal), precision, scale); } else { - byte[] bytes = getBinary(ordinal); - BigInteger bigInteger = new BigInteger(bytes); - BigDecimal javaDecimal = new BigDecimal(bigInteger, scale); - return Decimal.apply(javaDecimal, precision, scale); + return SqlTypesUnsafeHelper.getDecimalExceedingLong(getBinary(ordinal), precision, scale, + false); } } @Override public UTF8String getUTF8String(int ordinal) { if (isNullAt(ordinal)) return null; - final long offsetAndSize = getLong(ordinal); - final int offset = (int) (offsetAndSize >> 32); - final int size = (int) offsetAndSize; - return UTF8String.fromAddress(baseObject, baseOffset + offset, size); + return SqlTypesUnsafeHelper.getUTF8String(getLong(ordinal), baseObject, baseOffset); } @Override @@ -418,18 +413,7 @@ public byte[] getBinary(int ordinal) { if (isNullAt(ordinal)) { return null; } else { - final long offsetAndSize = getLong(ordinal); - final int offset = (int) (offsetAndSize >> 32); - final int size = (int) offsetAndSize; - final byte[] bytes = new byte[size]; - Platform.copyMemory( - baseObject, - baseOffset + offset, - bytes, - Platform.BYTE_ARRAY_OFFSET, - size - ); - return bytes; + return SqlTypesUnsafeHelper.getBinary(getLong(ordinal), baseObject, baseOffset); } } @@ -438,11 +422,7 @@ public CalendarInterval getInterval(int ordinal) { if (isNullAt(ordinal)) { return null; } else { - final long offsetAndSize = getLong(ordinal); - final int offset = (int) (offsetAndSize >> 32); - final int months = (int) Platform.getLong(baseObject, baseOffset + offset); - final long microseconds = Platform.getLong(baseObject, baseOffset + offset + 8); - return new CalendarInterval(months, microseconds); + return SqlTypesUnsafeHelper.getInterval(getLong(ordinal), baseObject, baseOffset); } } @@ -451,12 +431,7 @@ public UnsafeRow getStruct(int ordinal, int numFields) { if (isNullAt(ordinal)) { return null; } else { - final long offsetAndSize = getLong(ordinal); - final int offset = (int) (offsetAndSize >> 32); - final int size = (int) offsetAndSize; - final UnsafeRow row = new UnsafeRow(numFields); - row.pointTo(baseObject, baseOffset + offset, size); - return row; + return SqlTypesUnsafeHelper.getStruct(getLong(ordinal), baseObject, baseOffset, numFields); } } @@ -465,12 +440,7 @@ public UnsafeArrayData getArray(int ordinal) { if (isNullAt(ordinal)) { return null; } else { - final long offsetAndSize = getLong(ordinal); - final int offset = (int) (offsetAndSize >> 32); - final int size = (int) offsetAndSize; - final UnsafeArrayData array = new UnsafeArrayData(); - array.pointTo(baseObject, baseOffset + offset, size); - return array; + return SqlTypesUnsafeHelper.getArray(getLong(ordinal), baseObject, baseOffset); } } @@ -479,12 +449,7 @@ public UnsafeMapData getMap(int ordinal) { if (isNullAt(ordinal)) { return null; } else { - final long offsetAndSize = getLong(ordinal); - final int offset = (int) (offsetAndSize >> 32); - final int size = (int) offsetAndSize; - final UnsafeMapData map = new UnsafeMapData(); - map.pointTo(baseObject, baseOffset + offset, size); - return map; + return SqlTypesUnsafeHelper.getMap(getLong(ordinal), baseObject, baseOffset); } } @@ -495,14 +460,7 @@ public UnsafeMapData getMap(int ordinal) { @Override public UnsafeRow copy() { UnsafeRow rowCopy = new UnsafeRow(numFields); - final byte[] rowDataCopy = new byte[sizeInBytes]; - Platform.copyMemory( - baseObject, - baseOffset, - rowDataCopy, - Platform.BYTE_ARRAY_OFFSET, - sizeInBytes - ); + final byte[] rowDataCopy = UnsafeHelper.copyToMemory(baseObject, baseOffset, sizeInBytes); rowCopy.pointTo(rowDataCopy, Platform.BYTE_ARRAY_OFFSET, sizeInBytes); return rowCopy; } @@ -632,8 +590,8 @@ public void writeTo(ByteBuffer buffer) { */ public void writeFieldTo(int ordinal, ByteBuffer buffer) { final long offsetAndSize = getLong(ordinal); - final int offset = (int) (offsetAndSize >> 32); - final int size = (int) offsetAndSize; + final int offset = SqlTypesUnsafeHelper.getOffsetFromOffsetAndSize(offsetAndSize); + final int size = SqlTypesUnsafeHelper.getSizeFromOffsetAndSize(offsetAndSize); buffer.putInt(size); int pos = buffer.position();