Skip to content
Closed
Show file tree
Hide file tree
Changes from 1 commit
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Next Next commit
add unsafe array
  • Loading branch information
cloud-fan committed Aug 2, 2015
commit 821b8db8690aa20e5e71f5e8a55aa3d429429d30
Original file line number Diff line number Diff line change
@@ -0,0 +1,337 @@
/*
* 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.catalyst.InternalRow;
import org.apache.spark.sql.types.*;
import org.apache.spark.unsafe.PlatformDependent;
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;

import java.math.BigDecimal;
import java.math.BigInteger;

// todo: doc
// todo: there is a lof of duplicated code between UnsafeRow and UnsafeArrayData.
public class UnsafeArrayData extends ArrayData {
Copy link
Contributor

Choose a reason for hiding this comment

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

we should definitely document the binary format of UnsafeArrayData in the javadoc.


private Object baseObject;
private long baseOffset;

// The number of elements in this array
private int numElements;

// The size of this array's backing data, in bytes
private int sizeInBytes;

private int getElementOffset(int ordinal) {
return PlatformDependent.UNSAFE.getInt(baseObject, baseOffset + 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;
}

/**
* Construct a new UnsafeArrayData. The resulting UnsafeArrayData won't be usable until
* `pointTo()` has been called, since the value returned by this constructor is equivalent
* to a null pointer.
*/
public UnsafeArrayData() { }

public Object getBaseObject() { return baseObject; }
public long getBaseOffset() { return baseOffset; }
public int getSizeInBytes() { return sizeInBytes; }

@Override
public int numElements() { return numElements; }

/**
* Update this UnsafeArrayData to point to different backing data.
*
* @param baseObject the base object
* @param baseOffset the offset within the base object
* @param sizeInBytes the size of this row's backing data, in bytes
*/
public void pointTo(Object baseObject, long baseOffset, int numElements, int sizeInBytes) {
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;
}

@Override
public Object get(int ordinal, DataType dataType) {
if (isNullAt(ordinal) || dataType instanceof NullType) {
return null;
} else if (dataType instanceof BooleanType) {
return getBoolean(ordinal);
} else if (dataType instanceof ByteType) {
return getByte(ordinal);
} else if (dataType instanceof ShortType) {
return getShort(ordinal);
} else if (dataType instanceof IntegerType) {
return getInt(ordinal);
} else if (dataType instanceof LongType) {
return getLong(ordinal);
} else if (dataType instanceof FloatType) {
return getFloat(ordinal);
} else if (dataType instanceof DoubleType) {
return getDouble(ordinal);
} else if (dataType instanceof DecimalType) {
DecimalType dt = (DecimalType) dataType;
return getDecimal(ordinal, dt.precision(), dt.scale());
} else if (dataType instanceof DateType) {
return getInt(ordinal);
} else if (dataType instanceof TimestampType) {
return getLong(ordinal);
} else if (dataType instanceof BinaryType) {
return getBinary(ordinal);
} else if (dataType instanceof StringType) {
return getUTF8String(ordinal);
} else if (dataType instanceof CalendarIntervalType) {
return getInterval(ordinal);
} else if (dataType instanceof StructType) {
return getStruct(ordinal, ((StructType) dataType).size());
} else if (dataType instanceof ArrayType) {
return getArray(ordinal);
} else if (dataType instanceof MapType) {
return getMap(ordinal);
} else {
throw new UnsupportedOperationException("Unsupported data type " + dataType.simpleString());
}
}

@Override
public boolean getBoolean(int ordinal) {
assertIndexIsValid(ordinal);
final int offset = getElementOffset(ordinal);
if (offset < 0) {
return false;
} else {
return PlatformDependent.UNSAFE.getBoolean(baseObject, baseOffset + offset);
}
}

@Override
public byte getByte(int ordinal) {
assertIndexIsValid(ordinal);
final int offset = getElementOffset(ordinal);
if (offset < 0) {
return 0;
} else {
return PlatformDependent.UNSAFE.getByte(baseObject, baseOffset + offset);
}
}

@Override
public short getShort(int ordinal) {
assertIndexIsValid(ordinal);
final int offset = getElementOffset(ordinal);
if (offset < 0) {
return 0;
} else {
return PlatformDependent.UNSAFE.getShort(baseObject, baseOffset + offset);
}
}

@Override
public int getInt(int ordinal) {
assertIndexIsValid(ordinal);
final int offset = getElementOffset(ordinal);
if (offset < 0) {
return 0;
} else {
return PlatformDependent.UNSAFE.getInt(baseObject, baseOffset + offset);
}
}

