From fb9a42d2f7c72c6320608157a5765dafbb06122c Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Wed, 15 Jun 2016 04:46:29 +0900 Subject: [PATCH 01/37] add two implementations (sparse and dense) for UnsafeArrayData --- .../catalyst/expressions/UnsafeArrayData.java | 319 +++--------------- .../expressions/UnsafeArrayDataDense.java | 205 +++++++++++ .../expressions/UnsafeArrayDataSparse.java | 291 ++++++++++++++++ .../catalyst/expressions/UnsafeMapData.java | 4 +- .../sql/catalyst/expressions/UnsafeRow.java | 2 +- .../sql/execution/columnar/ColumnType.scala | 2 +- 6 files changed, 545 insertions(+), 278 deletions(-) create mode 100644 sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeArrayDataDense.java create mode 100644 sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeArrayDataSparse.java 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 6302660548ec1..09dbe5d1cdba9 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,68 +17,36 @@ 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.array.ByteArrayMethods; import org.apache.spark.unsafe.hash.Murmur3_x86_32; -import org.apache.spark.unsafe.types.CalendarInterval; -import org.apache.spark.unsafe.types.UTF8String; -/** - * An Unsafe implementation of Array which is backed by raw memory instead of Java objects. - * - * Each tuple has three parts: [numElements] [offsets] [values] - * - * The `numElements` is 4 bytes storing the number of elements of this array. - * - * In the `offsets` region, we store 4 bytes per element, represents the relative offset (w.r.t. the - * base address of the array) of this element in `values` region. We can get the length of this - * element by subtracting next offset. - * Note that offset can by negative which means this element is null. - * - * In the `values` region, we store the content of elements. As we can get length info, so elements - * can be variable-length. - * - * Instances of `UnsafeArrayData` act as pointers to row data stored in this format. - */ +import java.nio.ByteBuffer; + + // todo: there is a lof of duplicated code between UnsafeRow and UnsafeArrayData. -public final class UnsafeArrayData extends ArrayData { +public abstract class UnsafeArrayData extends ArrayData { + public enum Format { + Sparse, Dense + } + + protected Format format; - private Object baseObject; - private long baseOffset; + protected Object baseObject; + protected long baseOffset; // The number of elements in this array - private int numElements; + protected int numElements; // The size of this array's backing data, in bytes. // The 4-bytes header of `numElements` is also included. - private int sizeInBytes; + protected int sizeInBytes; - public Object getBaseObject() { return baseObject; } - public long getBaseOffset() { return baseOffset; } - public int getSizeInBytes() { return sizeInBytes; } - - private int getElementOffset(int ordinal) { - return Platform.getInt(baseObject, baseOffset + 4 + ordinal * 4L); - } - - private int getElementSize(int offset, int ordinal) { - if (ordinal == numElements - 1) { - return sizeInBytes - offset; - } else { - return Math.abs(getElementOffset(ordinal + 1)) - offset; - } - } - - private void assertIndexIsValid(int ordinal) { - assert ordinal >= 0 : "ordinal (" + ordinal + ") should >= 0"; - assert ordinal < numElements : "ordinal (" + ordinal + ") should < " + numElements; - } + public final Object getBaseObject() { return baseObject; } + public final long getBaseOffset() { return baseOffset; } + public final int getSizeInBytes() { return sizeInBytes; } public Object[] array() { throw new UnsupportedOperationException("Not supported on UnsafeArrayData."); @@ -89,10 +57,18 @@ public Object[] array() { * `pointTo()` has been called, since the value returned by this constructor is equivalent * to a null pointer. */ - public UnsafeArrayData() { } + public static final UnsafeArrayData allocate(Format format) { + if (format == Format.Sparse) { + return new UnsafeArrayDataSparse(); + } else { + return new UnsafeArrayDataDense(); + } + } + + public UnsafeArrayData() { } @Override - public int numElements() { return numElements; } + public final int numElements() { return numElements; } /** * Update this UnsafeArrayData to point to different backing data. @@ -101,25 +77,10 @@ public UnsafeArrayData() { } * @param baseOffset the offset within the base object * @param sizeInBytes the size of this array's backing data, in bytes */ - public void pointTo(Object baseObject, long baseOffset, int sizeInBytes) { - // Read the number of elements from the first 4 bytes. - final int numElements = Platform.getInt(baseObject, baseOffset); - assert numElements >= 0 : "numElements (" + numElements + ") should >= 0"; - - this.numElements = numElements; - this.baseObject = baseObject; - this.baseOffset = baseOffset; - this.sizeInBytes = sizeInBytes; - } - - @Override - public boolean isNullAt(int ordinal) { - assertIndexIsValid(ordinal); - return getElementOffset(ordinal) < 0; - } + public abstract void pointTo(Object baseObject, long baseOffset, int sizeInBytes); @Override - public Object get(int ordinal, DataType dataType) { + public final Object get(int ordinal, DataType dataType) { if (isNullAt(ordinal) || dataType instanceof NullType) { return null; } else if (dataType instanceof BooleanType) { @@ -162,140 +123,23 @@ public Object get(int ordinal, DataType dataType) { } } - @Override - public boolean getBoolean(int ordinal) { - assertIndexIsValid(ordinal); - final int offset = getElementOffset(ordinal); - if (offset < 0) return false; - return Platform.getBoolean(baseObject, baseOffset + offset); - } - - @Override - public byte getByte(int ordinal) { - assertIndexIsValid(ordinal); - final int offset = getElementOffset(ordinal); - if (offset < 0) return 0; - return Platform.getByte(baseObject, baseOffset + offset); - } - - @Override - public short getShort(int ordinal) { - assertIndexIsValid(ordinal); - final int offset = getElementOffset(ordinal); - if (offset < 0) return 0; - return Platform.getShort(baseObject, baseOffset + offset); - } - - @Override - public int getInt(int ordinal) { - assertIndexIsValid(ordinal); - final int offset = getElementOffset(ordinal); - if (offset < 0) return 0; - return Platform.getInt(baseObject, baseOffset + offset); - } + public abstract UnsafeRow getStruct(int ordinal, int numFields); - @Override - public long getLong(int ordinal) { - assertIndexIsValid(ordinal); - final int offset = getElementOffset(ordinal); - if (offset < 0) return 0; - return Platform.getLong(baseObject, baseOffset + offset); - } + public abstract UnsafeArrayData getArray(int ordinal); - @Override - public float getFloat(int ordinal) { - assertIndexIsValid(ordinal); - final int offset = getElementOffset(ordinal); - if (offset < 0) return 0; - return Platform.getFloat(baseObject, baseOffset + offset); - } + public abstract UnsafeMapData getMap(int ordinal); - @Override - public double getDouble(int ordinal) { - assertIndexIsValid(ordinal); - final int offset = getElementOffset(ordinal); - if (offset < 0) return 0; - return Platform.getDouble(baseObject, baseOffset + offset); - } - - @Override - public Decimal getDecimal(int ordinal, int precision, int scale) { - assertIndexIsValid(ordinal); - final int offset = getElementOffset(ordinal); - if (offset < 0) return null; - - if (precision <= Decimal.MAX_LONG_DIGITS()) { - final long value = Platform.getLong(baseObject, baseOffset + offset); - return Decimal.apply(value, 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); - } - } - - @Override - public UTF8String getUTF8String(int ordinal) { - assertIndexIsValid(ordinal); - final int offset = getElementOffset(ordinal); - if (offset < 0) return null; - final int size = getElementSize(offset, ordinal); - return UTF8String.fromAddress(baseObject, baseOffset + offset, size); - } - - @Override - public byte[] getBinary(int ordinal) { - assertIndexIsValid(ordinal); - final int offset = getElementOffset(ordinal); - if (offset < 0) return null; - final int size = getElementSize(offset, ordinal); - final byte[] bytes = new byte[size]; - Platform.copyMemory(baseObject, baseOffset + offset, bytes, Platform.BYTE_ARRAY_OFFSET, size); - return bytes; - } - - @Override - public CalendarInterval getInterval(int ordinal) { - assertIndexIsValid(ordinal); - final int offset = getElementOffset(ordinal); - if (offset < 0) return null; - final int months = (int) Platform.getLong(baseObject, baseOffset + offset); - final long microseconds = Platform.getLong(baseObject, baseOffset + offset + 8); - return new CalendarInterval(months, microseconds); - } - - @Override - public UnsafeRow getStruct(int ordinal, int numFields) { - assertIndexIsValid(ordinal); - final int offset = getElementOffset(ordinal); - if (offset < 0) return null; - final int size = getElementSize(offset, ordinal); - final UnsafeRow row = new UnsafeRow(numFields); - row.pointTo(baseObject, baseOffset + offset, size); - return row; - } - - @Override - public UnsafeArrayData getArray(int ordinal) { - assertIndexIsValid(ordinal); - final int offset = getElementOffset(ordinal); - if (offset < 0) return null; - final int size = getElementSize(offset, ordinal); - final UnsafeArrayData array = new UnsafeArrayData(); - array.pointTo(baseObject, baseOffset + offset, size); - return array; + public final void writeToMemory(Object target, long targetOffset) { + Platform.copyMemory(baseObject, baseOffset, target, targetOffset, sizeInBytes); } - @Override - public UnsafeMapData getMap(int ordinal) { - assertIndexIsValid(ordinal); - final int offset = getElementOffset(ordinal); - if (offset < 0) return null; - final int size = getElementSize(offset, ordinal); - final UnsafeMapData map = new UnsafeMapData(); - map.pointTo(baseObject, baseOffset + offset, size); - return map; + public final 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); } // This `hashCode` computation could consume much processor time for large data. @@ -312,92 +156,19 @@ public boolean equals(Object other) { if (other instanceof UnsafeArrayData) { UnsafeArrayData o = (UnsafeArrayData) other; return (sizeInBytes == o.sizeInBytes) && - ByteArrayMethods.arrayEquals(baseObject, baseOffset, o.baseObject, o.baseOffset, - sizeInBytes); + ByteArrayMethods.arrayEquals(baseObject, baseOffset, o.baseObject, o.baseOffset, + sizeInBytes); } 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); - } - - @Override - public UnsafeArrayData copy() { - UnsafeArrayData arrayCopy = new UnsafeArrayData(); - final byte[] arrayDataCopy = new byte[sizeInBytes]; - Platform.copyMemory( - baseObject, baseOffset, arrayDataCopy, Platform.BYTE_ARRAY_OFFSET, sizeInBytes); - arrayCopy.pointTo(arrayDataCopy, Platform.BYTE_ARRAY_OFFSET, sizeInBytes); - return arrayCopy; - } + public abstract UnsafeArrayData copy(); public static UnsafeArrayData fromPrimitiveArray(int[] arr) { - if (arr.length > (Integer.MAX_VALUE - 4) / 8) { - throw new UnsupportedOperationException("Cannot convert this array to unsafe format as " + - "it's too big."); - } - - final int offsetRegionSize = 4 * arr.length; - final int valueRegionSize = 4 * arr.length; - final int totalSize = 4 + offsetRegionSize + valueRegionSize; - final byte[] data = new byte[totalSize]; - - Platform.putInt(data, Platform.BYTE_ARRAY_OFFSET, arr.length); - - int offsetPosition = Platform.BYTE_ARRAY_OFFSET + 4; - int valueOffset = 4 + offsetRegionSize; - for (int i = 0; i < arr.length; i++) { - Platform.putInt(data, offsetPosition, valueOffset); - offsetPosition += 4; - valueOffset += 4; - } - - Platform.copyMemory(arr, Platform.INT_ARRAY_OFFSET, data, - Platform.BYTE_ARRAY_OFFSET + 4 + offsetRegionSize, valueRegionSize); - - UnsafeArrayData result = new UnsafeArrayData(); - result.pointTo(data, Platform.BYTE_ARRAY_OFFSET, totalSize); - return result; + return UnsafeArrayDataDense._fromPrimitiveArray(arr); } public static UnsafeArrayData fromPrimitiveArray(double[] arr) { - if (arr.length > (Integer.MAX_VALUE - 4) / 12) { - throw new UnsupportedOperationException("Cannot convert this array to unsafe format as " + - "it's too big."); - } - - final int offsetRegionSize = 4 * arr.length; - final int valueRegionSize = 8 * arr.length; - final int totalSize = 4 + offsetRegionSize + valueRegionSize; - final byte[] data = new byte[totalSize]; - - Platform.putInt(data, Platform.BYTE_ARRAY_OFFSET, arr.length); - - int offsetPosition = Platform.BYTE_ARRAY_OFFSET + 4; - int valueOffset = 4 + offsetRegionSize; - for (int i = 0; i < arr.length; i++) { - Platform.putInt(data, offsetPosition, valueOffset); - offsetPosition += 4; - valueOffset += 8; - } - - Platform.copyMemory(arr, Platform.DOUBLE_ARRAY_OFFSET, data, - Platform.BYTE_ARRAY_OFFSET + 4 + offsetRegionSize, valueRegionSize); - - UnsafeArrayData result = new UnsafeArrayData(); - result.pointTo(data, Platform.BYTE_ARRAY_OFFSET, totalSize); - return result; + return UnsafeArrayDataDense._fromPrimitiveArray(arr); } - - // TODO: add more specialized methods. } diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeArrayDataDense.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeArrayDataDense.java new file mode 100644 index 0000000000000..196ba2251af98 --- /dev/null +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeArrayDataDense.java @@ -0,0 +1,205 @@ +/* + * 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; + +/** + * An Unsafe implementation of Array which is backed by raw memory instead of Java objects. + * This array assumes that each element is non-null + * + * Each tuple has three parts: [numElements] [0] [values] + * + * The `numElements` is 4 bytes storing the number of elements of this array. + * + * [0] is a 4-byte identifier for a dense array + * + * In the `values` region, we store the content of elements. As we can get length info, so elements + * can be variable-length. + * + * Instances of `UnsafeArrayDataDense` act as pointers to row data stored in this format. + */ +public final class UnsafeArrayDataDense extends UnsafeArrayData { + + protected UnsafeArrayDataDense() { + format = Format.Dense; + } + + public void pointTo(Object baseObject, long baseOffset, int sizeInBytes) { + // Read the number of elements from the first 4 bytes. + final int numElements = Platform.getInt(baseObject, baseOffset); + assert numElements >= 0 : "numElements (" + numElements + ") should >= 0"; + assert(Platform.getInt(baseObject, baseOffset + 4) == 0); + + this.numElements = numElements; + this.baseObject = baseObject; + this.baseOffset = baseOffset; + this.sizeInBytes = sizeInBytes; + } + + private void assertIndexIsValid(int ordinal) { + assert ordinal >= 0 : "ordinal (" + ordinal + ") should >= 0"; + assert ordinal < numElements : "ordinal (" + ordinal + ") should < " + numElements; + } + + @Override + public boolean isNullAt(int ordinal) { + return false; + } + + @Override + public boolean getBoolean(int ordinal) { + assertIndexIsValid(ordinal); + return Platform.getBoolean(baseObject, baseOffset + 4 + ordinal); + } + + @Override + public byte getByte(int ordinal) { + assertIndexIsValid(ordinal); + return Platform.getByte(baseObject, baseOffset + 4 + ordinal); + } + + @Override + public short getShort(int ordinal) { + assertIndexIsValid(ordinal); + return Platform.getShort(baseObject, baseOffset + 4 + ordinal * 2); + } + + @Override + public int getInt(int ordinal) { + assertIndexIsValid(ordinal); + return Platform.getInt(baseObject, baseOffset + 4 + ordinal * 4); + } + + @Override + public long getLong(int ordinal) { + assertIndexIsValid(ordinal); + return Platform.getLong(baseObject, baseOffset + 4 + ordinal * 8); + } + + @Override + public float getFloat(int ordinal) { + assertIndexIsValid(ordinal); + return Platform.getFloat(baseObject, baseOffset + 4 + ordinal * 4); + } + + @Override + public double getDouble(int ordinal) { + assertIndexIsValid(ordinal); + return Platform.getDouble(baseObject, baseOffset + 4 + ordinal * 8); + } + + @Override + public Decimal getDecimal(int ordinal, int precision, int scale) { + throw new UnsupportedOperationException( + "Not support Decimal type in UnsafeArrayDataDense"); + } + + @Override + public UTF8String getUTF8String(int ordinal) { + throw new UnsupportedOperationException( + "Not support UTF8String type in UnsafeArrayDataDense"); + } + + @Override + public byte[] getBinary(int ordinal) { + throw new UnsupportedOperationException( + "Not support Binary type in UnsafeArrayDataDense"); + } + + @Override + public CalendarInterval getInterval(int ordinal) { + throw new UnsupportedOperationException( + "Not support Interval type in UnsafeArrayDataDense"); + } + + @Override + public UnsafeRow getStruct(int ordinal, int numFields) { + throw new UnsupportedOperationException( + "Not support Struct type in UnsafeArrayDataDense"); + } + + @Override + public UnsafeArrayData getArray(int ordinal) { + throw new UnsupportedOperationException( + "Not support Array type in UnsafeArrayDataDense"); + } + + @Override + public UnsafeMapData getMap(int ordinal) { + throw new UnsupportedOperationException( + "Not support Map type in UnsafeArrayDataDense"); + } + + @Override + public UnsafeArrayData copy() { + UnsafeArrayData arrayCopy = UnsafeArrayData.allocate(Format.Dense); + final byte[] arrayDataCopy = new byte[sizeInBytes]; + Platform.copyMemory( + baseObject, baseOffset, arrayDataCopy, Platform.BYTE_ARRAY_OFFSET, sizeInBytes); + arrayCopy.pointTo(arrayDataCopy, Platform.BYTE_ARRAY_OFFSET, sizeInBytes); + return arrayCopy; + } + + protected static UnsafeArrayData _fromPrimitiveArray(int[] arr) { + if (arr.length > (Integer.MAX_VALUE - 8) / 4) { + throw new UnsupportedOperationException("Cannot convert this array to unsafe format as " + + "it's too big."); + } + + final int valueRegionSize = 4 * arr.length; + final int totalSize = 4 + 4 + valueRegionSize; + final byte[] data = new byte[totalSize]; + + Platform.putInt(data, Platform.BYTE_ARRAY_OFFSET, arr.length); + Platform.putInt(data, Platform.BYTE_ARRAY_OFFSET + 4, 0); + + Platform.copyMemory(arr, Platform.INT_ARRAY_OFFSET, data, + Platform.BYTE_ARRAY_OFFSET + 8, valueRegionSize); + + UnsafeArrayData result = UnsafeArrayData.allocate(Format.Dense); + result.pointTo(data, Platform.BYTE_ARRAY_OFFSET, totalSize); + return result; + } + + protected static UnsafeArrayData _fromPrimitiveArray(double[] arr) { + if (arr.length > (Integer.MAX_VALUE - 4) / 8) { + throw new UnsupportedOperationException("Cannot convert this array to unsafe format as " + + "it's too big."); + } + + final int valueRegionSize = 8 * arr.length; + final int totalSize = 4 + 4 + valueRegionSize; + final byte[] data = new byte[totalSize]; + + Platform.putInt(data, Platform.BYTE_ARRAY_OFFSET, arr.length); + Platform.putInt(data, Platform.BYTE_ARRAY_OFFSET + 4, 0); + + Platform.copyMemory(arr, Platform.DOUBLE_ARRAY_OFFSET, data, + Platform.BYTE_ARRAY_OFFSET + 8, valueRegionSize); + + UnsafeArrayData result = UnsafeArrayData.allocate(Format.Dense); + result.pointTo(data, Platform.BYTE_ARRAY_OFFSET, totalSize); + return result; + } + + // TODO: add more specialized methods. +} diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeArrayDataSparse.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeArrayDataSparse.java new file mode 100644 index 0000000000000..f1f73187837a4 --- /dev/null +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeArrayDataSparse.java @@ -0,0 +1,291 @@ +/* + * 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; + +/** + * An Unsafe implementation of Array which is backed by raw memory instead of Java objects. + * + * Each tuple has three parts: [numElements] [offsets] [values] + * + * The `numElements` is 4 bytes storing the number of elements of this array. + * + * In the `offsets` region, we store 4 bytes per element, represents the relative offset (w.r.t. the + * base address of the array) of this element in `values` region. We can get the length of this + * element by subtracting next offset. + * Note that offset can by negative which means this element is null. + * + * In the `values` region, we store the content of elements. As we can get length info, so elements + * can be variable-length. + * + * Instances of `UnsafeArrayDataSparse` act as pointers to row data stored in this format. + */ +public final class UnsafeArrayDataSparse extends UnsafeArrayData { + + protected UnsafeArrayDataSparse() { + format = Format.Sparse; + } + + public void pointTo(Object baseObject, long baseOffset, int sizeInBytes) { + // Read the number of elements from the first 4 bytes. + final int numElements = Platform.getInt(baseObject, baseOffset); + assert numElements >= 0 : "numElements (" + numElements + ") should >= 0"; + assert(Platform.getInt(baseObject, baseOffset + 4) != 0); + + this.numElements = numElements; + this.baseObject = baseObject; + this.baseOffset = baseOffset; + this.sizeInBytes = sizeInBytes; + } + + private int getElementOffset(int ordinal) { + return Platform.getInt(baseObject, baseOffset + 4 + ordinal * 4L); + } + + private int getElementSize(int offset, int ordinal) { + if (ordinal == numElements - 1) { + return sizeInBytes - offset; + } else { + return Math.abs(getElementOffset(ordinal + 1)) - offset; + } + } + + private void assertIndexIsValid(int ordinal) { + assert ordinal >= 0 : "ordinal (" + ordinal + ") should >= 0"; + assert ordinal < numElements : "ordinal (" + ordinal + ") should < " + numElements; + } + + @Override + public boolean isNullAt(int ordinal) { + assertIndexIsValid(ordinal); + return getElementOffset(ordinal) < 0; + } + + @Override + public boolean getBoolean(int ordinal) { + assertIndexIsValid(ordinal); + final int offset = getElementOffset(ordinal); + if (offset < 0) return false; + return Platform.getBoolean(baseObject, baseOffset + offset); + } + + @Override + public byte getByte(int ordinal) { + assertIndexIsValid(ordinal); + final int offset = getElementOffset(ordinal); + if (offset < 0) return 0; + return Platform.getByte(baseObject, baseOffset + offset); + } + + @Override + public short getShort(int ordinal) { + assertIndexIsValid(ordinal); + final int offset = getElementOffset(ordinal); + if (offset < 0) return 0; + return Platform.getShort(baseObject, baseOffset + offset); + } + + @Override + public int getInt(int ordinal) { + assertIndexIsValid(ordinal); + final int offset = getElementOffset(ordinal); + if (offset < 0) return 0; + return Platform.getInt(baseObject, baseOffset + offset); + } + + @Override + public long getLong(int ordinal) { + assertIndexIsValid(ordinal); + final int offset = getElementOffset(ordinal); + if (offset < 0) return 0; + return Platform.getLong(baseObject, baseOffset + offset); + } + + @Override + public float getFloat(int ordinal) { + assertIndexIsValid(ordinal); + final int offset = getElementOffset(ordinal); + if (offset < 0) return 0; + return Platform.getFloat(baseObject, baseOffset + offset); + } + + @Override + public double getDouble(int ordinal) { + assertIndexIsValid(ordinal); + final int offset = getElementOffset(ordinal); + if (offset < 0) return 0; + return Platform.getDouble(baseObject, baseOffset + offset); + } + + @Override + public Decimal getDecimal(int ordinal, int precision, int scale) { + assertIndexIsValid(ordinal); + final int offset = getElementOffset(ordinal); + if (offset < 0) return null; + + if (precision <= Decimal.MAX_LONG_DIGITS()) { + final long value = Platform.getLong(baseObject, baseOffset + offset); + return Decimal.apply(value, 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); + } + } + + @Override + public UTF8String getUTF8String(int ordinal) { + assertIndexIsValid(ordinal); + final int offset = getElementOffset(ordinal); + if (offset < 0) return null; + final int size = getElementSize(offset, ordinal); + return UTF8String.fromAddress(baseObject, baseOffset + offset, size); + } + + @Override + public byte[] getBinary(int ordinal) { + assertIndexIsValid(ordinal); + final int offset = getElementOffset(ordinal); + if (offset < 0) return null; + final int size = getElementSize(offset, ordinal); + final byte[] bytes = new byte[size]; + Platform.copyMemory(baseObject, baseOffset + offset, bytes, Platform.BYTE_ARRAY_OFFSET, size); + return bytes; + } + + @Override + public CalendarInterval getInterval(int ordinal) { + assertIndexIsValid(ordinal); + final int offset = getElementOffset(ordinal); + if (offset < 0) return null; + final int months = (int) Platform.getLong(baseObject, baseOffset + offset); + final long microseconds = Platform.getLong(baseObject, baseOffset + offset + 8); + return new CalendarInterval(months, microseconds); + } + + @Override + public UnsafeRow getStruct(int ordinal, int numFields) { + assertIndexIsValid(ordinal); + final int offset = getElementOffset(ordinal); + if (offset < 0) return null; + final int size = getElementSize(offset, ordinal); + final UnsafeRow row = new UnsafeRow(numFields); + row.pointTo(baseObject, baseOffset + offset, size); + return row; + } + + @Override + public UnsafeArrayData getArray(int ordinal) { + assertIndexIsValid(ordinal); + final int offset = getElementOffset(ordinal); + if (offset < 0) return null; + final int size = getElementSize(offset, ordinal); + final UnsafeArrayData array = UnsafeArrayData.allocate(Format.Sparse); + array.pointTo(baseObject, baseOffset + offset, size); + return array; + } + + @Override + public UnsafeMapData getMap(int ordinal) { + assertIndexIsValid(ordinal); + final int offset = getElementOffset(ordinal); + if (offset < 0) return null; + final int size = getElementSize(offset, ordinal); + final UnsafeMapData map = new UnsafeMapData(); + map.pointTo(baseObject, baseOffset + offset, size); + return map; + } + + @Override + public UnsafeArrayData copy() { + UnsafeArrayData arrayCopy = UnsafeArrayData.allocate(Format.Sparse); + final byte[] arrayDataCopy = new byte[sizeInBytes]; + Platform.copyMemory( + baseObject, baseOffset, arrayDataCopy, Platform.BYTE_ARRAY_OFFSET, sizeInBytes); + arrayCopy.pointTo(arrayDataCopy, Platform.BYTE_ARRAY_OFFSET, sizeInBytes); + return arrayCopy; + } + + protected static UnsafeArrayData _fromPrimitiveArray(int[] arr) { + if (arr.length > (Integer.MAX_VALUE - 4) / 8) { + throw new UnsupportedOperationException("Cannot convert this array to unsafe format as " + + "it's too big."); + } + + final int offsetRegionSize = 4 * arr.length; + final int valueRegionSize = 4 * arr.length; + final int totalSize = 4 + offsetRegionSize + valueRegionSize; + final byte[] data = new byte[totalSize]; + + Platform.putInt(data, Platform.BYTE_ARRAY_OFFSET, arr.length); + + int offsetPosition = Platform.BYTE_ARRAY_OFFSET + 4; + int valueOffset = 4 + offsetRegionSize; + for (int i = 0; i < arr.length; i++) { + Platform.putInt(data, offsetPosition, valueOffset); + offsetPosition += 4; + valueOffset += 4; + } + + Platform.copyMemory(arr, Platform.INT_ARRAY_OFFSET, data, + Platform.BYTE_ARRAY_OFFSET + 4 + offsetRegionSize, valueRegionSize); + + UnsafeArrayData result = UnsafeArrayData.allocate(Format.Sparse); + result.pointTo(data, Platform.BYTE_ARRAY_OFFSET, totalSize); + return result; + } + + protected static UnsafeArrayData _fromPrimitiveArray(double[] arr) { + if (arr.length > (Integer.MAX_VALUE - 4) / 12) { + throw new UnsupportedOperationException("Cannot convert this array to unsafe format as " + + "it's too big."); + } + + final int offsetRegionSize = 4 * arr.length; + final int valueRegionSize = 8 * arr.length; + final int totalSize = 4 + offsetRegionSize + valueRegionSize; + final byte[] data = new byte[totalSize]; + + Platform.putInt(data, Platform.BYTE_ARRAY_OFFSET, arr.length); + + int offsetPosition = Platform.BYTE_ARRAY_OFFSET + 4; + int valueOffset = 4 + offsetRegionSize; + for (int i = 0; i < arr.length; i++) { + Platform.putInt(data, offsetPosition, valueOffset); + offsetPosition += 4; + valueOffset += 8; + } + + Platform.copyMemory(arr, Platform.DOUBLE_ARRAY_OFFSET, data, + Platform.BYTE_ARRAY_OFFSET + 4 + offsetRegionSize, valueRegionSize); + + UnsafeArrayData result = UnsafeArrayData.allocate(Format.Sparse); + result.pointTo(data, Platform.BYTE_ARRAY_OFFSET, totalSize); + return result; + } + + // TODO: add more specialized methods. +} 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 0700148becaba..2551177e7b086 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 @@ -53,8 +53,8 @@ public final class UnsafeMapData extends MapData { * to a null pointer. */ public UnsafeMapData() { - keys = new UnsafeArrayData(); - values = new UnsafeArrayData(); + keys = UnsafeArrayData.allocate(UnsafeArrayData.Format.Sparse); + values = UnsafeArrayData.allocate(UnsafeArrayData.Format.Sparse); } /** 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 dd2f39eb816f2..916f13481ae3a 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 @@ -473,7 +473,7 @@ public UnsafeArrayData getArray(int ordinal) { final long offsetAndSize = getLong(ordinal); final int offset = (int) (offsetAndSize >> 32); final int size = (int) offsetAndSize; - final UnsafeArrayData array = new UnsafeArrayData(); + final UnsafeArrayData array = UnsafeArrayData.allocate(UnsafeArrayData.Format.Sparse); array.pointTo(baseObject, baseOffset + offset, size); return array; } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnType.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnType.scala index f9d606e37ea89..9c8e8c7c4ba60 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnType.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnType.scala @@ -614,7 +614,7 @@ private[columnar] case class ARRAY(dataType: ArrayType) assert(buffer.hasArray) val cursor = buffer.position() buffer.position(cursor + numBytes) - val array = new UnsafeArrayData + val array = UnsafeArrayData.allocate(UnsafeArrayData.Format.Sparse); array.pointTo( buffer.array(), Platform.BYTE_ARRAY_OFFSET + buffer.arrayOffset() + cursor, From d9314282032acd8c9b2782511ba11f37a6cd3125 Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Wed, 15 Jun 2016 15:19:18 +0900 Subject: [PATCH 02/37] fix failures of testsuite --- .../sql/catalyst/expressions/UnsafeArrayData.java | 2 ++ .../catalyst/expressions/UnsafeArrayDataDense.java | 14 +++++++------- .../spark/sql/catalyst/util/UnsafeArraySuite.scala | 6 ++++-- 3 files changed, 13 insertions(+), 9 deletions(-) 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 09dbe5d1cdba9..1289eb9bcb670 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 @@ -34,6 +34,8 @@ public enum Format { protected Format format; + public final Format getFormat() { return format; } + protected Object baseObject; protected long baseOffset; diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeArrayDataDense.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeArrayDataDense.java index 196ba2251af98..eb015db8b59b3 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeArrayDataDense.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeArrayDataDense.java @@ -68,43 +68,43 @@ public boolean isNullAt(int ordinal) { @Override public boolean getBoolean(int ordinal) { assertIndexIsValid(ordinal); - return Platform.getBoolean(baseObject, baseOffset + 4 + ordinal); + return Platform.getBoolean(baseObject, baseOffset + 8 + ordinal); } @Override public byte getByte(int ordinal) { assertIndexIsValid(ordinal); - return Platform.getByte(baseObject, baseOffset + 4 + ordinal); + return Platform.getByte(baseObject, baseOffset + 8 + ordinal); } @Override public short getShort(int ordinal) { assertIndexIsValid(ordinal); - return Platform.getShort(baseObject, baseOffset + 4 + ordinal * 2); + return Platform.getShort(baseObject, baseOffset + 8 + ordinal * 2); } @Override public int getInt(int ordinal) { assertIndexIsValid(ordinal); - return Platform.getInt(baseObject, baseOffset + 4 + ordinal * 4); + return Platform.getInt(baseObject, baseOffset + 8 + ordinal * 4); } @Override public long getLong(int ordinal) { assertIndexIsValid(ordinal); - return Platform.getLong(baseObject, baseOffset + 4 + ordinal * 8); + return Platform.getLong(baseObject, baseOffset + 8 + ordinal * 8); } @Override public float getFloat(int ordinal) { assertIndexIsValid(ordinal); - return Platform.getFloat(baseObject, baseOffset + 4 + ordinal * 4); + return Platform.getFloat(baseObject, baseOffset + 8 + ordinal * 4); } @Override public double getDouble(int ordinal) { assertIndexIsValid(ordinal); - return Platform.getDouble(baseObject, baseOffset + 4 + ordinal * 8); + return Platform.getDouble(baseObject, baseOffset + 8 + ordinal * 8); } @Override diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/UnsafeArraySuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/UnsafeArraySuite.scala index 1685276ff1201..b7ebd170af953 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/UnsafeArraySuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/UnsafeArraySuite.scala @@ -26,7 +26,8 @@ class UnsafeArraySuite extends SparkFunSuite { val array = Array(1, 10, 100) val unsafe = UnsafeArrayData.fromPrimitiveArray(array) assert(unsafe.numElements == 3) - assert(unsafe.getSizeInBytes == 4 + 4 * 3 + 4 * 3) + assert(unsafe.getFormat == UnsafeArrayData.Format.Dense) + assert(unsafe.getSizeInBytes == 4 + 4 + 4 * 3) assert(unsafe.getInt(0) == 1) assert(unsafe.getInt(1) == 10) assert(unsafe.getInt(2) == 100) @@ -36,7 +37,8 @@ class UnsafeArraySuite extends SparkFunSuite { val array = Array(1.1, 2.2, 3.3) val unsafe = UnsafeArrayData.fromPrimitiveArray(array) assert(unsafe.numElements == 3) - assert(unsafe.getSizeInBytes == 4 + 4 * 3 + 8 * 3) + assert(unsafe.getFormat == UnsafeArrayData.Format.Dense) + assert(unsafe.getSizeInBytes == 4 + 4 + 8 * 3) assert(unsafe.getDouble(0) == 1.1) assert(unsafe.getDouble(1) == 2.2) assert(unsafe.getDouble(2) == 3.3) From 9777a2dd0ad385811e4c04dccaccdc9aea4763d3 Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Wed, 15 Jun 2016 19:15:04 +0900 Subject: [PATCH 03/37] fix errors of unit tests prepare specialized primitive array methods for UnsafeArrayDataDense --- .../catalyst/expressions/UnsafeArrayData.java | 11 +++ .../expressions/UnsafeArrayDataDense.java | 74 +++++++++++++++++-- .../sql/catalyst/expressions/UnsafeRow.java | 3 +- 3 files changed, 82 insertions(+), 6 deletions(-) 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 1289eb9bcb670..d67e9666f23f2 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 @@ -36,6 +36,17 @@ public enum Format { public final Format getFormat() { return format; } + protected static final int DenseID = 1; + + public static Format getFormat(Object baseObject, long baseOffset) { + // Read the number of elements from the first 4 bytes. + final int numElements = Platform.getInt(baseObject, baseOffset); + assert numElements >= 0 : "numElements (" + numElements + ") should >= 0"; + // Read the next 4 bytes to check its format id ( != 1) + int formatId = Platform.getInt(baseObject, baseOffset + 4); + return (formatId == DenseID) ? Format.Dense : Format.Sparse; + } + protected Object baseObject; protected long baseOffset; diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeArrayDataDense.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeArrayDataDense.java index eb015db8b59b3..86c4755a626b3 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeArrayDataDense.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeArrayDataDense.java @@ -26,11 +26,11 @@ * An Unsafe implementation of Array which is backed by raw memory instead of Java objects. * This array assumes that each element is non-null * - * Each tuple has three parts: [numElements] [0] [values] + * Each tuple has three parts: [numElements] [1] [values] * * The `numElements` is 4 bytes storing the number of elements of this array. * - * [0] is a 4-byte identifier for a dense array + * [1] is a 4-byte identifier for a dense array * * In the `values` region, we store the content of elements. As we can get length info, so elements * can be variable-length. @@ -47,7 +47,8 @@ public void pointTo(Object baseObject, long baseOffset, int sizeInBytes) { // Read the number of elements from the first 4 bytes. final int numElements = Platform.getInt(baseObject, baseOffset); assert numElements >= 0 : "numElements (" + numElements + ") should >= 0"; - assert(Platform.getInt(baseObject, baseOffset + 4) == 0); + // Read format id ( == DenseID) + assert(Platform.getInt(baseObject, baseOffset + 4) == DenseID); this.numElements = numElements; this.baseObject = baseObject; @@ -159,6 +160,69 @@ public UnsafeArrayData copy() { return arrayCopy; } + @Override + public boolean[] toBooleanArray() { + int size = numElements(); + boolean[] values = new boolean[size]; + Platform.copyMemory( + baseObject, baseOffset + 8, values, Platform.BYTE_ARRAY_OFFSET, size); + return values; + } + + @Override + public byte[] toByteArray() { + int size = numElements(); + byte[] values = new byte[size]; + Platform.copyMemory( + baseObject, baseOffset + 8, values, Platform.BYTE_ARRAY_OFFSET, size); + return values; + } + + @Override + public short[] toShortArray() { + int size = numElements(); + short[] values = new short[size]; + Platform.copyMemory( + baseObject, baseOffset + 8, values, Platform.BYTE_ARRAY_OFFSET, size * 2); + return values; + } + + @Override + public int[] toIntArray() { + int size = numElements(); + int[] values = new int[size]; + Platform.copyMemory( + baseObject, baseOffset + 8, values, Platform.BYTE_ARRAY_OFFSET, size * 4); + return values; + } + + @Override + public long[] toLongArray() { + int size = numElements(); + long[] values = new long[size]; + Platform.copyMemory( + baseObject, baseOffset + 8, values, Platform.BYTE_ARRAY_OFFSET, size * 8); + return values; + } + + @Override + public float[] toFloatArray() { + int size = numElements(); + float[] values = new float[size]; + Platform.copyMemory( + baseObject, baseOffset + 8, values, Platform.BYTE_ARRAY_OFFSET, size * 4); + return values; + } + + @Override + public double[] toDoubleArray() { + int size = numElements(); + double[] values = new double[size]; + Platform.copyMemory( + baseObject, baseOffset + 8, values, Platform.BYTE_ARRAY_OFFSET, size * 8); + return values; + } + protected static UnsafeArrayData _fromPrimitiveArray(int[] arr) { if (arr.length > (Integer.MAX_VALUE - 8) / 4) { throw new UnsupportedOperationException("Cannot convert this array to unsafe format as " + @@ -170,7 +234,7 @@ protected static UnsafeArrayData _fromPrimitiveArray(int[] arr) { final byte[] data = new byte[totalSize]; Platform.putInt(data, Platform.BYTE_ARRAY_OFFSET, arr.length); - Platform.putInt(data, Platform.BYTE_ARRAY_OFFSET + 4, 0); + Platform.putInt(data, Platform.BYTE_ARRAY_OFFSET + 4, DenseID); Platform.copyMemory(arr, Platform.INT_ARRAY_OFFSET, data, Platform.BYTE_ARRAY_OFFSET + 8, valueRegionSize); @@ -191,7 +255,7 @@ protected static UnsafeArrayData _fromPrimitiveArray(double[] arr) { final byte[] data = new byte[totalSize]; Platform.putInt(data, Platform.BYTE_ARRAY_OFFSET, arr.length); - Platform.putInt(data, Platform.BYTE_ARRAY_OFFSET + 4, 0); + Platform.putInt(data, Platform.BYTE_ARRAY_OFFSET + 4, DenseID); Platform.copyMemory(arr, Platform.DOUBLE_ARRAY_OFFSET, data, Platform.BYTE_ARRAY_OFFSET + 8, valueRegionSize); 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 916f13481ae3a..e132032f1ff38 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 @@ -473,7 +473,8 @@ public UnsafeArrayData getArray(int ordinal) { final long offsetAndSize = getLong(ordinal); final int offset = (int) (offsetAndSize >> 32); final int size = (int) offsetAndSize; - final UnsafeArrayData array = UnsafeArrayData.allocate(UnsafeArrayData.Format.Sparse); + final UnsafeArrayData.Format format = UnsafeArrayData.getFormat(baseObject, baseOffset + offset); + final UnsafeArrayData array = UnsafeArrayData.allocate(format); array.pointTo(baseObject, baseOffset + offset, size); return array; } From 000eda439d827eb1fc9ce0107f555b6a87cb557d Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Wed, 15 Jun 2016 21:11:49 +0900 Subject: [PATCH 04/37] fix failures of unit tests --- .../sql/catalyst/expressions/UnsafeArrayDataSparse.java | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeArrayDataSparse.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeArrayDataSparse.java index f1f73187837a4..9b70acf2a4c32 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeArrayDataSparse.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeArrayDataSparse.java @@ -52,7 +52,10 @@ public void pointTo(Object baseObject, long baseOffset, int sizeInBytes) { // Read the number of elements from the first 4 bytes. final int numElements = Platform.getInt(baseObject, baseOffset); assert numElements >= 0 : "numElements (" + numElements + ") should >= 0"; - assert(Platform.getInt(baseObject, baseOffset + 4) != 0); + // Read the first element of offset to check its format id ( != DenseID + assert numElements == 0 || Platform.getInt(baseObject, baseOffset + 4) != DenseID: + "numElements (" + numElements + ") should == 0 or " + + "first offset (" + Platform.getInt(baseObject, baseOffset + 4) + ") should != DenseID"; this.numElements = numElements; this.baseObject = baseObject; From 804f08199f9c13d05e19ad1af7dec0f6169ae2f4 Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Thu, 23 Jun 2016 13:23:37 +0900 Subject: [PATCH 05/37] make DenseID public --- .../apache/spark/sql/catalyst/expressions/UnsafeArrayData.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 d67e9666f23f2..8a886ffaf454a 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 @@ -36,7 +36,7 @@ public enum Format { public final Format getFormat() { return format; } - protected static final int DenseID = 1; + public static final int DenseID = 1; public static Format getFormat(Object baseObject, long baseOffset) { // Read the number of elements from the first 4 bytes. From e6fb2617d04107b79237bb8ecd0462be9622d6f3 Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Sat, 25 Jun 2016 11:02:36 +0900 Subject: [PATCH 06/37] Use one implementation approach --- .../catalyst/expressions/UnsafeArrayData.java | 345 +++++++++++++++--- .../expressions/UnsafeArrayDataDense.java | 269 -------------- .../expressions/UnsafeArrayDataSparse.java | 294 --------------- .../catalyst/expressions/UnsafeMapData.java | 4 +- .../sql/catalyst/expressions/UnsafeRow.java | 3 +- .../codegen/UnsafeArrayWriter.java | 147 +++++--- .../codegen/GenerateUnsafeProjection.scala | 15 +- .../sql/catalyst/util/UnsafeArraySuite.scala | 6 +- .../sql/execution/columnar/ColumnType.scala | 2 +- 9 files changed, 415 insertions(+), 670 deletions(-) delete mode 100644 sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeArrayDataDense.java delete mode 100644 sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeArrayDataSparse.java 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 8a886ffaf454a..0d4e4c11e353c 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,49 +17,71 @@ 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.array.ByteArrayMethods; +import org.apache.spark.unsafe.bitset.BitSetMethods; import org.apache.spark.unsafe.hash.Murmur3_x86_32; +import org.apache.spark.unsafe.types.CalendarInterval; +import org.apache.spark.unsafe.types.UTF8String; -import java.nio.ByteBuffer; - +/** + * An Unsafe implementation of Array which is backed by raw memory instead of Java objects. + * + * Each tuple has three four: [numElements] [null bits] [values] [variable length portion] + * + * The `numElements` is 4 bytes storing the number of elements of this array. + * + * In the `null bits` region, we store 1 bit per element, represents whether a element has null + * Its total size is ceil(numElements / 8) bytes, and it is aligned to 8-byte word boundaries. + * + * In the `offsets` region, we store 4 bytes per element, represents the relative offset (w.r.t. the + * base address of the array) of this element in `values` region. We can get the length of this + * element by subtracting next offset. + * Note that offset can by negative which means this element is null. + * + * In the `values` region, we store the content of elements. As we can get length info, so elements + * can be variable-length. + * + */ // todo: there is a lof of duplicated code between UnsafeRow and UnsafeArrayData. -public abstract class UnsafeArrayData extends ArrayData { - public enum Format { - Sparse, Dense - } +public final class UnsafeArrayData extends ArrayData { - protected Format format; - - public final Format getFormat() { return format; } - - public static final int DenseID = 1; - - public static Format getFormat(Object baseObject, long baseOffset) { - // Read the number of elements from the first 4 bytes. - final int numElements = Platform.getInt(baseObject, baseOffset); - assert numElements >= 0 : "numElements (" + numElements + ") should >= 0"; - // Read the next 4 bytes to check its format id ( != 1) - int formatId = Platform.getInt(baseObject, baseOffset + 4); - return (formatId == DenseID) ? Format.Dense : Format.Sparse; + public static int calculateHeaderPortionInBytes(int numFields) { + return 4 + ((numFields + 63)/ 64) * 8; } - protected Object baseObject; - protected long baseOffset; + private Object baseObject; + private long baseOffset; // The number of elements in this array - protected int numElements; + private int numElements; // The size of this array's backing data, in bytes. // The 4-bytes header of `numElements` is also included. - protected int sizeInBytes; + private int sizeInBytes; + + /** The width of the null tracking bit set, in bytes */ + private int headerInBytes; + + private long getFieldOffset(int ordinal, int scale) { + return baseOffset + headerInBytes + ordinal * scale; + } + + public Object getBaseObject() { return baseObject; } + public long getBaseOffset() { return baseOffset; } + public int getSizeInBytes() { return sizeInBytes; } - public final Object getBaseObject() { return baseObject; } - public final long getBaseOffset() { return baseOffset; } - public final int getSizeInBytes() { return sizeInBytes; } + private void assertIndexIsValid(int ordinal) { + assert ordinal >= 0 : "ordinal (" + ordinal + ") should >= 0"; + assert ordinal < numElements : "ordinal (" + ordinal + ") should < " + numElements; + } public Object[] array() { throw new UnsupportedOperationException("Not supported on UnsafeArrayData."); @@ -70,18 +92,10 @@ public Object[] array() { * `pointTo()` has been called, since the value returned by this constructor is equivalent * to a null pointer. */ - public static final UnsafeArrayData allocate(Format format) { - if (format == Format.Sparse) { - return new UnsafeArrayDataSparse(); - } else { - return new UnsafeArrayDataDense(); - } - } - public UnsafeArrayData() { } @Override - public final int numElements() { return numElements; } + public int numElements() { return numElements; } /** * Update this UnsafeArrayData to point to different backing data. @@ -90,7 +104,23 @@ public UnsafeArrayData() { } * @param baseOffset the offset within the base object * @param sizeInBytes the size of this array's backing data, in bytes */ - public abstract void pointTo(Object baseObject, long baseOffset, int sizeInBytes); + public void pointTo(Object baseObject, long baseOffset, int sizeInBytes) { + // Read the number of elements from the first 4 bytes. + final int numElements = Platform.getInt(baseObject, baseOffset); + assert numElements >= 0 : "numElements (" + numElements + ") should >= 0"; + + this.numElements = numElements; + this.baseObject = baseObject; + this.baseOffset = baseOffset; + this.sizeInBytes = sizeInBytes; + this.headerInBytes = calculateHeaderPortionInBytes(numElements); + } + + @Override + public boolean isNullAt(int ordinal) { + assertIndexIsValid(ordinal); + return BitSetMethods.isSet(baseObject, baseOffset + 4, ordinal); + } @Override public final Object get(int ordinal, DataType dataType) { @@ -136,11 +166,125 @@ public final Object get(int ordinal, DataType dataType) { } } - public abstract UnsafeRow getStruct(int ordinal, int numFields); + @Override + public boolean getBoolean(int ordinal) { + assertIndexIsValid(ordinal); + return Platform.getBoolean(baseObject, getFieldOffset(ordinal, 1)); + } + + @Override + public byte getByte(int ordinal) { + assertIndexIsValid(ordinal); + return Platform.getByte(baseObject, getFieldOffset(ordinal, 1)); + } + + @Override + public short getShort(int ordinal) { + assertIndexIsValid(ordinal); + return Platform.getShort(baseObject, getFieldOffset(ordinal, 2)); + } - public abstract UnsafeArrayData getArray(int ordinal); + @Override + public int getInt(int ordinal) { + assertIndexIsValid(ordinal); + return Platform.getInt(baseObject, getFieldOffset(ordinal, 4)); + } - public abstract UnsafeMapData getMap(int ordinal); + @Override + public long getLong(int ordinal) { + assertIndexIsValid(ordinal); + return Platform.getLong(baseObject, getFieldOffset(ordinal, 8)); + } + + @Override + public float getFloat(int ordinal) { + assertIndexIsValid(ordinal); + return Platform.getFloat(baseObject, getFieldOffset(ordinal, 4)); + } + + @Override + public double getDouble(int ordinal) { + assertIndexIsValid(ordinal); + return Platform.getDouble(baseObject, getFieldOffset(ordinal, 8)); + } + + @Override + public Decimal getDecimal(int ordinal, int precision, int scale) { + if (isNullAt(ordinal)) { + return null; + } + 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); + } + } + + @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); + } + + @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; + } + + @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); + } + + @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; + } + + @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; + } + + @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; + } public final void writeToMemory(Object target, long targetOffset) { Platform.copyMemory(baseObject, baseOffset, target, targetOffset, sizeInBytes); @@ -159,6 +303,119 @@ public final void writeTo(ByteBuffer buffer) { // If the computation becomes a bottleneck, we can use a light-weight logic; the first fixed bytes // are used to compute `hashCode` (See `Vector.hashCode`). // The same issue exists in `UnsafeRow.hashCode`. + @Override + public UnsafeArrayData copy() { + UnsafeArrayData arrayCopy = new UnsafeArrayData(); + final byte[] arrayDataCopy = new byte[sizeInBytes]; + Platform.copyMemory( + baseObject, baseOffset, arrayDataCopy, Platform.BYTE_ARRAY_OFFSET, sizeInBytes); + arrayCopy.pointTo(arrayDataCopy, Platform.BYTE_ARRAY_OFFSET, sizeInBytes); + return arrayCopy; + } + + @Override + public boolean[] toBooleanArray() { + int size = numElements(); + boolean[] values = new boolean[size]; + Platform.copyMemory( + baseObject, baseOffset + headerInBytes, values, Platform.BYTE_ARRAY_OFFSET, size); + return values; + } + + @Override + public byte[] toByteArray() { + int size = numElements(); + byte[] values = new byte[size]; + Platform.copyMemory( + baseObject, baseOffset + headerInBytes, values, Platform.BYTE_ARRAY_OFFSET, size); + return values; + } + + @Override + public short[] toShortArray() { + int size = numElements(); + short[] values = new short[size]; + Platform.copyMemory( + baseObject, baseOffset + headerInBytes, values, Platform.BYTE_ARRAY_OFFSET, size * 2); + return values; + } + + @Override + public int[] toIntArray() { + int size = numElements(); + int[] values = new int[size]; + Platform.copyMemory( + baseObject, baseOffset + headerInBytes, values, Platform.BYTE_ARRAY_OFFSET, size * 4); + return values; + } + + @Override + public long[] toLongArray() { + int size = numElements(); + long[] values = new long[size]; + Platform.copyMemory( + baseObject, baseOffset + headerInBytes, values, Platform.BYTE_ARRAY_OFFSET, size * 8); + return values; + } + + @Override + public float[] toFloatArray() { + int size = numElements(); + float[] values = new float[size]; + Platform.copyMemory( + baseObject, baseOffset + headerInBytes, values, Platform.BYTE_ARRAY_OFFSET, size * 4); + return values; + } + + @Override + public double[] toDoubleArray() { + int size = numElements(); + double[] values = new double[size]; + Platform.copyMemory( + baseObject, baseOffset + headerInBytes, values, Platform.BYTE_ARRAY_OFFSET, size * 8); + return values; + } + + public static UnsafeArrayData fromPrimitiveArray(int[] arr) { + final int elementSize = 4; + final int headerSize = calculateHeaderPortionInBytes(arr.length); + if (arr.length > (Integer.MAX_VALUE - headerSize) / elementSize) { + throw new UnsupportedOperationException("Cannot convert this array to unsafe format as " + + "it's too big."); + } + + final int valueRegionSize = elementSize * arr.length; + final byte[] data = new byte[valueRegionSize + headerSize]; + + Platform.putInt(data, Platform.BYTE_ARRAY_OFFSET, arr.length); + Platform.copyMemory(arr, Platform.INT_ARRAY_OFFSET, data, + Platform.BYTE_ARRAY_OFFSET + headerSize, valueRegionSize); + + UnsafeArrayData result = new UnsafeArrayData(); + result.pointTo(data, Platform.BYTE_ARRAY_OFFSET, valueRegionSize + headerSize); + return result; + } + + public static UnsafeArrayData fromPrimitiveArray(double[] arr) { + final int elementSize = 8; + final int headerSize = calculateHeaderPortionInBytes(arr.length); + if (arr.length > (Integer.MAX_VALUE - headerSize) / elementSize) { + throw new UnsupportedOperationException("Cannot convert this array to unsafe format as " + + "it's too big."); + } + + final int valueRegionSize = elementSize * arr.length; + final byte[] data = new byte[valueRegionSize + headerSize]; + + Platform.putInt(data, Platform.BYTE_ARRAY_OFFSET, arr.length); + Platform.copyMemory(arr, Platform.DOUBLE_ARRAY_OFFSET, data, + Platform.BYTE_ARRAY_OFFSET + headerSize, valueRegionSize); + + UnsafeArrayData result = new UnsafeArrayData(); + result.pointTo(data, Platform.BYTE_ARRAY_OFFSET, valueRegionSize + headerSize); + return result; + } + @Override public int hashCode() { return Murmur3_x86_32.hashUnsafeBytes(baseObject, baseOffset, sizeInBytes, 42); @@ -174,14 +431,4 @@ public boolean equals(Object other) { } return false; } - - public abstract UnsafeArrayData copy(); - - public static UnsafeArrayData fromPrimitiveArray(int[] arr) { - return UnsafeArrayDataDense._fromPrimitiveArray(arr); - } - - public static UnsafeArrayData fromPrimitiveArray(double[] arr) { - return UnsafeArrayDataDense._fromPrimitiveArray(arr); - } } diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeArrayDataDense.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeArrayDataDense.java deleted file mode 100644 index 86c4755a626b3..0000000000000 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeArrayDataDense.java +++ /dev/null @@ -1,269 +0,0 @@ -/* - * 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; - -/** - * An Unsafe implementation of Array which is backed by raw memory instead of Java objects. - * This array assumes that each element is non-null - * - * Each tuple has three parts: [numElements] [1] [values] - * - * The `numElements` is 4 bytes storing the number of elements of this array. - * - * [1] is a 4-byte identifier for a dense array - * - * In the `values` region, we store the content of elements. As we can get length info, so elements - * can be variable-length. - * - * Instances of `UnsafeArrayDataDense` act as pointers to row data stored in this format. - */ -public final class UnsafeArrayDataDense extends UnsafeArrayData { - - protected UnsafeArrayDataDense() { - format = Format.Dense; - } - - public void pointTo(Object baseObject, long baseOffset, int sizeInBytes) { - // Read the number of elements from the first 4 bytes. - final int numElements = Platform.getInt(baseObject, baseOffset); - assert numElements >= 0 : "numElements (" + numElements + ") should >= 0"; - // Read format id ( == DenseID) - assert(Platform.getInt(baseObject, baseOffset + 4) == DenseID); - - this.numElements = numElements; - this.baseObject = baseObject; - this.baseOffset = baseOffset; - this.sizeInBytes = sizeInBytes; - } - - private void assertIndexIsValid(int ordinal) { - assert ordinal >= 0 : "ordinal (" + ordinal + ") should >= 0"; - assert ordinal < numElements : "ordinal (" + ordinal + ") should < " + numElements; - } - - @Override - public boolean isNullAt(int ordinal) { - return false; - } - - @Override - public boolean getBoolean(int ordinal) { - assertIndexIsValid(ordinal); - return Platform.getBoolean(baseObject, baseOffset + 8 + ordinal); - } - - @Override - public byte getByte(int ordinal) { - assertIndexIsValid(ordinal); - return Platform.getByte(baseObject, baseOffset + 8 + ordinal); - } - - @Override - public short getShort(int ordinal) { - assertIndexIsValid(ordinal); - return Platform.getShort(baseObject, baseOffset + 8 + ordinal * 2); - } - - @Override - public int getInt(int ordinal) { - assertIndexIsValid(ordinal); - return Platform.getInt(baseObject, baseOffset + 8 + ordinal * 4); - } - - @Override - public long getLong(int ordinal) { - assertIndexIsValid(ordinal); - return Platform.getLong(baseObject, baseOffset + 8 + ordinal * 8); - } - - @Override - public float getFloat(int ordinal) { - assertIndexIsValid(ordinal); - return Platform.getFloat(baseObject, baseOffset + 8 + ordinal * 4); - } - - @Override - public double getDouble(int ordinal) { - assertIndexIsValid(ordinal); - return Platform.getDouble(baseObject, baseOffset + 8 + ordinal * 8); - } - - @Override - public Decimal getDecimal(int ordinal, int precision, int scale) { - throw new UnsupportedOperationException( - "Not support Decimal type in UnsafeArrayDataDense"); - } - - @Override - public UTF8String getUTF8String(int ordinal) { - throw new UnsupportedOperationException( - "Not support UTF8String type in UnsafeArrayDataDense"); - } - - @Override - public byte[] getBinary(int ordinal) { - throw new UnsupportedOperationException( - "Not support Binary type in UnsafeArrayDataDense"); - } - - @Override - public CalendarInterval getInterval(int ordinal) { - throw new UnsupportedOperationException( - "Not support Interval type in UnsafeArrayDataDense"); - } - - @Override - public UnsafeRow getStruct(int ordinal, int numFields) { - throw new UnsupportedOperationException( - "Not support Struct type in UnsafeArrayDataDense"); - } - - @Override - public UnsafeArrayData getArray(int ordinal) { - throw new UnsupportedOperationException( - "Not support Array type in UnsafeArrayDataDense"); - } - - @Override - public UnsafeMapData getMap(int ordinal) { - throw new UnsupportedOperationException( - "Not support Map type in UnsafeArrayDataDense"); - } - - @Override - public UnsafeArrayData copy() { - UnsafeArrayData arrayCopy = UnsafeArrayData.allocate(Format.Dense); - final byte[] arrayDataCopy = new byte[sizeInBytes]; - Platform.copyMemory( - baseObject, baseOffset, arrayDataCopy, Platform.BYTE_ARRAY_OFFSET, sizeInBytes); - arrayCopy.pointTo(arrayDataCopy, Platform.BYTE_ARRAY_OFFSET, sizeInBytes); - return arrayCopy; - } - - @Override - public boolean[] toBooleanArray() { - int size = numElements(); - boolean[] values = new boolean[size]; - Platform.copyMemory( - baseObject, baseOffset + 8, values, Platform.BYTE_ARRAY_OFFSET, size); - return values; - } - - @Override - public byte[] toByteArray() { - int size = numElements(); - byte[] values = new byte[size]; - Platform.copyMemory( - baseObject, baseOffset + 8, values, Platform.BYTE_ARRAY_OFFSET, size); - return values; - } - - @Override - public short[] toShortArray() { - int size = numElements(); - short[] values = new short[size]; - Platform.copyMemory( - baseObject, baseOffset + 8, values, Platform.BYTE_ARRAY_OFFSET, size * 2); - return values; - } - - @Override - public int[] toIntArray() { - int size = numElements(); - int[] values = new int[size]; - Platform.copyMemory( - baseObject, baseOffset + 8, values, Platform.BYTE_ARRAY_OFFSET, size * 4); - return values; - } - - @Override - public long[] toLongArray() { - int size = numElements(); - long[] values = new long[size]; - Platform.copyMemory( - baseObject, baseOffset + 8, values, Platform.BYTE_ARRAY_OFFSET, size * 8); - return values; - } - - @Override - public float[] toFloatArray() { - int size = numElements(); - float[] values = new float[size]; - Platform.copyMemory( - baseObject, baseOffset + 8, values, Platform.BYTE_ARRAY_OFFSET, size * 4); - return values; - } - - @Override - public double[] toDoubleArray() { - int size = numElements(); - double[] values = new double[size]; - Platform.copyMemory( - baseObject, baseOffset + 8, values, Platform.BYTE_ARRAY_OFFSET, size * 8); - return values; - } - - protected static UnsafeArrayData _fromPrimitiveArray(int[] arr) { - if (arr.length > (Integer.MAX_VALUE - 8) / 4) { - throw new UnsupportedOperationException("Cannot convert this array to unsafe format as " + - "it's too big."); - } - - final int valueRegionSize = 4 * arr.length; - final int totalSize = 4 + 4 + valueRegionSize; - final byte[] data = new byte[totalSize]; - - Platform.putInt(data, Platform.BYTE_ARRAY_OFFSET, arr.length); - Platform.putInt(data, Platform.BYTE_ARRAY_OFFSET + 4, DenseID); - - Platform.copyMemory(arr, Platform.INT_ARRAY_OFFSET, data, - Platform.BYTE_ARRAY_OFFSET + 8, valueRegionSize); - - UnsafeArrayData result = UnsafeArrayData.allocate(Format.Dense); - result.pointTo(data, Platform.BYTE_ARRAY_OFFSET, totalSize); - return result; - } - - protected static UnsafeArrayData _fromPrimitiveArray(double[] arr) { - if (arr.length > (Integer.MAX_VALUE - 4) / 8) { - throw new UnsupportedOperationException("Cannot convert this array to unsafe format as " + - "it's too big."); - } - - final int valueRegionSize = 8 * arr.length; - final int totalSize = 4 + 4 + valueRegionSize; - final byte[] data = new byte[totalSize]; - - Platform.putInt(data, Platform.BYTE_ARRAY_OFFSET, arr.length); - Platform.putInt(data, Platform.BYTE_ARRAY_OFFSET + 4, DenseID); - - Platform.copyMemory(arr, Platform.DOUBLE_ARRAY_OFFSET, data, - Platform.BYTE_ARRAY_OFFSET + 8, valueRegionSize); - - UnsafeArrayData result = UnsafeArrayData.allocate(Format.Dense); - result.pointTo(data, Platform.BYTE_ARRAY_OFFSET, totalSize); - return result; - } - - // TODO: add more specialized methods. -} diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeArrayDataSparse.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeArrayDataSparse.java deleted file mode 100644 index 9b70acf2a4c32..0000000000000 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeArrayDataSparse.java +++ /dev/null @@ -1,294 +0,0 @@ -/* - * 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; - -/** - * An Unsafe implementation of Array which is backed by raw memory instead of Java objects. - * - * Each tuple has three parts: [numElements] [offsets] [values] - * - * The `numElements` is 4 bytes storing the number of elements of this array. - * - * In the `offsets` region, we store 4 bytes per element, represents the relative offset (w.r.t. the - * base address of the array) of this element in `values` region. We can get the length of this - * element by subtracting next offset. - * Note that offset can by negative which means this element is null. - * - * In the `values` region, we store the content of elements. As we can get length info, so elements - * can be variable-length. - * - * Instances of `UnsafeArrayDataSparse` act as pointers to row data stored in this format. - */ -public final class UnsafeArrayDataSparse extends UnsafeArrayData { - - protected UnsafeArrayDataSparse() { - format = Format.Sparse; - } - - public void pointTo(Object baseObject, long baseOffset, int sizeInBytes) { - // Read the number of elements from the first 4 bytes. - final int numElements = Platform.getInt(baseObject, baseOffset); - assert numElements >= 0 : "numElements (" + numElements + ") should >= 0"; - // Read the first element of offset to check its format id ( != DenseID - assert numElements == 0 || Platform.getInt(baseObject, baseOffset + 4) != DenseID: - "numElements (" + numElements + ") should == 0 or " + - "first offset (" + Platform.getInt(baseObject, baseOffset + 4) + ") should != DenseID"; - - this.numElements = numElements; - this.baseObject = baseObject; - this.baseOffset = baseOffset; - this.sizeInBytes = sizeInBytes; - } - - private int getElementOffset(int ordinal) { - return Platform.getInt(baseObject, baseOffset + 4 + ordinal * 4L); - } - - private int getElementSize(int offset, int ordinal) { - if (ordinal == numElements - 1) { - return sizeInBytes - offset; - } else { - return Math.abs(getElementOffset(ordinal + 1)) - offset; - } - } - - private void assertIndexIsValid(int ordinal) { - assert ordinal >= 0 : "ordinal (" + ordinal + ") should >= 0"; - assert ordinal < numElements : "ordinal (" + ordinal + ") should < " + numElements; - } - - @Override - public boolean isNullAt(int ordinal) { - assertIndexIsValid(ordinal); - return getElementOffset(ordinal) < 0; - } - - @Override - public boolean getBoolean(int ordinal) { - assertIndexIsValid(ordinal); - final int offset = getElementOffset(ordinal); - if (offset < 0) return false; - return Platform.getBoolean(baseObject, baseOffset + offset); - } - - @Override - public byte getByte(int ordinal) { - assertIndexIsValid(ordinal); - final int offset = getElementOffset(ordinal); - if (offset < 0) return 0; - return Platform.getByte(baseObject, baseOffset + offset); - } - - @Override - public short getShort(int ordinal) { - assertIndexIsValid(ordinal); - final int offset = getElementOffset(ordinal); - if (offset < 0) return 0; - return Platform.getShort(baseObject, baseOffset + offset); - } - - @Override - public int getInt(int ordinal) { - assertIndexIsValid(ordinal); - final int offset = getElementOffset(ordinal); - if (offset < 0) return 0; - return Platform.getInt(baseObject, baseOffset + offset); - } - - @Override - public long getLong(int ordinal) { - assertIndexIsValid(ordinal); - final int offset = getElementOffset(ordinal); - if (offset < 0) return 0; - return Platform.getLong(baseObject, baseOffset + offset); - } - - @Override - public float getFloat(int ordinal) { - assertIndexIsValid(ordinal); - final int offset = getElementOffset(ordinal); - if (offset < 0) return 0; - return Platform.getFloat(baseObject, baseOffset + offset); - } - - @Override - public double getDouble(int ordinal) { - assertIndexIsValid(ordinal); - final int offset = getElementOffset(ordinal); - if (offset < 0) return 0; - return Platform.getDouble(baseObject, baseOffset + offset); - } - - @Override - public Decimal getDecimal(int ordinal, int precision, int scale) { - assertIndexIsValid(ordinal); - final int offset = getElementOffset(ordinal); - if (offset < 0) return null; - - if (precision <= Decimal.MAX_LONG_DIGITS()) { - final long value = Platform.getLong(baseObject, baseOffset + offset); - return Decimal.apply(value, 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); - } - } - - @Override - public UTF8String getUTF8String(int ordinal) { - assertIndexIsValid(ordinal); - final int offset = getElementOffset(ordinal); - if (offset < 0) return null; - final int size = getElementSize(offset, ordinal); - return UTF8String.fromAddress(baseObject, baseOffset + offset, size); - } - - @Override - public byte[] getBinary(int ordinal) { - assertIndexIsValid(ordinal); - final int offset = getElementOffset(ordinal); - if (offset < 0) return null; - final int size = getElementSize(offset, ordinal); - final byte[] bytes = new byte[size]; - Platform.copyMemory(baseObject, baseOffset + offset, bytes, Platform.BYTE_ARRAY_OFFSET, size); - return bytes; - } - - @Override - public CalendarInterval getInterval(int ordinal) { - assertIndexIsValid(ordinal); - final int offset = getElementOffset(ordinal); - if (offset < 0) return null; - final int months = (int) Platform.getLong(baseObject, baseOffset + offset); - final long microseconds = Platform.getLong(baseObject, baseOffset + offset + 8); - return new CalendarInterval(months, microseconds); - } - - @Override - public UnsafeRow getStruct(int ordinal, int numFields) { - assertIndexIsValid(ordinal); - final int offset = getElementOffset(ordinal); - if (offset < 0) return null; - final int size = getElementSize(offset, ordinal); - final UnsafeRow row = new UnsafeRow(numFields); - row.pointTo(baseObject, baseOffset + offset, size); - return row; - } - - @Override - public UnsafeArrayData getArray(int ordinal) { - assertIndexIsValid(ordinal); - final int offset = getElementOffset(ordinal); - if (offset < 0) return null; - final int size = getElementSize(offset, ordinal); - final UnsafeArrayData array = UnsafeArrayData.allocate(Format.Sparse); - array.pointTo(baseObject, baseOffset + offset, size); - return array; - } - - @Override - public UnsafeMapData getMap(int ordinal) { - assertIndexIsValid(ordinal); - final int offset = getElementOffset(ordinal); - if (offset < 0) return null; - final int size = getElementSize(offset, ordinal); - final UnsafeMapData map = new UnsafeMapData(); - map.pointTo(baseObject, baseOffset + offset, size); - return map; - } - - @Override - public UnsafeArrayData copy() { - UnsafeArrayData arrayCopy = UnsafeArrayData.allocate(Format.Sparse); - final byte[] arrayDataCopy = new byte[sizeInBytes]; - Platform.copyMemory( - baseObject, baseOffset, arrayDataCopy, Platform.BYTE_ARRAY_OFFSET, sizeInBytes); - arrayCopy.pointTo(arrayDataCopy, Platform.BYTE_ARRAY_OFFSET, sizeInBytes); - return arrayCopy; - } - - protected static UnsafeArrayData _fromPrimitiveArray(int[] arr) { - if (arr.length > (Integer.MAX_VALUE - 4) / 8) { - throw new UnsupportedOperationException("Cannot convert this array to unsafe format as " + - "it's too big."); - } - - final int offsetRegionSize = 4 * arr.length; - final int valueRegionSize = 4 * arr.length; - final int totalSize = 4 + offsetRegionSize + valueRegionSize; - final byte[] data = new byte[totalSize]; - - Platform.putInt(data, Platform.BYTE_ARRAY_OFFSET, arr.length); - - int offsetPosition = Platform.BYTE_ARRAY_OFFSET + 4; - int valueOffset = 4 + offsetRegionSize; - for (int i = 0; i < arr.length; i++) { - Platform.putInt(data, offsetPosition, valueOffset); - offsetPosition += 4; - valueOffset += 4; - } - - Platform.copyMemory(arr, Platform.INT_ARRAY_OFFSET, data, - Platform.BYTE_ARRAY_OFFSET + 4 + offsetRegionSize, valueRegionSize); - - UnsafeArrayData result = UnsafeArrayData.allocate(Format.Sparse); - result.pointTo(data, Platform.BYTE_ARRAY_OFFSET, totalSize); - return result; - } - - protected static UnsafeArrayData _fromPrimitiveArray(double[] arr) { - if (arr.length > (Integer.MAX_VALUE - 4) / 12) { - throw new UnsupportedOperationException("Cannot convert this array to unsafe format as " + - "it's too big."); - } - - final int offsetRegionSize = 4 * arr.length; - final int valueRegionSize = 8 * arr.length; - final int totalSize = 4 + offsetRegionSize + valueRegionSize; - final byte[] data = new byte[totalSize]; - - Platform.putInt(data, Platform.BYTE_ARRAY_OFFSET, arr.length); - - int offsetPosition = Platform.BYTE_ARRAY_OFFSET + 4; - int valueOffset = 4 + offsetRegionSize; - for (int i = 0; i < arr.length; i++) { - Platform.putInt(data, offsetPosition, valueOffset); - offsetPosition += 4; - valueOffset += 8; - } - - Platform.copyMemory(arr, Platform.DOUBLE_ARRAY_OFFSET, data, - Platform.BYTE_ARRAY_OFFSET + 4 + offsetRegionSize, valueRegionSize); - - UnsafeArrayData result = UnsafeArrayData.allocate(Format.Sparse); - result.pointTo(data, Platform.BYTE_ARRAY_OFFSET, totalSize); - return result; - } - - // TODO: add more specialized methods. -} 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 2551177e7b086..0700148becaba 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 @@ -53,8 +53,8 @@ public final class UnsafeMapData extends MapData { * to a null pointer. */ public UnsafeMapData() { - keys = UnsafeArrayData.allocate(UnsafeArrayData.Format.Sparse); - values = UnsafeArrayData.allocate(UnsafeArrayData.Format.Sparse); + keys = new UnsafeArrayData(); + values = new UnsafeArrayData(); } /** 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 e132032f1ff38..dd2f39eb816f2 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 @@ -473,8 +473,7 @@ public UnsafeArrayData getArray(int ordinal) { final long offsetAndSize = getLong(ordinal); final int offset = (int) (offsetAndSize >> 32); final int size = (int) offsetAndSize; - final UnsafeArrayData.Format format = UnsafeArrayData.getFormat(baseObject, baseOffset + offset); - final UnsafeArrayData array = UnsafeArrayData.allocate(format); + final UnsafeArrayData array = new UnsafeArrayData(); array.pointTo(baseObject, baseOffset + offset, size); return array; } diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeArrayWriter.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeArrayWriter.java index 7dd932d1981b7..fef30236a77e5 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeArrayWriter.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeArrayWriter.java @@ -19,9 +19,14 @@ import org.apache.spark.sql.types.Decimal; import org.apache.spark.unsafe.Platform; +import org.apache.spark.unsafe.bitset.BitSetMethods; import org.apache.spark.unsafe.types.CalendarInterval; import org.apache.spark.unsafe.types.UTF8String; +import java.util.Arrays; + +import static org.apache.spark.sql.catalyst.expressions.UnsafeArrayData.calculateHeaderPortionInBytes; + /** * A helper class to write data into global row buffer using `UnsafeArrayData` format, * used by {@link org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection}. @@ -33,91 +38,145 @@ public class UnsafeArrayWriter { // The offset of the global buffer where we start to write this array. private int startingOffset; + // The number of elements in this array + private int numElements; + + private int headerInBytes; + + private void assertIndexIsValid(int index) { + assert index >= 0 : "index (" + index + ") should >= 0"; + assert index < numElements : "index (" + index + ") should < " + numElements; + } + public void initialize(BufferHolder holder, int numElements, int fixedElementSize) { - // We need 4 bytes to store numElements and 4 bytes each element to store offset. - final int fixedSize = 4 + 4 * numElements; + this.numElements = numElements; + this.headerInBytes = calculateHeaderPortionInBytes(numElements); this.holder = holder; this.startingOffset = holder.cursor; - holder.grow(fixedSize); - Platform.putInt(holder.buffer, holder.cursor, numElements); - holder.cursor += fixedSize; - // Grows the global buffer ahead for fixed size data. - holder.grow(fixedElementSize * numElements); + holder.grow(headerInBytes + fixedElementSize * numElements); + + Platform.putInt(holder.buffer, startingOffset, numElements); + Arrays.fill(holder.buffer, startingOffset + 4, startingOffset + headerInBytes, (byte)0); + + holder.cursor += (headerInBytes + fixedElementSize * numElements); } - private long getElementOffset(int ordinal) { - return startingOffset + 4 + 4 * ordinal; + private long getElementOffset(int ordinal, int scale) { + return startingOffset + headerInBytes + ordinal * scale; + } + + public void setOffsetAndSize(int ordinal, long currentCursor, long size) { + final long relativeOffset = currentCursor - startingOffset; + final long offsetAndSize = (relativeOffset << 32) | size; + + write(ordinal, offsetAndSize); } public void setNullAt(int ordinal) { - final int relativeOffset = holder.cursor - startingOffset; - // Writes negative offset value to represent null element. - Platform.putInt(holder.buffer, getElementOffset(ordinal), -relativeOffset); + throw new UnsupportedOperationException("setNullAt() is not supported"); + } + + private void setNullBit(int ordinal) { + assertIndexIsValid(ordinal); + BitSetMethods.set(holder.buffer, startingOffset + 4, ordinal); + } + + public void setNullBoolean(int ordinal) { + setNullBit(ordinal); + // put zero into the corresponding field when set null + Platform.putBoolean(holder.buffer, getElementOffset(ordinal, 1), false); + } + + public void setNullByte(int ordinal) { + setNullBit(ordinal); + // put zero into the corresponding field when set null + Platform.putByte(holder.buffer, getElementOffset(ordinal, 1), (byte)0); + } + + public void setNullShort(int ordinal) { + setNullBit(ordinal); + // put zero into the corresponding field when set null + Platform.putShort(holder.buffer, getElementOffset(ordinal, 2), (short)0); + } + + public void setNullInt(int ordinal) { + setNullBit(ordinal); + // put zero into the corresponding field when set null + Platform.putInt(holder.buffer, getElementOffset(ordinal, 4), (int)0); + } + + public void setNullLong(int ordinal) { + setNullBit(ordinal); + // put zero into the corresponding field when set null + Platform.putLong(holder.buffer, getElementOffset(ordinal, 8), (long)0); + } + + public void setNullFloat(int ordinal) { + setNullBit(ordinal); + // put zero into the corresponding field when set null + Platform.putFloat(holder.buffer, getElementOffset(ordinal, 4), (float)0); + } + + public void setNullDouble(int ordinal) { + setNullBit(ordinal); + // put zero into the corresponding field when set null + Platform.putDouble(holder.buffer, getElementOffset(ordinal, 8), (double)0); } - public void setOffset(int ordinal) { - final int relativeOffset = holder.cursor - startingOffset; - Platform.putInt(holder.buffer, getElementOffset(ordinal), relativeOffset); + public void setNull(int ordinal) { + setNullLong(ordinal); } public void write(int ordinal, boolean value) { - Platform.putBoolean(holder.buffer, holder.cursor, value); - setOffset(ordinal); - holder.cursor += 1; + assertIndexIsValid(ordinal); + Platform.putBoolean(holder.buffer, getElementOffset(ordinal, 1), value); } public void write(int ordinal, byte value) { - Platform.putByte(holder.buffer, holder.cursor, value); - setOffset(ordinal); - holder.cursor += 1; + assertIndexIsValid(ordinal); + Platform.putByte(holder.buffer, getElementOffset(ordinal, 1), value); } public void write(int ordinal, short value) { - Platform.putShort(holder.buffer, holder.cursor, value); - setOffset(ordinal); - holder.cursor += 2; + assertIndexIsValid(ordinal); + Platform.putShort(holder.buffer, getElementOffset(ordinal, 2), value); } public void write(int ordinal, int value) { - Platform.putInt(holder.buffer, holder.cursor, value); - setOffset(ordinal); - holder.cursor += 4; + assertIndexIsValid(ordinal); + Platform.putInt(holder.buffer, getElementOffset(ordinal, 4), value); } public void write(int ordinal, long value) { - Platform.putLong(holder.buffer, holder.cursor, value); - setOffset(ordinal); - holder.cursor += 8; + assertIndexIsValid(ordinal); + Platform.putLong(holder.buffer, getElementOffset(ordinal, 8), value); } public void write(int ordinal, float value) { if (Float.isNaN(value)) { value = Float.NaN; } - Platform.putFloat(holder.buffer, holder.cursor, value); - setOffset(ordinal); - holder.cursor += 4; + assertIndexIsValid(ordinal); + Platform.putFloat(holder.buffer, getElementOffset(ordinal, 4), value); } public void write(int ordinal, double value) { if (Double.isNaN(value)) { value = Double.NaN; } - Platform.putDouble(holder.buffer, holder.cursor, value); - setOffset(ordinal); - holder.cursor += 8; + assertIndexIsValid(ordinal); + Platform.putDouble(holder.buffer, getElementOffset(ordinal, 8), value); } public void write(int ordinal, Decimal input, int precision, int scale) { // make sure Decimal object has the same scale as DecimalType + assertIndexIsValid(ordinal); if (input.changePrecision(precision, scale)) { if (precision <= Decimal.MAX_LONG_DIGITS()) { - Platform.putLong(holder.buffer, holder.cursor, input.toUnscaledLong()); - setOffset(ordinal); - holder.cursor += 8; + write(ordinal, input.toUnscaledLong()); } else { final byte[] bytes = input.toJavaBigDecimal().unscaledValue().toByteArray(); assert bytes.length <= 16; @@ -126,11 +185,11 @@ public void write(int ordinal, Decimal input, int precision, int scale) { // Write the bytes to the variable length portion. Platform.copyMemory( bytes, Platform.BYTE_ARRAY_OFFSET, holder.buffer, holder.cursor, bytes.length); - setOffset(ordinal); + write(ordinal, ((long)(holder.cursor - startingOffset) << 32) | ((long) bytes.length)); holder.cursor += bytes.length; } } else { - setNullAt(ordinal); + setNull(ordinal); } } @@ -143,7 +202,7 @@ public void write(int ordinal, UTF8String input) { // Write the bytes to the variable length portion. input.writeToMemory(holder.buffer, holder.cursor); - setOffset(ordinal); + write(ordinal, ((long)(holder.cursor - startingOffset) << 32) | ((long) numBytes)); // move the cursor forward. holder.cursor += numBytes; @@ -157,7 +216,7 @@ public void write(int ordinal, byte[] input) { Platform.copyMemory( input, Platform.BYTE_ARRAY_OFFSET, holder.buffer, holder.cursor, input.length); - setOffset(ordinal); + write(ordinal, ((long)(holder.cursor - startingOffset) << 32) | ((long) input.length)); // move the cursor forward. holder.cursor += input.length; @@ -171,7 +230,7 @@ public void write(int ordinal, CalendarInterval input) { Platform.putLong(holder.buffer, holder.cursor, input.months); Platform.putLong(holder.buffer, holder.cursor + 8, input.microseconds); - setOffset(ordinal); + write(ordinal, ((long)(holder.cursor - startingOffset) << 32) | ((long) 16)); // move the cursor forward. holder.cursor += 16; diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjection.scala index 5efba4b3a6087..0ec2425c0909d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjection.scala @@ -192,26 +192,30 @@ object GenerateUnsafeProjection extends CodeGenerator[Seq[Expression], UnsafePro val fixedElementSize = et match { case t: DecimalType if t.precision <= Decimal.MAX_LONG_DIGITS => 8 case _ if ctx.isPrimitiveType(jt) => et.defaultSize - case _ => 0 + case _ => 8 } + val tmpCursor = ctx.freshName("tmpCursor") val writeElement = et match { case t: StructType => s""" - $arrayWriter.setOffset($index); + final int $tmpCursor = $bufferHolder.cursor; ${writeStructToBuffer(ctx, element, t.map(_.dataType), bufferHolder)} + $arrayWriter.setOffsetAndSize($index, $tmpCursor, $bufferHolder.cursor - $tmpCursor); """ case a @ ArrayType(et, _) => s""" - $arrayWriter.setOffset($index); + final int $tmpCursor = $bufferHolder.cursor; ${writeArrayToBuffer(ctx, element, et, bufferHolder)} + $arrayWriter.setOffsetAndSize($index, $tmpCursor, $bufferHolder.cursor - $tmpCursor); """ case m @ MapType(kt, vt, _) => s""" - $arrayWriter.setOffset($index); + final int $tmpCursor = $bufferHolder.cursor; ${writeMapToBuffer(ctx, element, kt, vt, bufferHolder)} + $arrayWriter.setOffsetAndSize($index, $tmpCursor, $bufferHolder.cursor - $tmpCursor); """ case t: DecimalType => @@ -222,6 +226,7 @@ object GenerateUnsafeProjection extends CodeGenerator[Seq[Expression], UnsafePro case _ => s"$arrayWriter.write($index, $element);" } + val dataType = if (ctx.isPrimitiveType(jt)) ctx.primitiveTypeName(et) else "" s""" if ($input instanceof UnsafeArrayData) { ${writeUnsafeData(ctx, s"((UnsafeArrayData) $input)", bufferHolder)} @@ -231,7 +236,7 @@ object GenerateUnsafeProjection extends CodeGenerator[Seq[Expression], UnsafePro for (int $index = 0; $index < $numElements; $index++) { if ($input.isNullAt($index)) { - $arrayWriter.setNullAt($index); + $arrayWriter.setNull$dataType($index); } else { final $jt $element = ${ctx.getValue(input, et, index)}; $writeElement diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/UnsafeArraySuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/UnsafeArraySuite.scala index b7ebd170af953..b89825f52c932 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/UnsafeArraySuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/UnsafeArraySuite.scala @@ -26,8 +26,7 @@ class UnsafeArraySuite extends SparkFunSuite { val array = Array(1, 10, 100) val unsafe = UnsafeArrayData.fromPrimitiveArray(array) assert(unsafe.numElements == 3) - assert(unsafe.getFormat == UnsafeArrayData.Format.Dense) - assert(unsafe.getSizeInBytes == 4 + 4 + 4 * 3) + assert(unsafe.getSizeInBytes == 4 + scala.math.ceil(3/64.toDouble) * 8 + 4 * 3) assert(unsafe.getInt(0) == 1) assert(unsafe.getInt(1) == 10) assert(unsafe.getInt(2) == 100) @@ -37,8 +36,7 @@ class UnsafeArraySuite extends SparkFunSuite { val array = Array(1.1, 2.2, 3.3) val unsafe = UnsafeArrayData.fromPrimitiveArray(array) assert(unsafe.numElements == 3) - assert(unsafe.getFormat == UnsafeArrayData.Format.Dense) - assert(unsafe.getSizeInBytes == 4 + 4 + 8 * 3) + assert(unsafe.getSizeInBytes == 4 + scala.math.ceil(3/64.toDouble) * 8 + 8 * 3) assert(unsafe.getDouble(0) == 1.1) assert(unsafe.getDouble(1) == 2.2) assert(unsafe.getDouble(2) == 3.3) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnType.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnType.scala index 9c8e8c7c4ba60..0e874ae12c8ca 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnType.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnType.scala @@ -614,7 +614,7 @@ private[columnar] case class ARRAY(dataType: ArrayType) assert(buffer.hasArray) val cursor = buffer.position() buffer.position(cursor + numBytes) - val array = UnsafeArrayData.allocate(UnsafeArrayData.Format.Sparse); + val array = new UnsafeArrayData(); array.pointTo( buffer.array(), Platform.BYTE_ARRAY_OFFSET + buffer.arrayOffset() + cursor, From a313084f0aed2cfa801f26ceb548c5fa0f37e10c Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Sat, 25 Jun 2016 13:07:40 +0900 Subject: [PATCH 07/37] fix test failures --- .../sql/catalyst/expressions/codegen/UnsafeArrayWriter.java | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeArrayWriter.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeArrayWriter.java index fef30236a77e5..cfed1776623b0 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeArrayWriter.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeArrayWriter.java @@ -55,11 +55,13 @@ public void initialize(BufferHolder holder, int numElements, int fixedElementSiz this.holder = holder; this.startingOffset = holder.cursor; - // Grows the global buffer ahead for fixed size data. + // Grows the global buffer ahead for header and fixed size data. holder.grow(headerInBytes + fixedElementSize * numElements); + // Initialize information in header Platform.putInt(holder.buffer, startingOffset, numElements); - Arrays.fill(holder.buffer, startingOffset + 4, startingOffset + headerInBytes, (byte)0); + Arrays.fill(holder.buffer, startingOffset + 4 - Platform.BYTE_ARRAY_OFFSET, + startingOffset + headerInBytes - Platform.BYTE_ARRAY_OFFSET, (byte)0); holder.cursor += (headerInBytes + fixedElementSize * numElements); } From 68d92f7b5ad175b1403c23edce1c930678b42fe2 Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Sat, 25 Jun 2016 20:42:08 +0900 Subject: [PATCH 08/37] fix test failures --- .../apache/spark/sql/execution/columnar/ColumnTypeSuite.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/ColumnTypeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/ColumnTypeSuite.scala index 052f4cbaebc8e..597009490e40d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/ColumnTypeSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/ColumnTypeSuite.scala @@ -73,8 +73,8 @@ class ColumnTypeSuite extends SparkFunSuite with Logging { checkActualSize(BINARY, Array.fill[Byte](4)(0.toByte), 4 + 4) checkActualSize(COMPACT_DECIMAL(15, 10), Decimal(0, 15, 10), 8) checkActualSize(LARGE_DECIMAL(20, 10), Decimal(0, 20, 10), 5) - checkActualSize(ARRAY_TYPE, Array[Any](1), 16) - checkActualSize(MAP_TYPE, Map(1 -> "a"), 29) + checkActualSize(ARRAY_TYPE, Array[Any](1), 4 + 8 + 8) + checkActualSize(MAP_TYPE, Map(1 -> "a"), 4 + (4 + 8 + 8) * 2 + 1) checkActualSize(STRUCT_TYPE, Row("hello"), 28) } From 7f2da14f1235ca503df91f79daaf8284569427da Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Sat, 25 Jun 2016 23:39:38 +0900 Subject: [PATCH 09/37] update test suite --- .../expressions/UnsafeRowConverterSuite.scala | 15 ++++++++------- 1 file changed, 8 insertions(+), 7 deletions(-) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/UnsafeRowConverterSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/UnsafeRowConverterSuite.scala index 1265908182b3a..8ab54faddebf6 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/UnsafeRowConverterSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/UnsafeRowConverterSuite.scala @@ -300,7 +300,8 @@ class UnsafeRowConverterSuite extends SparkFunSuite with Matchers { private def testArrayInt(array: UnsafeArrayData, values: Seq[Int]): Unit = { assert(array.numElements == values.length) - assert(array.getSizeInBytes == 4 + (4 + 4) * values.length) + assert(array.getSizeInBytes == + 4 + scala.math.ceil(values.length / 64.toDouble) * 8 + 4 * values.length) values.zipWithIndex.foreach { case (value, index) => assert(array.getInt(index) == value) } @@ -339,7 +340,7 @@ class UnsafeRowConverterSuite extends SparkFunSuite with Matchers { val nestedArray = unsafeArray2.getArray(0) testArrayInt(nestedArray, Seq(3, 4)) - assert(unsafeArray2.getSizeInBytes == 4 + 4 + nestedArray.getSizeInBytes) + assert(unsafeArray2.getSizeInBytes == 4 + 8 + 8+ nestedArray.getSizeInBytes) val array1Size = roundedSize(unsafeArray1.getSizeInBytes) val array2Size = roundedSize(unsafeArray2.getSizeInBytes) @@ -382,7 +383,7 @@ class UnsafeRowConverterSuite extends SparkFunSuite with Matchers { val nestedMap = valueArray.getMap(0) testMapInt(nestedMap, Seq(5, 6), Seq(7, 8)) - assert(valueArray.getSizeInBytes == 4 + 4 + nestedMap.getSizeInBytes) + assert(valueArray.getSizeInBytes == 4 + 8 + 8 + nestedMap.getSizeInBytes) } assert(unsafeMap2.getSizeInBytes == 4 + keyArray.getSizeInBytes + valueArray.getSizeInBytes) @@ -425,7 +426,7 @@ class UnsafeRowConverterSuite extends SparkFunSuite with Matchers { assert(innerStruct.getLong(0) == 2L) } - assert(field2.getSizeInBytes == 4 + 4 + innerStruct.getSizeInBytes) + assert(field2.getSizeInBytes == 4 + 8 + 8 + innerStruct.getSizeInBytes) assert(unsafeRow.getSizeInBytes == 8 + 8 * 2 + field1.getSizeInBytes + roundedSize(field2.getSizeInBytes)) @@ -468,7 +469,7 @@ class UnsafeRowConverterSuite extends SparkFunSuite with Matchers { assert(innerStruct.getSizeInBytes == 8 + 8) assert(innerStruct.getLong(0) == 4L) - assert(valueArray.getSizeInBytes == 4 + 4 + innerStruct.getSizeInBytes) + assert(valueArray.getSizeInBytes == 4 + 8 + 8 + innerStruct.getSizeInBytes) } assert(field2.getSizeInBytes == 4 + keyArray.getSizeInBytes + valueArray.getSizeInBytes) @@ -497,7 +498,7 @@ class UnsafeRowConverterSuite extends SparkFunSuite with Matchers { val innerMap = field1.getMap(0) testMapInt(innerMap, Seq(1), Seq(2)) - assert(field1.getSizeInBytes == 4 + 4 + innerMap.getSizeInBytes) + assert(field1.getSizeInBytes == 4 + 8 + 8 + innerMap.getSizeInBytes) val field2 = unsafeRow.getMap(1) assert(field2.numElements == 1) @@ -513,7 +514,7 @@ class UnsafeRowConverterSuite extends SparkFunSuite with Matchers { val innerArray = valueArray.getArray(0) testArrayInt(innerArray, Seq(4)) - assert(valueArray.getSizeInBytes == 4 + (4 + innerArray.getSizeInBytes)) + assert(valueArray.getSizeInBytes == 4 + 8 + 8 + innerArray.getSizeInBytes) } assert(field2.getSizeInBytes == 4 + keyArray.getSizeInBytes + valueArray.getSizeInBytes) From 2f26f6f7a923c6ed61ae252551c1dc1e883ead07 Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Sun, 26 Jun 2016 00:26:30 +0900 Subject: [PATCH 10/37] fix scala style error --- .../sql/catalyst/expressions/UnsafeRowConverterSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/UnsafeRowConverterSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/UnsafeRowConverterSuite.scala index 8ab54faddebf6..3b5938e8a1a34 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/UnsafeRowConverterSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/UnsafeRowConverterSuite.scala @@ -340,7 +340,7 @@ class UnsafeRowConverterSuite extends SparkFunSuite with Matchers { val nestedArray = unsafeArray2.getArray(0) testArrayInt(nestedArray, Seq(3, 4)) - assert(unsafeArray2.getSizeInBytes == 4 + 8 + 8+ nestedArray.getSizeInBytes) + assert(unsafeArray2.getSizeInBytes == 4 + 8 + 8 + nestedArray.getSizeInBytes) val array1Size = roundedSize(unsafeArray1.getSizeInBytes) val array2Size = roundedSize(unsafeArray2.getSizeInBytes) From ccef63c9106b910a6b42a3e4a33360b3eea59ace Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Sun, 26 Jun 2016 02:13:26 +0900 Subject: [PATCH 11/37] revert changes --- .../org/apache/spark/sql/execution/columnar/ColumnType.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnType.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnType.scala index 0e874ae12c8ca..f9d606e37ea89 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnType.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnType.scala @@ -614,7 +614,7 @@ private[columnar] case class ARRAY(dataType: ArrayType) assert(buffer.hasArray) val cursor = buffer.position() buffer.position(cursor + numBytes) - val array = new UnsafeArrayData(); + val array = new UnsafeArrayData array.pointTo( buffer.array(), Platform.BYTE_ARRAY_OFFSET + buffer.arrayOffset() + cursor, From c4f1b5e269dbb31b41b40f8472f8827fbb997bfe Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Tue, 28 Jun 2016 22:15:45 +0900 Subject: [PATCH 12/37] addressed comments --- .../catalyst/expressions/UnsafeArrayData.java | 120 +++++++++--------- 1 file changed, 63 insertions(+), 57 deletions(-) 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 0d4e4c11e353c..268d841cf7ac2 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 @@ -33,21 +33,20 @@ /** * An Unsafe implementation of Array which is backed by raw memory instead of Java objects. * - * Each tuple has three four: [numElements] [null bits] [values] [variable length portion] + * Each tuple has four parts: [numElements][null bits][values or offset][variable length portion] * * The `numElements` is 4 bytes storing the number of elements of this array. * * In the `null bits` region, we store 1 bit per element, represents whether a element has null * Its total size is ceil(numElements / 8) bytes, and it is aligned to 8-byte word boundaries. * - * In the `offsets` region, we store 4 bytes per element, represents the relative offset (w.r.t. the - * base address of the array) of this element in `values` region. We can get the length of this - * element by subtracting next offset. - * Note that offset can by negative which means this element is null. - * - * In the `values` region, we store the content of elements. As we can get length info, so elements - * can be variable-length. + * In the `values or offset` region, we store the content of elements. For fields that hold + * fixed-length primitive types, such as long, double, or int, we store the value directly + * in the field. For fields with non-primitive or variable-length values, we store a relative + * offset (w.r.t. the base address of the row) that points to the beginning of the variable-length + * field, and length (they are combined into a long). * + * Instances of `UnsafeArrayData` act as pointers to row data stored in this format. */ // todo: there is a lof of duplicated code between UnsafeRow and UnsafeArrayData. @@ -67,11 +66,11 @@ public static int calculateHeaderPortionInBytes(int numFields) { // The 4-bytes header of `numElements` is also included. private int sizeInBytes; - /** The width of the null tracking bit set, in bytes */ + /** The width of the null tracking bit set plus `numElements`, in bytes */ private int headerInBytes; - private long getFieldOffset(int ordinal, int scale) { - return baseOffset + headerInBytes + ordinal * scale; + private long getFieldOffset(int ordinal, int elementSize) { + return baseOffset + headerInBytes + ordinal * elementSize; } public Object getBaseObject() { return baseObject; } @@ -92,7 +91,7 @@ public Object[] array() { * `pointTo()` has been called, since the value returned by this constructor is equivalent * to a null pointer. */ - public UnsafeArrayData() { } + public UnsafeArrayData() { } @Override public int numElements() { return numElements; } @@ -123,7 +122,7 @@ public boolean isNullAt(int ordinal) { } @Override - public final Object get(int ordinal, DataType dataType) { + public Object get(int ordinal, DataType dataType) { if (isNullAt(ordinal) || dataType instanceof NullType) { return null; } else if (dataType instanceof BooleanType) { @@ -210,6 +209,7 @@ public double getDouble(int ordinal) { @Override public Decimal getDecimal(int ordinal, int precision, int scale) { + assertIndexIsValid(ordinal); if (isNullAt(ordinal)) { return null; } @@ -286,11 +286,30 @@ public UnsafeMapData getMap(int ordinal) { return map; } - public final void writeToMemory(Object target, long targetOffset) { + // This `hashCode` computation could consume much processor time for large data. + // If the computation becomes a bottleneck, we can use a light-weight logic; the first fixed bytes + // are used to compute `hashCode` (See `Vector.hashCode`). + // The same issue exists in `UnsafeRow.hashCode`. + @Override + public int hashCode() { + return Murmur3_x86_32.hashUnsafeBytes(baseObject, baseOffset, sizeInBytes, 42); + } + + @Override + public boolean equals(Object other) { + if (other instanceof UnsafeArrayData) { + UnsafeArrayData o = (UnsafeArrayData) other; + return (sizeInBytes == o.sizeInBytes) && + ByteArrayMethods.arrayEquals(baseObject, baseOffset, o.baseObject, o.baseOffset, + sizeInBytes); + } + return false; + } + public void writeToMemory(Object target, long targetOffset) { Platform.copyMemory(baseObject, baseOffset, target, targetOffset, sizeInBytes); } - public final void writeTo(ByteBuffer buffer) { + public void writeTo(ByteBuffer buffer) { assert(buffer.hasArray()); byte[] target = buffer.array(); int offset = buffer.arrayOffset(); @@ -299,10 +318,6 @@ public final void writeTo(ByteBuffer buffer) { buffer.position(pos + sizeInBytes); } - // This `hashCode` computation could consume much processor time for large data. - // If the computation becomes a bottleneck, we can use a light-weight logic; the first fixed bytes - // are used to compute `hashCode` (See `Vector.hashCode`). - // The same issue exists in `UnsafeRow.hashCode`. @Override public UnsafeArrayData copy() { UnsafeArrayData arrayCopy = new UnsafeArrayData(); @@ -336,7 +351,7 @@ public short[] toShortArray() { int size = numElements(); short[] values = new short[size]; Platform.copyMemory( - baseObject, baseOffset + headerInBytes, values, Platform.BYTE_ARRAY_OFFSET, size * 2); + baseObject, baseOffset + headerInBytes, values, Platform.SHORT_ARRAY_OFFSET, size * 2); return values; } @@ -345,7 +360,7 @@ public int[] toIntArray() { int size = numElements(); int[] values = new int[size]; Platform.copyMemory( - baseObject, baseOffset + headerInBytes, values, Platform.BYTE_ARRAY_OFFSET, size * 4); + baseObject, baseOffset + headerInBytes, values, Platform.INT_ARRAY_OFFSET, size * 4); return values; } @@ -354,7 +369,7 @@ public long[] toLongArray() { int size = numElements(); long[] values = new long[size]; Platform.copyMemory( - baseObject, baseOffset + headerInBytes, values, Platform.BYTE_ARRAY_OFFSET, size * 8); + baseObject, baseOffset + headerInBytes, values, Platform.LONG_ARRAY_OFFSET, size * 8); return values; } @@ -363,7 +378,7 @@ public float[] toFloatArray() { int size = numElements(); float[] values = new float[size]; Platform.copyMemory( - baseObject, baseOffset + headerInBytes, values, Platform.BYTE_ARRAY_OFFSET, size * 4); + baseObject, baseOffset + headerInBytes, values, Platform.FLOAT_ARRAY_OFFSET, size * 4); return values; } @@ -372,22 +387,21 @@ public double[] toDoubleArray() { int size = numElements(); double[] values = new double[size]; Platform.copyMemory( - baseObject, baseOffset + headerInBytes, values, Platform.BYTE_ARRAY_OFFSET, size * 8); + baseObject, baseOffset + headerInBytes, values, Platform.DOUBLE_ARRAY_OFFSET, size * 8); return values; } - public static UnsafeArrayData fromPrimitiveArray(int[] arr) { - final int elementSize = 4; - final int headerSize = calculateHeaderPortionInBytes(arr.length); - if (arr.length > (Integer.MAX_VALUE - headerSize) / elementSize) { + private static UnsafeArrayData fromPrimitiveArray(Object arr, int length, final int elementSize) { + final int headerSize = calculateHeaderPortionInBytes(length); + if (length > (Integer.MAX_VALUE - headerSize) / elementSize) { throw new UnsupportedOperationException("Cannot convert this array to unsafe format as " + "it's too big."); } - final int valueRegionSize = elementSize * arr.length; + final int valueRegionSize = elementSize * length; final byte[] data = new byte[valueRegionSize + headerSize]; - Platform.putInt(data, Platform.BYTE_ARRAY_OFFSET, arr.length); + Platform.putInt(data, Platform.BYTE_ARRAY_OFFSET, length); Platform.copyMemory(arr, Platform.INT_ARRAY_OFFSET, data, Platform.BYTE_ARRAY_OFFSET + headerSize, valueRegionSize); @@ -396,39 +410,31 @@ public static UnsafeArrayData fromPrimitiveArray(int[] arr) { return result; } - public static UnsafeArrayData fromPrimitiveArray(double[] arr) { - final int elementSize = 8; - final int headerSize = calculateHeaderPortionInBytes(arr.length); - if (arr.length > (Integer.MAX_VALUE - headerSize) / elementSize) { - throw new UnsupportedOperationException("Cannot convert this array to unsafe format as " + - "it's too big."); - } + public static UnsafeArrayData fromPrimitiveArray(boolean[] arr) { + return fromPrimitiveArray(arr, arr.length, 1); + } - final int valueRegionSize = elementSize * arr.length; - final byte[] data = new byte[valueRegionSize + headerSize]; + public static UnsafeArrayData fromPrimitiveArray(byte[] arr) { + return fromPrimitiveArray(arr, arr.length, 1); + } - Platform.putInt(data, Platform.BYTE_ARRAY_OFFSET, arr.length); - Platform.copyMemory(arr, Platform.DOUBLE_ARRAY_OFFSET, data, - Platform.BYTE_ARRAY_OFFSET + headerSize, valueRegionSize); + public static UnsafeArrayData fromPrimitiveArray(short[] arr) { + return fromPrimitiveArray(arr, arr.length, 2); + } - UnsafeArrayData result = new UnsafeArrayData(); - result.pointTo(data, Platform.BYTE_ARRAY_OFFSET, valueRegionSize + headerSize); - return result; + public static UnsafeArrayData fromPrimitiveArray(int[] arr) { + return fromPrimitiveArray(arr, arr.length, 4); } - @Override - public int hashCode() { - return Murmur3_x86_32.hashUnsafeBytes(baseObject, baseOffset, sizeInBytes, 42); + public static UnsafeArrayData fromPrimitiveArray(long[] arr) { + return fromPrimitiveArray(arr, arr.length, 8); } - @Override - public boolean equals(Object other) { - if (other instanceof UnsafeArrayData) { - UnsafeArrayData o = (UnsafeArrayData) other; - return (sizeInBytes == o.sizeInBytes) && - ByteArrayMethods.arrayEquals(baseObject, baseOffset, o.baseObject, o.baseOffset, - sizeInBytes); - } - return false; + public static UnsafeArrayData fromPrimitiveArray(float[] arr) { + return fromPrimitiveArray(arr, arr.length, 4); + } + + public static UnsafeArrayData fromPrimitiveArray(double[] arr) { + return fromPrimitiveArray(arr, arr.length, 8); } } From 34a5c6a8376e8a31e6ec29d4dc54232ee19defd2 Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Tue, 28 Jun 2016 22:16:30 +0900 Subject: [PATCH 13/37] add benchmark --- .../benchmark/UnsafeArrayDataBenchmark.scala | 298 ++++++++++++++++++ 1 file changed, 298 insertions(+) create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/UnsafeArrayDataBenchmark.scala diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/UnsafeArrayDataBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/UnsafeArrayDataBenchmark.scala new file mode 100644 index 0000000000000..6fb3a970e5744 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/UnsafeArrayDataBenchmark.scala @@ -0,0 +1,298 @@ +/* + * 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.execution.benchmark + +import org.apache.spark.SparkConf +import org.apache.spark.sql.catalyst.expressions.{UnsafeArrayData, UnsafeRow} +import org.apache.spark.sql.catalyst.expressions.codegen.{BufferHolder, UnsafeArrayWriter} +import org.apache.spark.unsafe.Platform +import org.apache.spark.util.Benchmark + +/** + * Benchmark [[UnsafeArrayDataBenchmark]] for UnsafeArrayData + * To run this: + * build/sbt "sql/test-only *benchmark.UnsafeArrayDataBenchmark" + * + * Benchmarks in this file are skipped in normal builds. + */ +class UnsafeArrayDataBenchmark extends BenchmarkBase { + + new SparkConf() + .setMaster("local[1]") + .setAppName("microbenchmark") + .set("spark.driver.memory", "3g") + + def calculateHeaderPortionInBytes(count: Int) : Int = { + // Use this assignment for SPARK-15962 + // val size = 4 + 4 * count + val size = UnsafeArrayData.calculateHeaderPortionInBytes(count) + size + } + + def readUnsafeArray(iters: Int): Unit = { + val count = 1024 * 1024 * 16 + + val intUnsafeArray = new UnsafeArrayData + var intResult: Int = 0 + val intSize = calculateHeaderPortionInBytes(count) + 4 * count + val intBuffer = new Array[Byte](intSize) + Platform.putInt(intBuffer, Platform.BYTE_ARRAY_OFFSET, count) + intUnsafeArray.pointTo(intBuffer, Platform.BYTE_ARRAY_OFFSET, intSize) + val readIntArray = { i: Int => + var n = 0 + while (n < iters) { + val len = intUnsafeArray.numElements + var sum = 0.toInt + var i = 0 + while (i < len) { + sum += intUnsafeArray.getInt(i) + i += 1 + } + intResult = sum + n += 1 + } + } + + val doubleUnsafeArray = new UnsafeArrayData + var doubleResult: Double = 0 + val doubleSize = calculateHeaderPortionInBytes(count) + 8 * count + val doubleBuffer = new Array[Byte](doubleSize) + Platform.putInt(doubleBuffer, Platform.BYTE_ARRAY_OFFSET, count) + doubleUnsafeArray.pointTo(doubleBuffer, Platform.BYTE_ARRAY_OFFSET, doubleSize) + val readDoubleArray = { i: Int => + var n = 0 + while (n < iters) { + val len = doubleUnsafeArray.numElements + var sum = 0.toDouble + var i = 0 + while (i < len) { + sum += doubleUnsafeArray.getDouble(i) + i += 1 + } + doubleResult = sum + n += 1 + } + } + + val benchmark = new Benchmark("Read UnsafeArrayData", count * iters) + benchmark.addCase("Int")(readIntArray) + benchmark.addCase("Double")(readDoubleArray) + benchmark.run + /* + Without SPARK-15962 + OpenJDK 64-Bit Server VM 1.8.0_91-b14 on Linux 4.0.4-301.fc22.x86_64 + Intel Xeon E3-12xx v2 (Ivy Bridge) + Read UnsafeArrayData: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------------ + Int 370 / 471 454.0 2.2 1.0X + Double 351 / 466 477.5 2.1 1.1X + */ + /* + With SPARK-15962 + OpenJDK 64-Bit Server VM 1.8.0_91-b14 on Linux 4.0.4-301.fc22.x86_64 + Intel Xeon E3-12xx v2 (Ivy Bridge) + Read UnsafeArrayData: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------------ + Int 246 / 259 682.8 1.5 1.0X + Double 259 / 265 648.4 1.5 0.9X + */ + } + + def writeUnsafeArray(iters: Int): Unit = { + val count = 1024 * 1024 * 16 + + val intUnsafeRow = new UnsafeRow(1) + val intUnsafeArrayWriter = new UnsafeArrayWriter + val intBufferHolder = new BufferHolder(intUnsafeRow, 64) + intBufferHolder.reset() + intUnsafeArrayWriter.initialize(intBufferHolder, count, 4) + val intCursor = intBufferHolder.cursor + val writeIntArray = { i: Int => + var n = 0 + while (n < iters) { + intBufferHolder.cursor = intCursor + val len = count + var i = 0 + while (i < len) { + intUnsafeArrayWriter.write(i, 0.toInt) + i += 1 + } + n += 1 + } + } + + val doubleUnsafeRow = new UnsafeRow(1) + val doubleUnsafeArrayWriter = new UnsafeArrayWriter + val doubleBufferHolder = new BufferHolder(doubleUnsafeRow, 64) + doubleBufferHolder.reset() + doubleUnsafeArrayWriter.initialize(doubleBufferHolder, count, 8) + val doubleCursor = doubleBufferHolder.cursor + val writeDoubleArray = { i: Int => + var n = 0 + while (n < iters) { + doubleBufferHolder.cursor = doubleCursor + val len = count + var i = 0 + while (i < len) { + doubleUnsafeArrayWriter.write(i, 0.toDouble) + i += 1 + } + n += 1 + } + } + + val benchmark = new Benchmark("Write UnsafeArrayData", count * iters) + benchmark.addCase("Int")(writeIntArray) + benchmark.addCase("Double")(writeDoubleArray) + benchmark.run + /* + Without SPARK-15962 + OpenJDK 64-Bit Server VM 1.8.0_91-b14 on Linux 4.0.4-301.fc22.x86_64 + Intel Xeon E3-12xx v2 (Ivy Bridge) + Write UnsafeArrayData: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------------ + Int 337 / 407 498.0 2.0 1.0X + Double 458 / 496 366.2 2.7 0.7X + */ + /* + With SPARK-15962 + OpenJDK 64-Bit Server VM 1.8.0_91-b14 on Linux 4.0.4-301.fc22.x86_64 + Intel Xeon E3-12xx v2 (Ivy Bridge) + Write UnsafeArrayData: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------------ + Int 84 / 85 2005.4 0.5 1.0X + Double 152 / 154 1102.5 0.9 0.5X + */ + } + + def getPrimitiveArray(iters: Int): Unit = { + val count = 1024 * 1024 * 12 + + val intUnsafeArray = new UnsafeArrayData + val intSize = calculateHeaderPortionInBytes(count) + 4 * count + val intBuffer = new Array[Byte](intSize) + Platform.putInt(intBuffer, Platform.BYTE_ARRAY_OFFSET, count) + intUnsafeArray.pointTo(intBuffer, Platform.BYTE_ARRAY_OFFSET, intSize) + var intPrimitiveArray: Array[Int] = null + val readIntArray = { i: Int => + var n = 0 + while (n < iters) { + intPrimitiveArray = intUnsafeArray.toIntArray + n += 1 + } + } + + val doubleUnsafeArray = new UnsafeArrayData + val doubleSize = calculateHeaderPortionInBytes(count) + 8 * count + val doubleBuffer = new Array[Byte](doubleSize) + Platform.putInt(doubleBuffer, Platform.BYTE_ARRAY_OFFSET, count) + doubleUnsafeArray.pointTo(doubleBuffer, Platform.BYTE_ARRAY_OFFSET, doubleSize) + var doublePrimitiveArray: Array[Double] = null + val readDoubleArray = { i: Int => + var n = 0 + while (n < iters) { + doublePrimitiveArray = doubleUnsafeArray.toDoubleArray + n += 1 + } + } + + val benchmark = new Benchmark("Get primitive array from UnsafeArrayData", count * iters) + benchmark.addCase("Int")(readIntArray) + benchmark.addCase("Double")(readDoubleArray) + benchmark.run + /* + Without SPARK-15962 + OpenJDK 64-Bit Server VM 1.8.0_91-b14 on Linux 4.0.4-301.fc22.x86_64 + Intel Xeon E3-12xx v2 (Ivy Bridge) + Get primitive array from UnsafeArrayData: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------------ + Int 218 / 256 288.1 3.5 1.0X + Double 318 / 539 198.0 5.1 0.7X + */ + /* + With SPARK-15962 + OpenJDK 64-Bit Server VM 1.8.0_91-b14 on Linux 4.0.4-301.fc22.x86_64 + Intel Xeon E3-12xx v2 (Ivy Bridge) + Get primitive array from UnsafeArrayData: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------------ + Int 86 / 96 734.2 1.4 1.0X + Double 203 / 268 310.6 3.2 0.4X + */ + } + + def putPrimitiveArray(iters: Int): Unit = { + val count = 1024 * 1024 * 12 + + val intPrimitiveArray: Array[Int] = new Array[Int](count) + var intUnsafeArray: UnsafeArrayData = null + val createIntArray = { i: Int => + var n = 0 + while (n < iters) { + intUnsafeArray = UnsafeArrayData.fromPrimitiveArray(intPrimitiveArray) + n += 1 + } + } + + val doublePrimitiveArray: Array[Double] = new Array[Double](count) + var doubleUnsafeArray: UnsafeArrayData = null + val createDoubleArray = { i: Int => + var n = 0 + while (n < iters) { + doubleUnsafeArray = UnsafeArrayData.fromPrimitiveArray(doublePrimitiveArray) + n += 1 + } + } + + val benchmark = new Benchmark("Create UnsafeArrayData from primitive array", count * iters) + benchmark.addCase("Int")(createIntArray) + benchmark.addCase("Double")(createDoubleArray) + benchmark.run + /* + Without SPARK-15962 + OpenJDK 64-Bit Server VM 1.8.0_91-b14 on Linux 4.0.4-301.fc22.x86_64 + Intel Xeon E3-12xx v2 (Ivy Bridge) + Create UnsafeArrayData from primitive array: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------------ + Int 343 / 437 183.6 5.4 1.0X + Double 322 / 505 195.6 5.1 1.1X + */ + /* + With SPARK-15962 + OpenJDK 64-Bit Server VM 1.8.0_91-b14 on Linux 4.0.4-301.fc22.x86_64 + Intel Xeon E3-12xx v2 (Ivy Bridge) + Create UnsafeArrayData from primitive array: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------------ + Int 84 / 95 748.3 1.3 1.0X + Double 196 / 227 320.9 3.1 0.4X + */ + } + + ignore("Benchmark UnsafeArrayData") { + readUnsafeArray(10) + writeUnsafeArray(10) + getPrimitiveArray(5) + putPrimitiveArray(5) + } + + def main(args: Array[String]): Unit = { + readUnsafeArray(10) + writeUnsafeArray(10) + getPrimitiveArray(5) + putPrimitiveArray(5) + } +} From 7a77b20c670df6f6801c6f4150d9b76b8d6bebaa Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Tue, 28 Jun 2016 22:31:21 +0900 Subject: [PATCH 14/37] fix scala style error --- .../execution/benchmark/UnsafeArrayDataBenchmark.scala | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/UnsafeArrayDataBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/UnsafeArrayDataBenchmark.scala index 6fb3a970e5744..a83bc36bbb9fe 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/UnsafeArrayDataBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/UnsafeArrayDataBenchmark.scala @@ -219,7 +219,7 @@ class UnsafeArrayDataBenchmark extends BenchmarkBase { Without SPARK-15962 OpenJDK 64-Bit Server VM 1.8.0_91-b14 on Linux 4.0.4-301.fc22.x86_64 Intel Xeon E3-12xx v2 (Ivy Bridge) - Get primitive array from UnsafeArrayData: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + Get primitive array from UnsafeArrayData: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ Int 218 / 256 288.1 3.5 1.0X Double 318 / 539 198.0 5.1 0.7X @@ -228,7 +228,7 @@ class UnsafeArrayDataBenchmark extends BenchmarkBase { With SPARK-15962 OpenJDK 64-Bit Server VM 1.8.0_91-b14 on Linux 4.0.4-301.fc22.x86_64 Intel Xeon E3-12xx v2 (Ivy Bridge) - Get primitive array from UnsafeArrayData: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + Get primitive array from UnsafeArrayData: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ Int 86 / 96 734.2 1.4 1.0X Double 203 / 268 310.6 3.2 0.4X @@ -266,7 +266,7 @@ class UnsafeArrayDataBenchmark extends BenchmarkBase { Without SPARK-15962 OpenJDK 64-Bit Server VM 1.8.0_91-b14 on Linux 4.0.4-301.fc22.x86_64 Intel Xeon E3-12xx v2 (Ivy Bridge) - Create UnsafeArrayData from primitive array: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + Create UnsafeArrayData from primitive array: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ Int 343 / 437 183.6 5.4 1.0X Double 322 / 505 195.6 5.1 1.1X @@ -275,7 +275,7 @@ class UnsafeArrayDataBenchmark extends BenchmarkBase { With SPARK-15962 OpenJDK 64-Bit Server VM 1.8.0_91-b14 on Linux 4.0.4-301.fc22.x86_64 Intel Xeon E3-12xx v2 (Ivy Bridge) - Create UnsafeArrayData from primitive array: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + Create UnsafeArrayData from primitive array: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ Int 84 / 95 748.3 1.3 1.0X Double 196 / 227 320.9 3.1 0.4X From 7b0d4dad0c507ef491e2cdd18c3e76530ddeb94b Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Sat, 2 Jul 2016 00:20:40 +0900 Subject: [PATCH 15/37] addressed comments --- .../catalyst/expressions/UnsafeArrayData.java | 26 ++++++++----------- 1 file changed, 11 insertions(+), 15 deletions(-) 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 268d841cf7ac2..6beb737ec0527 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 @@ -49,7 +49,6 @@ * Instances of `UnsafeArrayData` act as pointers to row data stored in this format. */ -// todo: there is a lof of duplicated code between UnsafeRow and UnsafeArrayData. public final class UnsafeArrayData extends ArrayData { public static int calculateHeaderPortionInBytes(int numFields) { @@ -69,7 +68,7 @@ public static int calculateHeaderPortionInBytes(int numFields) { /** The width of the null tracking bit set plus `numElements`, in bytes */ private int headerInBytes; - private long getFieldOffset(int ordinal, int elementSize) { + private long getElementOffset(int ordinal, int elementSize) { return baseOffset + headerInBytes + ordinal * elementSize; } @@ -168,51 +167,48 @@ public Object get(int ordinal, DataType dataType) { @Override public boolean getBoolean(int ordinal) { assertIndexIsValid(ordinal); - return Platform.getBoolean(baseObject, getFieldOffset(ordinal, 1)); + return Platform.getBoolean(baseObject, getElementOffset(ordinal, 1)); } @Override public byte getByte(int ordinal) { assertIndexIsValid(ordinal); - return Platform.getByte(baseObject, getFieldOffset(ordinal, 1)); + return Platform.getByte(baseObject, getElementOffset(ordinal, 1)); } @Override public short getShort(int ordinal) { assertIndexIsValid(ordinal); - return Platform.getShort(baseObject, getFieldOffset(ordinal, 2)); + return Platform.getShort(baseObject, getElementOffset(ordinal, 2)); } @Override public int getInt(int ordinal) { assertIndexIsValid(ordinal); - return Platform.getInt(baseObject, getFieldOffset(ordinal, 4)); + return Platform.getInt(baseObject, getElementOffset(ordinal, 4)); } @Override public long getLong(int ordinal) { assertIndexIsValid(ordinal); - return Platform.getLong(baseObject, getFieldOffset(ordinal, 8)); + return Platform.getLong(baseObject, getElementOffset(ordinal, 8)); } @Override public float getFloat(int ordinal) { assertIndexIsValid(ordinal); - return Platform.getFloat(baseObject, getFieldOffset(ordinal, 4)); + return Platform.getFloat(baseObject, getElementOffset(ordinal, 4)); } @Override public double getDouble(int ordinal) { assertIndexIsValid(ordinal); - return Platform.getDouble(baseObject, getFieldOffset(ordinal, 8)); + return Platform.getDouble(baseObject, getElementOffset(ordinal, 8)); } @Override public Decimal getDecimal(int ordinal, int precision, int scale) { - assertIndexIsValid(ordinal); - if (isNullAt(ordinal)) { - return null; - } + if (isNullAt(ordinal)) return null; if (precision <= Decimal.MAX_LONG_DIGITS()) { return Decimal.apply(getLong(ordinal), precision, scale); } else { @@ -395,7 +391,7 @@ private static UnsafeArrayData fromPrimitiveArray(Object arr, int length, final final int headerSize = calculateHeaderPortionInBytes(length); if (length > (Integer.MAX_VALUE - headerSize) / elementSize) { throw new UnsupportedOperationException("Cannot convert this array to unsafe format as " + - "it's too big."); + "it's too big."); } final int valueRegionSize = elementSize * length; @@ -403,7 +399,7 @@ private static UnsafeArrayData fromPrimitiveArray(Object arr, int length, final Platform.putInt(data, Platform.BYTE_ARRAY_OFFSET, length); Platform.copyMemory(arr, Platform.INT_ARRAY_OFFSET, data, - Platform.BYTE_ARRAY_OFFSET + headerSize, valueRegionSize); + Platform.BYTE_ARRAY_OFFSET + headerSize, valueRegionSize); UnsafeArrayData result = new UnsafeArrayData(); result.pointTo(data, Platform.BYTE_ARRAY_OFFSET, valueRegionSize + headerSize); From b4eac29ebc8ea7b2c0e9e5717fbbbf13f653a4fb Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Sun, 3 Jul 2016 04:25:46 +0900 Subject: [PATCH 16/37] addressed comments --- .../catalyst/expressions/UnsafeArrayData.java | 33 +++++----- .../codegen/UnsafeArrayWriter.java | 23 +++---- .../codegen/GenerateUnsafeProjection.scala | 6 +- .../benchmark/UnsafeArrayDataBenchmark.scala | 64 +++---------------- 4 files changed, 40 insertions(+), 86 deletions(-) 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 6beb737ec0527..616cf489220cc 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 @@ -43,8 +43,8 @@ * In the `values or offset` region, we store the content of elements. For fields that hold * fixed-length primitive types, such as long, double, or int, we store the value directly * in the field. For fields with non-primitive or variable-length values, we store a relative - * offset (w.r.t. the base address of the row) that points to the beginning of the variable-length - * field, and length (they are combined into a long). + * offset (w.r.t. the base address of the array) that points to the beginning of + * the variable-length field, and length (they are combined into a long). * * Instances of `UnsafeArrayData` act as pointers to row data stored in this format. */ @@ -301,6 +301,7 @@ public boolean equals(Object other) { } return false; } + public void writeToMemory(Object target, long targetOffset) { Platform.copyMemory(baseObject, baseOffset, target, targetOffset, sizeInBytes); } @@ -387,50 +388,52 @@ public double[] toDoubleArray() { return values; } - private static UnsafeArrayData fromPrimitiveArray(Object arr, int length, final int elementSize) { - final int headerSize = calculateHeaderPortionInBytes(length); - if (length > (Integer.MAX_VALUE - headerSize) / elementSize) { + private static UnsafeArrayData fromPrimitiveArray( + Object arr, int offset, int length, int elementSize) { + final long headerSize = calculateHeaderPortionInBytes(length); + final long valueRegionSize = (long)elementSize * (long)length; + final long allocationSize = (headerSize + valueRegionSize + 7) / 8; + if (allocationSize > (long)Integer.MAX_VALUE) { throw new UnsupportedOperationException("Cannot convert this array to unsafe format as " + "it's too big."); } - final int valueRegionSize = elementSize * length; - final byte[] data = new byte[valueRegionSize + headerSize]; + final long[] data = new long[(int)allocationSize]; Platform.putInt(data, Platform.BYTE_ARRAY_OFFSET, length); Platform.copyMemory(arr, Platform.INT_ARRAY_OFFSET, data, Platform.BYTE_ARRAY_OFFSET + headerSize, valueRegionSize); UnsafeArrayData result = new UnsafeArrayData(); - result.pointTo(data, Platform.BYTE_ARRAY_OFFSET, valueRegionSize + headerSize); + result.pointTo(data, Platform.BYTE_ARRAY_OFFSET, (int)allocationSize * 8); return result; } public static UnsafeArrayData fromPrimitiveArray(boolean[] arr) { - return fromPrimitiveArray(arr, arr.length, 1); + return fromPrimitiveArray(arr, Platform.BYTE_ARRAY_OFFSET, arr.length, 1); } public static UnsafeArrayData fromPrimitiveArray(byte[] arr) { - return fromPrimitiveArray(arr, arr.length, 1); + return fromPrimitiveArray(arr, Platform.BYTE_ARRAY_OFFSET, arr.length, 1); } public static UnsafeArrayData fromPrimitiveArray(short[] arr) { - return fromPrimitiveArray(arr, arr.length, 2); + return fromPrimitiveArray(arr, Platform.SHORT_ARRAY_OFFSET, arr.length, 2); } public static UnsafeArrayData fromPrimitiveArray(int[] arr) { - return fromPrimitiveArray(arr, arr.length, 4); + return fromPrimitiveArray(arr, Platform.INT_ARRAY_OFFSET, arr.length, 4); } public static UnsafeArrayData fromPrimitiveArray(long[] arr) { - return fromPrimitiveArray(arr, arr.length, 8); + return fromPrimitiveArray(arr, Platform.LONG_ARRAY_OFFSET, arr.length, 8); } public static UnsafeArrayData fromPrimitiveArray(float[] arr) { - return fromPrimitiveArray(arr, arr.length, 4); + return fromPrimitiveArray(arr, Platform.FLOAT_ARRAY_OFFSET, arr.length, 4); } public static UnsafeArrayData fromPrimitiveArray(double[] arr) { - return fromPrimitiveArray(arr, arr.length, 8); + return fromPrimitiveArray(arr, Platform.DOUBLE_ARRAY_OFFSET, arr.length, 8); } } diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeArrayWriter.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeArrayWriter.java index cfed1776623b0..951d4f89feb2f 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeArrayWriter.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeArrayWriter.java @@ -49,6 +49,7 @@ private void assertIndexIsValid(int index) { } public void initialize(BufferHolder holder, int numElements, int fixedElementSize) { + // We need 4 bytes to store numElements in header this.numElements = numElements; this.headerInBytes = calculateHeaderPortionInBytes(numElements); @@ -60,14 +61,14 @@ public void initialize(BufferHolder holder, int numElements, int fixedElementSiz // Initialize information in header Platform.putInt(holder.buffer, startingOffset, numElements); - Arrays.fill(holder.buffer, startingOffset + 4 - Platform.BYTE_ARRAY_OFFSET, - startingOffset + headerInBytes - Platform.BYTE_ARRAY_OFFSET, (byte)0); - + for (int i = 4; i < headerInBytes; i += 8) { + Platform.putLong(holder.buffer, startingOffset + i, 0L); + } holder.cursor += (headerInBytes + fixedElementSize * numElements); } - private long getElementOffset(int ordinal, int scale) { - return startingOffset + headerInBytes + ordinal * scale; + private long getElementOffset(int ordinal, int elementSize) { + return startingOffset + headerInBytes + ordinal * elementSize; } public void setOffsetAndSize(int ordinal, long currentCursor, long size) { @@ -77,10 +78,6 @@ public void setOffsetAndSize(int ordinal, long currentCursor, long size) { write(ordinal, offsetAndSize); } - public void setNullAt(int ordinal) { - throw new UnsupportedOperationException("setNullAt() is not supported"); - } - private void setNullBit(int ordinal) { assertIndexIsValid(ordinal); BitSetMethods.set(holder.buffer, startingOffset + 4, ordinal); @@ -187,7 +184,7 @@ public void write(int ordinal, Decimal input, int precision, int scale) { // Write the bytes to the variable length portion. Platform.copyMemory( bytes, Platform.BYTE_ARRAY_OFFSET, holder.buffer, holder.cursor, bytes.length); - write(ordinal, ((long)(holder.cursor - startingOffset) << 32) | ((long) bytes.length)); + setOffsetAndSize(ordinal, holder.cursor, bytes.length); holder.cursor += bytes.length; } } else { @@ -204,7 +201,7 @@ public void write(int ordinal, UTF8String input) { // Write the bytes to the variable length portion. input.writeToMemory(holder.buffer, holder.cursor); - write(ordinal, ((long)(holder.cursor - startingOffset) << 32) | ((long) numBytes)); + setOffsetAndSize(ordinal, holder.cursor, numBytes); // move the cursor forward. holder.cursor += numBytes; @@ -218,7 +215,7 @@ public void write(int ordinal, byte[] input) { Platform.copyMemory( input, Platform.BYTE_ARRAY_OFFSET, holder.buffer, holder.cursor, input.length); - write(ordinal, ((long)(holder.cursor - startingOffset) << 32) | ((long) input.length)); + setOffsetAndSize(ordinal, holder.cursor, input.length); // move the cursor forward. holder.cursor += input.length; @@ -232,7 +229,7 @@ public void write(int ordinal, CalendarInterval input) { Platform.putLong(holder.buffer, holder.cursor, input.months); Platform.putLong(holder.buffer, holder.cursor + 8, input.microseconds); - write(ordinal, ((long)(holder.cursor - startingOffset) << 32) | ((long) 16)); + setOffsetAndSize(ordinal, holder.cursor, 16); // move the cursor forward. holder.cursor += 16; diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjection.scala index 0ec2425c0909d..7d7f7b9c393b2 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjection.scala @@ -189,10 +189,10 @@ object GenerateUnsafeProjection extends CodeGenerator[Seq[Expression], UnsafePro val jt = ctx.javaType(et) - val fixedElementSize = et match { + val elementOrOffsetSize = et match { case t: DecimalType if t.precision <= Decimal.MAX_LONG_DIGITS => 8 case _ if ctx.isPrimitiveType(jt) => et.defaultSize - case _ => 8 + case _ => 8 // we need 8 bytes to store offset and length for variable-length types] } val tmpCursor = ctx.freshName("tmpCursor") @@ -232,7 +232,7 @@ object GenerateUnsafeProjection extends CodeGenerator[Seq[Expression], UnsafePro ${writeUnsafeData(ctx, s"((UnsafeArrayData) $input)", bufferHolder)} } else { final int $numElements = $input.numElements(); - $arrayWriter.initialize($bufferHolder, $numElements, $fixedElementSize); + $arrayWriter.initialize($bufferHolder, $numElements, $elementOrOffsetSize); for (int $index = 0; $index < $numElements; $index++) { if ($input.isNullAt($index)) { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/UnsafeArrayDataBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/UnsafeArrayDataBenchmark.scala index a83bc36bbb9fe..20a17e9d8ec86 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/UnsafeArrayDataBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/UnsafeArrayDataBenchmark.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.execution.benchmark import org.apache.spark.SparkConf +import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder import org.apache.spark.sql.catalyst.expressions.{UnsafeArrayData, UnsafeRow} import org.apache.spark.sql.catalyst.expressions.codegen.{BufferHolder, UnsafeArrayWriter} import org.apache.spark.unsafe.Platform @@ -32,11 +33,6 @@ import org.apache.spark.util.Benchmark */ class UnsafeArrayDataBenchmark extends BenchmarkBase { - new SparkConf() - .setMaster("local[1]") - .setAppName("microbenchmark") - .set("spark.driver.memory", "3g") - def calculateHeaderPortionInBytes(count: Int) : Int = { // Use this assignment for SPARK-15962 // val size = 4 + 4 * count @@ -47,12 +43,11 @@ class UnsafeArrayDataBenchmark extends BenchmarkBase { def readUnsafeArray(iters: Int): Unit = { val count = 1024 * 1024 * 16 - val intUnsafeArray = new UnsafeArrayData var intResult: Int = 0 - val intSize = calculateHeaderPortionInBytes(count) + 4 * count - val intBuffer = new Array[Byte](intSize) - Platform.putInt(intBuffer, Platform.BYTE_ARRAY_OFFSET, count) - intUnsafeArray.pointTo(intBuffer, Platform.BYTE_ARRAY_OFFSET, intSize) + val intBuffer = new Array[Int](count) + val intEncoder = ExpressionEncoder[Array[Int]].resolveAndBind() + val intInternalRow = intEncoder.toRow(intBuffer) + val intUnsafeArray = intInternalRow.getArray(0) val readIntArray = { i: Int => var n = 0 while (n < iters) { @@ -68,12 +63,11 @@ class UnsafeArrayDataBenchmark extends BenchmarkBase { } } - val doubleUnsafeArray = new UnsafeArrayData var doubleResult: Double = 0 - val doubleSize = calculateHeaderPortionInBytes(count) + 8 * count - val doubleBuffer = new Array[Byte](doubleSize) - Platform.putInt(doubleBuffer, Platform.BYTE_ARRAY_OFFSET, count) - doubleUnsafeArray.pointTo(doubleBuffer, Platform.BYTE_ARRAY_OFFSET, doubleSize) + val doubleBuffer = new Array[Double](count) + val doubleEncoder = ExpressionEncoder[Array[Double]].resolveAndBind() + val doubleInternalRow = doubleEncoder.toRow(doubleBuffer) + val doubleUnsafeArray = doubleInternalRow.getArray(0) val readDoubleArray = { i: Int => var n = 0 while (n < iters) { @@ -94,16 +88,6 @@ class UnsafeArrayDataBenchmark extends BenchmarkBase { benchmark.addCase("Double")(readDoubleArray) benchmark.run /* - Without SPARK-15962 - OpenJDK 64-Bit Server VM 1.8.0_91-b14 on Linux 4.0.4-301.fc22.x86_64 - Intel Xeon E3-12xx v2 (Ivy Bridge) - Read UnsafeArrayData: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------------ - Int 370 / 471 454.0 2.2 1.0X - Double 351 / 466 477.5 2.1 1.1X - */ - /* - With SPARK-15962 OpenJDK 64-Bit Server VM 1.8.0_91-b14 on Linux 4.0.4-301.fc22.x86_64 Intel Xeon E3-12xx v2 (Ivy Bridge) Read UnsafeArrayData: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative @@ -161,16 +145,6 @@ class UnsafeArrayDataBenchmark extends BenchmarkBase { benchmark.addCase("Double")(writeDoubleArray) benchmark.run /* - Without SPARK-15962 - OpenJDK 64-Bit Server VM 1.8.0_91-b14 on Linux 4.0.4-301.fc22.x86_64 - Intel Xeon E3-12xx v2 (Ivy Bridge) - Write UnsafeArrayData: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------------ - Int 337 / 407 498.0 2.0 1.0X - Double 458 / 496 366.2 2.7 0.7X - */ - /* - With SPARK-15962 OpenJDK 64-Bit Server VM 1.8.0_91-b14 on Linux 4.0.4-301.fc22.x86_64 Intel Xeon E3-12xx v2 (Ivy Bridge) Write UnsafeArrayData: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative @@ -216,16 +190,6 @@ class UnsafeArrayDataBenchmark extends BenchmarkBase { benchmark.addCase("Double")(readDoubleArray) benchmark.run /* - Without SPARK-15962 - OpenJDK 64-Bit Server VM 1.8.0_91-b14 on Linux 4.0.4-301.fc22.x86_64 - Intel Xeon E3-12xx v2 (Ivy Bridge) - Get primitive array from UnsafeArrayData: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------------ - Int 218 / 256 288.1 3.5 1.0X - Double 318 / 539 198.0 5.1 0.7X - */ - /* - With SPARK-15962 OpenJDK 64-Bit Server VM 1.8.0_91-b14 on Linux 4.0.4-301.fc22.x86_64 Intel Xeon E3-12xx v2 (Ivy Bridge) Get primitive array from UnsafeArrayData: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative @@ -263,16 +227,6 @@ class UnsafeArrayDataBenchmark extends BenchmarkBase { benchmark.addCase("Double")(createDoubleArray) benchmark.run /* - Without SPARK-15962 - OpenJDK 64-Bit Server VM 1.8.0_91-b14 on Linux 4.0.4-301.fc22.x86_64 - Intel Xeon E3-12xx v2 (Ivy Bridge) - Create UnsafeArrayData from primitive array: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------------ - Int 343 / 437 183.6 5.4 1.0X - Double 322 / 505 195.6 5.1 1.1X - */ - /* - With SPARK-15962 OpenJDK 64-Bit Server VM 1.8.0_91-b14 on Linux 4.0.4-301.fc22.x86_64 Intel Xeon E3-12xx v2 (Ivy Bridge) Create UnsafeArrayData from primitive array: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative From eecf6bd1fcf21b30208ac3d51aaaf9412b21454b Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Sun, 3 Jul 2016 12:59:15 +0900 Subject: [PATCH 17/37] fix parameters of Platform.OFFSET --- .../spark/sql/catalyst/expressions/UnsafeArrayData.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) 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 616cf489220cc..acb616e6dc85f 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 @@ -400,12 +400,12 @@ private static UnsafeArrayData fromPrimitiveArray( final long[] data = new long[(int)allocationSize]; - Platform.putInt(data, Platform.BYTE_ARRAY_OFFSET, length); - Platform.copyMemory(arr, Platform.INT_ARRAY_OFFSET, data, - Platform.BYTE_ARRAY_OFFSET + headerSize, valueRegionSize); + Platform.putInt(data, Platform.LONG_ARRAY_OFFSET, length); + Platform.copyMemory(arr, offset, data, + Platform.LONG_ARRAY_OFFSET + headerSize, valueRegionSize); UnsafeArrayData result = new UnsafeArrayData(); - result.pointTo(data, Platform.BYTE_ARRAY_OFFSET, (int)allocationSize * 8); + result.pointTo(data, Platform.LONG_ARRAY_OFFSET, (int)allocationSize * 8); return result; } From d88a25ab8534dfcf21b8dd2a00df7cddd94cb57d Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Sun, 3 Jul 2016 13:00:21 +0900 Subject: [PATCH 18/37] update benchmark results --- .../benchmark/UnsafeArrayDataBenchmark.scala | 40 ++++++++++--------- 1 file changed, 22 insertions(+), 18 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/UnsafeArrayDataBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/UnsafeArrayDataBenchmark.scala index 20a17e9d8ec86..d7a910746d7df 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/UnsafeArrayDataBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/UnsafeArrayDataBenchmark.scala @@ -88,12 +88,13 @@ class UnsafeArrayDataBenchmark extends BenchmarkBase { benchmark.addCase("Double")(readDoubleArray) benchmark.run /* - OpenJDK 64-Bit Server VM 1.8.0_91-b14 on Linux 4.0.4-301.fc22.x86_64 - Intel Xeon E3-12xx v2 (Ivy Bridge) + Java HotSpot(TM) 64-Bit Server VM 1.8.0_92-b14 on Mac OS X 10.10.4 + Intel(R) Core(TM) i5-5257U CPU @ 2.70GHz + Read UnsafeArrayData: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Int 246 / 259 682.8 1.5 1.0X - Double 259 / 265 648.4 1.5 0.9X + Int 281 / 296 597.5 1.7 1.0X + Double 298 / 301 562.3 1.8 0.9X */ } @@ -145,12 +146,13 @@ class UnsafeArrayDataBenchmark extends BenchmarkBase { benchmark.addCase("Double")(writeDoubleArray) benchmark.run /* - OpenJDK 64-Bit Server VM 1.8.0_91-b14 on Linux 4.0.4-301.fc22.x86_64 - Intel Xeon E3-12xx v2 (Ivy Bridge) + Java HotSpot(TM) 64-Bit Server VM 1.8.0_92-b14 on Mac OS X 10.10.4 + Intel(R) Core(TM) i5-5257U CPU @ 2.70GHz + Write UnsafeArrayData: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Int 84 / 85 2005.4 0.5 1.0X - Double 152 / 154 1102.5 0.9 0.5X + Int 82 / 85 2056.9 0.5 1.0X + Double 139 / 144 1207.1 0.8 0.6X */ } @@ -190,12 +192,13 @@ class UnsafeArrayDataBenchmark extends BenchmarkBase { benchmark.addCase("Double")(readDoubleArray) benchmark.run /* - OpenJDK 64-Bit Server VM 1.8.0_91-b14 on Linux 4.0.4-301.fc22.x86_64 - Intel Xeon E3-12xx v2 (Ivy Bridge) - Get primitive array from UnsafeArrayData: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + Java HotSpot(TM) 64-Bit Server VM 1.8.0_92-b14 on Mac OS X 10.10.4 + Intel(R) Core(TM) i5-5257U CPU @ 2.70GHz + + Get primitive array from UnsafeArrayData: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Int 86 / 96 734.2 1.4 1.0X - Double 203 / 268 310.6 3.2 0.4X + Int 100 / 176 632.1 1.6 1.0X + Double 267 / 334 236.0 4.2 0.4X */ } @@ -227,12 +230,13 @@ class UnsafeArrayDataBenchmark extends BenchmarkBase { benchmark.addCase("Double")(createDoubleArray) benchmark.run /* - OpenJDK 64-Bit Server VM 1.8.0_91-b14 on Linux 4.0.4-301.fc22.x86_64 - Intel Xeon E3-12xx v2 (Ivy Bridge) - Create UnsafeArrayData from primitive array: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + Java HotSpot(TM) 64-Bit Server VM 1.8.0_92-b14 on Mac OS X 10.10.4 + Intel(R) Core(TM) i5-5257U CPU @ 2.70GHz + + Create UnsafeArrayData from primitive array: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Int 84 / 95 748.3 1.3 1.0X - Double 196 / 227 320.9 3.1 0.4X + Int 69 / 168 911.5 1.1 1.0X + Double 298 / 328 210.8 4.7 0.2X */ } From db15432c06f7bb72163236c12f8954ae821f6902 Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Sun, 3 Jul 2016 13:00:44 +0900 Subject: [PATCH 19/37] add test cases --- .../sql/catalyst/util/UnsafeArraySuite.scala | 115 +++++++++++++++--- 1 file changed, 99 insertions(+), 16 deletions(-) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/UnsafeArraySuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/UnsafeArraySuite.scala index b89825f52c932..9ff37f932fc34 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/UnsafeArraySuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/UnsafeArraySuite.scala @@ -18,27 +18,110 @@ package org.apache.spark.sql.catalyst.util import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder import org.apache.spark.sql.catalyst.expressions.UnsafeArrayData +import org.apache.spark.unsafe.Platform class UnsafeArraySuite extends SparkFunSuite { - test("from primitive int array") { - val array = Array(1, 10, 100) - val unsafe = UnsafeArrayData.fromPrimitiveArray(array) - assert(unsafe.numElements == 3) - assert(unsafe.getSizeInBytes == 4 + scala.math.ceil(3/64.toDouble) * 8 + 4 * 3) - assert(unsafe.getInt(0) == 1) - assert(unsafe.getInt(1) == 10) - assert(unsafe.getInt(2) == 100) + val booleanArray = Array(false, true) + val shortArray = Array(1.toShort, 10.toShort, 100.toShort) + val intArray = Array(1, 10, 100) + val longArray = Array(1.toLong, 10.toLong, 100.toLong) + val floatArray = Array(1.1.toFloat, 2.2.toFloat, 3.3.toFloat) + val doubleArray = Array(1.1, 2.2, 3.3) + + test("read array") { + val unsafeBoolean = ExpressionEncoder[Array[Boolean]].resolveAndBind(). + toRow(booleanArray).getArray(0) + assert(unsafeBoolean.isInstanceOf[UnsafeArrayData]) + assert(unsafeBoolean.numElements == 2) + booleanArray.zipWithIndex.map { case (e, i) => + assert(unsafeBoolean.getBoolean(i) == e) + } + + val unsafeShort = ExpressionEncoder[Array[Short]].resolveAndBind(). + toRow(shortArray).getArray(0) + assert(unsafeShort.isInstanceOf[UnsafeArrayData]) + assert(unsafeShort.numElements == 3) + shortArray.zipWithIndex.map { case (e, i) => + assert(unsafeShort.getShort(i) == e) + } + + val unsafeInt = ExpressionEncoder[Array[Int]].resolveAndBind(). + toRow(intArray).getArray(0) + assert(unsafeInt.isInstanceOf[UnsafeArrayData]) + assert(unsafeInt.numElements == 3) + intArray.zipWithIndex.map { case (e, i) => + assert(unsafeInt.getInt(i) == e) + } + + val unsafeLong = ExpressionEncoder[Array[Long]].resolveAndBind(). + toRow(longArray).getArray(0) + assert(unsafeLong.isInstanceOf[UnsafeArrayData]) + assert(unsafeLong.numElements == 3) + longArray.zipWithIndex.map { case (e, i) => + assert(unsafeLong.getLong(i) == e) + } + + val unsafeFloat = ExpressionEncoder[Array[Float]].resolveAndBind(). + toRow(floatArray).getArray(0) + assert(unsafeFloat.isInstanceOf[UnsafeArrayData]) + assert(unsafeFloat.numElements == 3) + floatArray.zipWithIndex.map { case (e, i) => + assert(unsafeFloat.getFloat(i) == e) + } + + val unsafeDouble = ExpressionEncoder[Array[Double]].resolveAndBind(). + toRow(doubleArray).getArray(0) + assert(unsafeDouble.isInstanceOf[UnsafeArrayData]) + assert(unsafeDouble.numElements == 3) + doubleArray.zipWithIndex.map { case (e, i) => + assert(unsafeDouble.getDouble(i) == e) + } } - test("from primitive double array") { - val array = Array(1.1, 2.2, 3.3) - val unsafe = UnsafeArrayData.fromPrimitiveArray(array) - assert(unsafe.numElements == 3) - assert(unsafe.getSizeInBytes == 4 + scala.math.ceil(3/64.toDouble) * 8 + 8 * 3) - assert(unsafe.getDouble(0) == 1.1) - assert(unsafe.getDouble(1) == 2.2) - assert(unsafe.getDouble(2) == 3.3) + test("from primitive array") { + val unsafeInt = UnsafeArrayData.fromPrimitiveArray(intArray) + assert(unsafeInt.numElements == 3) + assert(unsafeInt.getSizeInBytes == + ((4 + scala.math.ceil(3/64.toDouble) * 8 + 4 * 3 + 7).toInt / 8) * 8) + intArray.zipWithIndex.map { case (e, i) => + assert(unsafeInt.getInt(i) == e) + } + + val unsafeDouble = UnsafeArrayData.fromPrimitiveArray(doubleArray) + assert(unsafeDouble.numElements == 3) + assert(unsafeDouble.getSizeInBytes == + ((4 + scala.math.ceil(3/64.toDouble) * 8 + 8 * 3 + 7).toInt / 8) * 8) + doubleArray.zipWithIndex.map { case (e, i) => + assert(unsafeDouble.getDouble(i) == e) + } + } + + test("to primitive array") { + val intCount = intArray.length + val intUnsafeArray = new UnsafeArrayData + val intHeader = UnsafeArrayData.calculateHeaderPortionInBytes(intCount) + val intSize = intHeader + 4 * intCount + val intBuffer = new Array[Byte](intSize) + Platform.putInt(intBuffer, Platform.BYTE_ARRAY_OFFSET, intCount) + intUnsafeArray.pointTo(intBuffer, Platform.BYTE_ARRAY_OFFSET, intSize) + intArray.zipWithIndex.map { case (e, i) => + Platform.putInt(intBuffer, Platform.BYTE_ARRAY_OFFSET + intHeader + 4 * i, e) + } + assert(intUnsafeArray.toIntArray.sameElements(intArray)) + + val doubleCount = doubleArray.length + val doubleUnsafeArray = new UnsafeArrayData + val doubleHeader = UnsafeArrayData.calculateHeaderPortionInBytes(doubleCount) + val doubleSize = doubleHeader + 4 * doubleCount + val doubleBuffer = new Array[Byte](doubleSize) + Platform.putInt(doubleBuffer, Platform.BYTE_ARRAY_OFFSET, doubleCount) + doubleUnsafeArray.pointTo(doubleBuffer, Platform.BYTE_ARRAY_OFFSET, doubleSize) + doubleArray.zipWithIndex.map { case (e, i) => + Platform.putDouble(intBuffer, Platform.BYTE_ARRAY_OFFSET + doubleHeader + 8 * i, e) + } + assert(intUnsafeArray.toDoubleArray.sameElements(doubleArray)) } } From 3fa7052deaccbaf02e776a08316946d891bc4903 Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Tue, 5 Jul 2016 03:05:17 +0900 Subject: [PATCH 20/37] addressed comments --- .../org/apache/spark/unsafe/Platform.java | 4 ++ .../catalyst/expressions/UnsafeArrayData.java | 50 +++++++------ .../codegen/UnsafeArrayWriter.java | 11 +-- .../codegen/GenerateUnsafeProjection.scala | 6 +- .../sql/catalyst/util/UnsafeArraySuite.scala | 28 ++------ .../benchmark/UnsafeArrayDataBenchmark.scala | 71 +++++++++---------- 6 files changed, 80 insertions(+), 90 deletions(-) diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/Platform.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/Platform.java index c892b9cdaf49c..671b8c7475943 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/Platform.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/Platform.java @@ -29,6 +29,8 @@ public final class Platform { private static final Unsafe _UNSAFE; + public static final int BOOLEAN_ARRAY_OFFSET; + public static final int BYTE_ARRAY_OFFSET; public static final int SHORT_ARRAY_OFFSET; @@ -235,6 +237,7 @@ public static void throwException(Throwable t) { _UNSAFE = unsafe; if (_UNSAFE != null) { + BOOLEAN_ARRAY_OFFSET = _UNSAFE.arrayBaseOffset(boolean[].class); BYTE_ARRAY_OFFSET = _UNSAFE.arrayBaseOffset(byte[].class); SHORT_ARRAY_OFFSET = _UNSAFE.arrayBaseOffset(short[].class); INT_ARRAY_OFFSET = _UNSAFE.arrayBaseOffset(int[].class); @@ -242,6 +245,7 @@ public static void throwException(Throwable t) { FLOAT_ARRAY_OFFSET = _UNSAFE.arrayBaseOffset(float[].class); DOUBLE_ARRAY_OFFSET = _UNSAFE.arrayBaseOffset(double[].class); } else { + BOOLEAN_ARRAY_OFFSET = 0; BYTE_ARRAY_OFFSET = 0; SHORT_ARRAY_OFFSET = 0; INT_ARRAY_OFFSET = 0; 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 acb616e6dc85f..c7be8204e8657 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 @@ -44,7 +44,7 @@ * fixed-length primitive types, such as long, double, or int, we store the value directly * in the field. For fields with non-primitive or variable-length values, we store a relative * offset (w.r.t. the base address of the array) that points to the beginning of - * the variable-length field, and length (they are combined into a long). + * the variable-length field into int. its length can be got by subtracting 2 adjacent offsets * * Instances of `UnsafeArrayData` act as pointers to row data stored in this format. */ @@ -114,6 +114,14 @@ public void pointTo(Object baseObject, long baseOffset, int sizeInBytes) { this.headerInBytes = calculateHeaderPortionInBytes(numElements); } + private int getSize(int ordinal) { + if (ordinal != numElements) { + return getInt(ordinal + 1) - getInt(ordinal); + } else { + return sizeInBytes - getInt(ordinal); + } + } + @Override public boolean isNullAt(int ordinal) { assertIndexIsValid(ordinal); @@ -231,9 +239,8 @@ public UTF8String getUTF8String(int ordinal) { @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 int offset = getInt(ordinal); + final int size = getSize(ordinal); final byte[] bytes = new byte[size]; Platform.copyMemory(baseObject, baseOffset + offset, bytes, Platform.BYTE_ARRAY_OFFSET, size); return bytes; @@ -243,7 +250,7 @@ public byte[] getBinary(int ordinal) { public CalendarInterval getInterval(int ordinal) { if (isNullAt(ordinal)) return null; final long offsetAndSize = getLong(ordinal); - final int offset = (int) (offsetAndSize >> 32); + final int offset = getInt(ordinal); final int months = (int) Platform.getLong(baseObject, baseOffset + offset); final long microseconds = Platform.getLong(baseObject, baseOffset + offset + 8); return new CalendarInterval(months, microseconds); @@ -252,9 +259,8 @@ public CalendarInterval getInterval(int ordinal) { @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 int offset = getInt(ordinal); + final int size = getSize(ordinal); final UnsafeRow row = new UnsafeRow(numFields); row.pointTo(baseObject, baseOffset + offset, size); return row; @@ -263,9 +269,8 @@ public UnsafeRow getStruct(int ordinal, int 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 int offset = getInt(ordinal); + final int size = getSize(ordinal); final UnsafeArrayData array = new UnsafeArrayData(); array.pointTo(baseObject, baseOffset + offset, size); return array; @@ -274,9 +279,8 @@ public UnsafeArrayData getArray(int ordinal) { @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 int offset = getInt(ordinal); + final int size = getSize(ordinal); final UnsafeMapData map = new UnsafeMapData(); map.pointTo(baseObject, baseOffset + offset, size); return map; @@ -330,7 +334,7 @@ public boolean[] toBooleanArray() { int size = numElements(); boolean[] values = new boolean[size]; Platform.copyMemory( - baseObject, baseOffset + headerInBytes, values, Platform.BYTE_ARRAY_OFFSET, size); + baseObject, baseOffset + headerInBytes, values, Platform.BOOLEAN_ARRAY_OFFSET, size); return values; } @@ -390,27 +394,27 @@ public double[] toDoubleArray() { private static UnsafeArrayData fromPrimitiveArray( Object arr, int offset, int length, int elementSize) { - final long headerSize = calculateHeaderPortionInBytes(length); - final long valueRegionSize = (long)elementSize * (long)length; - final long allocationSize = (headerSize + valueRegionSize + 7) / 8; - if (allocationSize > (long)Integer.MAX_VALUE) { + final long headerInBytes = calculateHeaderPortionInBytes(length); + final long valueRegionInBytes = (long)elementSize * (long)length; + final long totalSizeInLongs = (headerInBytes + valueRegionInBytes + 7) / 8; + if (totalSizeInLongs * 8> Integer.MAX_VALUE) { throw new UnsupportedOperationException("Cannot convert this array to unsafe format as " + "it's too big."); } - final long[] data = new long[(int)allocationSize]; + final long[] data = new long[(int)totalSizeInLongs]; Platform.putInt(data, Platform.LONG_ARRAY_OFFSET, length); Platform.copyMemory(arr, offset, data, - Platform.LONG_ARRAY_OFFSET + headerSize, valueRegionSize); + Platform.LONG_ARRAY_OFFSET + headerInBytes, valueRegionInBytes); UnsafeArrayData result = new UnsafeArrayData(); - result.pointTo(data, Platform.LONG_ARRAY_OFFSET, (int)allocationSize * 8); + result.pointTo(data, Platform.LONG_ARRAY_OFFSET, (int)totalSizeInLongs * 8); return result; } public static UnsafeArrayData fromPrimitiveArray(boolean[] arr) { - return fromPrimitiveArray(arr, Platform.BYTE_ARRAY_OFFSET, arr.length, 1); + return fromPrimitiveArray(arr, Platform.BOOLEAN_ARRAY_OFFSET, arr.length, 1); } public static UnsafeArrayData fromPrimitiveArray(byte[] arr) { diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeArrayWriter.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeArrayWriter.java index 951d4f89feb2f..861ff48fb662e 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeArrayWriter.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeArrayWriter.java @@ -48,7 +48,7 @@ private void assertIndexIsValid(int index) { assert index < numElements : "index (" + index + ") should < " + numElements; } - public void initialize(BufferHolder holder, int numElements, int fixedElementSize) { + public void initialize(BufferHolder holder, int numElements, int elementSize) { // We need 4 bytes to store numElements in header this.numElements = numElements; this.headerInBytes = calculateHeaderPortionInBytes(numElements); @@ -57,14 +57,14 @@ public void initialize(BufferHolder holder, int numElements, int fixedElementSiz this.startingOffset = holder.cursor; // Grows the global buffer ahead for header and fixed size data. - holder.grow(headerInBytes + fixedElementSize * numElements); + holder.grow(headerInBytes + elementSize * numElements); - // Initialize information in header + // Write numElements and clear out null bits to header Platform.putInt(holder.buffer, startingOffset, numElements); for (int i = 4; i < headerInBytes; i += 8) { Platform.putLong(holder.buffer, startingOffset + i, 0L); } - holder.cursor += (headerInBytes + fixedElementSize * numElements); + holder.cursor += (headerInBytes + elementSize * numElements); } private long getElementOffset(int ordinal, int elementSize) { @@ -75,7 +75,8 @@ public void setOffsetAndSize(int ordinal, long currentCursor, long size) { final long relativeOffset = currentCursor - startingOffset; final long offsetAndSize = (relativeOffset << 32) | size; - write(ordinal, offsetAndSize); + //write(ordinal, offsetAndSize); + write(ordinal, (int)relativeOffset); } private void setNullBit(int ordinal) { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjection.scala index 7d7f7b9c393b2..c0e36cbbd0399 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjection.scala @@ -192,7 +192,7 @@ object GenerateUnsafeProjection extends CodeGenerator[Seq[Expression], UnsafePro val elementOrOffsetSize = et match { case t: DecimalType if t.precision <= Decimal.MAX_LONG_DIGITS => 8 case _ if ctx.isPrimitiveType(jt) => et.defaultSize - case _ => 8 // we need 8 bytes to store offset and length for variable-length types] + case _ => 8 // we need 8 bytes to store offset and length for variable-length types } val tmpCursor = ctx.freshName("tmpCursor") @@ -226,7 +226,7 @@ object GenerateUnsafeProjection extends CodeGenerator[Seq[Expression], UnsafePro case _ => s"$arrayWriter.write($index, $element);" } - val dataType = if (ctx.isPrimitiveType(jt)) ctx.primitiveTypeName(et) else "" + val primitiveTypeName = if (ctx.isPrimitiveType(jt)) ctx.primitiveTypeName(et) else "" s""" if ($input instanceof UnsafeArrayData) { ${writeUnsafeData(ctx, s"((UnsafeArrayData) $input)", bufferHolder)} @@ -236,7 +236,7 @@ object GenerateUnsafeProjection extends CodeGenerator[Seq[Expression], UnsafePro for (int $index = 0; $index < $numElements; $index++) { if ($input.isNullAt($index)) { - $arrayWriter.setNull$dataType($index); + $arrayWriter.setNull$primitiveTypeName($index); } else { final $jt $element = ${ctx.getValue(input, et, index)}; $writeElement diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/UnsafeArraySuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/UnsafeArraySuite.scala index 9ff37f932fc34..b1dacc3d9ddf2 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/UnsafeArraySuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/UnsafeArraySuite.scala @@ -100,28 +100,14 @@ class UnsafeArraySuite extends SparkFunSuite { } test("to primitive array") { - val intCount = intArray.length - val intUnsafeArray = new UnsafeArrayData - val intHeader = UnsafeArrayData.calculateHeaderPortionInBytes(intCount) - val intSize = intHeader + 4 * intCount - val intBuffer = new Array[Byte](intSize) - Platform.putInt(intBuffer, Platform.BYTE_ARRAY_OFFSET, intCount) - intUnsafeArray.pointTo(intBuffer, Platform.BYTE_ARRAY_OFFSET, intSize) - intArray.zipWithIndex.map { case (e, i) => - Platform.putInt(intBuffer, Platform.BYTE_ARRAY_OFFSET + intHeader + 4 * i, e) - } + val intEncoder = ExpressionEncoder[Array[Int]].resolveAndBind() + val intInternalRow = intEncoder.toRow(intArray) + val intUnsafeArray = intInternalRow.getArray(0) assert(intUnsafeArray.toIntArray.sameElements(intArray)) - val doubleCount = doubleArray.length - val doubleUnsafeArray = new UnsafeArrayData - val doubleHeader = UnsafeArrayData.calculateHeaderPortionInBytes(doubleCount) - val doubleSize = doubleHeader + 4 * doubleCount - val doubleBuffer = new Array[Byte](doubleSize) - Platform.putInt(doubleBuffer, Platform.BYTE_ARRAY_OFFSET, doubleCount) - doubleUnsafeArray.pointTo(doubleBuffer, Platform.BYTE_ARRAY_OFFSET, doubleSize) - doubleArray.zipWithIndex.map { case (e, i) => - Platform.putDouble(intBuffer, Platform.BYTE_ARRAY_OFFSET + doubleHeader + 8 * i, e) - } - assert(intUnsafeArray.toDoubleArray.sameElements(doubleArray)) + val doubleEncoder = ExpressionEncoder[Array[Double]].resolveAndBind() + val doubleInternalRow = doubleEncoder.toRow(doubleArray) + val doubleUnsafeArray = doubleInternalRow.getArray(0) + assert(doubleUnsafeArray.toDoubleArray.sameElements(doubleArray)) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/UnsafeArrayDataBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/UnsafeArrayDataBenchmark.scala index d7a910746d7df..e9a3dd69d15d6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/UnsafeArrayDataBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/UnsafeArrayDataBenchmark.scala @@ -17,17 +17,18 @@ package org.apache.spark.sql.execution.benchmark -import org.apache.spark.SparkConf +import scala.util.Random + import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder import org.apache.spark.sql.catalyst.expressions.{UnsafeArrayData, UnsafeRow} import org.apache.spark.sql.catalyst.expressions.codegen.{BufferHolder, UnsafeArrayWriter} -import org.apache.spark.unsafe.Platform import org.apache.spark.util.Benchmark /** * Benchmark [[UnsafeArrayDataBenchmark]] for UnsafeArrayData * To run this: - * build/sbt "sql/test-only *benchmark.UnsafeArrayDataBenchmark" + * 1. replace ignore(...) with test(...) + * 2. build/sbt "sql/test-only *benchmark.UnsafeArrayDataBenchmark" * * Benchmarks in this file are skipped in normal builds. */ @@ -42,9 +43,10 @@ class UnsafeArrayDataBenchmark extends BenchmarkBase { def readUnsafeArray(iters: Int): Unit = { val count = 1024 * 1024 * 16 + val rand = new Random(42) var intResult: Int = 0 - val intBuffer = new Array[Int](count) + val intBuffer = Array.fill[Int](count) { rand.nextInt } val intEncoder = ExpressionEncoder[Array[Int]].resolveAndBind() val intInternalRow = intEncoder.toRow(intBuffer) val intUnsafeArray = intInternalRow.getArray(0) @@ -64,7 +66,7 @@ class UnsafeArrayDataBenchmark extends BenchmarkBase { } var doubleResult: Double = 0 - val doubleBuffer = new Array[Double](count) + val doubleBuffer = Array.fill[Double](count) { rand.nextDouble } val doubleEncoder = ExpressionEncoder[Array[Double]].resolveAndBind() val doubleInternalRow = doubleEncoder.toRow(doubleBuffer) val doubleUnsafeArray = doubleInternalRow.getArray(0) @@ -93,8 +95,8 @@ class UnsafeArrayDataBenchmark extends BenchmarkBase { Read UnsafeArrayData: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Int 281 / 296 597.5 1.7 1.0X - Double 298 / 301 562.3 1.8 0.9X + Int 279 / 294 600.4 1.7 1.0X + Double 296 / 303 567.0 1.8 0.9X */ } @@ -151,38 +153,37 @@ class UnsafeArrayDataBenchmark extends BenchmarkBase { Write UnsafeArrayData: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Int 82 / 85 2056.9 0.5 1.0X - Double 139 / 144 1207.1 0.8 0.6X + Int 79 / 86 2124.2 0.5 1.0X + Double 140 / 147 1201.0 0.8 0.6X */ } def getPrimitiveArray(iters: Int): Unit = { val count = 1024 * 1024 * 12 + val rand = new Random(42) - val intUnsafeArray = new UnsafeArrayData - val intSize = calculateHeaderPortionInBytes(count) + 4 * count - val intBuffer = new Array[Byte](intSize) - Platform.putInt(intBuffer, Platform.BYTE_ARRAY_OFFSET, count) - intUnsafeArray.pointTo(intBuffer, Platform.BYTE_ARRAY_OFFSET, intSize) - var intPrimitiveArray: Array[Int] = null + var intTotalLength: Int = 0 + val intBuffer = Array.fill[Int](count) { rand.nextInt } + val intEncoder = ExpressionEncoder[Array[Int]].resolveAndBind() + val intInternalRow = intEncoder.toRow(intBuffer) + val intUnsafeArray = intInternalRow.getArray(0) val readIntArray = { i: Int => var n = 0 while (n < iters) { - intPrimitiveArray = intUnsafeArray.toIntArray + intTotalLength += intUnsafeArray.toIntArray.length n += 1 } } - val doubleUnsafeArray = new UnsafeArrayData - val doubleSize = calculateHeaderPortionInBytes(count) + 8 * count - val doubleBuffer = new Array[Byte](doubleSize) - Platform.putInt(doubleBuffer, Platform.BYTE_ARRAY_OFFSET, count) - doubleUnsafeArray.pointTo(doubleBuffer, Platform.BYTE_ARRAY_OFFSET, doubleSize) - var doublePrimitiveArray: Array[Double] = null + var doubleTotalLength: Int = 0 + val doubleBuffer = Array.fill[Double](count) { rand.nextDouble } + val doubleEncoder = ExpressionEncoder[Array[Double]].resolveAndBind() + val doubleInternalRow = doubleEncoder.toRow(doubleBuffer) + val doubleUnsafeArray = doubleInternalRow.getArray(0) val readDoubleArray = { i: Int => var n = 0 while (n < iters) { - doublePrimitiveArray = doubleUnsafeArray.toDoubleArray + doubleTotalLength += doubleUnsafeArray.toDoubleArray.length n += 1 } } @@ -195,17 +196,18 @@ class UnsafeArrayDataBenchmark extends BenchmarkBase { Java HotSpot(TM) 64-Bit Server VM 1.8.0_92-b14 on Mac OS X 10.10.4 Intel(R) Core(TM) i5-5257U CPU @ 2.70GHz - Get primitive array from UnsafeArrayData: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + Get primitive array from UnsafeArrayData: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Int 100 / 176 632.1 1.6 1.0X - Double 267 / 334 236.0 4.2 0.4X + Int 80 / 151 783.4 1.3 1.0X + Double 208 / 366 302.8 3.3 0.4X */ } def putPrimitiveArray(iters: Int): Unit = { val count = 1024 * 1024 * 12 + val rand = new Random(42) - val intPrimitiveArray: Array[Int] = new Array[Int](count) + val intPrimitiveArray = Array.fill[Int](count) { rand.nextInt } var intUnsafeArray: UnsafeArrayData = null val createIntArray = { i: Int => var n = 0 @@ -215,7 +217,7 @@ class UnsafeArrayDataBenchmark extends BenchmarkBase { } } - val doublePrimitiveArray: Array[Double] = new Array[Double](count) + val doublePrimitiveArray = Array.fill[Double](count) { rand.nextDouble } var doubleUnsafeArray: UnsafeArrayData = null val createDoubleArray = { i: Int => var n = 0 @@ -233,10 +235,10 @@ class UnsafeArrayDataBenchmark extends BenchmarkBase { Java HotSpot(TM) 64-Bit Server VM 1.8.0_92-b14 on Mac OS X 10.10.4 Intel(R) Core(TM) i5-5257U CPU @ 2.70GHz - Create UnsafeArrayData from primitive array: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + Create UnsafeArrayData from primitive array: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Int 69 / 168 911.5 1.1 1.0X - Double 298 / 328 210.8 4.7 0.2X + Int 68 / 144 920.4 1.1 1.0X + Double 240 / 302 261.7 3.8 0.3X */ } @@ -246,11 +248,4 @@ class UnsafeArrayDataBenchmark extends BenchmarkBase { getPrimitiveArray(5) putPrimitiveArray(5) } - - def main(args: Array[String]): Unit = { - readUnsafeArray(10) - writeUnsafeArray(10) - getPrimitiveArray(5) - putPrimitiveArray(5) - } } From 4c094c293d073ada13c5f429e26cf54fd9c8450f Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Wed, 6 Jul 2016 19:08:11 +0900 Subject: [PATCH 21/37] addressed comments --- .../catalyst/expressions/UnsafeArrayData.java | 7 +++---- .../codegen/UnsafeArrayWriter.java | 21 +++++++++---------- .../codegen/GenerateUnsafeProjection.scala | 9 +++----- 3 files changed, 16 insertions(+), 21 deletions(-) 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 c7be8204e8657..4d015d5ede9c1 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 @@ -115,7 +115,7 @@ public void pointTo(Object baseObject, long baseOffset, int sizeInBytes) { } private int getSize(int ordinal) { - if (ordinal != numElements) { + if (ordinal != numElements - 1) { return getInt(ordinal + 1) - getInt(ordinal); } else { return sizeInBytes - getInt(ordinal); @@ -230,9 +230,8 @@ public Decimal getDecimal(int ordinal, int precision, int scale) { @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; + final int offset = getInt(ordinal); + final int size = getSize(ordinal); return UTF8String.fromAddress(baseObject, baseOffset + offset, size); } diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeArrayWriter.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeArrayWriter.java index 861ff48fb662e..521de5666c962 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeArrayWriter.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeArrayWriter.java @@ -71,12 +71,8 @@ private long getElementOffset(int ordinal, int elementSize) { return startingOffset + headerInBytes + ordinal * elementSize; } - public void setOffsetAndSize(int ordinal, long currentCursor, long size) { - final long relativeOffset = currentCursor - startingOffset; - final long offsetAndSize = (relativeOffset << 32) | size; - - //write(ordinal, offsetAndSize); - write(ordinal, (int)relativeOffset); + public void setOffset(int ordinal, int currentCursor) { + write(ordinal, currentCursor - startingOffset); } private void setNullBit(int ordinal) { @@ -127,7 +123,10 @@ public void setNullDouble(int ordinal) { } public void setNull(int ordinal) { - setNullLong(ordinal); + setNullBit(ordinal); + // put relative offset for current cursor position + // while put non-zero for the corresponding null field, it is still deterministic + write(ordinal, holder.cursor - startingOffset); } public void write(int ordinal, boolean value) { @@ -185,7 +184,7 @@ public void write(int ordinal, Decimal input, int precision, int scale) { // Write the bytes to the variable length portion. Platform.copyMemory( bytes, Platform.BYTE_ARRAY_OFFSET, holder.buffer, holder.cursor, bytes.length); - setOffsetAndSize(ordinal, holder.cursor, bytes.length); + setOffset(ordinal, holder.cursor); holder.cursor += bytes.length; } } else { @@ -202,7 +201,7 @@ public void write(int ordinal, UTF8String input) { // Write the bytes to the variable length portion. input.writeToMemory(holder.buffer, holder.cursor); - setOffsetAndSize(ordinal, holder.cursor, numBytes); + setOffset(ordinal, holder.cursor); // move the cursor forward. holder.cursor += numBytes; @@ -216,7 +215,7 @@ public void write(int ordinal, byte[] input) { Platform.copyMemory( input, Platform.BYTE_ARRAY_OFFSET, holder.buffer, holder.cursor, input.length); - setOffsetAndSize(ordinal, holder.cursor, input.length); + setOffset(ordinal, holder.cursor); // move the cursor forward. holder.cursor += input.length; @@ -230,7 +229,7 @@ public void write(int ordinal, CalendarInterval input) { Platform.putLong(holder.buffer, holder.cursor, input.months); Platform.putLong(holder.buffer, holder.cursor + 8, input.microseconds); - setOffsetAndSize(ordinal, holder.cursor, 16); + setOffset(ordinal, holder.cursor); // move the cursor forward. holder.cursor += 16; diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjection.scala index c0e36cbbd0399..75fd02ff62b39 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjection.scala @@ -199,23 +199,20 @@ object GenerateUnsafeProjection extends CodeGenerator[Seq[Expression], UnsafePro val writeElement = et match { case t: StructType => s""" - final int $tmpCursor = $bufferHolder.cursor; + $arrayWriter.setOffset($index, $bufferHolder.cursor); ${writeStructToBuffer(ctx, element, t.map(_.dataType), bufferHolder)} - $arrayWriter.setOffsetAndSize($index, $tmpCursor, $bufferHolder.cursor - $tmpCursor); """ case a @ ArrayType(et, _) => s""" - final int $tmpCursor = $bufferHolder.cursor; + $arrayWriter.setOffset($index, $bufferHolder.cursor); ${writeArrayToBuffer(ctx, element, et, bufferHolder)} - $arrayWriter.setOffsetAndSize($index, $tmpCursor, $bufferHolder.cursor - $tmpCursor); """ case m @ MapType(kt, vt, _) => s""" - final int $tmpCursor = $bufferHolder.cursor; + $arrayWriter.setOffset($index, $bufferHolder.cursor); ${writeMapToBuffer(ctx, element, kt, vt, bufferHolder)} - $arrayWriter.setOffsetAndSize($index, $tmpCursor, $bufferHolder.cursor - $tmpCursor); """ case t: DecimalType => From 9887171d81c023f46a9f86dab3d2fc9e3931ea7b Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Wed, 6 Jul 2016 19:08:31 +0900 Subject: [PATCH 22/37] update test cases --- .../sql/catalyst/util/UnsafeArraySuite.scala | 42 ++++++++++++++++--- 1 file changed, 36 insertions(+), 6 deletions(-) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/UnsafeArraySuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/UnsafeArraySuite.scala index b1dacc3d9ddf2..40f936c973951 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/UnsafeArraySuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/UnsafeArraySuite.scala @@ -31,11 +31,15 @@ class UnsafeArraySuite extends SparkFunSuite { val floatArray = Array(1.1.toFloat, 2.2.toFloat, 3.3.toFloat) val doubleArray = Array(1.1, 2.2, 3.3) + val intMultiDimArray = Array(Array(1, 10), Array(2, 20, 200), Array(3, 30, 300, 3000)) + val doubleMultiDimArray = Array( + Array(1.1, 11.1), Array(2.2, 22.2, 222.2), Array(3.3, 33.3, 333.3, 3333.3)) + test("read array") { val unsafeBoolean = ExpressionEncoder[Array[Boolean]].resolveAndBind(). toRow(booleanArray).getArray(0) assert(unsafeBoolean.isInstanceOf[UnsafeArrayData]) - assert(unsafeBoolean.numElements == 2) + assert(unsafeBoolean.numElements == booleanArray.length) booleanArray.zipWithIndex.map { case (e, i) => assert(unsafeBoolean.getBoolean(i) == e) } @@ -43,7 +47,7 @@ class UnsafeArraySuite extends SparkFunSuite { val unsafeShort = ExpressionEncoder[Array[Short]].resolveAndBind(). toRow(shortArray).getArray(0) assert(unsafeShort.isInstanceOf[UnsafeArrayData]) - assert(unsafeShort.numElements == 3) + assert(unsafeShort.numElements == shortArray.length) shortArray.zipWithIndex.map { case (e, i) => assert(unsafeShort.getShort(i) == e) } @@ -51,7 +55,7 @@ class UnsafeArraySuite extends SparkFunSuite { val unsafeInt = ExpressionEncoder[Array[Int]].resolveAndBind(). toRow(intArray).getArray(0) assert(unsafeInt.isInstanceOf[UnsafeArrayData]) - assert(unsafeInt.numElements == 3) + assert(unsafeInt.numElements == intArray.length) intArray.zipWithIndex.map { case (e, i) => assert(unsafeInt.getInt(i) == e) } @@ -59,7 +63,7 @@ class UnsafeArraySuite extends SparkFunSuite { val unsafeLong = ExpressionEncoder[Array[Long]].resolveAndBind(). toRow(longArray).getArray(0) assert(unsafeLong.isInstanceOf[UnsafeArrayData]) - assert(unsafeLong.numElements == 3) + assert(unsafeLong.numElements == longArray.length) longArray.zipWithIndex.map { case (e, i) => assert(unsafeLong.getLong(i) == e) } @@ -67,7 +71,7 @@ class UnsafeArraySuite extends SparkFunSuite { val unsafeFloat = ExpressionEncoder[Array[Float]].resolveAndBind(). toRow(floatArray).getArray(0) assert(unsafeFloat.isInstanceOf[UnsafeArrayData]) - assert(unsafeFloat.numElements == 3) + assert(unsafeFloat.numElements == floatArray.length) floatArray.zipWithIndex.map { case (e, i) => assert(unsafeFloat.getFloat(i) == e) } @@ -75,10 +79,36 @@ class UnsafeArraySuite extends SparkFunSuite { val unsafeDouble = ExpressionEncoder[Array[Double]].resolveAndBind(). toRow(doubleArray).getArray(0) assert(unsafeDouble.isInstanceOf[UnsafeArrayData]) - assert(unsafeDouble.numElements == 3) + assert(unsafeDouble.numElements == doubleArray.length) doubleArray.zipWithIndex.map { case (e, i) => assert(unsafeDouble.getDouble(i) == e) } + + val unsafeMultiDimInt = ExpressionEncoder[Array[Array[Int]]].resolveAndBind(). + toRow(intMultiDimArray).getArray(0) + assert(unsafeMultiDimInt.isInstanceOf[UnsafeArrayData]) + assert(unsafeMultiDimInt.numElements == intMultiDimArray.length) + intMultiDimArray.zipWithIndex.map { case (a, j) => + val u = unsafeMultiDimInt.getArray(j) + assert(u.isInstanceOf[UnsafeArrayData]) + assert(u.numElements == a.length) + a.zipWithIndex.map { case (e, i) => + assert(u.getInt(i) == e) + } + } + + val unsafeMultiDimDouble = ExpressionEncoder[Array[Array[Double]]].resolveAndBind(). + toRow(doubleMultiDimArray).getArray(0) + assert(unsafeDouble.isInstanceOf[UnsafeArrayData]) + assert(unsafeMultiDimDouble.numElements == doubleMultiDimArray.length) + doubleMultiDimArray.zipWithIndex.map { case (a, j) => + val u = unsafeMultiDimDouble.getArray(j) + assert(u.isInstanceOf[UnsafeArrayData]) + assert(u.numElements == a.length) + a.zipWithIndex.map { case (e, i) => + assert(u.getDouble(i) == e) + } + } } test("from primitive array") { From 9fe7ad0823ce103db77627b74645764015a8f606 Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Fri, 8 Jul 2016 01:49:55 +0900 Subject: [PATCH 23/37] address comments --- .../sql/catalyst/expressions/UnsafeArrayData.java | 9 ++++----- .../expressions/codegen/UnsafeArrayWriter.java | 14 ++++++++------ .../codegen/GenerateUnsafeProjection.scala | 9 ++++----- 3 files changed, 16 insertions(+), 16 deletions(-) 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 4d015d5ede9c1..8413f746796f5 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 @@ -248,7 +248,6 @@ public byte[] getBinary(int ordinal) { @Override public CalendarInterval getInterval(int ordinal) { if (isNullAt(ordinal)) return null; - final long offsetAndSize = getLong(ordinal); final int offset = getInt(ordinal); final int months = (int) Platform.getLong(baseObject, baseOffset + offset); final long microseconds = Platform.getLong(baseObject, baseOffset + offset + 8); @@ -393,10 +392,10 @@ public double[] toDoubleArray() { private static UnsafeArrayData fromPrimitiveArray( Object arr, int offset, int length, int elementSize) { - final long headerInBytes = calculateHeaderPortionInBytes(length); - final long valueRegionInBytes = (long)elementSize * (long)length; - final long totalSizeInLongs = (headerInBytes + valueRegionInBytes + 7) / 8; - if (totalSizeInLongs * 8> Integer.MAX_VALUE) { + final int headerInBytes = calculateHeaderPortionInBytes(length); + final int valueRegionInBytes = elementSize * length; + final int totalSizeInLongs = (headerInBytes + valueRegionInBytes + 7) / 8; + if (totalSizeInLongs * 8 > Integer.MAX_VALUE) { throw new UnsupportedOperationException("Cannot convert this array to unsafe format as " + "it's too big."); } diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeArrayWriter.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeArrayWriter.java index 521de5666c962..7a6f4627bdb04 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeArrayWriter.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeArrayWriter.java @@ -71,8 +71,8 @@ private long getElementOffset(int ordinal, int elementSize) { return startingOffset + headerInBytes + ordinal * elementSize; } - public void setOffset(int ordinal, int currentCursor) { - write(ordinal, currentCursor - startingOffset); + public void setOffset(int ordinal) { + write(ordinal, holder.cursor - startingOffset); } private void setNullBit(int ordinal) { @@ -184,7 +184,9 @@ public void write(int ordinal, Decimal input, int precision, int scale) { // Write the bytes to the variable length portion. Platform.copyMemory( bytes, Platform.BYTE_ARRAY_OFFSET, holder.buffer, holder.cursor, bytes.length); - setOffset(ordinal, holder.cursor); + setOffset(ordinal); + + // move the cursor forward. holder.cursor += bytes.length; } } else { @@ -201,7 +203,7 @@ public void write(int ordinal, UTF8String input) { // Write the bytes to the variable length portion. input.writeToMemory(holder.buffer, holder.cursor); - setOffset(ordinal, holder.cursor); + setOffset(ordinal); // move the cursor forward. holder.cursor += numBytes; @@ -215,7 +217,7 @@ public void write(int ordinal, byte[] input) { Platform.copyMemory( input, Platform.BYTE_ARRAY_OFFSET, holder.buffer, holder.cursor, input.length); - setOffset(ordinal, holder.cursor); + setOffset(ordinal); // move the cursor forward. holder.cursor += input.length; @@ -229,7 +231,7 @@ public void write(int ordinal, CalendarInterval input) { Platform.putLong(holder.buffer, holder.cursor, input.months); Platform.putLong(holder.buffer, holder.cursor + 8, input.microseconds); - setOffset(ordinal, holder.cursor); + setOffset(ordinal); // move the cursor forward. holder.cursor += 16; diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjection.scala index 75fd02ff62b39..c2f503a98ffdc 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjection.scala @@ -192,26 +192,25 @@ object GenerateUnsafeProjection extends CodeGenerator[Seq[Expression], UnsafePro val elementOrOffsetSize = et match { case t: DecimalType if t.precision <= Decimal.MAX_LONG_DIGITS => 8 case _ if ctx.isPrimitiveType(jt) => et.defaultSize - case _ => 8 // we need 8 bytes to store offset and length for variable-length types + case _ => 4 // we need 4 bytes to store offset } - val tmpCursor = ctx.freshName("tmpCursor") val writeElement = et match { case t: StructType => s""" - $arrayWriter.setOffset($index, $bufferHolder.cursor); + $arrayWriter.setOffset($index); ${writeStructToBuffer(ctx, element, t.map(_.dataType), bufferHolder)} """ case a @ ArrayType(et, _) => s""" - $arrayWriter.setOffset($index, $bufferHolder.cursor); + $arrayWriter.setOffset($index); ${writeArrayToBuffer(ctx, element, et, bufferHolder)} """ case m @ MapType(kt, vt, _) => s""" - $arrayWriter.setOffset($index, $bufferHolder.cursor); + $arrayWriter.setOffset($index); ${writeMapToBuffer(ctx, element, kt, vt, bufferHolder)} """ From e4b4b52eb5c227ddddd603486e136538551be575 Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Fri, 8 Jul 2016 01:50:32 +0900 Subject: [PATCH 24/37] address comments for test cases and benchmark --- .../sql/catalyst/util/UnsafeArraySuite.scala | 17 ++-- .../benchmark/UnsafeArrayDataBenchmark.scala | 78 +++++-------------- 2 files changed, 32 insertions(+), 63 deletions(-) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/UnsafeArraySuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/UnsafeArraySuite.scala index 40f936c973951..7d6071111c060 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/UnsafeArraySuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/UnsafeArraySuite.scala @@ -30,6 +30,7 @@ class UnsafeArraySuite extends SparkFunSuite { val longArray = Array(1.toLong, 10.toLong, 100.toLong) val floatArray = Array(1.1.toFloat, 2.2.toFloat, 3.3.toFloat) val doubleArray = Array(1.1, 2.2, 3.3) + val stringArray = Array("1", "10", "100") val intMultiDimArray = Array(Array(1, 10), Array(2, 20, 200), Array(3, 30, 300, 3000)) val doubleMultiDimArray = Array( @@ -84,6 +85,14 @@ class UnsafeArraySuite extends SparkFunSuite { assert(unsafeDouble.getDouble(i) == e) } + val unsafeString = ExpressionEncoder[Array[String]].resolveAndBind(). + toRow(stringArray).getArray(0) + assert(unsafeString.isInstanceOf[UnsafeArrayData]) + assert(unsafeString.numElements == stringArray.length) + stringArray.zipWithIndex.map { case (e, i) => + assert(unsafeString.getUTF8String(i).toString().equals(e)) + } + val unsafeMultiDimInt = ExpressionEncoder[Array[Array[Int]]].resolveAndBind(). toRow(intMultiDimArray).getArray(0) assert(unsafeMultiDimInt.isInstanceOf[UnsafeArrayData]) @@ -131,13 +140,9 @@ class UnsafeArraySuite extends SparkFunSuite { test("to primitive array") { val intEncoder = ExpressionEncoder[Array[Int]].resolveAndBind() - val intInternalRow = intEncoder.toRow(intArray) - val intUnsafeArray = intInternalRow.getArray(0) - assert(intUnsafeArray.toIntArray.sameElements(intArray)) + assert(intEncoder.toRow(intArray).getArray(0).toIntArray.sameElements(intArray)) val doubleEncoder = ExpressionEncoder[Array[Double]].resolveAndBind() - val doubleInternalRow = doubleEncoder.toRow(doubleArray) - val doubleUnsafeArray = doubleInternalRow.getArray(0) - assert(doubleUnsafeArray.toDoubleArray.sameElements(doubleArray)) + assert(doubleEncoder.toRow(doubleArray).getArray(0).toDoubleArray.sameElements(doubleArray)) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/UnsafeArrayDataBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/UnsafeArrayDataBenchmark.scala index e9a3dd69d15d6..ffb8fd0f276fa 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/UnsafeArrayDataBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/UnsafeArrayDataBenchmark.scala @@ -35,52 +35,44 @@ import org.apache.spark.util.Benchmark class UnsafeArrayDataBenchmark extends BenchmarkBase { def calculateHeaderPortionInBytes(count: Int) : Int = { - // Use this assignment for SPARK-15962 - // val size = 4 + 4 * count - val size = UnsafeArrayData.calculateHeaderPortionInBytes(count) - size + /* 4 + 4 * count // Use this expression for SPARK-15962 */ + UnsafeArrayData.calculateHeaderPortionInBytes(count) } def readUnsafeArray(iters: Int): Unit = { val count = 1024 * 1024 * 16 val rand = new Random(42) - var intResult: Int = 0 - val intBuffer = Array.fill[Int](count) { rand.nextInt } + val intPrimitiveArray = Array.fill[Int](count) { rand.nextInt } val intEncoder = ExpressionEncoder[Array[Int]].resolveAndBind() - val intInternalRow = intEncoder.toRow(intBuffer) - val intUnsafeArray = intInternalRow.getArray(0) + val intUnsafeArray = intEncoder.toRow(intPrimitiveArray).getArray(0) val readIntArray = { i: Int => var n = 0 while (n < iters) { val len = intUnsafeArray.numElements - var sum = 0.toInt + var sum = 0 var i = 0 while (i < len) { sum += intUnsafeArray.getInt(i) i += 1 } - intResult = sum n += 1 } } - var doubleResult: Double = 0 - val doubleBuffer = Array.fill[Double](count) { rand.nextDouble } + val doublePrimitiveArray = Array.fill[Double](count) { rand.nextDouble } val doubleEncoder = ExpressionEncoder[Array[Double]].resolveAndBind() - val doubleInternalRow = doubleEncoder.toRow(doubleBuffer) - val doubleUnsafeArray = doubleInternalRow.getArray(0) + val doubleUnsafeArray = doubleEncoder.toRow(doublePrimitiveArray).getArray(0) val readDoubleArray = { i: Int => var n = 0 while (n < iters) { val len = doubleUnsafeArray.numElements - var sum = 0.toDouble + var sum = 0.0 var i = 0 while (i < len) { sum += doubleUnsafeArray.getDouble(i) i += 1 } - doubleResult = sum n += 1 } } @@ -102,43 +94,26 @@ class UnsafeArrayDataBenchmark extends BenchmarkBase { def writeUnsafeArray(iters: Int): Unit = { val count = 1024 * 1024 * 16 + val rand = new Random(42) - val intUnsafeRow = new UnsafeRow(1) - val intUnsafeArrayWriter = new UnsafeArrayWriter - val intBufferHolder = new BufferHolder(intUnsafeRow, 64) - intBufferHolder.reset() - intUnsafeArrayWriter.initialize(intBufferHolder, count, 4) - val intCursor = intBufferHolder.cursor + var intTotalLength: Int = 0 + val intPrimitiveArray = Array.fill[Int](count) { rand.nextInt } + val intEncoder = ExpressionEncoder[Array[Int]].resolveAndBind() val writeIntArray = { i: Int => var n = 0 while (n < iters) { - intBufferHolder.cursor = intCursor - val len = count - var i = 0 - while (i < len) { - intUnsafeArrayWriter.write(i, 0.toInt) - i += 1 - } + intTotalLength += intEncoder.toRow(intPrimitiveArray).getArray(0).numElements() n += 1 } } - val doubleUnsafeRow = new UnsafeRow(1) - val doubleUnsafeArrayWriter = new UnsafeArrayWriter - val doubleBufferHolder = new BufferHolder(doubleUnsafeRow, 64) - doubleBufferHolder.reset() - doubleUnsafeArrayWriter.initialize(doubleBufferHolder, count, 8) - val doubleCursor = doubleBufferHolder.cursor + var doubleTotalLength: Int = 0 + val doublePrimitiveArray = Array.fill[Double](count) { rand.nextDouble } + val doubleEncoder = ExpressionEncoder[Array[Double]].resolveAndBind() val writeDoubleArray = { i: Int => var n = 0 while (n < iters) { - doubleBufferHolder.cursor = doubleCursor - val len = count - var i = 0 - while (i < len) { - doubleUnsafeArrayWriter.write(i, 0.toDouble) - i += 1 - } + doubleTotalLength += doubleEncoder.toRow(doublePrimitiveArray).getArray(0).numElements() n += 1 } } @@ -147,15 +122,6 @@ class UnsafeArrayDataBenchmark extends BenchmarkBase { benchmark.addCase("Int")(writeIntArray) benchmark.addCase("Double")(writeDoubleArray) benchmark.run - /* - Java HotSpot(TM) 64-Bit Server VM 1.8.0_92-b14 on Mac OS X 10.10.4 - Intel(R) Core(TM) i5-5257U CPU @ 2.70GHz - - Write UnsafeArrayData: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------------ - Int 79 / 86 2124.2 0.5 1.0X - Double 140 / 147 1201.0 0.8 0.6X - */ } def getPrimitiveArray(iters: Int): Unit = { @@ -163,10 +129,9 @@ class UnsafeArrayDataBenchmark extends BenchmarkBase { val rand = new Random(42) var intTotalLength: Int = 0 - val intBuffer = Array.fill[Int](count) { rand.nextInt } + val intPrimitiveArray = Array.fill[Int](count) { rand.nextInt } val intEncoder = ExpressionEncoder[Array[Int]].resolveAndBind() - val intInternalRow = intEncoder.toRow(intBuffer) - val intUnsafeArray = intInternalRow.getArray(0) + val intUnsafeArray = intEncoder.toRow(intPrimitiveArray).getArray(0) val readIntArray = { i: Int => var n = 0 while (n < iters) { @@ -176,10 +141,9 @@ class UnsafeArrayDataBenchmark extends BenchmarkBase { } var doubleTotalLength: Int = 0 - val doubleBuffer = Array.fill[Double](count) { rand.nextDouble } + val doublePrimitiveArray = Array.fill[Double](count) { rand.nextDouble } val doubleEncoder = ExpressionEncoder[Array[Double]].resolveAndBind() - val doubleInternalRow = doubleEncoder.toRow(doubleBuffer) - val doubleUnsafeArray = doubleInternalRow.getArray(0) + val doubleUnsafeArray = doubleEncoder.toRow(doublePrimitiveArray).getArray(0) val readDoubleArray = { i: Int => var n = 0 while (n < iters) { From 585ca7b6d9eca691e1f88e341e1083efae882e58 Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Fri, 8 Jul 2016 15:55:43 +0900 Subject: [PATCH 25/37] addressed comments --- .../sql/catalyst/expressions/UnsafeArrayData.java | 6 +++--- .../expressions/codegen/UnsafeArrayWriter.java | 2 -- .../expressions/UnsafeRowConverterSuite.scala | 12 ++++++------ .../benchmark/UnsafeArrayDataBenchmark.scala | 9 +++++++++ .../sql/execution/columnar/ColumnTypeSuite.scala | 2 +- 5 files changed, 19 insertions(+), 12 deletions(-) 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 8413f746796f5..e6d406e91f3e5 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 @@ -392,9 +392,9 @@ public double[] toDoubleArray() { private static UnsafeArrayData fromPrimitiveArray( Object arr, int offset, int length, int elementSize) { - final int headerInBytes = calculateHeaderPortionInBytes(length); - final int valueRegionInBytes = elementSize * length; - final int totalSizeInLongs = (headerInBytes + valueRegionInBytes + 7) / 8; + final long headerInBytes = calculateHeaderPortionInBytes(length); + final long valueRegionInBytes = elementSize * length; + final long totalSizeInLongs = (headerInBytes + valueRegionInBytes + 7) / 8; if (totalSizeInLongs * 8 > Integer.MAX_VALUE) { throw new UnsupportedOperationException("Cannot convert this array to unsafe format as " + "it's too big."); diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeArrayWriter.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeArrayWriter.java index 7a6f4627bdb04..54adc82cbae77 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeArrayWriter.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeArrayWriter.java @@ -23,8 +23,6 @@ import org.apache.spark.unsafe.types.CalendarInterval; import org.apache.spark.unsafe.types.UTF8String; -import java.util.Arrays; - import static org.apache.spark.sql.catalyst.expressions.UnsafeArrayData.calculateHeaderPortionInBytes; /** diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/UnsafeRowConverterSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/UnsafeRowConverterSuite.scala index 3b5938e8a1a34..d41b23cd3d244 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/UnsafeRowConverterSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/UnsafeRowConverterSuite.scala @@ -340,7 +340,7 @@ class UnsafeRowConverterSuite extends SparkFunSuite with Matchers { val nestedArray = unsafeArray2.getArray(0) testArrayInt(nestedArray, Seq(3, 4)) - assert(unsafeArray2.getSizeInBytes == 4 + 8 + 8 + nestedArray.getSizeInBytes) + assert(unsafeArray2.getSizeInBytes == 4 + 8 + 4 + nestedArray.getSizeInBytes) val array1Size = roundedSize(unsafeArray1.getSizeInBytes) val array2Size = roundedSize(unsafeArray2.getSizeInBytes) @@ -383,7 +383,7 @@ class UnsafeRowConverterSuite extends SparkFunSuite with Matchers { val nestedMap = valueArray.getMap(0) testMapInt(nestedMap, Seq(5, 6), Seq(7, 8)) - assert(valueArray.getSizeInBytes == 4 + 8 + 8 + nestedMap.getSizeInBytes) + assert(valueArray.getSizeInBytes == 4 + 8 + 4 + nestedMap.getSizeInBytes) } assert(unsafeMap2.getSizeInBytes == 4 + keyArray.getSizeInBytes + valueArray.getSizeInBytes) @@ -426,7 +426,7 @@ class UnsafeRowConverterSuite extends SparkFunSuite with Matchers { assert(innerStruct.getLong(0) == 2L) } - assert(field2.getSizeInBytes == 4 + 8 + 8 + innerStruct.getSizeInBytes) + assert(field2.getSizeInBytes == 4 + 8 + 4 + innerStruct.getSizeInBytes) assert(unsafeRow.getSizeInBytes == 8 + 8 * 2 + field1.getSizeInBytes + roundedSize(field2.getSizeInBytes)) @@ -469,7 +469,7 @@ class UnsafeRowConverterSuite extends SparkFunSuite with Matchers { assert(innerStruct.getSizeInBytes == 8 + 8) assert(innerStruct.getLong(0) == 4L) - assert(valueArray.getSizeInBytes == 4 + 8 + 8 + innerStruct.getSizeInBytes) + assert(valueArray.getSizeInBytes == 4 + 8 + 4 + innerStruct.getSizeInBytes) } assert(field2.getSizeInBytes == 4 + keyArray.getSizeInBytes + valueArray.getSizeInBytes) @@ -498,7 +498,7 @@ class UnsafeRowConverterSuite extends SparkFunSuite with Matchers { val innerMap = field1.getMap(0) testMapInt(innerMap, Seq(1), Seq(2)) - assert(field1.getSizeInBytes == 4 + 8 + 8 + innerMap.getSizeInBytes) + assert(field1.getSizeInBytes == 4 + 8 + 4 + innerMap.getSizeInBytes) val field2 = unsafeRow.getMap(1) assert(field2.numElements == 1) @@ -514,7 +514,7 @@ class UnsafeRowConverterSuite extends SparkFunSuite with Matchers { val innerArray = valueArray.getArray(0) testArrayInt(innerArray, Seq(4)) - assert(valueArray.getSizeInBytes == 4 + 8 + 8 + innerArray.getSizeInBytes) + assert(valueArray.getSizeInBytes == 4 + 8 + 4 + innerArray.getSizeInBytes) } assert(field2.getSizeInBytes == 4 + keyArray.getSizeInBytes + valueArray.getSizeInBytes) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/UnsafeArrayDataBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/UnsafeArrayDataBenchmark.scala index ffb8fd0f276fa..7952454832cb9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/UnsafeArrayDataBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/UnsafeArrayDataBenchmark.scala @@ -122,6 +122,15 @@ class UnsafeArrayDataBenchmark extends BenchmarkBase { benchmark.addCase("Int")(writeIntArray) benchmark.addCase("Double")(writeDoubleArray) benchmark.run + /* + Java HotSpot(TM) 64-Bit Server VM 1.8.0_92-b14 on Mac OS X 10.10.4 + Intel(R) Core(TM) i5-5257U CPU @ 2.70GHz + + Write UnsafeArrayData: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------------ + Int 135 / 180 154.9 6.5 1.0X + Double 195 / 300 107.8 9.3 0.7X + */ } def getPrimitiveArray(iters: Int): Unit = { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/ColumnTypeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/ColumnTypeSuite.scala index 597009490e40d..a9912eeababa0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/ColumnTypeSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/ColumnTypeSuite.scala @@ -74,7 +74,7 @@ class ColumnTypeSuite extends SparkFunSuite with Logging { checkActualSize(COMPACT_DECIMAL(15, 10), Decimal(0, 15, 10), 8) checkActualSize(LARGE_DECIMAL(20, 10), Decimal(0, 20, 10), 5) checkActualSize(ARRAY_TYPE, Array[Any](1), 4 + 8 + 8) - checkActualSize(MAP_TYPE, Map(1 -> "a"), 4 + (4 + 8 + 8) * 2 + 1) + checkActualSize(MAP_TYPE, Map(1 -> "a"), 4 + (4 + 8 + 8) + (4 + 8 + 4 + 1)) checkActualSize(STRUCT_TYPE, Row("hello"), 28) } From 9933a06b0a58b1bf3ffc9cab574c699d902ca39c Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Sat, 6 Aug 2016 10:03:44 +0900 Subject: [PATCH 26/37] addressed review comments --- .../catalyst/expressions/UnsafeArrayData.java | 52 +++++++++++-------- .../catalyst/expressions/UnsafeMapData.java | 11 ++-- .../codegen/UnsafeArrayWriter.java | 4 +- .../spark/sql/catalyst/ScalaReflection.scala | 3 ++ .../codegen/GenerateUnsafeProjection.scala | 10 ++-- .../codegen/GenerateUnsafeRowJoiner.scala | 6 ++- .../expressions/UnsafeRowConverterSuite.scala | 22 ++++---- .../sql/catalyst/util/UnsafeArraySuite.scala | 51 ++++++++++++++++-- .../execution/columnar/ColumnTypeSuite.scala | 4 +- 9 files changed, 111 insertions(+), 52 deletions(-) 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 e6d406e91f3e5..791ade1e20151 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 @@ -33,9 +33,9 @@ /** * An Unsafe implementation of Array which is backed by raw memory instead of Java objects. * - * Each tuple has four parts: [numElements][null bits][values or offset][variable length portion] + * Each array has four parts: [numElements][null bits][values or offset][variable length portion] * - * The `numElements` is 4 bytes storing the number of elements of this array. + * The `numElements` is 8 bytes storing the number of elements of this array. * * In the `null bits` region, we store 1 bit per element, represents whether a element has null * Its total size is ceil(numElements / 8) bytes, and it is aligned to 8-byte word boundaries. @@ -44,7 +44,8 @@ * fixed-length primitive types, such as long, double, or int, we store the value directly * in the field. For fields with non-primitive or variable-length values, we store a relative * offset (w.r.t. the base address of the array) that points to the beginning of - * the variable-length field into int. its length can be got by subtracting 2 adjacent offsets + * the variable-length field into int. It can only be calculated by knowing the total bytes of + * the array. Its length can be got by subtracting 2 adjacent offsets, * * Instances of `UnsafeArrayData` act as pointers to row data stored in this format. */ @@ -52,7 +53,7 @@ public final class UnsafeArrayData extends ArrayData { public static int calculateHeaderPortionInBytes(int numFields) { - return 4 + ((numFields + 63)/ 64) * 8; + return 8 + ((numFields + 63)/ 64) * 8; } private Object baseObject; @@ -62,14 +63,14 @@ public static int calculateHeaderPortionInBytes(int numFields) { private int numElements; // The size of this array's backing data, in bytes. - // The 4-bytes header of `numElements` is also included. + // The 8-bytes header of `numElements` is also included. private int sizeInBytes; - /** The width of the null tracking bit set plus `numElements`, in bytes */ - private int headerInBytes; + /** The position to start storing array elements, */ + private long elementOffset; private long getElementOffset(int ordinal, int elementSize) { - return baseOffset + headerInBytes + ordinal * elementSize; + return elementOffset + ordinal * elementSize; } public Object getBaseObject() { return baseObject; } @@ -103,15 +104,16 @@ public UnsafeArrayData() { } * @param sizeInBytes the size of this array's backing data, in bytes */ public void pointTo(Object baseObject, long baseOffset, int sizeInBytes) { - // Read the number of elements from the first 4 bytes. - final int numElements = Platform.getInt(baseObject, baseOffset); + // Read the number of elements from the first 8 bytes. + final long numElements = Platform.getLong(baseObject, baseOffset); assert numElements >= 0 : "numElements (" + numElements + ") should >= 0"; + assert numElements <= Integer.MAX_VALUE : "numElements (" + numElements + ") should <= Integer.MAX_VALUE"; - this.numElements = numElements; + this.numElements = (int)numElements; this.baseObject = baseObject; this.baseOffset = baseOffset; this.sizeInBytes = sizeInBytes; - this.headerInBytes = calculateHeaderPortionInBytes(numElements); + this.elementOffset = baseOffset + calculateHeaderPortionInBytes(this.numElements); } private int getSize(int ordinal) { @@ -332,7 +334,7 @@ public boolean[] toBooleanArray() { int size = numElements(); boolean[] values = new boolean[size]; Platform.copyMemory( - baseObject, baseOffset + headerInBytes, values, Platform.BOOLEAN_ARRAY_OFFSET, size); + baseObject, elementOffset, values, Platform.BOOLEAN_ARRAY_OFFSET, size); return values; } @@ -341,16 +343,20 @@ public byte[] toByteArray() { int size = numElements(); byte[] values = new byte[size]; Platform.copyMemory( - baseObject, baseOffset + headerInBytes, values, Platform.BYTE_ARRAY_OFFSET, size); + baseObject, elementOffset, values, Platform.BYTE_ARRAY_OFFSET, size); return values; } @Override public short[] toShortArray() { - int size = numElements(); + if (numElements > Integer.MAX_VALUE) { + throw new UnsupportedOperationException("Cannot convert this unsafe array to array as " + + "it's too big."); + } + int size = (int)numElements; short[] values = new short[size]; Platform.copyMemory( - baseObject, baseOffset + headerInBytes, values, Platform.SHORT_ARRAY_OFFSET, size * 2); + baseObject, elementOffset, values, Platform.SHORT_ARRAY_OFFSET, size * 2); return values; } @@ -359,16 +365,20 @@ public int[] toIntArray() { int size = numElements(); int[] values = new int[size]; Platform.copyMemory( - baseObject, baseOffset + headerInBytes, values, Platform.INT_ARRAY_OFFSET, size * 4); + baseObject, elementOffset, values, Platform.INT_ARRAY_OFFSET, size * 4); return values; } @Override public long[] toLongArray() { - int size = numElements(); + if (numElements > Integer.MAX_VALUE) { + throw new UnsupportedOperationException("Cannot convert this unsafe array to array as " + + "it's too big."); + } + int size = (int)numElements; long[] values = new long[size]; Platform.copyMemory( - baseObject, baseOffset + headerInBytes, values, Platform.LONG_ARRAY_OFFSET, size * 8); + baseObject, elementOffset, values, Platform.LONG_ARRAY_OFFSET, size * 8); return values; } @@ -377,7 +387,7 @@ public float[] toFloatArray() { int size = numElements(); float[] values = new float[size]; Platform.copyMemory( - baseObject, baseOffset + headerInBytes, values, Platform.FLOAT_ARRAY_OFFSET, size * 4); + baseObject, elementOffset, values, Platform.FLOAT_ARRAY_OFFSET, size * 4); return values; } @@ -386,7 +396,7 @@ public double[] toDoubleArray() { int size = numElements(); double[] values = new double[size]; Platform.copyMemory( - baseObject, baseOffset + headerInBytes, values, Platform.DOUBLE_ARRAY_OFFSET, size * 8); + baseObject, elementOffset, values, Platform.DOUBLE_ARRAY_OFFSET, size * 8); return values; } 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 0700148becaba..823454236ca6d 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 @@ -65,14 +65,15 @@ public UnsafeMapData() { * @param sizeInBytes the size of this map's backing data, in bytes */ public void pointTo(Object baseObject, long baseOffset, int sizeInBytes) { - // Read the numBytes of key array from the first 4 bytes. - final int keyArraySize = Platform.getInt(baseObject, baseOffset); - final int valueArraySize = sizeInBytes - keyArraySize - 4; + // Read the numBytes of key array from the first 8 bytes. + final long keyArraySize = Platform.getLong(baseObject, baseOffset); + final long valueArraySize = sizeInBytes - keyArraySize - 8; assert keyArraySize >= 0 : "keyArraySize (" + keyArraySize + ") should >= 0"; + assert keyArraySize <= Integer.MAX_VALUE : "keyArraySize (" + keyArraySize + ") should <= Integer.MAX_VALUE"; assert valueArraySize >= 0 : "valueArraySize (" + valueArraySize + ") should >= 0"; - keys.pointTo(baseObject, baseOffset + 4, keyArraySize); - values.pointTo(baseObject, baseOffset + 4 + keyArraySize, valueArraySize); + keys.pointTo(baseObject, baseOffset + 8, (int)keyArraySize); + values.pointTo(baseObject, baseOffset + 8 + keyArraySize, (int)valueArraySize); assert keys.numElements() == values.numElements(); diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeArrayWriter.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeArrayWriter.java index 54adc82cbae77..b2a62839c5707 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeArrayWriter.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeArrayWriter.java @@ -47,7 +47,7 @@ private void assertIndexIsValid(int index) { } public void initialize(BufferHolder holder, int numElements, int elementSize) { - // We need 4 bytes to store numElements in header + // We need 8 bytes to store numElements in header this.numElements = numElements; this.headerInBytes = calculateHeaderPortionInBytes(numElements); @@ -58,7 +58,7 @@ public void initialize(BufferHolder holder, int numElements, int elementSize) { holder.grow(headerInBytes + elementSize * numElements); // Write numElements and clear out null bits to header - Platform.putInt(holder.buffer, startingOffset, numElements); + Platform.putLong(holder.buffer, startingOffset, numElements); for (int i = 4; i < headerInBytes; i += 8) { Platform.putLong(holder.buffer, startingOffset + i, 0L); } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala index 7923cfce82100..528286d72de06 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala @@ -101,6 +101,8 @@ object ScalaReflection extends ScalaReflection { case t if t <:< definitions.ShortTpe => classOf[Array[Short]] case t if t <:< definitions.ByteTpe => classOf[Array[Byte]] case t if t <:< definitions.BooleanTpe => classOf[Array[Boolean]] + case t if t <:< localTypeOf[CalendarInterval] => classOf[Array[CalendarInterval]] + case t if t <:< localTypeOf[Decimal] => classOf[Array[Decimal]] case other => // There is probably a better way to do this, but I couldn't find it... val elementType = dataTypeFor(other).asInstanceOf[ObjectType].cls @@ -678,6 +680,7 @@ object ScalaReflection extends ScalaReflection { Schema(DecimalType.BigIntDecimal, nullable = true) case t if t <:< localTypeOf[scala.math.BigInt] => Schema(DecimalType.BigIntDecimal, nullable = true) + case t if t <:< localTypeOf[CalendarInterval] => Schema(CalendarIntervalType, nullable = true) case t if t <:< localTypeOf[Decimal] => Schema(DecimalType.SYSTEM_DEFAULT, nullable = true) case t if t <:< localTypeOf[java.lang.Integer] => Schema(IntegerType, nullable = true) case t if t <:< localTypeOf[java.lang.Long] => Schema(LongType, nullable = true) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjection.scala index c2f503a98ffdc..0ae3f86743f7e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjection.scala @@ -262,16 +262,16 @@ object GenerateUnsafeProjection extends CodeGenerator[Seq[Expression], UnsafePro final ArrayData $keys = $input.keyArray(); final ArrayData $values = $input.valueArray(); - // preserve 4 bytes to write the key array numBytes later. - $bufferHolder.grow(4); - $bufferHolder.cursor += 4; + // preserve 8 bytes to write the key array numBytes later. + $bufferHolder.grow(8); + $bufferHolder.cursor += 8; // Remember the current cursor so that we can write numBytes of key array later. final int $tmpCursor = $bufferHolder.cursor; ${writeArrayToBuffer(ctx, keys, keyType, bufferHolder)} - // Write the numBytes of key array into the first 4 bytes. - Platform.putInt($bufferHolder.buffer, $tmpCursor - 4, $bufferHolder.cursor - $tmpCursor); + // Write the numBytes of key array into the first 8 bytes. + Platform.putLong($bufferHolder.buffer, $tmpCursor - 8, $bufferHolder.cursor - $tmpCursor); ${writeArrayToBuffer(ctx, values, valueType, bufferHolder)} } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeRowJoiner.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeRowJoiner.scala index 4aa5ec82471ec..55ace42222a5f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeRowJoiner.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeRowJoiner.scala @@ -170,8 +170,12 @@ object GenerateUnsafeRowJoiner extends CodeGenerator[(StructType, StructType), U | // row1: ${schema1.size} fields, $bitset1Words words in bitset | // row2: ${schema2.size}, $bitset2Words words in bitset | // output: ${schema1.size + schema2.size} fields, $outputBitsetWords words in bitset - | final int sizeInBytes = row1.getSizeInBytes() + row2.getSizeInBytes() - $sizeReduction; + | final long sizeInBytes = row1.getSizeInBytes() + row2.getSizeInBytes() - $sizeReduction; | if (sizeInBytes > buf.length) { + | if (sizeInBytes > Integer.MAX_VALUE) { + | throw new UnsupportedOperationException("Cannot allocate an array as " + + | "it's too big."); + | } | buf = new byte[sizeInBytes]; | } | diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/UnsafeRowConverterSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/UnsafeRowConverterSuite.scala index d41b23cd3d244..b1403304b5dfc 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/UnsafeRowConverterSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/UnsafeRowConverterSuite.scala @@ -301,7 +301,7 @@ class UnsafeRowConverterSuite extends SparkFunSuite with Matchers { private def testArrayInt(array: UnsafeArrayData, values: Seq[Int]): Unit = { assert(array.numElements == values.length) assert(array.getSizeInBytes == - 4 + scala.math.ceil(values.length / 64.toDouble) * 8 + 4 * values.length) + 8 + scala.math.ceil(values.length / 64.toDouble) * 8 + 4 * values.length) values.zipWithIndex.foreach { case (value, index) => assert(array.getInt(index) == value) } @@ -314,7 +314,7 @@ class UnsafeRowConverterSuite extends SparkFunSuite with Matchers { testArrayInt(map.keyArray, keys) testArrayInt(map.valueArray, values) - assert(map.getSizeInBytes == 4 + map.keyArray.getSizeInBytes + map.valueArray.getSizeInBytes) + assert(map.getSizeInBytes == 8 + map.keyArray.getSizeInBytes + map.valueArray.getSizeInBytes) } test("basic conversion with array type") { @@ -340,7 +340,7 @@ class UnsafeRowConverterSuite extends SparkFunSuite with Matchers { val nestedArray = unsafeArray2.getArray(0) testArrayInt(nestedArray, Seq(3, 4)) - assert(unsafeArray2.getSizeInBytes == 4 + 8 + 4 + nestedArray.getSizeInBytes) + assert(unsafeArray2.getSizeInBytes == 8 + 8 + 4 + nestedArray.getSizeInBytes) val array1Size = roundedSize(unsafeArray1.getSizeInBytes) val array2Size = roundedSize(unsafeArray2.getSizeInBytes) @@ -383,10 +383,10 @@ class UnsafeRowConverterSuite extends SparkFunSuite with Matchers { val nestedMap = valueArray.getMap(0) testMapInt(nestedMap, Seq(5, 6), Seq(7, 8)) - assert(valueArray.getSizeInBytes == 4 + 8 + 4 + nestedMap.getSizeInBytes) + assert(valueArray.getSizeInBytes == 8 + 8 + 4 + nestedMap.getSizeInBytes) } - assert(unsafeMap2.getSizeInBytes == 4 + keyArray.getSizeInBytes + valueArray.getSizeInBytes) + assert(unsafeMap2.getSizeInBytes == 8 + keyArray.getSizeInBytes + valueArray.getSizeInBytes) val map1Size = roundedSize(unsafeMap1.getSizeInBytes) val map2Size = roundedSize(unsafeMap2.getSizeInBytes) @@ -426,7 +426,7 @@ class UnsafeRowConverterSuite extends SparkFunSuite with Matchers { assert(innerStruct.getLong(0) == 2L) } - assert(field2.getSizeInBytes == 4 + 8 + 4 + innerStruct.getSizeInBytes) + assert(field2.getSizeInBytes == 8 + 8 + 4 + innerStruct.getSizeInBytes) assert(unsafeRow.getSizeInBytes == 8 + 8 * 2 + field1.getSizeInBytes + roundedSize(field2.getSizeInBytes)) @@ -469,10 +469,10 @@ class UnsafeRowConverterSuite extends SparkFunSuite with Matchers { assert(innerStruct.getSizeInBytes == 8 + 8) assert(innerStruct.getLong(0) == 4L) - assert(valueArray.getSizeInBytes == 4 + 8 + 4 + innerStruct.getSizeInBytes) + assert(valueArray.getSizeInBytes == 8 + 8 + 4 + innerStruct.getSizeInBytes) } - assert(field2.getSizeInBytes == 4 + keyArray.getSizeInBytes + valueArray.getSizeInBytes) + assert(field2.getSizeInBytes == 8 + keyArray.getSizeInBytes + valueArray.getSizeInBytes) assert(unsafeRow.getSizeInBytes == 8 + 8 * 2 + field1.getSizeInBytes + roundedSize(field2.getSizeInBytes)) @@ -498,7 +498,7 @@ class UnsafeRowConverterSuite extends SparkFunSuite with Matchers { val innerMap = field1.getMap(0) testMapInt(innerMap, Seq(1), Seq(2)) - assert(field1.getSizeInBytes == 4 + 8 + 4 + innerMap.getSizeInBytes) + assert(field1.getSizeInBytes == 8 + 8 + 4 + innerMap.getSizeInBytes) val field2 = unsafeRow.getMap(1) assert(field2.numElements == 1) @@ -514,10 +514,10 @@ class UnsafeRowConverterSuite extends SparkFunSuite with Matchers { val innerArray = valueArray.getArray(0) testArrayInt(innerArray, Seq(4)) - assert(valueArray.getSizeInBytes == 4 + 8 + 4 + innerArray.getSizeInBytes) + assert(valueArray.getSizeInBytes == 8 + 8 + 4 + innerArray.getSizeInBytes) } - assert(field2.getSizeInBytes == 4 + keyArray.getSizeInBytes + valueArray.getSizeInBytes) + assert(field2.getSizeInBytes == 8 + keyArray.getSizeInBytes + valueArray.getSizeInBytes) assert(unsafeRow.getSizeInBytes == 8 + 8 * 2 + roundedSize(field1.getSizeInBytes) + roundedSize(field2.getSizeInBytes)) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/UnsafeArraySuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/UnsafeArraySuite.scala index 7d6071111c060..c3f39a0e8fa76 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/UnsafeArraySuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/UnsafeArraySuite.scala @@ -20,7 +20,8 @@ package org.apache.spark.sql.catalyst.util import org.apache.spark.SparkFunSuite import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder import org.apache.spark.sql.catalyst.expressions.UnsafeArrayData -import org.apache.spark.unsafe.Platform +import org.apache.spark.sql.types.{DateType, Decimal, TimestampType} +import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} class UnsafeArraySuite extends SparkFunSuite { @@ -31,8 +32,16 @@ class UnsafeArraySuite extends SparkFunSuite { val floatArray = Array(1.1.toFloat, 2.2.toFloat, 3.3.toFloat) val doubleArray = Array(1.1, 2.2, 3.3) val stringArray = Array("1", "10", "100") - - val intMultiDimArray = Array(Array(1, 10), Array(2, 20, 200), Array(3, 30, 300, 3000)) + val dateArray = Array( + DateTimeUtils.stringToDate(UTF8String.fromString("1970-1-1")).get, + DateTimeUtils.stringToDate(UTF8String.fromString("2016-7-26")).get) + val timestampArray = Array( + DateTimeUtils.stringToTimestamp(UTF8String.fromString("1970-1-1 00:00:00")).get, + DateTimeUtils.stringToTimestamp(UTF8String.fromString("2016-7-26 00:00:00")).get) + val decimalArray = Array(Decimal(77L, 2, 1), Decimal(77L, 12, 1), Decimal(77L, 20, 1)) + val calenderintervalArray = Array(new CalendarInterval(3, 321), new CalendarInterval(1, 123)) + + val intMultiDimArray = Array(Array(1), Array(2, 20), Array(3, 30, 300)) val doubleMultiDimArray = Array( Array(1.1, 11.1), Array(2.2, 22.2, 222.2), Array(3.3, 33.3, 333.3, 3333.3)) @@ -93,6 +102,38 @@ class UnsafeArraySuite extends SparkFunSuite { assert(unsafeString.getUTF8String(i).toString().equals(e)) } + val unsafeDate = ExpressionEncoder[Array[Int]].resolveAndBind(). + toRow(dateArray).getArray(0) + assert(unsafeDate.isInstanceOf[UnsafeArrayData]) + assert(unsafeDate.numElements == dateArray.length) + dateArray.zipWithIndex.map { case (e, i) => + assert(unsafeDate.get(i, DateType) == e) + } + + val unsafeTimestamp = ExpressionEncoder[Array[Long]].resolveAndBind(). + toRow(timestampArray).getArray(0) + assert(unsafeTimestamp.isInstanceOf[UnsafeArrayData]) + assert(unsafeTimestamp.numElements == timestampArray.length) + timestampArray.zipWithIndex.map { case (e, i) => + assert(unsafeTimestamp.get(i, TimestampType) == e) + } + + val unsafeDecimal = ExpressionEncoder[Array[Decimal]].resolveAndBind(). + toRow(decimalArray).getArray(0) + assert(unsafeDecimal.isInstanceOf[UnsafeArrayData]) + assert(unsafeDecimal.numElements == decimalArray.length) + decimalArray.zipWithIndex.map { case (e, i) => + assert(unsafeDecimal.getDecimal(i, e.precision, e.scale) == e) + } + + val unsafeInterval = ExpressionEncoder[Array[CalendarInterval]].resolveAndBind(). + toRow(calenderintervalArray).getArray(0) + assert(unsafeInterval.isInstanceOf[UnsafeArrayData]) + assert(unsafeInterval.numElements == calenderintervalArray.length) + calenderintervalArray.zipWithIndex.map { case (e, i) => + assert(unsafeInterval.getInterval(i) == e) + } + val unsafeMultiDimInt = ExpressionEncoder[Array[Array[Int]]].resolveAndBind(). toRow(intMultiDimArray).getArray(0) assert(unsafeMultiDimInt.isInstanceOf[UnsafeArrayData]) @@ -124,7 +165,7 @@ class UnsafeArraySuite extends SparkFunSuite { val unsafeInt = UnsafeArrayData.fromPrimitiveArray(intArray) assert(unsafeInt.numElements == 3) assert(unsafeInt.getSizeInBytes == - ((4 + scala.math.ceil(3/64.toDouble) * 8 + 4 * 3 + 7).toInt / 8) * 8) + ((8 + scala.math.ceil(3/64.toDouble) * 8 + 4 * 3 + 7).toInt / 8) * 8) intArray.zipWithIndex.map { case (e, i) => assert(unsafeInt.getInt(i) == e) } @@ -132,7 +173,7 @@ class UnsafeArraySuite extends SparkFunSuite { val unsafeDouble = UnsafeArrayData.fromPrimitiveArray(doubleArray) assert(unsafeDouble.numElements == 3) assert(unsafeDouble.getSizeInBytes == - ((4 + scala.math.ceil(3/64.toDouble) * 8 + 8 * 3 + 7).toInt / 8) * 8) + ((8 + scala.math.ceil(3/64.toDouble) * 8 + 8 * 3 + 7).toInt / 8) * 8) doubleArray.zipWithIndex.map { case (e, i) => assert(unsafeDouble.getDouble(i) == e) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/ColumnTypeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/ColumnTypeSuite.scala index a9912eeababa0..95b616133f8df 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/ColumnTypeSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/ColumnTypeSuite.scala @@ -73,8 +73,8 @@ class ColumnTypeSuite extends SparkFunSuite with Logging { checkActualSize(BINARY, Array.fill[Byte](4)(0.toByte), 4 + 4) checkActualSize(COMPACT_DECIMAL(15, 10), Decimal(0, 15, 10), 8) checkActualSize(LARGE_DECIMAL(20, 10), Decimal(0, 20, 10), 5) - checkActualSize(ARRAY_TYPE, Array[Any](1), 4 + 8 + 8) - checkActualSize(MAP_TYPE, Map(1 -> "a"), 4 + (4 + 8 + 8) + (4 + 8 + 4 + 1)) + checkActualSize(ARRAY_TYPE, Array[Any](1), 8 + 8 + 8) + checkActualSize(MAP_TYPE, Map(1 -> "a"), 8 + (8 + 8 + 8) + (8 + 8 + 4 + 1)) checkActualSize(STRUCT_TYPE, Row("hello"), 28) } From 919e832cc4f27361fde6711ac6055c4b19ad1046 Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Sun, 7 Aug 2016 13:04:15 +0900 Subject: [PATCH 27/37] fixed test failures --- .../spark/sql/catalyst/expressions/UnsafeArrayData.java | 4 ++-- .../sql/catalyst/expressions/codegen/UnsafeArrayWriter.java | 4 ++-- .../expressions/codegen/GenerateUnsafeRowJoiner.scala | 6 +----- 3 files changed, 5 insertions(+), 9 deletions(-) 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 791ade1e20151..dbfd68644c7c6 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 @@ -127,7 +127,7 @@ private int getSize(int ordinal) { @Override public boolean isNullAt(int ordinal) { assertIndexIsValid(ordinal); - return BitSetMethods.isSet(baseObject, baseOffset + 4, ordinal); + return BitSetMethods.isSet(baseObject, baseOffset + 8, ordinal); } @Override @@ -412,7 +412,7 @@ private static UnsafeArrayData fromPrimitiveArray( final long[] data = new long[(int)totalSizeInLongs]; - Platform.putInt(data, Platform.LONG_ARRAY_OFFSET, length); + Platform.putLong(data, Platform.LONG_ARRAY_OFFSET, length); Platform.copyMemory(arr, offset, data, Platform.LONG_ARRAY_OFFSET + headerInBytes, valueRegionInBytes); diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeArrayWriter.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeArrayWriter.java index b2a62839c5707..afd67236cf538 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeArrayWriter.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeArrayWriter.java @@ -59,7 +59,7 @@ public void initialize(BufferHolder holder, int numElements, int elementSize) { // Write numElements and clear out null bits to header Platform.putLong(holder.buffer, startingOffset, numElements); - for (int i = 4; i < headerInBytes; i += 8) { + for (int i = 8; i < headerInBytes; i += 8) { Platform.putLong(holder.buffer, startingOffset + i, 0L); } holder.cursor += (headerInBytes + elementSize * numElements); @@ -75,7 +75,7 @@ public void setOffset(int ordinal) { private void setNullBit(int ordinal) { assertIndexIsValid(ordinal); - BitSetMethods.set(holder.buffer, startingOffset + 4, ordinal); + BitSetMethods.set(holder.buffer, startingOffset + 8, ordinal); } public void setNullBoolean(int ordinal) { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeRowJoiner.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeRowJoiner.scala index 55ace42222a5f..4aa5ec82471ec 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeRowJoiner.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeRowJoiner.scala @@ -170,12 +170,8 @@ object GenerateUnsafeRowJoiner extends CodeGenerator[(StructType, StructType), U | // row1: ${schema1.size} fields, $bitset1Words words in bitset | // row2: ${schema2.size}, $bitset2Words words in bitset | // output: ${schema1.size + schema2.size} fields, $outputBitsetWords words in bitset - | final long sizeInBytes = row1.getSizeInBytes() + row2.getSizeInBytes() - $sizeReduction; + | final int sizeInBytes = row1.getSizeInBytes() + row2.getSizeInBytes() - $sizeReduction; | if (sizeInBytes > buf.length) { - | if (sizeInBytes > Integer.MAX_VALUE) { - | throw new UnsupportedOperationException("Cannot allocate an array as " + - | "it's too big."); - | } | buf = new byte[sizeInBytes]; | } | From 0886e3a6b0ae03eeaebde6e374f192685c375b7d Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Wed, 10 Aug 2016 06:05:49 +0900 Subject: [PATCH 28/37] update test suites --- .../sql/catalyst/util/UnsafeArraySuite.scala | 33 ++++++++++++++----- 1 file changed, 24 insertions(+), 9 deletions(-) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/UnsafeArraySuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/UnsafeArraySuite.scala index c3f39a0e8fa76..b4353e8d495d0 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/UnsafeArraySuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/UnsafeArraySuite.scala @@ -18,9 +18,10 @@ package org.apache.spark.sql.catalyst.util import org.apache.spark.SparkFunSuite -import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder +import org.apache.spark.sql.catalyst.encoders.{ExpressionEncoder, RowEncoder} import org.apache.spark.sql.catalyst.expressions.UnsafeArrayData -import org.apache.spark.sql.types.{DateType, Decimal, TimestampType} +import org.apache.spark.sql.Row +import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} class UnsafeArraySuite extends SparkFunSuite { @@ -38,7 +39,13 @@ class UnsafeArraySuite extends SparkFunSuite { val timestampArray = Array( DateTimeUtils.stringToTimestamp(UTF8String.fromString("1970-1-1 00:00:00")).get, DateTimeUtils.stringToTimestamp(UTF8String.fromString("2016-7-26 00:00:00")).get) - val decimalArray = Array(Decimal(77L, 2, 1), Decimal(77L, 12, 1), Decimal(77L, 20, 1)) + val decimalArray4_1 = Array( + BigDecimal("123.4").setScale(1, BigDecimal.RoundingMode.FLOOR), + BigDecimal("567.8").setScale(1, BigDecimal.RoundingMode.FLOOR)) + val decimalArray20_20 = Array( + BigDecimal("1.2345678901234567890123456").setScale(21, BigDecimal.RoundingMode.FLOOR), + BigDecimal("2.3456789012345678901234567").setScale(21, BigDecimal.RoundingMode.FLOOR)) + val calenderintervalArray = Array(new CalendarInterval(3, 321), new CalendarInterval(1, 123)) val intMultiDimArray = Array(Array(1), Array(2, 20), Array(3, 30, 300)) @@ -118,12 +125,20 @@ class UnsafeArraySuite extends SparkFunSuite { assert(unsafeTimestamp.get(i, TimestampType) == e) } - val unsafeDecimal = ExpressionEncoder[Array[Decimal]].resolveAndBind(). - toRow(decimalArray).getArray(0) - assert(unsafeDecimal.isInstanceOf[UnsafeArrayData]) - assert(unsafeDecimal.numElements == decimalArray.length) - decimalArray.zipWithIndex.map { case (e, i) => - assert(unsafeDecimal.getDecimal(i, e.precision, e.scale) == e) + Seq(decimalArray4_1, decimalArray20_20).map { decimalArray => + val decimal = decimalArray(0) + val schema = new StructType().add( + "array", ArrayType(DecimalType(decimal.precision, decimal.scale))) + val encoder = RowEncoder(schema).resolveAndBind() + val externalRow = Row(decimalArray) + val ir = encoder.toRow(externalRow) + + val unsafeDecimal = ir.getArray(0) + assert(unsafeDecimal.isInstanceOf[UnsafeArrayData]) + assert(unsafeDecimal.numElements == decimalArray.length) + decimalArray.zipWithIndex.map { case (e, i) => + assert(unsafeDecimal.getDecimal(i, e.precision, e.scale).toBigDecimal == e) + } } val unsafeInterval = ExpressionEncoder[Array[CalendarInterval]].resolveAndBind(). From c385bf485af2ed33465aae906abd8246b512a5e2 Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Fri, 19 Aug 2016 04:25:30 +0900 Subject: [PATCH 29/37] align each of variable length elements to 8 bytes --- .../catalyst/expressions/UnsafeArrayData.java | 47 +++++++++-------- .../codegen/UnsafeArrayWriter.java | 50 +++++++++++++------ .../codegen/GenerateUnsafeProjection.scala | 15 ++++-- .../sql/execution/columnar/ColumnType.scala | 4 +- .../execution/columnar/ColumnTypeSuite.scala | 4 +- 5 files changed, 74 insertions(+), 46 deletions(-) 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 dbfd68644c7c6..c9314666001be 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 @@ -33,19 +33,20 @@ /** * An Unsafe implementation of Array which is backed by raw memory instead of Java objects. * - * Each array has four parts: [numElements][null bits][values or offset][variable length portion] + * Each array has four parts: + * [numElements][null bits][values or offset&length][variable length portion] * * The `numElements` is 8 bytes storing the number of elements of this array. * * In the `null bits` region, we store 1 bit per element, represents whether a element has null - * Its total size is ceil(numElements / 8) bytes, and it is aligned to 8-byte word boundaries. + * Its total size is ceil(numElements / 8) bytes, and it is aligned to 8-byte boundaries. * - * In the `values or offset` region, we store the content of elements. For fields that hold + * In the `values or offset&length` region, we store the content of elements. For fields that hold * fixed-length primitive types, such as long, double, or int, we store the value directly * in the field. For fields with non-primitive or variable-length values, we store a relative * offset (w.r.t. the base address of the array) that points to the beginning of - * the variable-length field into int. It can only be calculated by knowing the total bytes of - * the array. Its length can be got by subtracting 2 adjacent offsets, + * the variable-length field and length (they are combined into a long). For variable length + * portion, each is aligned to 8-byte boundaries. * * Instances of `UnsafeArrayData` act as pointers to row data stored in this format. */ @@ -116,14 +117,6 @@ public void pointTo(Object baseObject, long baseOffset, int sizeInBytes) { this.elementOffset = baseOffset + calculateHeaderPortionInBytes(this.numElements); } - private int getSize(int ordinal) { - if (ordinal != numElements - 1) { - return getInt(ordinal + 1) - getInt(ordinal); - } else { - return sizeInBytes - getInt(ordinal); - } - } - @Override public boolean isNullAt(int ordinal) { assertIndexIsValid(ordinal); @@ -232,16 +225,18 @@ public Decimal getDecimal(int ordinal, int precision, int scale) { @Override public UTF8String getUTF8String(int ordinal) { if (isNullAt(ordinal)) return null; - final int offset = getInt(ordinal); - final int size = getSize(ordinal); + final long offsetAndSize = getLong(ordinal); + final int offset = (int) (offsetAndSize >> 32); + final int size = (int) offsetAndSize; return UTF8String.fromAddress(baseObject, baseOffset + offset, size); } @Override public byte[] getBinary(int ordinal) { if (isNullAt(ordinal)) return null; - final int offset = getInt(ordinal); - final int size = getSize(ordinal); + 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; @@ -250,7 +245,8 @@ public byte[] getBinary(int ordinal) { @Override public CalendarInterval getInterval(int ordinal) { if (isNullAt(ordinal)) return null; - final int offset = getInt(ordinal); + 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); @@ -259,8 +255,9 @@ public CalendarInterval getInterval(int ordinal) { @Override public UnsafeRow getStruct(int ordinal, int numFields) { if (isNullAt(ordinal)) return null; - final int offset = getInt(ordinal); - final int size = getSize(ordinal); + 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; @@ -269,8 +266,9 @@ public UnsafeRow getStruct(int ordinal, int numFields) { @Override public UnsafeArrayData getArray(int ordinal) { if (isNullAt(ordinal)) return null; - final int offset = getInt(ordinal); - final int size = getSize(ordinal); + 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; @@ -279,8 +277,9 @@ public UnsafeArrayData getArray(int ordinal) { @Override public UnsafeMapData getMap(int ordinal) { if (isNullAt(ordinal)) return null; - final int offset = getInt(ordinal); - final int size = getSize(ordinal); + 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; diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeArrayWriter.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeArrayWriter.java index afd67236cf538..226cca03afa5d 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeArrayWriter.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeArrayWriter.java @@ -55,22 +55,41 @@ public void initialize(BufferHolder holder, int numElements, int elementSize) { this.startingOffset = holder.cursor; // Grows the global buffer ahead for header and fixed size data. - holder.grow(headerInBytes + elementSize * numElements); + int fixedPartLength = ((elementSize * numElements + 7) / 8) * 8; + holder.grow(headerInBytes + fixedPartLength); // Write numElements and clear out null bits to header Platform.putLong(holder.buffer, startingOffset, numElements); for (int i = 8; i < headerInBytes; i += 8) { Platform.putLong(holder.buffer, startingOffset + i, 0L); } - holder.cursor += (headerInBytes + elementSize * numElements); + holder.cursor += (headerInBytes + fixedPartLength); } private long getElementOffset(int ordinal, int elementSize) { return startingOffset + headerInBytes + ordinal * elementSize; } - public void setOffset(int ordinal) { - write(ordinal, holder.cursor - startingOffset); + public void setOffsetAndSize(int ordinal, long currentCursor, long size) { + final long relativeOffset = currentCursor - startingOffset; + final long offsetAndSize = (relativeOffset << 32) | size; + + write(ordinal, offsetAndSize); + } + + // Do word alignment for this row and grow the row buffer if needed. + public void alignToEightBytes(int numBytes) { + final int remainder = numBytes & 0x07; + + if (remainder > 0) { + final int paddingBytes = 8 - remainder; + holder.grow(paddingBytes); + + for (int i = 0; i < paddingBytes; i++) { + Platform.putByte(holder.buffer, holder.cursor, (byte) 0); + holder.cursor++; + } + } } private void setNullBit(int ordinal) { @@ -182,10 +201,10 @@ public void write(int ordinal, Decimal input, int precision, int scale) { // Write the bytes to the variable length portion. Platform.copyMemory( bytes, Platform.BYTE_ARRAY_OFFSET, holder.buffer, holder.cursor, bytes.length); - setOffset(ordinal); + write(ordinal, ((long)(holder.cursor - startingOffset) << 32) | ((long) bytes.length)); - // move the cursor forward. - holder.cursor += bytes.length; + // move the cursor forward with 8-bytes boundary + holder.cursor += ((bytes.length + 7) / 8) * 8; } } else { setNull(ordinal); @@ -194,31 +213,34 @@ public void write(int ordinal, Decimal input, int precision, int scale) { public void write(int ordinal, UTF8String input) { final int numBytes = input.numBytes(); + final int bufferLength = ((numBytes + 7) / 8) * 8; // 8-bytes boundary // grow the global buffer before writing data. - holder.grow(numBytes); + holder.grow(bufferLength); // Write the bytes to the variable length portion. input.writeToMemory(holder.buffer, holder.cursor); - setOffset(ordinal); + write(ordinal, ((long)(holder.cursor - startingOffset) << 32) | ((long) numBytes)); // move the cursor forward. - holder.cursor += numBytes; + holder.cursor += bufferLength; } public void write(int ordinal, byte[] input) { + final int bufferLength = ((input.length + 7) / 8) * 8; // 8-bytes boundary + // grow the global buffer before writing data. - holder.grow(input.length); + holder.grow(bufferLength); // Write the bytes to the variable length portion. Platform.copyMemory( input, Platform.BYTE_ARRAY_OFFSET, holder.buffer, holder.cursor, input.length); - setOffset(ordinal); + write(ordinal, ((long)(holder.cursor - startingOffset) << 32) | ((long) input.length)); // move the cursor forward. - holder.cursor += input.length; + holder.cursor += bufferLength; } public void write(int ordinal, CalendarInterval input) { @@ -229,7 +251,7 @@ public void write(int ordinal, CalendarInterval input) { Platform.putLong(holder.buffer, holder.cursor, input.months); Platform.putLong(holder.buffer, holder.cursor + 8, input.microseconds); - setOffset(ordinal); + write(ordinal, ((long)(holder.cursor - startingOffset) << 32) | ((long) 16)); // move the cursor forward. holder.cursor += 16; diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjection.scala index 0ae3f86743f7e..3feab3b092854 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjection.scala @@ -192,26 +192,33 @@ object GenerateUnsafeProjection extends CodeGenerator[Seq[Expression], UnsafePro val elementOrOffsetSize = et match { case t: DecimalType if t.precision <= Decimal.MAX_LONG_DIGITS => 8 case _ if ctx.isPrimitiveType(jt) => et.defaultSize - case _ => 4 // we need 4 bytes to store offset + case _ => 8 // we need 8 bytes to store offset and length } + val tmpCursor = ctx.freshName("tmpCursor") val writeElement = et match { case t: StructType => s""" - $arrayWriter.setOffset($index); + final int $tmpCursor = $bufferHolder.cursor; ${writeStructToBuffer(ctx, element, t.map(_.dataType), bufferHolder)} + $arrayWriter.setOffsetAndSize($index, $tmpCursor, $bufferHolder.cursor - $tmpCursor); + $arrayWriter.alignToEightBytes($bufferHolder.cursor - $tmpCursor); """ case a @ ArrayType(et, _) => s""" - $arrayWriter.setOffset($index); + final int $tmpCursor = $bufferHolder.cursor; ${writeArrayToBuffer(ctx, element, et, bufferHolder)} + $arrayWriter.setOffsetAndSize($index, $tmpCursor, $bufferHolder.cursor - $tmpCursor); + $arrayWriter.alignToEightBytes($bufferHolder.cursor - $tmpCursor); """ case m @ MapType(kt, vt, _) => s""" - $arrayWriter.setOffset($index); + final int $tmpCursor = $bufferHolder.cursor; ${writeMapToBuffer(ctx, element, kt, vt, bufferHolder)} + $arrayWriter.setOffsetAndSize($index, $tmpCursor, $bufferHolder.cursor - $tmpCursor); + $arrayWriter.alignToEightBytes($bufferHolder.cursor - $tmpCursor); """ case t: DecimalType => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnType.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnType.scala index f9d606e37ea89..fa9619eb07fec 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnType.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnType.scala @@ -601,7 +601,7 @@ private[columnar] case class ARRAY(dataType: ArrayType) override def actualSize(row: InternalRow, ordinal: Int): Int = { val unsafeArray = getField(row, ordinal) - 4 + unsafeArray.getSizeInBytes + 8 + unsafeArray.getSizeInBytes } override def append(value: UnsafeArrayData, buffer: ByteBuffer): Unit = { @@ -640,7 +640,7 @@ private[columnar] case class MAP(dataType: MapType) override def actualSize(row: InternalRow, ordinal: Int): Int = { val unsafeMap = getField(row, ordinal) - 4 + unsafeMap.getSizeInBytes + 8 + unsafeMap.getSizeInBytes } override def append(value: UnsafeMapData, buffer: ByteBuffer): Unit = { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/ColumnTypeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/ColumnTypeSuite.scala index 95b616133f8df..0b93c633b2d93 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/ColumnTypeSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/ColumnTypeSuite.scala @@ -73,8 +73,8 @@ class ColumnTypeSuite extends SparkFunSuite with Logging { checkActualSize(BINARY, Array.fill[Byte](4)(0.toByte), 4 + 4) checkActualSize(COMPACT_DECIMAL(15, 10), Decimal(0, 15, 10), 8) checkActualSize(LARGE_DECIMAL(20, 10), Decimal(0, 20, 10), 5) - checkActualSize(ARRAY_TYPE, Array[Any](1), 8 + 8 + 8) - checkActualSize(MAP_TYPE, Map(1 -> "a"), 8 + (8 + 8 + 8) + (8 + 8 + 4 + 1)) + checkActualSize(ARRAY_TYPE, Array[Any](1), 8 + 8 + 8 + 8) + checkActualSize(MAP_TYPE, Map(1 -> "a"), 8 + (8 + 8 + 8 + 8) + (8 + 8 + 8 + 8)) checkActualSize(STRUCT_TYPE, Row("hello"), 28) } From c8813db0ac1d2945e88933ffd08ba3557731a976 Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Sat, 20 Aug 2016 13:38:08 +0900 Subject: [PATCH 30/37] fixed test failures some refinements --- .../codegen/UnsafeArrayWriter.java | 69 +++++++++---------- .../codegen/GenerateUnsafeProjection.scala | 3 - .../expressions/UnsafeRowConverterSuite.scala | 14 ++-- 3 files changed, 39 insertions(+), 47 deletions(-) diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeArrayWriter.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeArrayWriter.java index 226cca03afa5d..e7c5989416440 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeArrayWriter.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeArrayWriter.java @@ -55,15 +55,21 @@ public void initialize(BufferHolder holder, int numElements, int elementSize) { this.startingOffset = holder.cursor; // Grows the global buffer ahead for header and fixed size data. - int fixedPartLength = ((elementSize * numElements + 7) / 8) * 8; - holder.grow(headerInBytes + fixedPartLength); + int fixedPartInBytes = ((elementSize * numElements + 7) / 8) * 8; + holder.grow(headerInBytes + fixedPartInBytes); // Write numElements and clear out null bits to header Platform.putLong(holder.buffer, startingOffset, numElements); for (int i = 8; i < headerInBytes; i += 8) { Platform.putLong(holder.buffer, startingOffset + i, 0L); } - holder.cursor += (headerInBytes + fixedPartLength); + holder.cursor += (headerInBytes + fixedPartInBytes); + } + + private void zeroOutPaddingBytes(int numBytes) { + if ((numBytes & 0x07) > 0) { + Platform.putLong(holder.buffer, holder.cursor + ((numBytes >> 3) << 3), 0L); + } } private long getElementOffset(int ordinal, int elementSize) { @@ -77,21 +83,6 @@ public void setOffsetAndSize(int ordinal, long currentCursor, long size) { write(ordinal, offsetAndSize); } - // Do word alignment for this row and grow the row buffer if needed. - public void alignToEightBytes(int numBytes) { - final int remainder = numBytes & 0x07; - - if (remainder > 0) { - final int paddingBytes = 8 - remainder; - holder.grow(paddingBytes); - - for (int i = 0; i < paddingBytes; i++) { - Platform.putByte(holder.buffer, holder.cursor, (byte) 0); - holder.cursor++; - } - } - } - private void setNullBit(int ordinal) { assertIndexIsValid(ordinal); BitSetMethods.set(holder.buffer, startingOffset + 8, ordinal); @@ -139,12 +130,7 @@ public void setNullDouble(int ordinal) { Platform.putDouble(holder.buffer, getElementOffset(ordinal, 8), (double)0); } - public void setNull(int ordinal) { - setNullBit(ordinal); - // put relative offset for current cursor position - // while put non-zero for the corresponding null field, it is still deterministic - write(ordinal, holder.cursor - startingOffset); - } + public void setNull(int ordinal) { setNullLong(ordinal); } public void write(int ordinal, boolean value) { assertIndexIsValid(ordinal); @@ -195,16 +181,20 @@ public void write(int ordinal, Decimal input, int precision, int scale) { write(ordinal, input.toUnscaledLong()); } else { final byte[] bytes = input.toJavaBigDecimal().unscaledValue().toByteArray(); - assert bytes.length <= 16; - holder.grow(bytes.length); + final int numBytes = bytes.length; + assert numBytes <= 16; + int roundedSize = ((numBytes + 7) / 8) * 8; // 8-bytes boundary + holder.grow(roundedSize); + + zeroOutPaddingBytes(numBytes); // Write the bytes to the variable length portion. Platform.copyMemory( - bytes, Platform.BYTE_ARRAY_OFFSET, holder.buffer, holder.cursor, bytes.length); - write(ordinal, ((long)(holder.cursor - startingOffset) << 32) | ((long) bytes.length)); + bytes, Platform.BYTE_ARRAY_OFFSET, holder.buffer, holder.cursor, numBytes); + write(ordinal, ((long)(holder.cursor - startingOffset) << 32) | ((long) numBytes)); // move the cursor forward with 8-bytes boundary - holder.cursor += ((bytes.length + 7) / 8) * 8; + holder.cursor += roundedSize; } } else { setNull(ordinal); @@ -213,10 +203,12 @@ public void write(int ordinal, Decimal input, int precision, int scale) { public void write(int ordinal, UTF8String input) { final int numBytes = input.numBytes(); - final int bufferLength = ((numBytes + 7) / 8) * 8; // 8-bytes boundary + final int roundedSize = ((numBytes + 7) / 8) * 8; // 8-bytes boundary // grow the global buffer before writing data. - holder.grow(bufferLength); + holder.grow(roundedSize); + + zeroOutPaddingBytes(numBytes); // Write the bytes to the variable length portion. input.writeToMemory(holder.buffer, holder.cursor); @@ -224,23 +216,26 @@ public void write(int ordinal, UTF8String input) { write(ordinal, ((long)(holder.cursor - startingOffset) << 32) | ((long) numBytes)); // move the cursor forward. - holder.cursor += bufferLength; + holder.cursor += roundedSize; } public void write(int ordinal, byte[] input) { - final int bufferLength = ((input.length + 7) / 8) * 8; // 8-bytes boundary + final int numBytes = input.length; + final int roundedSize = ((input.length + 7) / 8) * 8; // 8-bytes boundary // grow the global buffer before writing data. - holder.grow(bufferLength); + holder.grow(roundedSize); + + zeroOutPaddingBytes(numBytes); // Write the bytes to the variable length portion. Platform.copyMemory( - input, Platform.BYTE_ARRAY_OFFSET, holder.buffer, holder.cursor, input.length); + input, Platform.BYTE_ARRAY_OFFSET, holder.buffer, holder.cursor, numBytes); - write(ordinal, ((long)(holder.cursor - startingOffset) << 32) | ((long) input.length)); + write(ordinal, ((long)(holder.cursor - startingOffset) << 32) | ((long) numBytes)); // move the cursor forward. - holder.cursor += bufferLength; + holder.cursor += roundedSize; } public void write(int ordinal, CalendarInterval input) { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjection.scala index 3feab3b092854..ef4d1990f842d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjection.scala @@ -202,7 +202,6 @@ object GenerateUnsafeProjection extends CodeGenerator[Seq[Expression], UnsafePro final int $tmpCursor = $bufferHolder.cursor; ${writeStructToBuffer(ctx, element, t.map(_.dataType), bufferHolder)} $arrayWriter.setOffsetAndSize($index, $tmpCursor, $bufferHolder.cursor - $tmpCursor); - $arrayWriter.alignToEightBytes($bufferHolder.cursor - $tmpCursor); """ case a @ ArrayType(et, _) => @@ -210,7 +209,6 @@ object GenerateUnsafeProjection extends CodeGenerator[Seq[Expression], UnsafePro final int $tmpCursor = $bufferHolder.cursor; ${writeArrayToBuffer(ctx, element, et, bufferHolder)} $arrayWriter.setOffsetAndSize($index, $tmpCursor, $bufferHolder.cursor - $tmpCursor); - $arrayWriter.alignToEightBytes($bufferHolder.cursor - $tmpCursor); """ case m @ MapType(kt, vt, _) => @@ -218,7 +216,6 @@ object GenerateUnsafeProjection extends CodeGenerator[Seq[Expression], UnsafePro final int $tmpCursor = $bufferHolder.cursor; ${writeMapToBuffer(ctx, element, kt, vt, bufferHolder)} $arrayWriter.setOffsetAndSize($index, $tmpCursor, $bufferHolder.cursor - $tmpCursor); - $arrayWriter.alignToEightBytes($bufferHolder.cursor - $tmpCursor); """ case t: DecimalType => diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/UnsafeRowConverterSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/UnsafeRowConverterSuite.scala index b1403304b5dfc..90790dda753f8 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/UnsafeRowConverterSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/UnsafeRowConverterSuite.scala @@ -301,7 +301,7 @@ class UnsafeRowConverterSuite extends SparkFunSuite with Matchers { private def testArrayInt(array: UnsafeArrayData, values: Seq[Int]): Unit = { assert(array.numElements == values.length) assert(array.getSizeInBytes == - 8 + scala.math.ceil(values.length / 64.toDouble) * 8 + 4 * values.length) + 8 + scala.math.ceil(values.length / 64.toDouble) * 8 + roundedSize(4 * values.length)) values.zipWithIndex.foreach { case (value, index) => assert(array.getInt(index) == value) } @@ -340,7 +340,7 @@ class UnsafeRowConverterSuite extends SparkFunSuite with Matchers { val nestedArray = unsafeArray2.getArray(0) testArrayInt(nestedArray, Seq(3, 4)) - assert(unsafeArray2.getSizeInBytes == 8 + 8 + 4 + nestedArray.getSizeInBytes) + assert(unsafeArray2.getSizeInBytes == 8 + 8 + 8 + nestedArray.getSizeInBytes) val array1Size = roundedSize(unsafeArray1.getSizeInBytes) val array2Size = roundedSize(unsafeArray2.getSizeInBytes) @@ -383,7 +383,7 @@ class UnsafeRowConverterSuite extends SparkFunSuite with Matchers { val nestedMap = valueArray.getMap(0) testMapInt(nestedMap, Seq(5, 6), Seq(7, 8)) - assert(valueArray.getSizeInBytes == 8 + 8 + 4 + nestedMap.getSizeInBytes) + assert(valueArray.getSizeInBytes == 8 + 8 + 8 + roundedSize(nestedMap.getSizeInBytes)) } assert(unsafeMap2.getSizeInBytes == 8 + keyArray.getSizeInBytes + valueArray.getSizeInBytes) @@ -426,7 +426,7 @@ class UnsafeRowConverterSuite extends SparkFunSuite with Matchers { assert(innerStruct.getLong(0) == 2L) } - assert(field2.getSizeInBytes == 8 + 8 + 4 + innerStruct.getSizeInBytes) + assert(field2.getSizeInBytes == 8 + 8 + 8 + innerStruct.getSizeInBytes) assert(unsafeRow.getSizeInBytes == 8 + 8 * 2 + field1.getSizeInBytes + roundedSize(field2.getSizeInBytes)) @@ -469,7 +469,7 @@ class UnsafeRowConverterSuite extends SparkFunSuite with Matchers { assert(innerStruct.getSizeInBytes == 8 + 8) assert(innerStruct.getLong(0) == 4L) - assert(valueArray.getSizeInBytes == 8 + 8 + 4 + innerStruct.getSizeInBytes) + assert(valueArray.getSizeInBytes == 8 + 8 + 8 + innerStruct.getSizeInBytes) } assert(field2.getSizeInBytes == 8 + keyArray.getSizeInBytes + valueArray.getSizeInBytes) @@ -498,7 +498,7 @@ class UnsafeRowConverterSuite extends SparkFunSuite with Matchers { val innerMap = field1.getMap(0) testMapInt(innerMap, Seq(1), Seq(2)) - assert(field1.getSizeInBytes == 8 + 8 + 4 + innerMap.getSizeInBytes) + assert(field1.getSizeInBytes == 8 + 8 + 8 + roundedSize(innerMap.getSizeInBytes)) val field2 = unsafeRow.getMap(1) assert(field2.numElements == 1) @@ -514,7 +514,7 @@ class UnsafeRowConverterSuite extends SparkFunSuite with Matchers { val innerArray = valueArray.getArray(0) testArrayInt(innerArray, Seq(4)) - assert(valueArray.getSizeInBytes == 8 + 8 + 4 + innerArray.getSizeInBytes) + assert(valueArray.getSizeInBytes == 8 + 8 + 8 + innerArray.getSizeInBytes) } assert(field2.getSizeInBytes == 8 + keyArray.getSizeInBytes + valueArray.getSizeInBytes) From aa7cfdbac35140dc9b58871ff07002f7e46a4533 Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Sat, 10 Sep 2016 01:24:57 +0900 Subject: [PATCH 31/37] fixed test failures --- .../catalyst/expressions/codegen/UnsafeArrayWriter.java | 7 +++++++ .../expressions/codegen/GenerateUnsafeProjection.scala | 2 -- 2 files changed, 7 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeArrayWriter.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeArrayWriter.java index e7c5989416440..b1fa55d18458e 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeArrayWriter.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeArrayWriter.java @@ -63,6 +63,13 @@ public void initialize(BufferHolder holder, int numElements, int elementSize) { for (int i = 8; i < headerInBytes; i += 8) { Platform.putLong(holder.buffer, startingOffset + i, 0L); } + + // fill 0 into reminder part of 8-bytes alignment in unsafe array + if ((fixedPartInBytes - elementSize * numElements) != 0) { + for (int i = elementSize * numElements; i < fixedPartInBytes; i++) { + Platform.putByte(holder.buffer, startingOffset + headerInBytes + i, (byte) 0); + } + } holder.cursor += (headerInBytes + fixedPartInBytes); } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjection.scala index ef4d1990f842d..75bb6936b49e0 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjection.scala @@ -124,7 +124,6 @@ object GenerateUnsafeProjection extends CodeGenerator[Seq[Expression], UnsafePro final int $tmpCursor = $bufferHolder.cursor; ${writeArrayToBuffer(ctx, input.value, et, bufferHolder)} $rowWriter.setOffsetAndSize($index, $tmpCursor, $bufferHolder.cursor - $tmpCursor); - $rowWriter.alignToWords($bufferHolder.cursor - $tmpCursor); """ case m @ MapType(kt, vt, _) => @@ -134,7 +133,6 @@ object GenerateUnsafeProjection extends CodeGenerator[Seq[Expression], UnsafePro final int $tmpCursor = $bufferHolder.cursor; ${writeMapToBuffer(ctx, input.value, kt, vt, bufferHolder)} $rowWriter.setOffsetAndSize($index, $tmpCursor, $bufferHolder.cursor - $tmpCursor); - $rowWriter.alignToWords($bufferHolder.cursor - $tmpCursor); """ case t: DecimalType => From 0b7867b82728e33d7f3a5a0a6092fd18a78cc266 Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Tue, 20 Sep 2016 20:52:57 +0900 Subject: [PATCH 32/37] address review comments --- .../catalyst/expressions/UnsafeArrayData.java | 57 +++++++------------ .../catalyst/expressions/UnsafeMapData.java | 4 +- .../codegen/UnsafeArrayWriter.java | 18 +++--- .../spark/sql/catalyst/ScalaReflection.scala | 2 - .../sql/catalyst/util/UnsafeArraySuite.scala | 18 ++++-- .../benchmark/UnsafeArrayDataBenchmark.scala | 28 ++++++--- 6 files changed, 65 insertions(+), 62 deletions(-) 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 c9314666001be..bc38916a1f7b0 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 @@ -38,15 +38,15 @@ * * The `numElements` is 8 bytes storing the number of elements of this array. * - * In the `null bits` region, we store 1 bit per element, represents whether a element has null - * Its total size is ceil(numElements / 8) bytes, and it is aligned to 8-byte boundaries. + * In the `null bits` region, we store 1 bit per element, represents whether an element is null + * Its total size is ceil(numElements / 8) bytes, and it is aligned to 8-byte boundaries. * * In the `values or offset&length` region, we store the content of elements. For fields that hold * fixed-length primitive types, such as long, double, or int, we store the value directly - * in the field. For fields with non-primitive or variable-length values, we store a relative - * offset (w.r.t. the base address of the array) that points to the beginning of - * the variable-length field and length (they are combined into a long). For variable length - * portion, each is aligned to 8-byte boundaries. + * in the field. For fixed-length portion, each is word-aligned. For fields with non-primitive or + * variable-length values, we store a relative offset (w.r.t. the base address of the array) + * that points to the beginning of the variable-length field and length (they are combined into + * a long). For variable length portion, each is aligned to 8-byte boundaries. * * Instances of `UnsafeArrayData` act as pointers to row data stored in this format. */ @@ -330,72 +330,57 @@ public UnsafeArrayData copy() { @Override public boolean[] toBooleanArray() { - int size = numElements(); - boolean[] values = new boolean[size]; + boolean[] values = new boolean[numElements]; Platform.copyMemory( - baseObject, elementOffset, values, Platform.BOOLEAN_ARRAY_OFFSET, size); + baseObject, elementOffset, values, Platform.BOOLEAN_ARRAY_OFFSET, numElements); return values; } @Override public byte[] toByteArray() { - int size = numElements(); - byte[] values = new byte[size]; + byte[] values = new byte[numElements]; Platform.copyMemory( - baseObject, elementOffset, values, Platform.BYTE_ARRAY_OFFSET, size); + baseObject, elementOffset, values, Platform.BYTE_ARRAY_OFFSET, numElements); return values; } @Override public short[] toShortArray() { - if (numElements > Integer.MAX_VALUE) { - throw new UnsupportedOperationException("Cannot convert this unsafe array to array as " + - "it's too big."); - } - int size = (int)numElements; - short[] values = new short[size]; + short[] values = new short[numElements]; Platform.copyMemory( - baseObject, elementOffset, values, Platform.SHORT_ARRAY_OFFSET, size * 2); + baseObject, elementOffset, values, Platform.SHORT_ARRAY_OFFSET, numElements * 2); return values; } @Override public int[] toIntArray() { - int size = numElements(); - int[] values = new int[size]; + int[] values = new int[numElements]; Platform.copyMemory( - baseObject, elementOffset, values, Platform.INT_ARRAY_OFFSET, size * 4); + baseObject, elementOffset, values, Platform.INT_ARRAY_OFFSET, numElements * 4); return values; } @Override public long[] toLongArray() { - if (numElements > Integer.MAX_VALUE) { - throw new UnsupportedOperationException("Cannot convert this unsafe array to array as " + - "it's too big."); - } - int size = (int)numElements; - long[] values = new long[size]; + long[] values = new long[numElements]; Platform.copyMemory( - baseObject, elementOffset, values, Platform.LONG_ARRAY_OFFSET, size * 8); + baseObject, elementOffset, values, Platform.LONG_ARRAY_OFFSET, numElements * 8); return values; } @Override public float[] toFloatArray() { - int size = numElements(); - float[] values = new float[size]; + float[] values = new float[numElements]; Platform.copyMemory( - baseObject, elementOffset, values, Platform.FLOAT_ARRAY_OFFSET, size * 4); + baseObject, elementOffset, values, Platform.FLOAT_ARRAY_OFFSET, numElements * 4); return values; } @Override public double[] toDoubleArray() { - int size = numElements(); - double[] values = new double[size]; + double[] values = new double[numElements]; Platform.copyMemory( - baseObject, elementOffset, values, Platform.DOUBLE_ARRAY_OFFSET, size * 8); + baseObject, elementOffset, values, Platform.DOUBLE_ARRAY_OFFSET, numElements * 8); return values; } @@ -404,7 +389,7 @@ private static UnsafeArrayData fromPrimitiveArray( final long headerInBytes = calculateHeaderPortionInBytes(length); final long valueRegionInBytes = elementSize * length; final long totalSizeInLongs = (headerInBytes + valueRegionInBytes + 7) / 8; - if (totalSizeInLongs * 8 > Integer.MAX_VALUE) { + if (totalSizeInLongs > Integer.MAX_VALUE / 8) { throw new UnsupportedOperationException("Cannot convert this array to unsafe format as " + "it's too big."); } 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 823454236ca6d..2c2166005c201 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 @@ -25,7 +25,7 @@ /** * An Unsafe implementation of Map which is backed by raw memory instead of Java objects. * - * Currently we just use 2 UnsafeArrayData to represent UnsafeMapData, with extra 4 bytes at head + * Currently we just use 2 UnsafeArrayData to represent UnsafeMapData, with extra 8 bytes at head * to indicate the number of bytes of the unsafe key array. * [unsafe key array numBytes] [unsafe key array] [unsafe value array] */ @@ -67,9 +67,9 @@ public UnsafeMapData() { public void pointTo(Object baseObject, long baseOffset, int sizeInBytes) { // Read the numBytes of key array from the first 8 bytes. final long keyArraySize = Platform.getLong(baseObject, baseOffset); - final long valueArraySize = sizeInBytes - keyArraySize - 8; assert keyArraySize >= 0 : "keyArraySize (" + keyArraySize + ") should >= 0"; assert keyArraySize <= Integer.MAX_VALUE : "keyArraySize (" + keyArraySize + ") should <= Integer.MAX_VALUE"; + final long valueArraySize = sizeInBytes - (int)keyArraySize - 8; assert valueArraySize >= 0 : "valueArraySize (" + valueArraySize + ") should >= 0"; keys.pointTo(baseObject, baseOffset + 8, (int)keyArraySize); diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeArrayWriter.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeArrayWriter.java index b1fa55d18458e..a84347082105b 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeArrayWriter.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeArrayWriter.java @@ -19,6 +19,7 @@ import org.apache.spark.sql.types.Decimal; import org.apache.spark.unsafe.Platform; +import org.apache.spark.unsafe.array.ByteArrayMethods; import org.apache.spark.unsafe.bitset.BitSetMethods; import org.apache.spark.unsafe.types.CalendarInterval; import org.apache.spark.unsafe.types.UTF8String; @@ -55,7 +56,8 @@ public void initialize(BufferHolder holder, int numElements, int elementSize) { this.startingOffset = holder.cursor; // Grows the global buffer ahead for header and fixed size data. - int fixedPartInBytes = ((elementSize * numElements + 7) / 8) * 8; + int fixedPartInBytes = + ByteArrayMethods.roundNumberOfBytesToNearestWord(elementSize * numElements); holder.grow(headerInBytes + fixedPartInBytes); // Write numElements and clear out null bits to header @@ -65,10 +67,8 @@ public void initialize(BufferHolder holder, int numElements, int elementSize) { } // fill 0 into reminder part of 8-bytes alignment in unsafe array - if ((fixedPartInBytes - elementSize * numElements) != 0) { - for (int i = elementSize * numElements; i < fixedPartInBytes; i++) { - Platform.putByte(holder.buffer, startingOffset + headerInBytes + i, (byte) 0); - } + for (int i = elementSize * numElements; i < fixedPartInBytes; i++) { + Platform.putByte(holder.buffer, startingOffset + headerInBytes + i, (byte) 0); } holder.cursor += (headerInBytes + fixedPartInBytes); } @@ -84,6 +84,7 @@ private long getElementOffset(int ordinal, int elementSize) { } public void setOffsetAndSize(int ordinal, long currentCursor, long size) { + assertIndexIsValid(ordinal); final long relativeOffset = currentCursor - startingOffset; final long offsetAndSize = (relativeOffset << 32) | size; @@ -190,7 +191,7 @@ public void write(int ordinal, Decimal input, int precision, int scale) { final byte[] bytes = input.toJavaBigDecimal().unscaledValue().toByteArray(); final int numBytes = bytes.length; assert numBytes <= 16; - int roundedSize = ((numBytes + 7) / 8) * 8; // 8-bytes boundary + int roundedSize = ByteArrayMethods.roundNumberOfBytesToNearestWord(numBytes); holder.grow(roundedSize); zeroOutPaddingBytes(numBytes); @@ -199,6 +200,7 @@ public void write(int ordinal, Decimal input, int precision, int scale) { Platform.copyMemory( bytes, Platform.BYTE_ARRAY_OFFSET, holder.buffer, holder.cursor, numBytes); write(ordinal, ((long)(holder.cursor - startingOffset) << 32) | ((long) numBytes)); + setOffsetAndSize(ordinal, holder.cursor, (long)numBytes); // move the cursor forward with 8-bytes boundary holder.cursor += roundedSize; @@ -210,7 +212,7 @@ public void write(int ordinal, Decimal input, int precision, int scale) { public void write(int ordinal, UTF8String input) { final int numBytes = input.numBytes(); - final int roundedSize = ((numBytes + 7) / 8) * 8; // 8-bytes boundary + final int roundedSize = ByteArrayMethods.roundNumberOfBytesToNearestWord(numBytes); // grow the global buffer before writing data. holder.grow(roundedSize); @@ -228,7 +230,7 @@ public void write(int ordinal, UTF8String input) { public void write(int ordinal, byte[] input) { final int numBytes = input.length; - final int roundedSize = ((input.length + 7) / 8) * 8; // 8-bytes boundary + final int roundedSize = ByteArrayMethods.roundNumberOfBytesToNearestWord(input.length); // grow the global buffer before writing data. holder.grow(roundedSize); diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala index 528286d72de06..b3ae146cbbb03 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala @@ -101,8 +101,6 @@ object ScalaReflection extends ScalaReflection { case t if t <:< definitions.ShortTpe => classOf[Array[Short]] case t if t <:< definitions.ByteTpe => classOf[Array[Byte]] case t if t <:< definitions.BooleanTpe => classOf[Array[Boolean]] - case t if t <:< localTypeOf[CalendarInterval] => classOf[Array[CalendarInterval]] - case t if t <:< localTypeOf[Decimal] => classOf[Array[Decimal]] case other => // There is probably a better way to do this, but I couldn't find it... val elementType = dataTypeFor(other).asInstanceOf[ObjectType].cls diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/UnsafeArraySuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/UnsafeArraySuite.scala index b4353e8d495d0..26ec65cd64e33 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/UnsafeArraySuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/UnsafeArraySuite.scala @@ -141,12 +141,18 @@ class UnsafeArraySuite extends SparkFunSuite { } } - val unsafeInterval = ExpressionEncoder[Array[CalendarInterval]].resolveAndBind(). - toRow(calenderintervalArray).getArray(0) - assert(unsafeInterval.isInstanceOf[UnsafeArrayData]) - assert(unsafeInterval.numElements == calenderintervalArray.length) - calenderintervalArray.zipWithIndex.map { case (e, i) => - assert(unsafeInterval.getInterval(i) == e) + Seq(calenderintervalArray).map { calendarArray => + val schema = new StructType().add("array", ArrayType(CalendarIntervalType)) + val encoder = RowEncoder(schema).resolveAndBind() + val externalRow = Row(calendarArray) + val ir = encoder.toRow(externalRow) + + val unsafeCalendar = ir.getArray(0) + assert(unsafeCalendar.isInstanceOf[UnsafeArrayData]) + assert(unsafeCalendar.numElements == calendarArray.length) + calendarArray.zipWithIndex.map { case (e, i) => + assert(unsafeCalendar.getInterval(i) == e) + } } val unsafeMultiDimInt = ExpressionEncoder[Array[Array[Int]]].resolveAndBind(). diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/UnsafeArrayDataBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/UnsafeArrayDataBenchmark.scala index 7952454832cb9..00875296a0680 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/UnsafeArrayDataBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/UnsafeArrayDataBenchmark.scala @@ -100,22 +100,26 @@ class UnsafeArrayDataBenchmark extends BenchmarkBase { val intPrimitiveArray = Array.fill[Int](count) { rand.nextInt } val intEncoder = ExpressionEncoder[Array[Int]].resolveAndBind() val writeIntArray = { i: Int => + var len = 0 var n = 0 while (n < iters) { - intTotalLength += intEncoder.toRow(intPrimitiveArray).getArray(0).numElements() + len += intEncoder.toRow(intPrimitiveArray).getArray(0).numElements() n += 1 } + intTotalLength = len } var doubleTotalLength: Int = 0 val doublePrimitiveArray = Array.fill[Double](count) { rand.nextDouble } val doubleEncoder = ExpressionEncoder[Array[Double]].resolveAndBind() val writeDoubleArray = { i: Int => + var len = 0 var n = 0 while (n < iters) { - doubleTotalLength += doubleEncoder.toRow(doublePrimitiveArray).getArray(0).numElements() + len += doubleEncoder.toRow(doublePrimitiveArray).getArray(0).numElements() n += 1 } + doubleTotalLength = len } val benchmark = new Benchmark("Write UnsafeArrayData", count * iters) @@ -142,11 +146,13 @@ class UnsafeArrayDataBenchmark extends BenchmarkBase { val intEncoder = ExpressionEncoder[Array[Int]].resolveAndBind() val intUnsafeArray = intEncoder.toRow(intPrimitiveArray).getArray(0) val readIntArray = { i: Int => + var len = 0 var n = 0 while (n < iters) { - intTotalLength += intUnsafeArray.toIntArray.length + len += intUnsafeArray.toIntArray.length n += 1 } + intTotalLength = len } var doubleTotalLength: Int = 0 @@ -154,11 +160,13 @@ class UnsafeArrayDataBenchmark extends BenchmarkBase { val doubleEncoder = ExpressionEncoder[Array[Double]].resolveAndBind() val doubleUnsafeArray = doubleEncoder.toRow(doublePrimitiveArray).getArray(0) val readDoubleArray = { i: Int => + var len = 0 var n = 0 while (n < iters) { - doubleTotalLength += doubleUnsafeArray.toDoubleArray.length + len += doubleUnsafeArray.toDoubleArray.length n += 1 } + doubleTotalLength = len } val benchmark = new Benchmark("Get primitive array from UnsafeArrayData", count * iters) @@ -180,24 +188,28 @@ class UnsafeArrayDataBenchmark extends BenchmarkBase { val count = 1024 * 1024 * 12 val rand = new Random(42) + var intTotalLen: Int = 0 val intPrimitiveArray = Array.fill[Int](count) { rand.nextInt } - var intUnsafeArray: UnsafeArrayData = null val createIntArray = { i: Int => + var len = 0 var n = 0 while (n < iters) { - intUnsafeArray = UnsafeArrayData.fromPrimitiveArray(intPrimitiveArray) + len += UnsafeArrayData.fromPrimitiveArray(intPrimitiveArray).numElements n += 1 } + intTotalLen = len } + var doubleTotalLen: Int = 0 val doublePrimitiveArray = Array.fill[Double](count) { rand.nextDouble } - var doubleUnsafeArray: UnsafeArrayData = null val createDoubleArray = { i: Int => + var len = 0 var n = 0 while (n < iters) { - doubleUnsafeArray = UnsafeArrayData.fromPrimitiveArray(doublePrimitiveArray) + len += UnsafeArrayData.fromPrimitiveArray(doublePrimitiveArray).numElements n += 1 } + doubleTotalLen = len } val benchmark = new Benchmark("Create UnsafeArrayData from primitive array", count * iters) From ab9a16a4649cf6fceda95b7447b709002ac3a2aa Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Tue, 20 Sep 2016 23:42:45 +0900 Subject: [PATCH 33/37] address review comments --- .../catalyst/expressions/UnsafeArrayData.java | 9 ++++---- .../catalyst/expressions/UnsafeMapData.java | 4 ++-- .../codegen/UnsafeArrayWriter.java | 1 - .../sql/catalyst/util/UnsafeArraySuite.scala | 21 ++++++++----------- 4 files changed, 16 insertions(+), 19 deletions(-) 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 bc38916a1f7b0..cb04d49cea285 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 @@ -43,10 +43,11 @@ * * In the `values or offset&length` region, we store the content of elements. For fields that hold * fixed-length primitive types, such as long, double, or int, we store the value directly - * in the field. For fixed-length portion, each is word-aligned. For fields with non-primitive or - * variable-length values, we store a relative offset (w.r.t. the base address of the array) - * that points to the beginning of the variable-length field and length (they are combined into - * a long). For variable length portion, each is aligned to 8-byte boundaries. + * in the field. For fixed-length portion, each element value (even for byte) is word-aligned. + * For fields with non-primitive or variable-length values, we store a relative offset + * (w.r.t. the base address of the array) that points to the beginning of the variable-length field + * and length (they are combined into a long). For variable length portion, each is aligned + * to 8-byte boundaries. * * Instances of `UnsafeArrayData` act as pointers to row data stored in this format. */ 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 2c2166005c201..35029f5a50e3e 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 @@ -69,11 +69,11 @@ public void pointTo(Object baseObject, long baseOffset, int sizeInBytes) { final long keyArraySize = Platform.getLong(baseObject, baseOffset); assert keyArraySize >= 0 : "keyArraySize (" + keyArraySize + ") should >= 0"; assert keyArraySize <= Integer.MAX_VALUE : "keyArraySize (" + keyArraySize + ") should <= Integer.MAX_VALUE"; - final long valueArraySize = sizeInBytes - (int)keyArraySize - 8; + final int valueArraySize = sizeInBytes - (int)keyArraySize - 8; assert valueArraySize >= 0 : "valueArraySize (" + valueArraySize + ") should >= 0"; keys.pointTo(baseObject, baseOffset + 8, (int)keyArraySize); - values.pointTo(baseObject, baseOffset + 8 + keyArraySize, (int)valueArraySize); + values.pointTo(baseObject, baseOffset + 8 + keyArraySize, valueArraySize); assert keys.numElements() == values.numElements(); diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeArrayWriter.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeArrayWriter.java index a84347082105b..0c4ab95a8846a 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeArrayWriter.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeArrayWriter.java @@ -199,7 +199,6 @@ public void write(int ordinal, Decimal input, int precision, int scale) { // Write the bytes to the variable length portion. Platform.copyMemory( bytes, Platform.BYTE_ARRAY_OFFSET, holder.buffer, holder.cursor, numBytes); - write(ordinal, ((long)(holder.cursor - startingOffset) << 32) | ((long) numBytes)); setOffsetAndSize(ordinal, holder.cursor, (long)numBytes); // move the cursor forward with 8-bytes boundary diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/UnsafeArraySuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/UnsafeArraySuite.scala index 26ec65cd64e33..f0e247bf46c44 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/UnsafeArraySuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/UnsafeArraySuite.scala @@ -141,18 +141,15 @@ class UnsafeArraySuite extends SparkFunSuite { } } - Seq(calenderintervalArray).map { calendarArray => - val schema = new StructType().add("array", ArrayType(CalendarIntervalType)) - val encoder = RowEncoder(schema).resolveAndBind() - val externalRow = Row(calendarArray) - val ir = encoder.toRow(externalRow) - - val unsafeCalendar = ir.getArray(0) - assert(unsafeCalendar.isInstanceOf[UnsafeArrayData]) - assert(unsafeCalendar.numElements == calendarArray.length) - calendarArray.zipWithIndex.map { case (e, i) => - assert(unsafeCalendar.getInterval(i) == e) - } + val schema = new StructType().add("array", ArrayType(CalendarIntervalType)) + val encoder = RowEncoder(schema).resolveAndBind() + val externalRow = Row(calenderintervalArray) + val ir = encoder.toRow(externalRow) + val unsafeCalendar = ir.getArray(0) + assert(unsafeCalendar.isInstanceOf[UnsafeArrayData]) + assert(unsafeCalendar.numElements == calenderintervalArray.length) + calenderintervalArray.zipWithIndex.map { case (e, i) => + assert(unsafeCalendar.getInterval(i) == e) } val unsafeMultiDimInt = ExpressionEncoder[Array[Array[Int]]].resolveAndBind(). From 515701be63796b6c25b7e4142486a15fc1f2b49b Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Wed, 21 Sep 2016 03:56:03 +0900 Subject: [PATCH 34/37] address review comments --- .../spark/sql/catalyst/expressions/UnsafeArrayData.java | 2 +- .../sql/catalyst/expressions/codegen/UnsafeArrayWriter.java | 6 +++--- .../org/apache/spark/sql/catalyst/ScalaReflection.scala | 1 - 3 files changed, 4 insertions(+), 5 deletions(-) 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 cb04d49cea285..86523c1474015 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 @@ -43,7 +43,7 @@ * * In the `values or offset&length` region, we store the content of elements. For fields that hold * fixed-length primitive types, such as long, double, or int, we store the value directly - * in the field. For fixed-length portion, each element value (even for byte) is word-aligned. + * in the field. The whole fixed-length portion (even for byte) is aligned to 8-byte boundaries. * For fields with non-primitive or variable-length values, we store a relative offset * (w.r.t. the base address of the array) that points to the beginning of the variable-length field * and length (they are combined into a long). For variable length portion, each is aligned diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeArrayWriter.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeArrayWriter.java index 0c4ab95a8846a..ef003ca6f5863 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeArrayWriter.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeArrayWriter.java @@ -221,7 +221,7 @@ public void write(int ordinal, UTF8String input) { // Write the bytes to the variable length portion. input.writeToMemory(holder.buffer, holder.cursor); - write(ordinal, ((long)(holder.cursor - startingOffset) << 32) | ((long) numBytes)); + setOffsetAndSize(ordinal, holder.cursor, (long)numBytes); // move the cursor forward. holder.cursor += roundedSize; @@ -240,7 +240,7 @@ public void write(int ordinal, byte[] input) { Platform.copyMemory( input, Platform.BYTE_ARRAY_OFFSET, holder.buffer, holder.cursor, numBytes); - write(ordinal, ((long)(holder.cursor - startingOffset) << 32) | ((long) numBytes)); + setOffsetAndSize(ordinal, holder.cursor, (long)numBytes); // move the cursor forward. holder.cursor += roundedSize; @@ -254,7 +254,7 @@ public void write(int ordinal, CalendarInterval input) { Platform.putLong(holder.buffer, holder.cursor, input.months); Platform.putLong(holder.buffer, holder.cursor + 8, input.microseconds); - write(ordinal, ((long)(holder.cursor - startingOffset) << 32) | ((long) 16)); + setOffsetAndSize(ordinal, holder.cursor, (long)16); // move the cursor forward. holder.cursor += 16; diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala index b3ae146cbbb03..7923cfce82100 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala @@ -678,7 +678,6 @@ object ScalaReflection extends ScalaReflection { Schema(DecimalType.BigIntDecimal, nullable = true) case t if t <:< localTypeOf[scala.math.BigInt] => Schema(DecimalType.BigIntDecimal, nullable = true) - case t if t <:< localTypeOf[CalendarInterval] => Schema(CalendarIntervalType, nullable = true) case t if t <:< localTypeOf[Decimal] => Schema(DecimalType.SYSTEM_DEFAULT, nullable = true) case t if t <:< localTypeOf[java.lang.Integer] => Schema(IntegerType, nullable = true) case t if t <:< localTypeOf[java.lang.Long] => Schema(LongType, nullable = true) From 8169abdc9062ac96a7497b7b012cfc824c7f16eb Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Tue, 27 Sep 2016 00:58:02 +0900 Subject: [PATCH 35/37] change benchmark size --- .../sql/execution/benchmark/UnsafeArrayDataBenchmark.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/UnsafeArrayDataBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/UnsafeArrayDataBenchmark.scala index 00875296a0680..3631c8a3ee0b4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/UnsafeArrayDataBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/UnsafeArrayDataBenchmark.scala @@ -93,7 +93,7 @@ class UnsafeArrayDataBenchmark extends BenchmarkBase { } def writeUnsafeArray(iters: Int): Unit = { - val count = 1024 * 1024 * 16 + val count = 1024 * 1024 * 2 val rand = new Random(42) var intTotalLength: Int = 0 From e356a796a4e78fb2e7f853b57ef3ce681c41676d Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Tue, 27 Sep 2016 01:09:36 +0900 Subject: [PATCH 36/37] addressed comments --- .../expressions/codegen/UnsafeArrayWriter.java | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeArrayWriter.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeArrayWriter.java index ef003ca6f5863..afea4676893ed 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeArrayWriter.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeArrayWriter.java @@ -83,10 +83,10 @@ private long getElementOffset(int ordinal, int elementSize) { return startingOffset + headerInBytes + ordinal * elementSize; } - public void setOffsetAndSize(int ordinal, long currentCursor, long size) { + public void setOffsetAndSize(int ordinal, long currentCursor, int size) { assertIndexIsValid(ordinal); final long relativeOffset = currentCursor - startingOffset; - final long offsetAndSize = (relativeOffset << 32) | size; + final long offsetAndSize = (relativeOffset << 32) | (long)size; write(ordinal, offsetAndSize); } @@ -199,7 +199,7 @@ public void write(int ordinal, Decimal input, int precision, int scale) { // Write the bytes to the variable length portion. Platform.copyMemory( bytes, Platform.BYTE_ARRAY_OFFSET, holder.buffer, holder.cursor, numBytes); - setOffsetAndSize(ordinal, holder.cursor, (long)numBytes); + setOffsetAndSize(ordinal, holder.cursor, numBytes); // move the cursor forward with 8-bytes boundary holder.cursor += roundedSize; @@ -221,7 +221,7 @@ public void write(int ordinal, UTF8String input) { // Write the bytes to the variable length portion. input.writeToMemory(holder.buffer, holder.cursor); - setOffsetAndSize(ordinal, holder.cursor, (long)numBytes); + setOffsetAndSize(ordinal, holder.cursor, numBytes); // move the cursor forward. holder.cursor += roundedSize; @@ -240,7 +240,7 @@ public void write(int ordinal, byte[] input) { Platform.copyMemory( input, Platform.BYTE_ARRAY_OFFSET, holder.buffer, holder.cursor, numBytes); - setOffsetAndSize(ordinal, holder.cursor, (long)numBytes); + setOffsetAndSize(ordinal, holder.cursor, numBytes); // move the cursor forward. holder.cursor += roundedSize; @@ -254,7 +254,7 @@ public void write(int ordinal, CalendarInterval input) { Platform.putLong(holder.buffer, holder.cursor, input.months); Platform.putLong(holder.buffer, holder.cursor + 8, input.microseconds); - setOffsetAndSize(ordinal, holder.cursor, (long)16); + setOffsetAndSize(ordinal, holder.cursor, 16); // move the cursor forward. holder.cursor += 16; From 2ef6e3bdbd16c7f7b9ff006d48382e108ed37eef Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Tue, 27 Sep 2016 01:24:51 +0900 Subject: [PATCH 37/37] update performance results --- .../linalg/UDTSerializationBenchmark.scala | 13 +++---- .../benchmark/UnsafeArrayDataBenchmark.scala | 38 +++++++++---------- 2 files changed, 23 insertions(+), 28 deletions(-) diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/UDTSerializationBenchmark.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/UDTSerializationBenchmark.scala index 8b439e6b7a017..5973479dfb5ed 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/UDTSerializationBenchmark.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/UDTSerializationBenchmark.scala @@ -57,13 +57,12 @@ object UDTSerializationBenchmark { } /* - Java HotSpot(TM) 64-Bit Server VM 1.8.0_60-b27 on Mac OS X 10.11.4 - Intel(R) Core(TM) i7-4960HQ CPU @ 2.60GHz - - VectorUDT de/serialization: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------- - serialize 380 / 392 0.0 379730.0 1.0X - deserialize 138 / 142 0.0 137816.6 2.8X + OpenJDK 64-Bit Server VM 1.8.0_91-b14 on Linux 4.4.11-200.fc22.x86_64 + Intel Xeon E3-12xx v2 (Ivy Bridge) + VectorUDT de/serialization: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------------ + serialize 265 / 318 0.0 265138.5 1.0X + deserialize 155 / 197 0.0 154611.4 1.7X */ benchmark.run() } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/UnsafeArrayDataBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/UnsafeArrayDataBenchmark.scala index 3631c8a3ee0b4..6c7779b5790d0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/UnsafeArrayDataBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/UnsafeArrayDataBenchmark.scala @@ -82,13 +82,12 @@ class UnsafeArrayDataBenchmark extends BenchmarkBase { benchmark.addCase("Double")(readDoubleArray) benchmark.run /* - Java HotSpot(TM) 64-Bit Server VM 1.8.0_92-b14 on Mac OS X 10.10.4 - Intel(R) Core(TM) i5-5257U CPU @ 2.70GHz - + OpenJDK 64-Bit Server VM 1.8.0_91-b14 on Linux 4.4.11-200.fc22.x86_64 + Intel Xeon E3-12xx v2 (Ivy Bridge) Read UnsafeArrayData: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Int 279 / 294 600.4 1.7 1.0X - Double 296 / 303 567.0 1.8 0.9X + Int 252 / 260 666.1 1.5 1.0X + Double 281 / 292 597.7 1.7 0.9X */ } @@ -127,13 +126,12 @@ class UnsafeArrayDataBenchmark extends BenchmarkBase { benchmark.addCase("Double")(writeDoubleArray) benchmark.run /* - Java HotSpot(TM) 64-Bit Server VM 1.8.0_92-b14 on Mac OS X 10.10.4 - Intel(R) Core(TM) i5-5257U CPU @ 2.70GHz - + OpenJDK 64-Bit Server VM 1.8.0_91-b14 on Linux 4.4.11-200.fc22.x86_64 + Intel Xeon E3-12xx v2 (Ivy Bridge) Write UnsafeArrayData: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Int 135 / 180 154.9 6.5 1.0X - Double 195 / 300 107.8 9.3 0.7X + Int 196 / 249 107.0 9.3 1.0X + Double 227 / 367 92.3 10.8 0.9X */ } @@ -174,13 +172,12 @@ class UnsafeArrayDataBenchmark extends BenchmarkBase { benchmark.addCase("Double")(readDoubleArray) benchmark.run /* - Java HotSpot(TM) 64-Bit Server VM 1.8.0_92-b14 on Mac OS X 10.10.4 - Intel(R) Core(TM) i5-5257U CPU @ 2.70GHz - + OpenJDK 64-Bit Server VM 1.8.0_91-b14 on Linux 4.4.11-200.fc22.x86_64 + Intel Xeon E3-12xx v2 (Ivy Bridge) Get primitive array from UnsafeArrayData: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Int 80 / 151 783.4 1.3 1.0X - Double 208 / 366 302.8 3.3 0.4X + Int 151 / 198 415.8 2.4 1.0X + Double 214 / 394 293.6 3.4 0.7X */ } @@ -217,13 +214,12 @@ class UnsafeArrayDataBenchmark extends BenchmarkBase { benchmark.addCase("Double")(createDoubleArray) benchmark.run /* - Java HotSpot(TM) 64-Bit Server VM 1.8.0_92-b14 on Mac OS X 10.10.4 - Intel(R) Core(TM) i5-5257U CPU @ 2.70GHz - - Create UnsafeArrayData from primitive array: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + OpenJDK 64-Bit Server VM 1.8.0_91-b14 on Linux 4.4.11-200.fc22.x86_64 + Intel Xeon E3-12xx v2 (Ivy Bridge) + Create UnsafeArrayData from primitive array: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Int 68 / 144 920.4 1.1 1.0X - Double 240 / 302 261.7 3.8 0.3X + Int 206 / 211 306.0 3.3 1.0X + Double 232 / 406 271.6 3.7 0.9X */ }