Skip to content
Original file line number Diff line number Diff line change
Expand Up @@ -154,19 +154,22 @@ public double getDouble(int rowId) {

@Override
public Decimal getDecimal(int rowId, int precision, int scale) {
if (isNullAt(rowId)) return null;
BigDecimal data = decimalData.vector[getRowIndex(rowId)].getHiveDecimal().bigDecimalValue();
return Decimal.apply(data, precision, scale);
}

@Override
public UTF8String getUTF8String(int rowId) {
if (isNullAt(rowId)) return null;
int index = getRowIndex(rowId);
BytesColumnVector col = bytesData;
return UTF8String.fromBytes(col.vector[index], col.start[index], col.length[index]);
}

@Override
public byte[] getBinary(int rowId) {
if (isNullAt(rowId)) return null;
int index = getRowIndex(rowId);
byte[] binary = new byte[bytesData.length[index]];
System.arraycopy(bytesData.vector[index], bytesData.start[index], binary, 0, binary.length);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -127,43 +127,36 @@ public boolean anyNull() {

@Override
public Decimal getDecimal(int ordinal, int precision, int scale) {
if (columns[ordinal].isNullAt(rowId)) return null;
return columns[ordinal].getDecimal(rowId, precision, scale);
}

@Override
public UTF8String getUTF8String(int ordinal) {
if (columns[ordinal].isNullAt(rowId)) return null;
return columns[ordinal].getUTF8String(rowId);
}

@Override
public byte[] getBinary(int ordinal) {
if (columns[ordinal].isNullAt(rowId)) return null;
return columns[ordinal].getBinary(rowId);
}

@Override
public CalendarInterval getInterval(int ordinal) {
if (columns[ordinal].isNullAt(rowId)) return null;
return columns[ordinal].getInterval(rowId);
}

@Override
public ColumnarRow getStruct(int ordinal, int numFields) {
if (columns[ordinal].isNullAt(rowId)) return null;
return columns[ordinal].getStruct(rowId);
}

@Override
public ColumnarArray getArray(int ordinal) {
if (columns[ordinal].isNullAt(rowId)) return null;
return columns[ordinal].getArray(rowId);
}

@Override
public ColumnarMap getMap(int ordinal) {
if (columns[ordinal].isNullAt(rowId)) return null;
return columns[ordinal].getMap(rowId);
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -341,6 +341,7 @@ public final int putByteArray(int rowId, byte[] value) {

@Override
public Decimal getDecimal(int rowId, int precision, int scale) {
if (isNullAt(rowId)) return null;
if (precision <= Decimal.MAX_INT_DIGITS()) {
return Decimal.createUnsafe(getInt(rowId), precision, scale);
} else if (precision <= Decimal.MAX_LONG_DIGITS()) {
Expand All @@ -367,6 +368,7 @@ public void putDecimal(int rowId, Decimal value, int precision) {

@Override
public UTF8String getUTF8String(int rowId) {
if (isNullAt(rowId)) return null;
if (dictionary == null) {
return arrayData().getBytesAsUTF8String(getArrayOffset(rowId), getArrayLength(rowId));
} else {
Expand All @@ -384,6 +386,7 @@ public UTF8String getUTF8String(int rowId) {

@Override
public byte[] getBinary(int rowId) {
if (isNullAt(rowId)) return null;
if (dictionary == null) {
return arrayData().getBytes(getArrayOffset(rowId), getArrayLength(rowId));
} else {
Expand Down Expand Up @@ -613,13 +616,15 @@ public final int appendStruct(boolean isNull) {
// array offsets and lengths in the current column vector.
@Override
public final ColumnarArray getArray(int rowId) {
if (isNullAt(rowId)) return null;
return new ColumnarArray(arrayData(), getArrayOffset(rowId), getArrayLength(rowId));
}

// `WritableColumnVector` puts the key array in the first child column vector, value array in the
// second child column vector, and puts the offsets and lengths in the current column vector.
@Override
public final ColumnarMap getMap(int rowId) {
if (isNullAt(rowId)) return null;
return new ColumnarMap(getChild(0), getChild(1), getArrayOffset(rowId), getArrayLength(rowId));
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -101,21 +101,25 @@ public double getDouble(int rowId) {

@Override
public Decimal getDecimal(int rowId, int precision, int scale) {
if (isNullAt(rowId)) return null;
return accessor.getDecimal(rowId, precision, scale);
}

@Override
public UTF8String getUTF8String(int rowId) {
if (isNullAt(rowId)) return null;
return accessor.getUTF8String(rowId);
}

@Override
public byte[] getBinary(int rowId) {
if (isNullAt(rowId)) return null;
return accessor.getBinary(rowId);
}

@Override
public ColumnarArray getArray(int rowId) {
if (isNullAt(rowId)) return null;
return accessor.getArray(rowId);
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -80,12 +80,14 @@ public abstract class ColumnVector implements AutoCloseable {
public abstract boolean isNullAt(int rowId);

/**
* Returns the boolean type value for rowId.
* Returns the boolean type value for rowId. The return value is undefined and can be anything,
* if the slot for rowId is null.
*/
public abstract boolean getBoolean(int rowId);

/**
* Gets boolean type values from [rowId, rowId + count)
* Gets boolean type values from [rowId, rowId + count). The return values for the null slots
* are undefined and can be anything.
*/
public boolean[] getBooleans(int rowId, int count) {
boolean[] res = new boolean[count];
Expand All @@ -96,12 +98,14 @@ public boolean[] getBooleans(int rowId, int count) {
}

/**
* Returns the byte type value for rowId.
* Returns the byte type value for rowId. The return value is undefined and can be anything,
* if the slot for rowId is null.
*/
public abstract byte getByte(int rowId);

/**
* Gets byte type values from [rowId, rowId + count)
* Gets byte type values from [rowId, rowId + count). The return values for the null slots
* are undefined and can be anything.
*/
public byte[] getBytes(int rowId, int count) {
byte[] res = new byte[count];
Expand All @@ -112,12 +116,14 @@ public byte[] getBytes(int rowId, int count) {
}

/**
* Returns the short type value for rowId.
* Returns the short type value for rowId. The return value is undefined and can be anything,
* if the slot for rowId is null.
*/
public abstract short getShort(int rowId);

/**
* Gets short type values from [rowId, rowId + count)
* Gets short type values from [rowId, rowId + count). The return values for the null slots
* are undefined and can be anything.
*/
public short[] getShorts(int rowId, int count) {
short[] res = new short[count];
Expand All @@ -128,12 +134,14 @@ public short[] getShorts(int rowId, int count) {
}

/**
* Returns the int type value for rowId.
* Returns the int type value for rowId. The return value is undefined and can be anything,
* if the slot for rowId is null.
*/
public abstract int getInt(int rowId);

/**
* Gets int type values from [rowId, rowId + count)
* Gets int type values from [rowId, rowId + count). The return values for the null slots
* are undefined and can be anything.
*/
public int[] getInts(int rowId, int count) {
int[] res = new int[count];
Expand All @@ -144,12 +152,14 @@ public int[] getInts(int rowId, int count) {
}

/**
* Returns the long type value for rowId.
* Returns the long type value for rowId. The return value is undefined and can be anything,
* if the slot for rowId is null.
*/
public abstract long getLong(int rowId);

/**
* Gets long type values from [rowId, rowId + count)
* Gets long type values from [rowId, rowId + count). The return values for the null slots
* are undefined and can be anything.
*/
public long[] getLongs(int rowId, int count) {
long[] res = new long[count];
Expand All @@ -160,12 +170,14 @@ public long[] getLongs(int rowId, int count) {
}

/**
* Returns the float type value for rowId.
* Returns the float type value for rowId. The return value is undefined and can be anything,
* if the slot for rowId is null.
*/
public abstract float getFloat(int rowId);

/**
* Gets float type values from [rowId, rowId + count)
* Gets float type values from [rowId, rowId + count). The return values for the null slots
* are undefined and can be anything.
*/
public float[] getFloats(int rowId, int count) {
float[] res = new float[count];
Expand All @@ -176,12 +188,14 @@ public float[] getFloats(int rowId, int count) {
}

/**
* Returns the double type value for rowId.
* Returns the double type value for rowId. The return value is undefined and can be anything,
* if the slot for rowId is null.
*/
public abstract double getDouble(int rowId);

/**
* Gets double type values from [rowId, rowId + count)
* Gets double type values from [rowId, rowId + count). The return values for the null slots
* are undefined and can be anything.
*/
public double[] getDoubles(int rowId, int count) {
double[] res = new double[count];
Expand All @@ -192,7 +206,7 @@ public double[] getDoubles(int rowId, int count) {
}

/**
* Returns the struct type value for rowId.
* Returns the struct type value for rowId. If the slot for rowId is null, it should return null.
*
* To support struct type, implementations must implement {@link #getChild(int)} and make this
* vector a tree structure. The number of child vectors must be same as the number of fields of
Expand All @@ -205,7 +219,7 @@ public final ColumnarRow getStruct(int rowId) {
}

/**
* Returns the array type value for rowId.
* Returns the array type value for rowId. If the slot for rowId is null, it should return null.
*
* To support array type, implementations must construct an {@link ColumnarArray} and return it in
* this method. {@link ColumnarArray} requires a {@link ColumnVector} that stores the data of all
Expand All @@ -218,13 +232,13 @@ public final ColumnarRow getStruct(int rowId) {
public abstract ColumnarArray getArray(int rowId);

/**
* Returns the map type value for rowId.
* Returns the map type value for rowId. If the slot for rowId is null, it should return null.
*
* In Spark, map type value is basically a key data array and a value data array. A key from the
* key array with a index and a value from the value array with the same index contribute to
* an entry of this map type value.
*
* To support map type, implementations must construct an {@link ColumnarMap} and return it in
* To support map type, implementations must construct a {@link ColumnarMap} and return it in
* this method. {@link ColumnarMap} requires a {@link ColumnVector} that stores the data of all
* the keys of all the maps in this vector, and another {@link ColumnVector} that stores the data
* of all the values of all the maps in this vector, and a pair of offset and length which
Expand All @@ -233,24 +247,25 @@ public final ColumnarRow getStruct(int rowId) {
public abstract ColumnarMap getMap(int ordinal);

/**
* Returns the decimal type value for rowId.
* Returns the decimal type value for rowId. If the slot for rowId is null, it should return null.
*/
public abstract Decimal getDecimal(int rowId, int precision, int scale);

/**
* Returns the string type value for rowId. Note that the returned UTF8String may point to the
* data of this column vector, please copy it if you want to keep it after this column vector is
* freed.
* Returns the string type value for rowId. If the slot for rowId is null, it should return null.
* Note that the returned UTF8String may point to the data of this column vector, please copy it
* if you want to keep it after this column vector is freed.
*/
public abstract UTF8String getUTF8String(int rowId);

/**
* Returns the binary type value for rowId.
* Returns the binary type value for rowId. If the slot for rowId is null, it should return null.
*/
public abstract byte[] getBinary(int rowId);

/**
* Returns the calendar interval type value for rowId.
* Returns the calendar interval type value for rowId. If the slot for rowId is null, it should
* return null.
*
* In Spark, calendar interval type value is basically an integer value representing the number of
* months in this interval, and a long value representing the number of microseconds in this
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -119,43 +119,36 @@ public boolean anyNull() {

@Override
public Decimal getDecimal(int ordinal, int precision, int scale) {
if (data.getChild(ordinal).isNullAt(rowId)) return null;
return data.getChild(ordinal).getDecimal(rowId, precision, scale);
}

@Override
public UTF8String getUTF8String(int ordinal) {
if (data.getChild(ordinal).isNullAt(rowId)) return null;
return data.getChild(ordinal).getUTF8String(rowId);
}

@Override
public byte[] getBinary(int ordinal) {
if (data.getChild(ordinal).isNullAt(rowId)) return null;
return data.getChild(ordinal).getBinary(rowId);
}

@Override
public CalendarInterval getInterval(int ordinal) {
if (data.getChild(ordinal).isNullAt(rowId)) return null;
return data.getChild(ordinal).getInterval(rowId);
}

@Override
public ColumnarRow getStruct(int ordinal, int numFields) {
if (data.getChild(ordinal).isNullAt(rowId)) return null;
return data.getChild(ordinal).getStruct(rowId);
}

@Override
public ColumnarArray getArray(int ordinal) {
if (data.getChild(ordinal).isNullAt(rowId)) return null;
return data.getChild(ordinal).getArray(rowId);
}

@Override
public ColumnarMap getMap(int ordinal) {
if (data.getChild(ordinal).isNullAt(rowId)) return null;
return data.getChild(ordinal).getMap(rowId);
}

Expand Down
Loading