@Override
public long getLong(int ordinal) {
assertIndexIsValid(ordinal);
final int offset = getElementOffset(ordinal);
if (offset < 0) {
return 0;
} else {
return PlatformDependent.UNSAFE.getLong(baseObject, baseOffset + offset);
}
}

@Override
public float getFloat(int ordinal) {
assertIndexIsValid(ordinal);
final int offset = getElementOffset(ordinal);
if (offset < 0) {
return 0;
} else {
return PlatformDependent.UNSAFE.getFloat(baseObject, baseOffset + offset);
}
}

@Override
public double getDouble(int ordinal) {
assertIndexIsValid(ordinal);
final int offset = getElementOffset(ordinal);
if (offset < 0) {
return 0;
} else {
return PlatformDependent.UNSAFE.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;
} else {
if (precision <= Decimal.MAX_LONG_DIGITS()) {
final long value = PlatformDependent.UNSAFE.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) {
final byte[] bytes = getBinary(ordinal);
if (bytes == null) {
return null;
} else {
return UTF8String.fromBytes(bytes);
}
}

@Override
public byte[] getBinary(int ordinal) {
assertIndexIsValid(ordinal);
final int offset = getElementOffset(ordinal);
if (offset < 0) {
return null;
} else {
final int size = getElementSize(offset, ordinal);
final byte[] bytes = new byte[size];
PlatformDependent.copyMemory(
baseObject,
baseOffset + offset,
bytes,
PlatformDependent.BYTE_ARRAY_OFFSET,
size);
return bytes;
}
}

@Override
public CalendarInterval getInterval(int ordinal) {
assertIndexIsValid(ordinal);
final int offset = getElementOffset(ordinal);
if (offset < 0) {
return null;
} else {
final int months = (int) PlatformDependent.UNSAFE.getLong(baseObject, baseOffset + offset);
final long microseconds =
PlatformDependent.UNSAFE.getLong(baseObject, baseOffset + offset + 8);
return new CalendarInterval(months, microseconds);
}
}

@Override
public InternalRow getStruct(int ordinal, int numFields) {
assertIndexIsValid(ordinal);
final int offset = getElementOffset(ordinal);
if (offset < 0) {
return null;
} else {
final int size = getElementSize(offset, ordinal);
final UnsafeRow row = new UnsafeRow();
row.pointTo(baseObject, baseOffset + offset, numFields, size);
return row;
}
}

@Override
public ArrayData getArray(int ordinal) {
assertIndexIsValid(ordinal);
final int offset = getElementOffset(ordinal);
if (offset < 0) {
return null;
} else {
final int numElements = PlatformDependent.UNSAFE.getInt(baseObject, baseOffset + offset);
final int size = getElementSize(offset, ordinal);
final UnsafeArrayData array = new UnsafeArrayData();
// Skip the first 4 bytes.
array.pointTo(baseObject, baseOffset + offset + 4, numElements, size - 4);
return array;
}
}

@Override
public MapData getMap(int ordinal) {
return null;
}

@Override
public int hashCode() {
return Murmur3_x86_32.hashUnsafeWords(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) {
PlatformDependent.copyMemory(
baseObject,
baseOffset,
target,
targetOffset,
sizeInBytes
);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -291,6 +291,10 @@ public Object get(int ordinal, DataType dataType) {
return getInterval(ordinal);
} else if (dataType instanceof StructType) {
return getStruct(ordinal, ((StructType) dataType).size());
} else if (dataType instanceof ArrayType) {
return getArray(ordinal);
} else if (dataType instanceof MapType) {
return getMap(ordinal);
} else {
throw new UnsupportedOperationException("Unsupported data type " + dataType.simpleString());
}
Expand Down Expand Up @@ -420,6 +424,23 @@ public UnsafeRow getStruct(int ordinal, int numFields) {
}
}

@Override
public ArrayData getArray(int ordinal) {
if (isNullAt(ordinal)) {
return null;
} else {
assertIndexIsValid(ordinal);
Copy link
Contributor

Choose a reason for hiding this comment

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

this should go before the if

final long offsetAndSize = getLong(ordinal);
final int offset = (int) (offsetAndSize >> 32);
final int size = (int) (offsetAndSize & ((1L << 32) - 1));
final int numElements = PlatformDependent.UNSAFE.getInt(baseObject, baseOffset + offset);
final UnsafeArrayData array = new UnsafeArrayData();
// Skip the first 4 bytes.
array.pointTo(baseObject, baseOffset + offset + 4, numElements, size - 4);
return array;
}
}

/**
* Copies this row, returning a self-contained UnsafeRow that stores its data in an internal
* byte array rather than referencing data stored in a data page.
Expand Down
Loading